HBASE-18241 Change client.Table, client.Admin, Region, Store, and HBaseTestingUtility to not use HTableDescriptor or HColumnDescriptor
This commit is contained in:
parent
c05a408093
commit
53ec9c5bd3
|
@ -149,17 +149,6 @@ public class HTableDescriptor implements TableDescriptor, Comparable<HTableDescr
|
|||
return delegatee.isMetaTable();
|
||||
}
|
||||
|
||||
/**
|
||||
* Getter for accessing the metadata associated with the key
|
||||
*
|
||||
* @param key The key.
|
||||
* @return The value.
|
||||
*/
|
||||
public String getValue(String key) {
|
||||
byte[] value = getValue(Bytes.toBytes(key));
|
||||
return value == null ? null : Bytes.toString(value);
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Getter for fetching an unmodifiable map.
|
||||
*/
|
||||
|
@ -860,6 +849,11 @@ public class HTableDescriptor implements TableDescriptor, Comparable<HTableDescr
|
|||
return delegatee.getValue(key);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getValue(String key) {
|
||||
return delegatee.getValue(key);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getValue(byte[] key) {
|
||||
return delegatee.getValue(key);
|
||||
|
|
|
@ -98,9 +98,20 @@ public interface Admin extends Abortable, Closeable {
|
|||
*
|
||||
* @return - returns an array of read-only HTableDescriptors
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version.
|
||||
* use {@link #listTableDescriptors()}
|
||||
*/
|
||||
@Deprecated
|
||||
HTableDescriptor[] listTables() throws IOException;
|
||||
|
||||
/**
|
||||
* List all the userspace tables.
|
||||
*
|
||||
* @return - returns a list of TableDescriptors
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
*/
|
||||
List<TableDescriptor> listTableDescriptors() throws IOException;
|
||||
|
||||
/**
|
||||
* List all the userspace tables matching the given pattern.
|
||||
*
|
||||
|
@ -108,18 +119,44 @@ public interface Admin extends Abortable, Closeable {
|
|||
* @return - returns an array of read-only HTableDescriptors
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
* @see #listTables()
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version.
|
||||
* use {@link #listTableDescriptors(java.util.regex.Pattern)}
|
||||
*/
|
||||
@Deprecated
|
||||
HTableDescriptor[] listTables(Pattern pattern) throws IOException;
|
||||
|
||||
/**
|
||||
* List all the userspace tables matching the given pattern.
|
||||
*
|
||||
* @param pattern The compiled regular expression to match against
|
||||
* @return - returns a list of TableDescriptors
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
* @see #listTables()
|
||||
*/
|
||||
List<TableDescriptor> listTableDescriptors(Pattern pattern) throws IOException;
|
||||
|
||||
/**
|
||||
* List all the userspace tables matching the given regular expression.
|
||||
*
|
||||
* @param regex The regular expression to match against
|
||||
* @return - returns an array of HTableDescriptors
|
||||
* @return - returns an array of read-only HTableDescriptors
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
* @see #listTables(java.util.regex.Pattern)
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version.
|
||||
* use {@link #listTableDescriptors(java.lang.String)}
|
||||
*/
|
||||
@Deprecated
|
||||
HTableDescriptor[] listTables(String regex) throws IOException;
|
||||
|
||||
/**
|
||||
* List all the userspace tables matching the given regular expression.
|
||||
*
|
||||
* @param regex The regular expression to match against
|
||||
* @return - returns a list of TableDescriptors
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
* @see #listTables(java.util.regex.Pattern)
|
||||
*/
|
||||
HTableDescriptor[] listTables(String regex) throws IOException;
|
||||
List<TableDescriptor> listTableDescriptors(String regex) throws IOException;
|
||||
|
||||
/**
|
||||
* List all the tables matching the given pattern.
|
||||
|
@ -129,10 +166,25 @@ public interface Admin extends Abortable, Closeable {
|
|||
* @return - returns an array of read-only HTableDescriptors
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
* @see #listTables()
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version.
|
||||
* use {@link #listTableDescriptors(java.util.regex.Pattern, boolean)}
|
||||
*/
|
||||
@Deprecated
|
||||
HTableDescriptor[] listTables(Pattern pattern, boolean includeSysTables)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* List all the tables matching the given pattern.
|
||||
*
|
||||
* @param pattern The compiled regular expression to match against
|
||||
* @param includeSysTables False to match only against userspace tables
|
||||
* @return - returns a list of TableDescriptors
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
* @see #listTables()
|
||||
*/
|
||||
List<TableDescriptor> listTableDescriptors(Pattern pattern, boolean includeSysTables)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* List all the tables matching the given pattern.
|
||||
*
|
||||
|
@ -141,10 +193,25 @@ public interface Admin extends Abortable, Closeable {
|
|||
* @return - returns an array of read-only HTableDescriptors
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
* @see #listTables(java.util.regex.Pattern, boolean)
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version.
|
||||
* use {@link #listTableDescriptors(java.lang.String, boolean)}
|
||||
*/
|
||||
@Deprecated
|
||||
HTableDescriptor[] listTables(String regex, boolean includeSysTables)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* List all the tables matching the given pattern.
|
||||
*
|
||||
* @param regex The regular expression to match against
|
||||
* @param includeSysTables False to match only against userspace tables
|
||||
* @return - returns a list of TableDescriptors
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
* @see #listTables(java.util.regex.Pattern, boolean)
|
||||
*/
|
||||
List<TableDescriptor> listTableDescriptors(String regex, boolean includeSysTables)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* List all of the names of userspace tables.
|
||||
*
|
||||
|
@ -196,10 +263,24 @@ public interface Admin extends Abortable, Closeable {
|
|||
* @return the read-only tableDescriptor
|
||||
* @throws org.apache.hadoop.hbase.TableNotFoundException
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version.
|
||||
* use {@link #listTableDescriptor(TableName)}
|
||||
*/
|
||||
@Deprecated
|
||||
HTableDescriptor getTableDescriptor(final TableName tableName)
|
||||
throws TableNotFoundException, IOException;
|
||||
|
||||
/**
|
||||
* Method for getting the tableDescriptor
|
||||
*
|
||||
* @param tableName as a {@link TableName}
|
||||
* @return the tableDescriptor
|
||||
* @throws org.apache.hadoop.hbase.TableNotFoundException
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
*/
|
||||
TableDescriptor listTableDescriptor(final TableName tableName)
|
||||
throws TableNotFoundException, IOException;
|
||||
|
||||
/**
|
||||
* Creates a new table. Synchronous operation.
|
||||
*
|
||||
|
@ -209,8 +290,50 @@ public interface Admin extends Abortable, Closeable {
|
|||
* @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 if a remote or network exception occurs
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version.
|
||||
* use {@link #createTable(TableDescriptor)}
|
||||
*/
|
||||
void createTable(HTableDescriptor desc) throws IOException;
|
||||
@Deprecated
|
||||
default void createTable(HTableDescriptor desc) throws IOException {
|
||||
createTable((TableDescriptor) desc);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new table. Synchronous operation.
|
||||
*
|
||||
* @param desc table descriptor for table
|
||||
* @throws IllegalArgumentException if the table name is reserved
|
||||
* @throws org.apache.hadoop.hbase.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 if a remote or network exception occurs
|
||||
*/
|
||||
void createTable(TableDescriptor desc) throws IOException;
|
||||
|
||||
/**
|
||||
* Creates a new table with the specified number of regions. The start key specified will become
|
||||
* the end key of the first region of the table, and the end key specified will become the start
|
||||
* key of the last region of the table (the first region has a null start key and the last region
|
||||
* has a null end key). BigInteger math will be used to divide the key range specified into enough
|
||||
* segments to make the required number of total regions. Synchronous operation.
|
||||
*
|
||||
* @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 org.apache.hadoop.hbase.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
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version.
|
||||
* use {@link #createTable(TableDescriptor, byte[], byte[], int)}
|
||||
*/
|
||||
@Deprecated
|
||||
default void createTable(HTableDescriptor desc, byte[] startKey, byte[] endKey, int numRegions)
|
||||
throws IOException {
|
||||
createTable((TableDescriptor) desc, startKey, endKey, numRegions);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new table with the specified number of regions. The start key specified will become
|
||||
|
@ -229,7 +352,7 @@ public interface Admin extends Abortable, Closeable {
|
|||
* threads, the table may have been created between test-for-existence and attempt-at-creation).
|
||||
* @throws IOException
|
||||
*/
|
||||
void createTable(HTableDescriptor desc, byte[] startKey, byte[] endKey, int numRegions)
|
||||
void createTable(TableDescriptor desc, byte[] startKey, byte[] endKey, int numRegions)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
|
@ -245,8 +368,52 @@ public interface Admin extends Abortable, Closeable {
|
|||
* @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
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version.
|
||||
* use {@link #createTable(TableDescriptor, byte[][])}
|
||||
*/
|
||||
void createTable(final HTableDescriptor desc, byte[][] splitKeys) throws IOException;
|
||||
@Deprecated
|
||||
default void createTable(final HTableDescriptor desc, byte[][] splitKeys) throws IOException {
|
||||
createTable((TableDescriptor) 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 org.apache.hadoop.hbase.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
|
||||
*/
|
||||
void createTable(final TableDescriptor desc, byte[][] splitKeys) throws IOException;
|
||||
|
||||
/**
|
||||
* Creates a new table but does not block and wait for it to come online.
|
||||
* You can use Future.get(long, TimeUnit) to wait on the operation to complete.
|
||||
* It may throw ExecutionException if there was an error while executing the operation
|
||||
* or TimeoutException in case the wait timeout was not long enough to allow the
|
||||
* operation to complete.
|
||||
* Throws IllegalArgumentException Bad table name, if the split keys
|
||||
* are repeated and if the split key has empty byte array.
|
||||
*
|
||||
* @param desc table descriptor for table
|
||||
* @param splitKeys keys to check if the table has been created with all split keys
|
||||
* @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.
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version.
|
||||
* use {@link #createTableAsync(TableDescriptor, byte[][])}
|
||||
*/
|
||||
@Deprecated
|
||||
default Future<Void> createTableAsync(final HTableDescriptor desc, final byte[][] splitKeys)
|
||||
throws IOException {
|
||||
return createTableAsync((TableDescriptor) desc, splitKeys);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new table but does not block and wait for it to come online.
|
||||
|
@ -263,7 +430,7 @@ public interface Admin extends Abortable, Closeable {
|
|||
* @return the result of the async creation. You can use Future.get(long, TimeUnit)
|
||||
* to wait on the operation to complete.
|
||||
*/
|
||||
Future<Void> createTableAsync(final HTableDescriptor desc, final byte[][] splitKeys)
|
||||
Future<Void> createTableAsync(final TableDescriptor desc, final byte[][] splitKeys)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
|
@ -291,7 +458,8 @@ public interface Admin extends Abortable, Closeable {
|
|||
/**
|
||||
* 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(org.apache.hadoop.hbase.TableName)}
|
||||
* #listTableDescriptors(java.lang.String)}
|
||||
* and {@link #deleteTable(org.apache.hadoop.hbase.TableName)}
|
||||
*
|
||||
* @param regex The regular expression to match table names against
|
||||
* @return Table descriptors for tables that couldn't be deleted.
|
||||
|
@ -299,20 +467,30 @@ public interface Admin extends Abortable, Closeable {
|
|||
* @throws IOException
|
||||
* @see #deleteTables(java.util.regex.Pattern)
|
||||
* @see #deleteTable(org.apache.hadoop.hbase.TableName)
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version
|
||||
* This is just a trivial helper method without any magic.
|
||||
* Consider using {@link #listTableDescriptors(java.lang.String)}
|
||||
* and {@link #enableTable(org.apache.hadoop.hbase.TableName)}
|
||||
*/
|
||||
@Deprecated
|
||||
HTableDescriptor[] deleteTables(String regex) throws IOException;
|
||||
|
||||
/**
|
||||
* Delete tables matching the passed in pattern and wait on completion. Warning: Use this method
|
||||
* carefully, there is no prompting and the effect is immediate. Consider using {@link
|
||||
* #listTables(java.util.regex.Pattern) } and
|
||||
* #listTableDescriptors(java.util.regex.Pattern)} and
|
||||
* {@link #deleteTable(org.apache.hadoop.hbase.TableName)}
|
||||
*
|
||||
* @param pattern The pattern to match table names against
|
||||
* @return Table descriptors for tables that couldn't be deleted
|
||||
* The return htds are read-only
|
||||
* @throws IOException
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version
|
||||
* This is just a trivial helper method without any magic.
|
||||
* Consider using {@link #listTableDescriptors(java.util.regex.Pattern)}
|
||||
* and {@link #enableTable(org.apache.hadoop.hbase.TableName)}
|
||||
*/
|
||||
@Deprecated
|
||||
HTableDescriptor[] deleteTables(Pattern pattern) throws IOException;
|
||||
|
||||
/**
|
||||
|
@ -372,7 +550,7 @@ public interface Admin extends Abortable, Closeable {
|
|||
/**
|
||||
* 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(org.apache.hadoop.hbase.TableName)}
|
||||
* #listTableDescriptors(java.lang.String)} and {@link #enableTable(org.apache.hadoop.hbase.TableName)}
|
||||
*
|
||||
* @param regex The regular expression to match table names against
|
||||
* @throws IOException
|
||||
|
@ -380,20 +558,30 @@ public interface Admin extends Abortable, Closeable {
|
|||
* The return HTDs are read-only.
|
||||
* @see #enableTables(java.util.regex.Pattern)
|
||||
* @see #enableTable(org.apache.hadoop.hbase.TableName)
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version
|
||||
* This is just a trivial helper method without any magic.
|
||||
* Consider using {@link #listTableDescriptors(java.lang.String)}
|
||||
* and {@link #enableTable(org.apache.hadoop.hbase.TableName)}
|
||||
*/
|
||||
@Deprecated
|
||||
HTableDescriptor[] enableTables(String regex) throws IOException;
|
||||
|
||||
/**
|
||||
* 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
|
||||
* #listTableDescriptors(java.util.regex.Pattern)} and
|
||||
* {@link #enableTable(org.apache.hadoop.hbase.TableName)}
|
||||
*
|
||||
* @param pattern The pattern to match table names against
|
||||
* @throws IOException
|
||||
* @return Table descriptors for tables that couldn't be enabled.
|
||||
* The return HTDs are read-only.
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version
|
||||
* This is just a trivial helper method without any magic.
|
||||
* Consider using {@link #listTableDescriptors(java.util.regex.Pattern)}
|
||||
* and {@link #enableTable(org.apache.hadoop.hbase.TableName)}
|
||||
*/
|
||||
@Deprecated
|
||||
HTableDescriptor[] enableTables(Pattern pattern) throws IOException;
|
||||
|
||||
/**
|
||||
|
@ -425,7 +613,8 @@ public interface Admin extends Abortable, Closeable {
|
|||
/**
|
||||
* 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(org.apache.hadoop.hbase.TableName)}
|
||||
* #listTableDescriptors(java.lang.String)}
|
||||
* and {@link #disableTable(org.apache.hadoop.hbase.TableName)}
|
||||
*
|
||||
* @param regex The regular expression to match table names against
|
||||
* @return Table descriptors for tables that couldn't be disabled
|
||||
|
@ -433,20 +622,30 @@ public interface Admin extends Abortable, Closeable {
|
|||
* @throws IOException
|
||||
* @see #disableTables(java.util.regex.Pattern)
|
||||
* @see #disableTable(org.apache.hadoop.hbase.TableName)
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version
|
||||
* This is just a trivial helper method without any magic.
|
||||
* Consider using {@link #listTableDescriptors(java.lang.String)}
|
||||
* and {@link #disableTable(org.apache.hadoop.hbase.TableName)}
|
||||
*/
|
||||
@Deprecated
|
||||
HTableDescriptor[] disableTables(String regex) throws IOException;
|
||||
|
||||
/**
|
||||
* 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
|
||||
* #listTableDescriptors(java.util.regex.Pattern)} and
|
||||
* {@link #disableTable(org.apache.hadoop.hbase.TableName)}
|
||||
*
|
||||
* @param pattern The pattern to match table names against
|
||||
* @return Table descriptors for tables that couldn't be disabled
|
||||
* The return htds are read-only
|
||||
* @throws IOException
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version
|
||||
* This is just a trivial helper method without any magic.
|
||||
* Consider using {@link #listTableDescriptors(java.util.regex.Pattern)}
|
||||
* and {@link #disableTable(org.apache.hadoop.hbase.TableName)}
|
||||
*/
|
||||
@Deprecated
|
||||
HTableDescriptor[] disableTables(Pattern pattern) throws IOException;
|
||||
|
||||
/**
|
||||
|
@ -1014,10 +1213,21 @@ public interface Admin extends Abortable, Closeable {
|
|||
* @param tableName name of table.
|
||||
* @param htd modified description of the table
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version.
|
||||
* use {@link #modifyTable(TableDescriptor)}
|
||||
*/
|
||||
@Deprecated
|
||||
void modifyTable(final TableName tableName, final HTableDescriptor htd)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Modify an existing table, more IRB friendly version.
|
||||
*
|
||||
* @param td modified description of the table
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
*/
|
||||
void modifyTable(final TableDescriptor td) throws IOException;
|
||||
|
||||
/**
|
||||
* 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.
|
||||
|
@ -1031,10 +1241,29 @@ public interface Admin extends Abortable, Closeable {
|
|||
* @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
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version.
|
||||
* use {@link #modifyTableAsync(TableDescriptor)}
|
||||
*/
|
||||
@Deprecated
|
||||
Future<Void> modifyTableAsync(final TableName tableName, final HTableDescriptor htd)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* 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 td 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
|
||||
*/
|
||||
Future<Void> modifyTableAsync(TableDescriptor td)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Shuts down the HBase cluster
|
||||
*
|
||||
|
@ -1177,10 +1406,23 @@ public interface Admin extends Abortable, Closeable {
|
|||
* @param name namespace name
|
||||
* @return HTD[] the read-only tableDescriptors
|
||||
* @throws IOException
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version.
|
||||
* use {@link #listTableDescriptorsByNamespace(byte[])}
|
||||
*/
|
||||
@Deprecated
|
||||
HTableDescriptor[] listTableDescriptorsByNamespace(final String name)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Get list of table descriptors by namespace
|
||||
*
|
||||
* @param name namespace name
|
||||
* @return returns a list of TableDescriptors
|
||||
* @throws IOException
|
||||
*/
|
||||
List<TableDescriptor> listTableDescriptorsByNamespace(final byte[] name)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Get list of table names by namespace
|
||||
*
|
||||
|
@ -1210,17 +1452,33 @@ public interface Admin extends Abortable, Closeable {
|
|||
* @param tableNames List of table names
|
||||
* @return HTD[] the read-only tableDescriptors
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version.
|
||||
* use {@link #listTableDescriptors(List)}
|
||||
*/
|
||||
@Deprecated
|
||||
HTableDescriptor[] getTableDescriptorsByTableName(List<TableName> tableNames)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Get tableDescriptors
|
||||
*
|
||||
* @param tableNames List of table names
|
||||
* @return returns a list of TableDescriptors
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
*/
|
||||
List<TableDescriptor> listTableDescriptors(List<TableName> tableNames)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Get tableDescriptors
|
||||
*
|
||||
* @param names List of table names
|
||||
* @return HTD[] the read-only tableDescriptors
|
||||
* @throws IOException if a remote or network exception occurs
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version.
|
||||
* use {@link #listTableDescriptors(List)}
|
||||
*/
|
||||
@Deprecated
|
||||
HTableDescriptor[] getTableDescriptors(List<String> names)
|
||||
throws IOException;
|
||||
|
||||
|
|
|
@ -206,6 +206,7 @@ import com.google.common.annotations.VisibleForTesting;
|
|||
import com.google.protobuf.Descriptors;
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.RpcController;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* HBaseAdmin is no longer a client API. It is marked InterfaceAudience.Private indicating that
|
||||
|
@ -307,6 +308,96 @@ public class HBaseAdmin implements Admin {
|
|||
return new AbortProcedureFuture(this, procId, abortProcResponse);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<TableDescriptor> listTableDescriptors() throws IOException {
|
||||
return listTableDescriptors((Pattern)null, false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<TableDescriptor> listTableDescriptors(Pattern pattern) throws IOException {
|
||||
return listTableDescriptors(pattern, false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<TableDescriptor> listTableDescriptors(String regex) throws IOException {
|
||||
return listTableDescriptors(Pattern.compile(regex), false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<TableDescriptor> listTableDescriptors(Pattern pattern, boolean includeSysTables) throws IOException {
|
||||
return executeCallable(new MasterCallable<List<TableDescriptor>>(getConnection(),
|
||||
getRpcControllerFactory()) {
|
||||
@Override
|
||||
protected List<TableDescriptor> rpcCall() throws Exception {
|
||||
GetTableDescriptorsRequest req =
|
||||
RequestConverter.buildGetTableDescriptorsRequest(pattern, includeSysTables);
|
||||
return ProtobufUtil.toTableDescriptorList(master.getTableDescriptors(getRpcController(),
|
||||
req));
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<TableDescriptor> listTableDescriptors(String regex, boolean includeSysTables) throws IOException {
|
||||
return listTableDescriptors(Pattern.compile(regex), includeSysTables);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TableDescriptor listTableDescriptor(TableName tableName) throws TableNotFoundException, IOException {
|
||||
return getTableDescriptor(tableName, getConnection(), rpcCallerFactory, rpcControllerFactory,
|
||||
operationTimeout, rpcTimeout);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void modifyTable(TableDescriptor td) throws IOException {
|
||||
get(modifyTableAsync(td), syncWaitTimeout, TimeUnit.MILLISECONDS);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Future<Void> modifyTableAsync(TableDescriptor td) throws IOException {
|
||||
ModifyTableResponse response = executeCallable(
|
||||
new MasterCallable<ModifyTableResponse>(getConnection(), getRpcControllerFactory()) {
|
||||
@Override
|
||||
protected ModifyTableResponse rpcCall() throws Exception {
|
||||
setPriority(td.getTableName());
|
||||
ModifyTableRequest request = RequestConverter.buildModifyTableRequest(
|
||||
td.getTableName(), td, ng.getNonceGroup(), ng.newNonce());
|
||||
return master.modifyTable(getRpcController(), request);
|
||||
}
|
||||
});
|
||||
return new ModifyTableFuture(this, td.getTableName(), response);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<TableDescriptor> listTableDescriptorsByNamespace(byte[] name) throws IOException {
|
||||
return executeCallable(new MasterCallable<List<TableDescriptor>>(getConnection(),
|
||||
getRpcControllerFactory()) {
|
||||
@Override
|
||||
protected List<TableDescriptor> rpcCall() throws Exception {
|
||||
return master.listTableDescriptorsByNamespace(getRpcController(),
|
||||
ListTableDescriptorsByNamespaceRequest.newBuilder()
|
||||
.setNamespaceName(Bytes.toString(name)).build())
|
||||
.getTableSchemaList()
|
||||
.stream()
|
||||
.map(ProtobufUtil::convertToTableDesc)
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<TableDescriptor> listTableDescriptors(List<TableName> tableNames) throws IOException {
|
||||
return executeCallable(new MasterCallable<List<TableDescriptor>>(getConnection(),
|
||||
getRpcControllerFactory()) {
|
||||
@Override
|
||||
protected List<TableDescriptor> rpcCall() throws Exception {
|
||||
GetTableDescriptorsRequest req =
|
||||
RequestConverter.buildGetTableDescriptorsRequest(tableNames);
|
||||
return ProtobufUtil.toTableDescriptorList(master.getTableDescriptors(getRpcController(), req));
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
private static class AbortProcedureFuture extends ProcedureFuture<Boolean> {
|
||||
private boolean isAbortInProgress;
|
||||
|
||||
|
@ -419,11 +510,40 @@ public class HBaseAdmin implements Admin {
|
|||
|
||||
@Override
|
||||
public HTableDescriptor getTableDescriptor(final TableName tableName) throws IOException {
|
||||
return getTableDescriptor(tableName, getConnection(), rpcCallerFactory, rpcControllerFactory,
|
||||
return getHTableDescriptor(tableName, getConnection(), rpcCallerFactory, rpcControllerFactory,
|
||||
operationTimeout, rpcTimeout);
|
||||
}
|
||||
|
||||
static HTableDescriptor getTableDescriptor(final TableName tableName, Connection connection,
|
||||
static TableDescriptor getTableDescriptor(final TableName tableName, Connection connection,
|
||||
RpcRetryingCallerFactory rpcCallerFactory, final RpcControllerFactory rpcControllerFactory,
|
||||
int operationTimeout, int rpcTimeout) throws IOException {
|
||||
if (tableName == null) return null;
|
||||
TableDescriptor td =
|
||||
executeCallable(new MasterCallable<TableDescriptor>(connection, rpcControllerFactory) {
|
||||
@Override
|
||||
protected TableDescriptor rpcCall() throws Exception {
|
||||
GetTableDescriptorsRequest req =
|
||||
RequestConverter.buildGetTableDescriptorsRequest(tableName);
|
||||
GetTableDescriptorsResponse htds = master.getTableDescriptors(getRpcController(), req);
|
||||
if (!htds.getTableSchemaList().isEmpty()) {
|
||||
return ProtobufUtil.convertToTableDesc(htds.getTableSchemaList().get(0));
|
||||
}
|
||||
return null;
|
||||
}
|
||||
}, rpcCallerFactory, operationTimeout, rpcTimeout);
|
||||
if (td != null) {
|
||||
return td;
|
||||
}
|
||||
throw new TableNotFoundException(tableName.getNameAsString());
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version.
|
||||
* use {@link #getTableDescriptor(TableName,
|
||||
* Connection, RpcRetryingCallerFactory,RpcControllerFactory,int,int)}
|
||||
*/
|
||||
@Deprecated
|
||||
static HTableDescriptor getHTableDescriptor(final TableName tableName, Connection connection,
|
||||
RpcRetryingCallerFactory rpcCallerFactory, final RpcControllerFactory rpcControllerFactory,
|
||||
int operationTimeout, int rpcTimeout) throws IOException {
|
||||
if (tableName == null) return null;
|
||||
|
@ -455,13 +575,13 @@ public class HBaseAdmin implements Admin {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void createTable(HTableDescriptor desc)
|
||||
public void createTable(TableDescriptor desc)
|
||||
throws IOException {
|
||||
createTable(desc, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void createTable(HTableDescriptor desc, byte [] startKey,
|
||||
public void createTable(TableDescriptor desc, byte [] startKey,
|
||||
byte [] endKey, int numRegions)
|
||||
throws IOException {
|
||||
if(numRegions < 3) {
|
||||
|
@ -481,13 +601,13 @@ public class HBaseAdmin implements Admin {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void createTable(final HTableDescriptor desc, byte [][] splitKeys)
|
||||
public void createTable(final TableDescriptor desc, byte [][] splitKeys)
|
||||
throws IOException {
|
||||
get(createTableAsync(desc, splitKeys), syncWaitTimeout, TimeUnit.MILLISECONDS);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Future<Void> createTableAsync(final HTableDescriptor desc, final byte[][] splitKeys)
|
||||
public Future<Void> createTableAsync(final TableDescriptor desc, final byte[][] splitKeys)
|
||||
throws IOException {
|
||||
if (desc.getTableName() == null) {
|
||||
throw new IllegalArgumentException("TableName cannot be null");
|
||||
|
@ -524,19 +644,19 @@ public class HBaseAdmin implements Admin {
|
|||
}
|
||||
|
||||
private static class CreateTableFuture extends TableFuture<Void> {
|
||||
private final HTableDescriptor desc;
|
||||
private final TableDescriptor desc;
|
||||
private final byte[][] splitKeys;
|
||||
|
||||
public CreateTableFuture(final HBaseAdmin admin, final HTableDescriptor desc,
|
||||
public CreateTableFuture(final HBaseAdmin admin, final TableDescriptor desc,
|
||||
final byte[][] splitKeys, final CreateTableResponse response) {
|
||||
super(admin, desc.getTableName(),
|
||||
(response != null && response.hasProcId()) ? response.getProcId() : null);
|
||||
this.splitKeys = splitKeys;
|
||||
this.desc = new ImmutableHTableDescriptor(desc);
|
||||
this.desc = desc;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HTableDescriptor getTableDescriptor() {
|
||||
protected TableDescriptor getTableDescriptor() {
|
||||
return desc;
|
||||
}
|
||||
|
||||
|
@ -3546,7 +3666,7 @@ public class HBaseAdmin implements Admin {
|
|||
/**
|
||||
* @return the table descriptor
|
||||
*/
|
||||
protected HTableDescriptor getTableDescriptor() throws IOException {
|
||||
protected TableDescriptor getTableDescriptor() throws IOException {
|
||||
return getAdmin().getTableDescriptorByTableName(getTableName());
|
||||
}
|
||||
|
||||
|
@ -3642,7 +3762,7 @@ public class HBaseAdmin implements Admin {
|
|||
|
||||
protected void waitForAllRegionsOnline(final long deadlineTs, final byte[][] splitKeys)
|
||||
throws IOException, TimeoutException {
|
||||
final HTableDescriptor desc = getTableDescriptor();
|
||||
final TableDescriptor desc = getTableDescriptor();
|
||||
final AtomicInteger actualRegCount = new AtomicInteger(0);
|
||||
final MetaTableAccessor.Visitor visitor = new MetaTableAccessor.Visitor() {
|
||||
@Override
|
||||
|
|
|
@ -263,7 +263,17 @@ public class HTable implements Table {
|
|||
*/
|
||||
@Override
|
||||
public HTableDescriptor getTableDescriptor() throws IOException {
|
||||
HTableDescriptor htd = HBaseAdmin.getTableDescriptor(tableName, connection, rpcCallerFactory,
|
||||
HTableDescriptor htd = HBaseAdmin.getHTableDescriptor(tableName, connection, rpcCallerFactory,
|
||||
rpcControllerFactory, operationTimeout, readRpcTimeout);
|
||||
if (htd != null) {
|
||||
return new ImmutableHTableDescriptor(htd);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TableDescriptor getDescriptor() throws IOException {
|
||||
HTableDescriptor htd = HBaseAdmin.getHTableDescriptor(tableName, connection, rpcCallerFactory,
|
||||
rpcControllerFactory, operationTimeout, readRpcTimeout);
|
||||
if (htd != null) {
|
||||
return new ImmutableHTableDescriptor(htd);
|
||||
|
|
|
@ -65,9 +65,18 @@ public interface Table extends Closeable {
|
|||
/**
|
||||
* Gets the {@link org.apache.hadoop.hbase.HTableDescriptor table descriptor} for this table.
|
||||
* @throws java.io.IOException if a remote or network exception occurs.
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version.
|
||||
* use {@link #getDescriptor()}
|
||||
*/
|
||||
@Deprecated
|
||||
HTableDescriptor getTableDescriptor() throws IOException;
|
||||
|
||||
/**
|
||||
* Gets the {@link org.apache.hadoop.hbase.client.TableDescriptor table descriptor} for this table.
|
||||
* @throws java.io.IOException if a remote or network exception occurs.
|
||||
*/
|
||||
TableDescriptor getDescriptor() throws IOException;
|
||||
|
||||
/**
|
||||
* Test for the existence of columns in the table, as specified by the Get.
|
||||
* <p>
|
||||
|
@ -604,7 +613,7 @@ public interface Table extends Closeable {
|
|||
/**
|
||||
* Get timeout (millisecond) of each rpc request in this Table instance.
|
||||
*
|
||||
* @returns Currently configured read timeout
|
||||
* @return Currently configured read timeout
|
||||
* @deprecated Use getReadRpcTimeout or getWriteRpcTimeout instead
|
||||
*/
|
||||
@Deprecated
|
||||
|
|
|
@ -207,6 +207,14 @@ public interface TableDescriptor {
|
|||
*/
|
||||
byte[] getValue(byte[] key);
|
||||
|
||||
/**
|
||||
* Getter for accessing the metadata associated with the key.
|
||||
*
|
||||
* @param key The key.
|
||||
* @return Null if no mapping for the key
|
||||
*/
|
||||
String getValue(String key);
|
||||
|
||||
/**
|
||||
* @return Getter for fetching an unmodifiable map.
|
||||
*/
|
||||
|
|
|
@ -500,13 +500,20 @@ public class TableDescriptorBuilder {
|
|||
|
||||
@Override
|
||||
public Bytes getValue(Bytes key) {
|
||||
return values.get(key);
|
||||
Bytes rval = values.get(key);
|
||||
return rval == null ? null : new Bytes(rval.copyBytes());
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getValue(String key) {
|
||||
Bytes rval = values.get(new Bytes(Bytes.toBytes(key)));
|
||||
return rval == null ? null : Bytes.toString(rval.get(), rval.getOffset(), rval.getLength());
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getValue(byte[] key) {
|
||||
Bytes value = values.get(new Bytes(key));
|
||||
return value == null ? null : value.get();
|
||||
return value == null ? null : value.copyBytes();
|
||||
}
|
||||
|
||||
private <T> T getOrDefault(Bytes key, Function<String, T> function, T defaultValue) {
|
||||
|
|
|
@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
|
|||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.io.crypto.Cipher;
|
||||
import org.apache.hadoop.hbase.io.crypto.Encryption;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
|
||||
|
@ -182,7 +183,7 @@ public final class EncryptionUtil {
|
|||
* @throws IOException if an encryption key for the column cannot be unwrapped
|
||||
*/
|
||||
public static Encryption.Context createEncryptionContext(Configuration conf,
|
||||
HColumnDescriptor family) throws IOException {
|
||||
ColumnFamilyDescriptor family) throws IOException {
|
||||
Encryption.Context cryptoContext = Encryption.Context.NONE;
|
||||
String cipherName = family.getEncryptionType();
|
||||
if (cipherName != null) {
|
||||
|
|
|
@ -1305,17 +1305,17 @@ public final class RequestConverter {
|
|||
* Creates a protocol buffer ModifyTableRequest
|
||||
*
|
||||
* @param tableName
|
||||
* @param hTableDesc
|
||||
* @param tableDesc
|
||||
* @return a ModifyTableRequest
|
||||
*/
|
||||
public static ModifyTableRequest buildModifyTableRequest(
|
||||
final TableName tableName,
|
||||
final HTableDescriptor hTableDesc,
|
||||
final TableDescriptor tableDesc,
|
||||
final long nonceGroup,
|
||||
final long nonce) {
|
||||
ModifyTableRequest.Builder builder = ModifyTableRequest.newBuilder();
|
||||
builder.setTableName(ProtobufUtil.toProtoTableName((tableName)));
|
||||
builder.setTableSchema(ProtobufUtil.convertToTableSchema(hTableDesc));
|
||||
builder.setTableSchema(ProtobufUtil.convertToTableSchema(tableDesc));
|
||||
builder.setNonceGroup(nonceGroup);
|
||||
builder.setNonce(nonce);
|
||||
return builder.build();
|
||||
|
|
|
@ -182,7 +182,7 @@ public class ZooKeeperScanPolicyObserver implements RegionObserver {
|
|||
}
|
||||
long ttl = Math.max(EnvironmentEdgeManager.currentTime() -
|
||||
Bytes.toLong(data), oldSI.getTtl());
|
||||
return new ScanInfo(oldSI.getConfiguration(), store.getFamily(), ttl,
|
||||
return new ScanInfo(oldSI.getConfiguration(), store.getColumnFamilyDescriptor(), ttl,
|
||||
oldSI.getTimeToPurgeDeletes(), oldSI.getComparator());
|
||||
}
|
||||
|
||||
|
|
|
@ -54,6 +54,7 @@ import org.apache.hadoop.hbase.client.Row;
|
|||
import org.apache.hadoop.hbase.client.RowMutations;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.coprocessor.Batch;
|
||||
import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
|
||||
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
|
||||
|
@ -508,6 +509,11 @@ public class RemoteHTable implements Table {
|
|||
// no-op
|
||||
}
|
||||
|
||||
@Override
|
||||
public TableDescriptor getDescriptor() throws IOException {
|
||||
return getTableDescriptor();
|
||||
}
|
||||
|
||||
class Scanner implements ResultScanner {
|
||||
|
||||
String uri;
|
||||
|
|
|
@ -126,7 +126,7 @@ public class HFileSplitterJob extends Configured implements Tool {
|
|||
try (Connection conn = ConnectionFactory.createConnection(conf);
|
||||
Table table = conn.getTable(tableName);
|
||||
RegionLocator regionLocator = conn.getRegionLocator(tableName)) {
|
||||
HFileOutputFormat2.configureIncrementalLoad(job, table.getTableDescriptor(), regionLocator);
|
||||
HFileOutputFormat2.configureIncrementalLoad(job, table.getDescriptor(), regionLocator);
|
||||
}
|
||||
LOG.debug("success configuring load incremental job");
|
||||
|
||||
|
|
|
@ -219,6 +219,11 @@ public final class HTableWrapper implements Table {
|
|||
return table.getTableDescriptor();
|
||||
}
|
||||
|
||||
@Override
|
||||
public TableDescriptor getDescriptor() throws IOException {
|
||||
return table.getDescriptor();
|
||||
}
|
||||
|
||||
@Override
|
||||
public TableName getName() {
|
||||
return table.getName();
|
||||
|
|
|
@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.CoprocessorEnvironment;
|
|||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Durability;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
|
||||
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.coprocessor.RegionObserver;
|
||||
|
@ -67,7 +68,7 @@ public class ConstraintProcessor implements RegionObserver {
|
|||
"Constraints only act on regions - started in an environment that was not a region");
|
||||
}
|
||||
RegionCoprocessorEnvironment env = (RegionCoprocessorEnvironment) environment;
|
||||
HTableDescriptor desc = env.getRegion().getTableDesc();
|
||||
TableDescriptor desc = env.getRegion().getTableDescriptor();
|
||||
// load all the constraints from the HTD
|
||||
try {
|
||||
this.constraints = Constraints.getConstraints(desc, classloader);
|
||||
|
|
|
@ -34,6 +34,7 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
|
||||
|
@ -555,7 +556,7 @@ public final class Constraints {
|
|||
* @throws IOException
|
||||
* if any part of reading/arguments fails
|
||||
*/
|
||||
static List<? extends Constraint> getConstraints(HTableDescriptor desc,
|
||||
static List<? extends Constraint> getConstraints(TableDescriptor desc,
|
||||
ClassLoader classloader) throws IOException {
|
||||
List<Constraint> constraints = new ArrayList<>();
|
||||
// loop through all the key, values looking for constraints
|
||||
|
|
|
@ -160,7 +160,7 @@ public class CopyTable extends Configured implements Tool {
|
|||
try (Connection conn = ConnectionFactory.createConnection(getConf());
|
||||
Admin admin = conn.getAdmin()) {
|
||||
HFileOutputFormat2.configureIncrementalLoadMap(job,
|
||||
admin.getTableDescriptor((TableName.valueOf(dstTableName))));
|
||||
admin.listTableDescriptor((TableName.valueOf(dstTableName))));
|
||||
}
|
||||
} else {
|
||||
TableMapReduceUtil.initTableMapperJob(tableName, scan,
|
||||
|
|
|
@ -25,7 +25,6 @@ import java.net.URLEncoder;
|
|||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
@ -44,20 +43,19 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.RegionLocator;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.fs.HFileSystem;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
||||
import org.apache.hadoop.hbase.io.compress.Compression;
|
||||
import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
|
||||
|
@ -67,10 +65,13 @@ import org.apache.hadoop.hbase.io.hfile.HFile;
|
|||
import org.apache.hadoop.hbase.io.hfile.HFileContext;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileWriterImpl;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.regionserver.BloomType;
|
||||
import org.apache.hadoop.hbase.regionserver.HStore;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
|
@ -94,23 +95,33 @@ import com.google.common.annotations.VisibleForTesting;
|
|||
* all HFiles being written.
|
||||
* <p>
|
||||
* Using this class as part of a MapReduce job is best done
|
||||
* using {@link #configureIncrementalLoad(Job, HTableDescriptor, RegionLocator)}.
|
||||
* using {@link #configureIncrementalLoad(Job, TableDescriptor, RegionLocator)}.
|
||||
*/
|
||||
@InterfaceAudience.Public
|
||||
public class HFileOutputFormat2
|
||||
extends FileOutputFormat<ImmutableBytesWritable, Cell> {
|
||||
private static final Log LOG = LogFactory.getLog(HFileOutputFormat2.class);
|
||||
static class TableInfo {
|
||||
private HTableDescriptor hTableDescriptor;
|
||||
private TableDescriptor tableDesctiptor;
|
||||
private RegionLocator regionLocator;
|
||||
|
||||
public TableInfo(HTableDescriptor hTableDescriptor, RegionLocator regionLocator) {
|
||||
this.hTableDescriptor = hTableDescriptor;
|
||||
public TableInfo(TableDescriptor tableDesctiptor, RegionLocator regionLocator) {
|
||||
this.tableDesctiptor = tableDesctiptor;
|
||||
this.regionLocator = regionLocator;
|
||||
}
|
||||
|
||||
/**
|
||||
* The modification for the returned HTD doesn't affect the inner TD.
|
||||
* @return A clone of inner table descriptor
|
||||
* @deprecated use {@link #getTableDescriptor}
|
||||
*/
|
||||
@Deprecated
|
||||
public HTableDescriptor getHTableDescriptor() {
|
||||
return hTableDescriptor;
|
||||
return new HTableDescriptor(tableDesctiptor);
|
||||
}
|
||||
|
||||
public TableDescriptor getTableDescriptor() {
|
||||
return tableDesctiptor;
|
||||
}
|
||||
|
||||
public RegionLocator getRegionLocator() {
|
||||
|
@ -539,7 +550,7 @@ public class HFileOutputFormat2
|
|||
*/
|
||||
public static void configureIncrementalLoad(Job job, Table table, RegionLocator regionLocator)
|
||||
throws IOException {
|
||||
configureIncrementalLoad(job, table.getTableDescriptor(), regionLocator);
|
||||
configureIncrementalLoad(job, table.getDescriptor(), regionLocator);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -556,7 +567,7 @@ public class HFileOutputFormat2
|
|||
* The user should be sure to set the map output value class to either KeyValue or Put before
|
||||
* running this function.
|
||||
*/
|
||||
public static void configureIncrementalLoad(Job job, HTableDescriptor tableDescriptor,
|
||||
public static void configureIncrementalLoad(Job job, TableDescriptor tableDescriptor,
|
||||
RegionLocator regionLocator) throws IOException {
|
||||
ArrayList<TableInfo> singleTableInfo = new ArrayList<>();
|
||||
singleTableInfo.add(new TableInfo(tableDescriptor, regionLocator));
|
||||
|
@ -601,13 +612,13 @@ public class HFileOutputFormat2
|
|||
/* Now get the region start keys for every table required */
|
||||
List<String> allTableNames = new ArrayList<>(multiTableInfo.size());
|
||||
List<RegionLocator> regionLocators = new ArrayList<>( multiTableInfo.size());
|
||||
List<HTableDescriptor> tableDescriptors = new ArrayList<>( multiTableInfo.size());
|
||||
List<TableDescriptor> tableDescriptors = new ArrayList<>( multiTableInfo.size());
|
||||
|
||||
for( TableInfo tableInfo : multiTableInfo )
|
||||
{
|
||||
regionLocators.add(tableInfo.getRegionLocator());
|
||||
allTableNames.add(tableInfo.getRegionLocator().getName().getNameAsString());
|
||||
tableDescriptors.add(tableInfo.getHTableDescriptor());
|
||||
tableDescriptors.add(tableInfo.getTableDescriptor());
|
||||
}
|
||||
// Record tablenames for creating writer by favored nodes, and decoding compression, block size and other attributes of columnfamily per table
|
||||
conf.set(OUTPUT_TABLE_NAME_CONF_KEY, StringUtils.join(allTableNames, Bytes
|
||||
|
@ -635,7 +646,7 @@ public class HFileOutputFormat2
|
|||
LOG.info("Incremental output configured for tables: " + StringUtils.join(allTableNames, ","));
|
||||
}
|
||||
|
||||
public static void configureIncrementalLoadMap(Job job, HTableDescriptor tableDescriptor) throws
|
||||
public static void configureIncrementalLoadMap(Job job, TableDescriptor tableDescriptor) throws
|
||||
IOException {
|
||||
Configuration conf = job.getConfiguration();
|
||||
|
||||
|
@ -643,10 +654,10 @@ public class HFileOutputFormat2
|
|||
job.setOutputValueClass(KeyValue.class);
|
||||
job.setOutputFormatClass(HFileOutputFormat2.class);
|
||||
|
||||
ArrayList<HTableDescriptor> singleTableDescriptor = new ArrayList<>(1);
|
||||
ArrayList<TableDescriptor> singleTableDescriptor = new ArrayList<>(1);
|
||||
singleTableDescriptor.add(tableDescriptor);
|
||||
|
||||
conf.set(OUTPUT_TABLE_NAME_CONF_KEY, tableDescriptor.getNameAsString());
|
||||
conf.set(OUTPUT_TABLE_NAME_CONF_KEY, tableDescriptor.getTableName().getNameAsString());
|
||||
// Set compression algorithms based on column families
|
||||
conf.set(COMPRESSION_FAMILIES_CONF_KEY,
|
||||
serializeColumnFamilyAttribute(compressionDetails, singleTableDescriptor));
|
||||
|
@ -793,18 +804,17 @@ public class HFileOutputFormat2
|
|||
|
||||
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "RCN_REDUNDANT_NULLCHECK_OF_NONNULL_VALUE")
|
||||
@VisibleForTesting
|
||||
static String serializeColumnFamilyAttribute(Function<HColumnDescriptor, String> fn, List<HTableDescriptor> allTables)
|
||||
static String serializeColumnFamilyAttribute(Function<ColumnFamilyDescriptor, String> fn, List<TableDescriptor> allTables)
|
||||
throws UnsupportedEncodingException {
|
||||
StringBuilder attributeValue = new StringBuilder();
|
||||
int i = 0;
|
||||
for (HTableDescriptor tableDescriptor : allTables) {
|
||||
for (TableDescriptor tableDescriptor : allTables) {
|
||||
if (tableDescriptor == null) {
|
||||
// could happen with mock table instance
|
||||
// CODEREVIEW: Can I set an empty string in conf if mock table instance?
|
||||
return "";
|
||||
}
|
||||
Collection<HColumnDescriptor> families = tableDescriptor.getFamilies();
|
||||
for (HColumnDescriptor familyDescriptor : families) {
|
||||
for (ColumnFamilyDescriptor familyDescriptor : tableDescriptor.getColumnFamilies()) {
|
||||
if (i++ > 0) {
|
||||
attributeValue.append('&');
|
||||
}
|
||||
|
@ -829,7 +839,7 @@ public class HFileOutputFormat2
|
|||
* on failure to read column family descriptors
|
||||
*/
|
||||
@VisibleForTesting
|
||||
static Function<HColumnDescriptor, String> compressionDetails = familyDescriptor ->
|
||||
static Function<ColumnFamilyDescriptor, String> compressionDetails = familyDescriptor ->
|
||||
familyDescriptor.getCompressionType().getName();
|
||||
|
||||
/**
|
||||
|
@ -845,7 +855,7 @@ public class HFileOutputFormat2
|
|||
* on failure to read column family descriptors
|
||||
*/
|
||||
@VisibleForTesting
|
||||
static Function<HColumnDescriptor, String> blockSizeDetails = familyDescriptor -> String
|
||||
static Function<ColumnFamilyDescriptor, String> blockSizeDetails = familyDescriptor -> String
|
||||
.valueOf(familyDescriptor.getBlocksize());
|
||||
|
||||
/**
|
||||
|
@ -861,10 +871,10 @@ public class HFileOutputFormat2
|
|||
* on failure to read column family descriptors
|
||||
*/
|
||||
@VisibleForTesting
|
||||
static Function<HColumnDescriptor, String> bloomTypeDetails = familyDescriptor -> {
|
||||
static Function<ColumnFamilyDescriptor, String> bloomTypeDetails = familyDescriptor -> {
|
||||
String bloomType = familyDescriptor.getBloomFilterType().toString();
|
||||
if (bloomType == null) {
|
||||
bloomType = HColumnDescriptor.DEFAULT_BLOOMFILTER;
|
||||
bloomType = ColumnFamilyDescriptorBuilder.DEFAULT_BLOOMFILTER.name();
|
||||
}
|
||||
return bloomType;
|
||||
};
|
||||
|
@ -881,7 +891,7 @@ public class HFileOutputFormat2
|
|||
* on failure to read column family descriptors
|
||||
*/
|
||||
@VisibleForTesting
|
||||
static Function<HColumnDescriptor, String> dataBlockEncodingDetails = familyDescriptor -> {
|
||||
static Function<ColumnFamilyDescriptor, String> dataBlockEncodingDetails = familyDescriptor -> {
|
||||
DataBlockEncoding encoding = familyDescriptor.getDataBlockEncoding();
|
||||
if (encoding == null) {
|
||||
encoding = DataBlockEncoding.NONE;
|
||||
|
|
|
@ -230,7 +230,7 @@ public class Import extends Configured implements Tool {
|
|||
int reduceNum = context.getNumReduceTasks();
|
||||
Configuration conf = context.getConfiguration();
|
||||
TableName tableName = TableName.valueOf(context.getConfiguration().get(TABLE_NAME));
|
||||
try (Connection conn = ConnectionFactory.createConnection(conf);
|
||||
try (Connection conn = ConnectionFactory.createConnection(conf);
|
||||
RegionLocator regionLocator = conn.getRegionLocator(tableName)) {
|
||||
byte[][] startKeys = regionLocator.getStartKeys();
|
||||
if (startKeys.length != reduceNum) {
|
||||
|
@ -622,10 +622,10 @@ public class Import extends Configured implements Tool {
|
|||
|
||||
if (hfileOutPath != null && conf.getBoolean(HAS_LARGE_RESULT, false)) {
|
||||
LOG.info("Use Large Result!!");
|
||||
try (Connection conn = ConnectionFactory.createConnection(conf);
|
||||
try (Connection conn = ConnectionFactory.createConnection(conf);
|
||||
Table table = conn.getTable(tableName);
|
||||
RegionLocator regionLocator = conn.getRegionLocator(tableName)) {
|
||||
HFileOutputFormat2.configureIncrementalLoad(job, table.getTableDescriptor(), regionLocator);
|
||||
HFileOutputFormat2.configureIncrementalLoad(job, table.getDescriptor(), regionLocator);
|
||||
job.setMapperClass(KeyValueSortImporter.class);
|
||||
job.setReducerClass(KeyValueReducer.class);
|
||||
Path outputDir = new Path(hfileOutPath);
|
||||
|
@ -655,7 +655,7 @@ public class Import extends Configured implements Tool {
|
|||
FileOutputFormat.setOutputPath(job, outputDir);
|
||||
job.setMapOutputKeyClass(ImmutableBytesWritable.class);
|
||||
job.setMapOutputValueClass(KeyValue.class);
|
||||
HFileOutputFormat2.configureIncrementalLoad(job, table.getTableDescriptor(), regionLocator);
|
||||
HFileOutputFormat2.configureIncrementalLoad(job, table.getDescriptor(), regionLocator);
|
||||
TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(),
|
||||
com.google.common.base.Preconditions.class);
|
||||
}
|
||||
|
|
|
@ -63,6 +63,8 @@ import org.apache.hadoop.util.ToolRunner;
|
|||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Splitter;
|
||||
import com.google.common.collect.Lists;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
|
||||
/**
|
||||
* Tool to import data from a TSV file.
|
||||
|
@ -521,15 +523,15 @@ public class ImportTsv extends Configured implements Tool {
|
|||
if(!noStrict) {
|
||||
ArrayList<String> unmatchedFamilies = new ArrayList<>();
|
||||
Set<String> cfSet = getColumnFamilies(columns);
|
||||
HTableDescriptor tDesc = table.getTableDescriptor();
|
||||
TableDescriptor tDesc = table.getDescriptor();
|
||||
for (String cf : cfSet) {
|
||||
if(tDesc.getFamily(Bytes.toBytes(cf)) == null) {
|
||||
if(!tDesc.hasColumnFamily(Bytes.toBytes(cf))) {
|
||||
unmatchedFamilies.add(cf);
|
||||
}
|
||||
}
|
||||
if(unmatchedFamilies.size() > 0) {
|
||||
ArrayList<String> familyNames = new ArrayList<>();
|
||||
for (HColumnDescriptor family : table.getTableDescriptor().getFamilies()) {
|
||||
for (ColumnFamilyDescriptor family : table.getDescriptor().getColumnFamilies()) {
|
||||
familyNames.add(family.getNameAsString());
|
||||
}
|
||||
String msg =
|
||||
|
@ -553,7 +555,7 @@ public class ImportTsv extends Configured implements Tool {
|
|||
if (!isDryRun) {
|
||||
Path outputDir = new Path(hfileOutPath);
|
||||
FileOutputFormat.setOutputPath(job, outputDir);
|
||||
HFileOutputFormat2.configureIncrementalLoad(job, table.getTableDescriptor(),
|
||||
HFileOutputFormat2.configureIncrementalLoad(job, table.getDescriptor(),
|
||||
regionLocator);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -70,6 +70,7 @@ import org.apache.hadoop.hbase.TableNotFoundException;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.ClientServiceCallable;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.RegionLocator;
|
||||
|
@ -647,9 +648,9 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
|
|||
*/
|
||||
private void validateFamiliesInHFiles(Table table, Deque<LoadQueueItem> queue, boolean silence)
|
||||
throws IOException {
|
||||
Collection<HColumnDescriptor> families = table.getTableDescriptor().getFamilies();
|
||||
List<String> familyNames = new ArrayList<>(families.size());
|
||||
for (HColumnDescriptor family : families) {
|
||||
ColumnFamilyDescriptor[] families = table.getDescriptor().getColumnFamilies();
|
||||
List<String> familyNames = new ArrayList<>(families.length);
|
||||
for (ColumnFamilyDescriptor family : families) {
|
||||
familyNames.add(family.getNameAsString());
|
||||
}
|
||||
Iterator<LoadQueueItem> queueIter = queue.iterator();
|
||||
|
|
|
@ -21,9 +21,8 @@ package org.apache.hadoop.hbase.mapreduce;
|
|||
import com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.RegionLocator;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.mapreduce.Job;
|
||||
|
@ -84,7 +83,7 @@ public class MultiTableHFileOutputFormat extends HFileOutputFormat2 {
|
|||
|
||||
/**
|
||||
* Analogous to
|
||||
* {@link HFileOutputFormat2#configureIncrementalLoad(Job, HTableDescriptor, RegionLocator)},
|
||||
* {@link HFileOutputFormat2#configureIncrementalLoad(Job, TableDescriptor, RegionLocator)},
|
||||
* this function will configure the requisite number of reducers to write HFiles for multple
|
||||
* tables simultaneously
|
||||
*
|
||||
|
|
|
@ -307,7 +307,7 @@ public class WALPlayer extends Configured implements Tool {
|
|||
try (Connection conn = ConnectionFactory.createConnection(conf);
|
||||
Table table = conn.getTable(tableName);
|
||||
RegionLocator regionLocator = conn.getRegionLocator(tableName)) {
|
||||
HFileOutputFormat2.configureIncrementalLoad(job, table.getTableDescriptor(), regionLocator);
|
||||
HFileOutputFormat2.configureIncrementalLoad(job, table.getDescriptor(), regionLocator);
|
||||
}
|
||||
TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(),
|
||||
com.google.common.base.Preconditions.class);
|
||||
|
|
|
@ -75,7 +75,7 @@ public class DefaultMobStoreCompactor extends DefaultCompactor {
|
|||
public InternalScanner createScanner(List<StoreFileScanner> scanners,
|
||||
ScanType scanType, FileDetails fd, long smallestReadPoint) throws IOException {
|
||||
Scan scan = new Scan();
|
||||
scan.setMaxVersions(store.getFamily().getMaxVersions());
|
||||
scan.setMaxVersions(store.getColumnFamilyDescriptor().getMaxVersions());
|
||||
return new StoreScanner(store, store.getScanInfo(), scan, scanners, scanType,
|
||||
smallestReadPoint, fd.earliestPutTs);
|
||||
}
|
||||
|
@ -103,7 +103,7 @@ public class DefaultMobStoreCompactor extends DefaultCompactor {
|
|||
throw new IllegalArgumentException("The store " + store + " is not a HMobStore");
|
||||
}
|
||||
mobStore = (HMobStore) store;
|
||||
mobSizeThreshold = store.getFamily().getMobThreshold();
|
||||
mobSizeThreshold = store.getColumnFamilyDescriptor().getMobThreshold();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -195,7 +195,7 @@ public class DefaultMobStoreCompactor extends DefaultCompactor {
|
|||
ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build();
|
||||
throughputController.start(compactionName);
|
||||
KeyValueScanner kvs = (scanner instanceof KeyValueScanner)? (KeyValueScanner)scanner : null;
|
||||
long shippedCallSizeLimit = (long) numofFilesToCompact * this.store.getFamily().getBlocksize();
|
||||
long shippedCallSizeLimit = (long) numofFilesToCompact * this.store.getColumnFamilyDescriptor().getBlocksize();
|
||||
try {
|
||||
try {
|
||||
// If the mob file writer could not be created, directly write the cell to the store file.
|
||||
|
|
|
@ -72,7 +72,7 @@ public class DefaultMobStoreFlusher extends DefaultStoreFlusher {
|
|||
|
||||
public DefaultMobStoreFlusher(Configuration conf, Store store) throws IOException {
|
||||
super(conf, store);
|
||||
mobCellValueSizeThreshold = store.getFamily().getMobThreshold();
|
||||
mobCellValueSizeThreshold = store.getColumnFamilyDescriptor().getMobThreshold();
|
||||
this.targetPath = MobUtils.getMobFamilyPath(conf, store.getTableName(),
|
||||
store.getColumnFamilyName());
|
||||
if (!this.store.getFileSystem().exists(targetPath)) {
|
||||
|
@ -115,7 +115,7 @@ public class DefaultMobStoreFlusher extends DefaultStoreFlusher {
|
|||
synchronized (flushLock) {
|
||||
status.setStatus("Flushing " + store + ": creating writer");
|
||||
// Write the map out to the disk
|
||||
writer = store.createWriterInTmp(cellsCount, store.getFamily().getCompressionType(),
|
||||
writer = store.createWriterInTmp(cellsCount, store.getColumnFamilyDescriptor().getCompressionType(),
|
||||
false, true, true, false, snapshot.getTimeRangeTracker());
|
||||
IOException e = null;
|
||||
try {
|
||||
|
@ -173,7 +173,7 @@ public class DefaultMobStoreFlusher extends DefaultStoreFlusher {
|
|||
long mobSize = 0;
|
||||
long time = snapshot.getTimeRangeTracker().getMax();
|
||||
mobFileWriter = mobStore.createWriterInTmp(new Date(time), snapshot.getCellsCount(),
|
||||
store.getFamily().getCompressionType(), store.getRegionInfo().getStartKey(), false);
|
||||
store.getColumnFamilyDescriptor().getCompressionType(), store.getRegionInfo().getStartKey(), false);
|
||||
// the target path is {tableName}/.mob/{cfName}/mobFiles
|
||||
// the relative path is mobFiles
|
||||
byte[] fileName = Bytes.toBytes(mobFileWriter.getPath().getName());
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.mob;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
|
||||
/**
|
||||
|
@ -31,7 +32,7 @@ public class MobCacheConfig extends CacheConfig {
|
|||
|
||||
private static MobFileCache mobFileCache;
|
||||
|
||||
public MobCacheConfig(Configuration conf, HColumnDescriptor family) {
|
||||
public MobCacheConfig(Configuration conf, ColumnFamilyDescriptor family) {
|
||||
super(conf, family);
|
||||
instantiateMobFileCache(conf);
|
||||
}
|
||||
|
|
|
@ -54,6 +54,7 @@ import org.apache.hadoop.hbase.TagType;
|
|||
import org.apache.hadoop.hbase.TagUtil;
|
||||
import org.apache.hadoop.hbase.backup.HFileArchiver;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.MobCompactPartitionPolicy;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.io.HFileLink;
|
||||
|
@ -651,7 +652,7 @@ public final class MobUtils {
|
|||
* @throws IOException
|
||||
*/
|
||||
public static StoreFileWriter createWriter(Configuration conf, FileSystem fs,
|
||||
HColumnDescriptor family, Path path, long maxKeyCount,
|
||||
ColumnFamilyDescriptor family, Path path, long maxKeyCount,
|
||||
Compression.Algorithm compression, CacheConfig cacheConfig, Encryption.Context cryptoContext,
|
||||
ChecksumType checksumType, int bytesPerChecksum, int blocksize, BloomType bloomType,
|
||||
boolean isCompaction)
|
||||
|
|
|
@ -69,7 +69,7 @@ public class ActivePolicyEnforcement {
|
|||
* @see #getPolicyEnforcement(TableName)
|
||||
*/
|
||||
public SpaceViolationPolicyEnforcement getPolicyEnforcement(Region r) {
|
||||
return getPolicyEnforcement(Objects.requireNonNull(r).getTableDesc().getTableName());
|
||||
return getPolicyEnforcement(Objects.requireNonNull(r).getTableDescriptor().getTableName());
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -183,7 +183,7 @@ public class RegionServerRpcQuotaManager {
|
|||
} else {
|
||||
ugi = User.getCurrent().getUGI();
|
||||
}
|
||||
TableName table = region.getTableDesc().getTableName();
|
||||
TableName table = region.getTableDescriptor().getTableName();
|
||||
|
||||
OperationQuota quota = getQuota(ugi, table);
|
||||
try {
|
||||
|
|
|
@ -332,7 +332,7 @@ public class CompactSplit implements CompactionRequestor, PropagatingConfigurati
|
|||
final String why, int priority, CompactionRequest request, boolean selectNow, User user)
|
||||
throws IOException {
|
||||
if (this.server.isStopped()
|
||||
|| (r.getTableDesc() != null && !r.getTableDesc().isCompactionEnabled())) {
|
||||
|| (r.getTableDescriptor() != null && !r.getTableDescriptor().isCompactionEnabled())) {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
@ -345,7 +345,7 @@ public class CompactSplit implements CompactionRequestor, PropagatingConfigurati
|
|||
final RegionServerSpaceQuotaManager spaceQuotaManager =
|
||||
this.server.getRegionServerSpaceQuotaManager();
|
||||
if (spaceQuotaManager != null && spaceQuotaManager.areCompactionsDisabled(
|
||||
r.getTableDesc().getTableName())) {
|
||||
r.getTableDescriptor().getTableName())) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Ignoring compaction request for " + r + " as an active space quota violation "
|
||||
+ " policy disallows compactions.");
|
||||
|
@ -562,7 +562,7 @@ public class CompactSplit implements CompactionRequestor, PropagatingConfigurati
|
|||
public void run() {
|
||||
Preconditions.checkNotNull(server);
|
||||
if (server.isStopped()
|
||||
|| (region.getTableDesc() != null && !region.getTableDesc().isCompactionEnabled())) {
|
||||
|| (region.getTableDescriptor() != null && !region.getTableDescriptor().isCompactionEnabled())) {
|
||||
return;
|
||||
}
|
||||
doCompaction(user);
|
||||
|
|
|
@ -408,7 +408,7 @@ public class CompactingMemStore extends AbstractMemStore {
|
|||
}
|
||||
|
||||
private byte[] getFamilyNameInBytes() {
|
||||
return store.getFamily().getName();
|
||||
return store.getColumnFamilyDescriptor().getName();
|
||||
}
|
||||
|
||||
private ThreadPoolExecutor getPool() {
|
||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
|||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
|
||||
/**
|
||||
* A {@link RegionSplitPolicy} implementation which splits a region
|
||||
|
@ -47,7 +48,7 @@ public class ConstantSizeRegionSplitPolicy extends RegionSplitPolicy {
|
|||
protected void configureForRegion(HRegion region) {
|
||||
super.configureForRegion(region);
|
||||
Configuration conf = getConf();
|
||||
HTableDescriptor desc = region.getTableDesc();
|
||||
TableDescriptor desc = region.getTableDescriptor();
|
||||
if (desc != null) {
|
||||
this.desiredMaxFileSize = desc.getMaxFileSize();
|
||||
}
|
||||
|
|
|
@ -64,7 +64,7 @@ public class DefaultStoreFlusher extends StoreFlusher {
|
|||
synchronized (flushLock) {
|
||||
status.setStatus("Flushing " + store + ": creating writer");
|
||||
// Write the map out to the disk
|
||||
writer = store.createWriterInTmp(cellsCount, store.getFamily().getCompressionType(),
|
||||
writer = store.createWriterInTmp(cellsCount, store.getColumnFamilyDescriptor().getCompressionType(),
|
||||
/* isCompaction = */ false,
|
||||
/* includeMVCCReadpoint = */ true,
|
||||
/* includesTags = */ snapshot.isTagsPresent(),
|
||||
|
|
|
@ -51,9 +51,9 @@ public class DelimitedKeyPrefixRegionSplitPolicy extends IncreasingToUpperBoundR
|
|||
protected void configureForRegion(HRegion region) {
|
||||
super.configureForRegion(region);
|
||||
// read the prefix length from the table descriptor
|
||||
String delimiterString = region.getTableDesc().getValue(DELIMITER_KEY);
|
||||
String delimiterString = region.getTableDescriptor().getValue(DELIMITER_KEY);
|
||||
if (delimiterString == null || delimiterString.length() == 0) {
|
||||
LOG.error(DELIMITER_KEY + " not specified for table " + region.getTableDesc().getTableName() +
|
||||
LOG.error(DELIMITER_KEY + " not specified for table " + region.getTableDescriptor().getTableName() +
|
||||
". Using default RegionSplitPolicy");
|
||||
return;
|
||||
}
|
||||
|
|
|
@ -38,7 +38,7 @@ public class FlushAllLargeStoresPolicy extends FlushLargeStoresPolicy{
|
|||
@Override
|
||||
protected void configureForRegion(HRegion region) {
|
||||
super.configureForRegion(region);
|
||||
int familyNumber = region.getTableDesc().getFamilies().size();
|
||||
int familyNumber = region.getTableDescriptor().getColumnFamilyCount();
|
||||
if (familyNumber <= 1) {
|
||||
// No need to parse and set flush size lower bound if only one family
|
||||
// Family number might also be zero in some of our unit test case
|
||||
|
@ -50,7 +50,7 @@ public class FlushAllLargeStoresPolicy extends FlushLargeStoresPolicy{
|
|||
@Override
|
||||
public Collection<Store> selectStoresToFlush() {
|
||||
// no need to select stores if only one family
|
||||
if (region.getTableDesc().getFamilies().size() == 1) {
|
||||
if (region.getTableDescriptor().getColumnFamilyCount() == 1) {
|
||||
return region.stores.values();
|
||||
}
|
||||
// start selection
|
||||
|
|
|
@ -43,7 +43,7 @@ public abstract class FlushLargeStoresPolicy extends FlushPolicy {
|
|||
protected long flushSizeLowerBound = -1;
|
||||
|
||||
protected long getFlushSizeLowerBound(HRegion region) {
|
||||
int familyNumber = region.getTableDesc().getFamilies().size();
|
||||
int familyNumber = region.getTableDescriptor().getColumnFamilyCount();
|
||||
// For multiple families, lower bound is the "average flush size" by default
|
||||
// unless setting in configuration is larger.
|
||||
long flushSizeLowerBound = region.getMemstoreFlushSize() / familyNumber;
|
||||
|
@ -55,11 +55,11 @@ public abstract class FlushLargeStoresPolicy extends FlushPolicy {
|
|||
}
|
||||
// use the setting in table description if any
|
||||
String flushedSizeLowerBoundString =
|
||||
region.getTableDesc().getValue(HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND);
|
||||
region.getTableDescriptor().getValue(HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND);
|
||||
if (flushedSizeLowerBoundString == null) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("No " + HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND
|
||||
+ " set in description of table " + region.getTableDesc().getTableName()
|
||||
+ " set in description of table " + region.getTableDescriptor().getTableName()
|
||||
+ ", use config (" + flushSizeLowerBound + ") instead");
|
||||
}
|
||||
} else {
|
||||
|
@ -69,7 +69,7 @@ public abstract class FlushLargeStoresPolicy extends FlushPolicy {
|
|||
// fall back for fault setting
|
||||
LOG.warn("Number format exception when parsing "
|
||||
+ HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND + " for table "
|
||||
+ region.getTableDesc().getTableName() + ":" + flushedSizeLowerBoundString + ". " + nfe
|
||||
+ region.getTableDescriptor().getTableName() + ":" + flushedSizeLowerBoundString + ". " + nfe
|
||||
+ ", use config (" + flushSizeLowerBound + ") instead");
|
||||
|
||||
}
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.util.ReflectionUtils;
|
||||
|
||||
/**
|
||||
|
@ -47,7 +48,7 @@ public class FlushPolicyFactory {
|
|||
* Create the FlushPolicy configured for the given table.
|
||||
*/
|
||||
public static FlushPolicy create(HRegion region, Configuration conf) throws IOException {
|
||||
Class<? extends FlushPolicy> clazz = getFlushPolicyClass(region.getTableDesc(), conf);
|
||||
Class<? extends FlushPolicy> clazz = getFlushPolicyClass(region.getTableDescriptor(), conf);
|
||||
FlushPolicy policy = ReflectionUtils.newInstance(clazz, conf);
|
||||
policy.configureForRegion(region);
|
||||
return policy;
|
||||
|
@ -56,7 +57,7 @@ public class FlushPolicyFactory {
|
|||
/**
|
||||
* Get FlushPolicy class for the given table.
|
||||
*/
|
||||
public static Class<? extends FlushPolicy> getFlushPolicyClass(HTableDescriptor htd,
|
||||
public static Class<? extends FlushPolicy> getFlushPolicyClass(TableDescriptor htd,
|
||||
Configuration conf) throws IOException {
|
||||
String className = htd.getFlushPolicyClassName();
|
||||
if (className == null) {
|
||||
|
|
|
@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.Tag;
|
|||
import org.apache.hadoop.hbase.TagType;
|
||||
import org.apache.hadoop.hbase.TagUtil;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.filter.Filter;
|
||||
import org.apache.hadoop.hbase.filter.FilterList;
|
||||
|
@ -91,7 +92,7 @@ public class HMobStore extends HStore {
|
|||
private volatile long mobFlushedCellsSize = 0;
|
||||
private volatile long mobScanCellsCount = 0;
|
||||
private volatile long mobScanCellsSize = 0;
|
||||
private HColumnDescriptor family;
|
||||
private ColumnFamilyDescriptor family;
|
||||
private Map<String, List<Path>> map = new ConcurrentHashMap<>();
|
||||
private final IdLock keyLock = new IdLock();
|
||||
// When we add a MOB reference cell to the HFile, we will add 2 tags along with it
|
||||
|
@ -102,7 +103,7 @@ public class HMobStore extends HStore {
|
|||
// cloning snapshot for mob files.
|
||||
private final byte[] refCellTags;
|
||||
|
||||
public HMobStore(final HRegion region, final HColumnDescriptor family,
|
||||
public HMobStore(final HRegion region, final ColumnFamilyDescriptor family,
|
||||
final Configuration confParam) throws IOException {
|
||||
super(region, family, confParam);
|
||||
this.family = family;
|
||||
|
@ -112,7 +113,7 @@ public class HMobStore extends HStore {
|
|||
family.getNameAsString());
|
||||
List<Path> locations = new ArrayList<>(2);
|
||||
locations.add(mobFamilyPath);
|
||||
TableName tn = region.getTableDesc().getTableName();
|
||||
TableName tn = region.getTableDescriptor().getTableName();
|
||||
locations.add(HFileArchiveUtil.getStoreArchivePath(conf, tn, MobUtils.getMobRegionInfo(tn)
|
||||
.getEncodedName(), family.getNameAsString()));
|
||||
map.put(Bytes.toString(tn.getName()), locations);
|
||||
|
@ -128,7 +129,7 @@ public class HMobStore extends HStore {
|
|||
* Creates the mob cache config.
|
||||
*/
|
||||
@Override
|
||||
protected void createCacheConf(HColumnDescriptor family) {
|
||||
protected void createCacheConf(ColumnFamilyDescriptor family) {
|
||||
cacheConf = new MobCacheConfig(conf, family);
|
||||
}
|
||||
|
||||
|
|
|
@ -88,12 +88,10 @@ import org.apache.hadoop.hbase.CellUtil;
|
|||
import org.apache.hadoop.hbase.CompoundConfiguration;
|
||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
import org.apache.hadoop.hbase.DroppedSnapshotException;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
|
||||
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||
|
@ -105,6 +103,7 @@ import org.apache.hadoop.hbase.TagUtil;
|
|||
import org.apache.hadoop.hbase.UnknownScannerException;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Append;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.Durability;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
|
@ -117,6 +116,8 @@ import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
|||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.RowMutations;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.conf.ConfigurationManager;
|
||||
import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver;
|
||||
import org.apache.hadoop.hbase.coprocessor.RegionObserver.MutationType;
|
||||
|
@ -635,7 +636,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
// Coprocessor host
|
||||
private RegionCoprocessorHost coprocessorHost;
|
||||
|
||||
private HTableDescriptor htableDescriptor = null;
|
||||
private TableDescriptor htableDescriptor = null;
|
||||
private RegionSplitPolicy splitPolicy;
|
||||
private FlushPolicy flushPolicy;
|
||||
|
||||
|
@ -675,7 +676,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
@VisibleForTesting
|
||||
public HRegion(final Path tableDir, final WAL wal, final FileSystem fs,
|
||||
final Configuration confParam, final HRegionInfo regionInfo,
|
||||
final HTableDescriptor htd, final RegionServerServices rsServices) {
|
||||
final TableDescriptor htd, final RegionServerServices rsServices) {
|
||||
this(new HRegionFileSystem(confParam, fs, tableDir, regionInfo),
|
||||
wal, confParam, htd, rsServices);
|
||||
}
|
||||
|
@ -697,7 +698,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
* @param rsServices reference to {@link RegionServerServices} or null
|
||||
*/
|
||||
public HRegion(final HRegionFileSystem fs, final WAL wal, final Configuration confParam,
|
||||
final HTableDescriptor htd, final RegionServerServices rsServices) {
|
||||
final TableDescriptor htd, final RegionServerServices rsServices) {
|
||||
if (htd == null) {
|
||||
throw new IllegalArgumentException("Need table descriptor");
|
||||
}
|
||||
|
@ -727,10 +728,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
|
||||
this.isLoadingCfsOnDemandDefault = conf.getBoolean(LOAD_CFS_ON_DEMAND_CONFIG_KEY, true);
|
||||
this.htableDescriptor = htd;
|
||||
Set<byte[]> families = this.htableDescriptor.getFamiliesKeys();
|
||||
Set<byte[]> families = this.htableDescriptor.getColumnFamilyNames();
|
||||
for (byte[] family : families) {
|
||||
if (!replicationScope.containsKey(family)) {
|
||||
int scope = htd.getFamily(family).getScope();
|
||||
int scope = htd.getColumnFamily(family).getScope();
|
||||
// Only store those families that has NON-DEFAULT scope
|
||||
if (scope != REPLICATION_SCOPE_LOCAL) {
|
||||
// Do a copy before storing it here.
|
||||
|
@ -826,7 +827,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
|
||||
if (flushSize <= 0) {
|
||||
flushSize = conf.getLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE,
|
||||
HTableDescriptor.DEFAULT_MEMSTORE_FLUSH_SIZE);
|
||||
TableDescriptorBuilder.DEFAULT_MEMSTORE_FLUSH_SIZE);
|
||||
}
|
||||
this.memstoreFlushSize = flushSize;
|
||||
this.blockingMemStoreSize = this.memstoreFlushSize *
|
||||
|
@ -858,7 +859,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
|
||||
//Refuse to open the region if there is no column family in the table
|
||||
if (htableDescriptor.getColumnFamilyCount() == 0) {
|
||||
throw new DoNotRetryIOException("Table " + htableDescriptor.getNameAsString() +
|
||||
throw new DoNotRetryIOException("Table " + htableDescriptor.getTableName().getNameAsString()+
|
||||
" should have at least one column family.");
|
||||
}
|
||||
|
||||
|
@ -987,14 +988,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
// initialized to -1 so that we pick up MemstoreTS from column families
|
||||
long maxMemstoreTS = -1;
|
||||
|
||||
if (!htableDescriptor.getFamilies().isEmpty()) {
|
||||
if (htableDescriptor.getColumnFamilyCount() != 0) {
|
||||
// initialize the thread pool for opening stores in parallel.
|
||||
ThreadPoolExecutor storeOpenerThreadPool =
|
||||
getStoreOpenAndCloseThreadPool("StoreOpener-" + this.getRegionInfo().getShortNameToLog());
|
||||
CompletionService<HStore> completionService = new ExecutorCompletionService<>(storeOpenerThreadPool);
|
||||
|
||||
// initialize each store in parallel
|
||||
for (final HColumnDescriptor family : htableDescriptor.getFamilies()) {
|
||||
for (final ColumnFamilyDescriptor family : htableDescriptor.getColumnFamilies()) {
|
||||
status.setStatus("Instantiating store for column family " + family);
|
||||
completionService.submit(new Callable<HStore>() {
|
||||
@Override
|
||||
|
@ -1006,10 +1007,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
boolean allStoresOpened = false;
|
||||
boolean hasSloppyStores = false;
|
||||
try {
|
||||
for (int i = 0; i < htableDescriptor.getFamilies().size(); i++) {
|
||||
for (int i = 0; i < htableDescriptor.getColumnFamilyCount(); i++) {
|
||||
Future<HStore> future = completionService.take();
|
||||
HStore store = future.get();
|
||||
this.stores.put(store.getFamily().getName(), store);
|
||||
this.stores.put(store.getColumnFamilyDescriptor().getName(), store);
|
||||
if (store.isSloppyMemstore()) {
|
||||
hasSloppyStores = true;
|
||||
}
|
||||
|
@ -1027,8 +1028,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
}
|
||||
allStoresOpened = true;
|
||||
if(hasSloppyStores) {
|
||||
htableDescriptor.setFlushPolicyClassName(FlushNonSloppyStoresFirstPolicy.class
|
||||
.getName());
|
||||
htableDescriptor = TableDescriptorBuilder.newBuilder(htableDescriptor)
|
||||
.setFlushPolicyClassName(FlushNonSloppyStoresFirstPolicy.class.getName())
|
||||
.build();
|
||||
LOG.info("Setting FlushNonSloppyStoresFirstPolicy for the region=" + this);
|
||||
}
|
||||
} catch (InterruptedException e) {
|
||||
|
@ -1076,7 +1078,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
for (StoreFile storeFile: storeFiles) {
|
||||
storeFileNames.add(storeFile.getPath());
|
||||
}
|
||||
allStoreFiles.put(store.getFamily().getName(), storeFileNames);
|
||||
allStoreFiles.put(store.getColumnFamilyDescriptor().getName(), storeFileNames);
|
||||
}
|
||||
return allStoreFiles;
|
||||
}
|
||||
|
@ -1146,13 +1148,13 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
/**
|
||||
* This is a helper function to compute HDFS block distribution on demand
|
||||
* @param conf configuration
|
||||
* @param tableDescriptor HTableDescriptor of the table
|
||||
* @param tableDescriptor TableDescriptor of the table
|
||||
* @param regionInfo encoded name of the region
|
||||
* @return The HDFS blocks distribution for the given region.
|
||||
* @throws IOException
|
||||
*/
|
||||
public static HDFSBlocksDistribution computeHDFSBlocksDistribution(final Configuration conf,
|
||||
final HTableDescriptor tableDescriptor, final HRegionInfo regionInfo) throws IOException {
|
||||
final TableDescriptor tableDescriptor, final HRegionInfo regionInfo) throws IOException {
|
||||
Path tablePath = FSUtils.getTableDir(FSUtils.getRootDir(conf), tableDescriptor.getTableName());
|
||||
return computeHDFSBlocksDistribution(conf, tableDescriptor, regionInfo, tablePath);
|
||||
}
|
||||
|
@ -1160,20 +1162,20 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
/**
|
||||
* This is a helper function to compute HDFS block distribution on demand
|
||||
* @param conf configuration
|
||||
* @param tableDescriptor HTableDescriptor of the table
|
||||
* @param tableDescriptor TableDescriptor of the table
|
||||
* @param regionInfo encoded name of the region
|
||||
* @param tablePath the table directory
|
||||
* @return The HDFS blocks distribution for the given region.
|
||||
* @throws IOException
|
||||
*/
|
||||
public static HDFSBlocksDistribution computeHDFSBlocksDistribution(final Configuration conf,
|
||||
final HTableDescriptor tableDescriptor, final HRegionInfo regionInfo, Path tablePath)
|
||||
final TableDescriptor tableDescriptor, final HRegionInfo regionInfo, Path tablePath)
|
||||
throws IOException {
|
||||
HDFSBlocksDistribution hdfsBlocksDistribution = new HDFSBlocksDistribution();
|
||||
FileSystem fs = tablePath.getFileSystem(conf);
|
||||
|
||||
HRegionFileSystem regionFs = new HRegionFileSystem(conf, fs, tablePath, regionInfo);
|
||||
for (HColumnDescriptor family : tableDescriptor.getFamilies()) {
|
||||
for (ColumnFamilyDescriptor family : tableDescriptor.getColumnFamilies()) {
|
||||
List<LocatedFileStatus> locatedFileStatusList = HRegionFileSystem
|
||||
.getStoreFilesLocatedStatus(regionFs, family.getNameAsString(), true);
|
||||
if (locatedFileStatusList == null) {
|
||||
|
@ -1338,7 +1340,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
&& wasRecovering && !newState) {
|
||||
|
||||
// force a flush only if region replication is set up for this region. Otherwise no need.
|
||||
boolean forceFlush = getTableDesc().getRegionReplication() > 1;
|
||||
boolean forceFlush = getTableDescriptor().getRegionReplication() > 1;
|
||||
|
||||
MonitoredTask status = TaskMonitor.get().createStatus("Recovering region " + this);
|
||||
|
||||
|
@ -1672,7 +1674,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
.submit(new Callable<Pair<byte[], Collection<StoreFile>>>() {
|
||||
@Override
|
||||
public Pair<byte[], Collection<StoreFile>> call() throws IOException {
|
||||
return new Pair<>(store.getFamily().getName(), store.close());
|
||||
return new Pair<>(store.getColumnFamilyDescriptor().getName(), store.close());
|
||||
}
|
||||
});
|
||||
}
|
||||
|
@ -1799,7 +1801,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
}
|
||||
protected ThreadPoolExecutor getStoreOpenAndCloseThreadPool(
|
||||
final String threadNamePrefix) {
|
||||
int numStores = Math.max(1, this.htableDescriptor.getFamilies().size());
|
||||
int numStores = Math.max(1, this.htableDescriptor.getColumnFamilyCount());
|
||||
int maxThreads = Math.min(numStores,
|
||||
conf.getInt(HConstants.HSTORE_OPEN_AND_CLOSE_THREADS_MAX,
|
||||
HConstants.DEFAULT_HSTORE_OPEN_AND_CLOSE_THREADS_MAX));
|
||||
|
@ -1808,7 +1810,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
|
||||
protected ThreadPoolExecutor getStoreFileOpenAndCloseThreadPool(
|
||||
final String threadNamePrefix) {
|
||||
int numStores = Math.max(1, this.htableDescriptor.getFamilies().size());
|
||||
int numStores = Math.max(1, this.htableDescriptor.getColumnFamilyCount());
|
||||
int maxThreads = Math.max(1,
|
||||
conf.getInt(HConstants.HSTORE_OPEN_AND_CLOSE_THREADS_MAX,
|
||||
HConstants.DEFAULT_HSTORE_OPEN_AND_CLOSE_THREADS_MAX)
|
||||
|
@ -1842,10 +1844,15 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
//////////////////////////////////////////////////////////////////////////////
|
||||
|
||||
@Override
|
||||
public HTableDescriptor getTableDesc() {
|
||||
public TableDescriptor getTableDescriptor() {
|
||||
return this.htableDescriptor;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
void setTableDescriptor(TableDescriptor desc) {
|
||||
htableDescriptor = desc;
|
||||
}
|
||||
|
||||
/** @return WAL in use for this region */
|
||||
public WAL getWAL() {
|
||||
return this.wal;
|
||||
|
@ -2280,7 +2287,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
*/
|
||||
boolean shouldFlushStore(Store store) {
|
||||
long earliest = this.wal.getEarliestMemstoreSeqNum(getRegionInfo().getEncodedNameAsBytes(),
|
||||
store.getFamily().getName()) - 1;
|
||||
store.getColumnFamilyDescriptor().getName()) - 1;
|
||||
if (earliest > 0 && earliest + flushPerChanges < mvcc.getReadPoint()) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Flush column family " + store.getColumnFamilyName() + " of " +
|
||||
|
@ -2461,7 +2468,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
|
||||
Map<byte[], Long> flushedFamilyNamesToSeq = new HashMap<>();
|
||||
for (Store store: storesToFlush) {
|
||||
flushedFamilyNamesToSeq.put(store.getFamily().getName(),
|
||||
flushedFamilyNamesToSeq.put(store.getColumnFamilyDescriptor().getName(),
|
||||
((HStore) store).preFlushSeqIDEstimation());
|
||||
}
|
||||
|
||||
|
@ -2501,9 +2508,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
for (Store s : storesToFlush) {
|
||||
MemstoreSize flushableSize = s.getSizeToFlush();
|
||||
totalSizeOfFlushableStores.incMemstoreSize(flushableSize);
|
||||
storeFlushCtxs.put(s.getFamily().getName(), s.createFlushContext(flushOpSeqId));
|
||||
committedFiles.put(s.getFamily().getName(), null); // for writing stores to WAL
|
||||
storeFlushableSize.put(s.getFamily().getName(), flushableSize);
|
||||
storeFlushCtxs.put(s.getColumnFamilyDescriptor().getName(), s.createFlushContext(flushOpSeqId));
|
||||
committedFiles.put(s.getColumnFamilyDescriptor().getName(), null); // for writing stores to WAL
|
||||
storeFlushableSize.put(s.getColumnFamilyDescriptor().getName(), flushableSize);
|
||||
}
|
||||
|
||||
// write the snapshot start to WAL
|
||||
|
@ -2661,7 +2668,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
if (needsCompaction) {
|
||||
compactionRequested = true;
|
||||
}
|
||||
byte[] storeName = it.next().getFamily().getName();
|
||||
byte[] storeName = it.next().getColumnFamilyDescriptor().getName();
|
||||
List<Path> storeCommittedFiles = flush.getCommittedFiles();
|
||||
committedFiles.put(storeName, storeCommittedFiles);
|
||||
// Flush committed no files, indicating flush is empty or flush was canceled
|
||||
|
@ -2796,7 +2803,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
// Verify families are all valid
|
||||
if (!scan.hasFamilies()) {
|
||||
// Adding all families to scanner
|
||||
for (byte[] family : this.htableDescriptor.getFamiliesKeys()) {
|
||||
for (byte[] family : this.htableDescriptor.getColumnFamilyNames()) {
|
||||
scan.addFamily(family);
|
||||
}
|
||||
} else {
|
||||
|
@ -2831,7 +2838,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
public void prepareDelete(Delete delete) throws IOException {
|
||||
// Check to see if this is a deleteRow insert
|
||||
if(delete.getFamilyCellMap().isEmpty()){
|
||||
for(byte [] family : this.htableDescriptor.getFamiliesKeys()){
|
||||
for(byte [] family : this.htableDescriptor.getColumnFamilyNames()){
|
||||
// Don't eat the timestamp
|
||||
delete.addFamily(family, delete.getTimeStamp());
|
||||
}
|
||||
|
@ -3643,7 +3650,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
private void removeNonExistentColumnFamilyForReplay(final Map<byte[], List<Cell>> familyMap) {
|
||||
List<byte[]> nonExistentList = null;
|
||||
for (byte[] family : familyMap.keySet()) {
|
||||
if (!this.htableDescriptor.hasFamily(family)) {
|
||||
if (!this.htableDescriptor.hasColumnFamily(family)) {
|
||||
if (nonExistentList == null) {
|
||||
nonExistentList = new ArrayList<>();
|
||||
}
|
||||
|
@ -3997,7 +4004,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
private void applyToMemstore(final Store store, final List<Cell> cells, final boolean delta,
|
||||
MemstoreSize memstoreSize) throws IOException {
|
||||
// Any change in how we update Store/MemStore needs to also be done in other applyToMemstore!!!!
|
||||
boolean upsert = delta && store.getFamily().getMaxVersions() == 1;
|
||||
boolean upsert = delta && store.getColumnFamilyDescriptor().getMaxVersions() == 1;
|
||||
if (upsert) {
|
||||
((HStore) store).upsert(cells, getSmallestReadPoint(), memstoreSize);
|
||||
} else {
|
||||
|
@ -4352,7 +4359,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
continue;
|
||||
}
|
||||
// Figure which store the edit is meant for.
|
||||
if (store == null || !CellUtil.matchingFamily(cell, store.getFamily().getName())) {
|
||||
if (store == null || !CellUtil.matchingFamily(cell,
|
||||
store.getColumnFamilyDescriptor().getName())) {
|
||||
store = getHStore(cell);
|
||||
}
|
||||
if (store == null) {
|
||||
|
@ -4369,7 +4377,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
continue;
|
||||
}
|
||||
// Now, figure if we should skip this edit.
|
||||
if (key.getLogSeqNum() <= maxSeqIdInStores.get(store.getFamily()
|
||||
if (key.getLogSeqNum() <= maxSeqIdInStores.get(store.getColumnFamilyDescriptor()
|
||||
.getName())) {
|
||||
skippedEdits++;
|
||||
continue;
|
||||
|
@ -5170,12 +5178,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
// only drop memstore snapshots if they are smaller than last flush for the store
|
||||
if (this.prepareFlushResult.flushOpSeqId <= storeSeqId) {
|
||||
StoreFlushContext ctx = this.prepareFlushResult.storeFlushCtxs == null ?
|
||||
null : this.prepareFlushResult.storeFlushCtxs.get(store.getFamily().getName());
|
||||
null : this.prepareFlushResult.storeFlushCtxs.get(
|
||||
store.getColumnFamilyDescriptor().getName());
|
||||
if (ctx != null) {
|
||||
MemstoreSize snapshotSize = store.getSizeToFlush();
|
||||
ctx.abort();
|
||||
this.decrMemstoreSize(snapshotSize);
|
||||
this.prepareFlushResult.storeFlushCtxs.remove(store.getFamily().getName());
|
||||
this.prepareFlushResult.storeFlushCtxs.remove(
|
||||
store.getColumnFamilyDescriptor().getName());
|
||||
totalFreedDataSize += snapshotSize.getDataSize();
|
||||
}
|
||||
}
|
||||
|
@ -5280,7 +5290,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
return true;
|
||||
}
|
||||
|
||||
protected HStore instantiateHStore(final HColumnDescriptor family) throws IOException {
|
||||
protected HStore instantiateHStore(final ColumnFamilyDescriptor family) throws IOException {
|
||||
if (family.isMobEnabled()) {
|
||||
if (HFile.getFormatVersion(this.conf) < HFile.MIN_FORMAT_VERSION_WITH_TAGS) {
|
||||
throw new IOException("A minimum HFile version of "
|
||||
|
@ -6500,7 +6510,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
* @return the new instance
|
||||
*/
|
||||
static HRegion newHRegion(Path tableDir, WAL wal, FileSystem fs,
|
||||
Configuration conf, HRegionInfo regionInfo, final HTableDescriptor htd,
|
||||
Configuration conf, HRegionInfo regionInfo, final TableDescriptor htd,
|
||||
RegionServerServices rsServices) {
|
||||
try {
|
||||
@SuppressWarnings("unchecked")
|
||||
|
@ -6509,7 +6519,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
|
||||
Constructor<? extends HRegion> c =
|
||||
regionClass.getConstructor(Path.class, WAL.class, FileSystem.class,
|
||||
Configuration.class, HRegionInfo.class, HTableDescriptor.class,
|
||||
Configuration.class, HRegionInfo.class, TableDescriptor.class,
|
||||
RegionServerServices.class);
|
||||
|
||||
return c.newInstance(tableDir, wal, fs, conf, regionInfo, htd, rsServices);
|
||||
|
@ -6530,7 +6540,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
* @throws IOException
|
||||
*/
|
||||
public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
|
||||
final Configuration conf, final HTableDescriptor hTableDescriptor,
|
||||
final Configuration conf, final TableDescriptor hTableDescriptor,
|
||||
final WAL wal, final boolean initialize)
|
||||
throws IOException {
|
||||
LOG.info("creating HRegion " + info.getTable().getNameAsString()
|
||||
|
@ -6546,7 +6556,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
|
||||
public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
|
||||
final Configuration conf,
|
||||
final HTableDescriptor hTableDescriptor,
|
||||
final TableDescriptor hTableDescriptor,
|
||||
final WAL wal)
|
||||
throws IOException {
|
||||
return createHRegion(info, rootDir, conf, hTableDescriptor, wal, true);
|
||||
|
@ -6565,7 +6575,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
* @throws IOException
|
||||
*/
|
||||
public static HRegion openHRegion(final HRegionInfo info,
|
||||
final HTableDescriptor htd, final WAL wal,
|
||||
final TableDescriptor htd, final WAL wal,
|
||||
final Configuration conf)
|
||||
throws IOException {
|
||||
return openHRegion(info, htd, wal, conf, null, null);
|
||||
|
@ -6587,7 +6597,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
* @throws IOException
|
||||
*/
|
||||
public static HRegion openHRegion(final HRegionInfo info,
|
||||
final HTableDescriptor htd, final WAL wal, final Configuration conf,
|
||||
final TableDescriptor htd, final WAL wal, final Configuration conf,
|
||||
final RegionServerServices rsServices,
|
||||
final CancelableProgressable reporter)
|
||||
throws IOException {
|
||||
|
@ -6608,7 +6618,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
* @throws IOException
|
||||
*/
|
||||
public static HRegion openHRegion(Path rootDir, final HRegionInfo info,
|
||||
final HTableDescriptor htd, final WAL wal, final Configuration conf)
|
||||
final TableDescriptor htd, final WAL wal, final Configuration conf)
|
||||
throws IOException {
|
||||
return openHRegion(rootDir, info, htd, wal, conf, null, null);
|
||||
}
|
||||
|
@ -6629,7 +6639,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
* @throws IOException
|
||||
*/
|
||||
public static HRegion openHRegion(final Path rootDir, final HRegionInfo info,
|
||||
final HTableDescriptor htd, final WAL wal, final Configuration conf,
|
||||
final TableDescriptor htd, final WAL wal, final Configuration conf,
|
||||
final RegionServerServices rsServices,
|
||||
final CancelableProgressable reporter)
|
||||
throws IOException {
|
||||
|
@ -6658,7 +6668,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
* @throws IOException
|
||||
*/
|
||||
public static HRegion openHRegion(final Configuration conf, final FileSystem fs,
|
||||
final Path rootDir, final HRegionInfo info, final HTableDescriptor htd, final WAL wal)
|
||||
final Path rootDir, final HRegionInfo info, final TableDescriptor htd, final WAL wal)
|
||||
throws IOException {
|
||||
return openHRegion(conf, fs, rootDir, info, htd, wal, null, null);
|
||||
}
|
||||
|
@ -6680,7 +6690,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
* @throws IOException
|
||||
*/
|
||||
public static HRegion openHRegion(final Configuration conf, final FileSystem fs,
|
||||
final Path rootDir, final HRegionInfo info, final HTableDescriptor htd, final WAL wal,
|
||||
final Path rootDir, final HRegionInfo info, final TableDescriptor htd, final WAL wal,
|
||||
final RegionServerServices rsServices, final CancelableProgressable reporter)
|
||||
throws IOException {
|
||||
Path tableDir = FSUtils.getTableDir(rootDir, info.getTable());
|
||||
|
@ -6704,7 +6714,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
* @throws IOException
|
||||
*/
|
||||
public static HRegion openHRegion(final Configuration conf, final FileSystem fs,
|
||||
final Path rootDir, final Path tableDir, final HRegionInfo info, final HTableDescriptor htd,
|
||||
final Path rootDir, final Path tableDir, final HRegionInfo info, final TableDescriptor htd,
|
||||
final WAL wal, final RegionServerServices rsServices,
|
||||
final CancelableProgressable reporter)
|
||||
throws IOException {
|
||||
|
@ -6732,7 +6742,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
throws IOException {
|
||||
HRegionFileSystem regionFs = other.getRegionFileSystem();
|
||||
HRegion r = newHRegion(regionFs.getTableDir(), other.getWAL(), regionFs.getFileSystem(),
|
||||
other.baseConf, other.getRegionInfo(), other.getTableDesc(), null);
|
||||
other.baseConf, other.getRegionInfo(), other.getTableDescriptor(), null);
|
||||
return r.openHRegion(reporter);
|
||||
}
|
||||
|
||||
|
@ -6769,7 +6779,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
}
|
||||
|
||||
public static void warmupHRegion(final HRegionInfo info,
|
||||
final HTableDescriptor htd, final WAL wal, final Configuration conf,
|
||||
final TableDescriptor htd, final WAL wal, final Configuration conf,
|
||||
final RegionServerServices rsServices,
|
||||
final CancelableProgressable reporter)
|
||||
throws IOException {
|
||||
|
@ -6797,14 +6807,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
|
||||
|
||||
private void checkCompressionCodecs() throws IOException {
|
||||
for (HColumnDescriptor fam: this.htableDescriptor.getColumnFamilies()) {
|
||||
for (ColumnFamilyDescriptor fam: this.htableDescriptor.getColumnFamilies()) {
|
||||
CompressionTest.testCompression(fam.getCompressionType());
|
||||
CompressionTest.testCompression(fam.getCompactionCompressionType());
|
||||
}
|
||||
}
|
||||
|
||||
private void checkEncryption() throws IOException {
|
||||
for (HColumnDescriptor fam: this.htableDescriptor.getColumnFamilies()) {
|
||||
for (ColumnFamilyDescriptor fam: this.htableDescriptor.getColumnFamilies()) {
|
||||
EncryptionTest.testEncryption(conf, fam.getEncryptionType(), fam.getEncryptionKey());
|
||||
}
|
||||
}
|
||||
|
@ -6825,7 +6835,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
|
||||
// Create the daughter HRegion instance
|
||||
HRegion r = HRegion.newHRegion(this.fs.getTableDir(), this.getWAL(), fs.getFileSystem(),
|
||||
this.getBaseConf(), hri, this.getTableDesc(), rsServices);
|
||||
this.getBaseConf(), hri, this.getTableDescriptor(), rsServices);
|
||||
r.readRequestsCount.add(this.getReadRequestsCount() / 2);
|
||||
r.filteredReadRequestsCount.add(this.getFilteredReadRequestsCount() / 2);
|
||||
r.writeRequestsCount.add(this.getWriteRequestsCount() / 2);
|
||||
|
@ -6842,7 +6852,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
final HRegion region_b) throws IOException {
|
||||
HRegion r = HRegion.newHRegion(this.fs.getTableDir(), this.getWAL(),
|
||||
fs.getFileSystem(), this.getBaseConf(), mergedRegionInfo,
|
||||
this.getTableDesc(), this.rsServices);
|
||||
this.getTableDescriptor(), this.rsServices);
|
||||
r.readRequestsCount.add(this.getReadRequestsCount()
|
||||
+ region_b.getReadRequestsCount());
|
||||
r.filteredReadRequestsCount.add(this.getFilteredReadRequestsCount()
|
||||
|
@ -6949,7 +6959,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
checkFamily(family);
|
||||
}
|
||||
} else { // Adding all families to scanner
|
||||
for (byte[] family : this.htableDescriptor.getFamiliesKeys()) {
|
||||
for (byte[] family : this.htableDescriptor.getColumnFamilyNames()) {
|
||||
get.addFamily(family);
|
||||
}
|
||||
}
|
||||
|
@ -7472,7 +7482,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
final Mutation mutation, final Durability effectiveDurability, final long now,
|
||||
final List<Cell> deltas, final List<Cell> results)
|
||||
throws IOException {
|
||||
byte [] columnFamily = store.getFamily().getName();
|
||||
byte [] columnFamily = store.getColumnFamilyDescriptor().getName();
|
||||
List<Cell> toApply = new ArrayList<>(deltas.size());
|
||||
// Get previous values for all columns in this family.
|
||||
List<Cell> currentValues = get(mutation, store, deltas,
|
||||
|
@ -7642,7 +7652,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
get.setIsolationLevel(isolation);
|
||||
}
|
||||
for (Cell cell: coordinates) {
|
||||
get.addColumn(store.getFamily().getName(), CellUtil.cloneQualifier(cell));
|
||||
get.addColumn(store.getColumnFamilyDescriptor().getName(), CellUtil.cloneQualifier(cell));
|
||||
}
|
||||
// Increments carry time range. If an Increment instance, put it on the Get.
|
||||
if (tr != null) {
|
||||
|
@ -7665,7 +7675,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
|
||||
void checkFamily(final byte [] family)
|
||||
throws NoSuchColumnFamilyException {
|
||||
if (!this.htableDescriptor.hasFamily(family)) {
|
||||
if (!this.htableDescriptor.hasColumnFamily(family)) {
|
||||
throw new NoSuchColumnFamilyException("Column family " +
|
||||
Bytes.toString(family) + " does not exist in region " + this
|
||||
+ " in table " + this.htableDescriptor);
|
||||
|
@ -8204,7 +8214,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
buf.append(getRegionInfo().isMetaTable() ? " meta table " : " ");
|
||||
buf.append("stores: ");
|
||||
for (Store s : getStores()) {
|
||||
buf.append(s.getFamily().getNameAsString());
|
||||
buf.append(s.getColumnFamilyDescriptor().getNameAsString());
|
||||
buf.append(" size: ");
|
||||
buf.append(s.getSizeOfMemStore().getDataSize());
|
||||
buf.append(" ");
|
||||
|
|
|
@ -3013,7 +3013,7 @@ public class HRegionServer extends HasThread implements
|
|||
Set<TableName> tables = new HashSet<>();
|
||||
synchronized (this.onlineRegions) {
|
||||
for (Region region: this.onlineRegions.values()) {
|
||||
tables.add(region.getTableDesc().getTableName());
|
||||
tables.add(region.getTableDescriptor().getTableName());
|
||||
}
|
||||
}
|
||||
return tables;
|
||||
|
@ -3167,7 +3167,7 @@ public class HRegionServer extends HasThread implements
|
|||
if (exceptionToThrow instanceof IOException) throw (IOException)exceptionToThrow;
|
||||
throw new IOException(exceptionToThrow);
|
||||
}
|
||||
if (regionToClose.getTableDesc().hasSerialReplicationScope()) {
|
||||
if (regionToClose.getTableDescriptor().hasSerialReplicationScope()) {
|
||||
// For serial replication, we need add a final barrier on this region. But the splitting
|
||||
// or merging may be reverted, so we should make sure if we reopen this region, the open
|
||||
// barrier is same as this final barrier
|
||||
|
@ -3185,7 +3185,7 @@ public class HRegionServer extends HasThread implements
|
|||
Put finalBarrier = MetaTableAccessor.makeBarrierPut(
|
||||
Bytes.toBytes(regionEncodedName.get(i)),
|
||||
seq,
|
||||
regionToClose.getTableDesc().getTableName().getName());
|
||||
regionToClose.getTableDescriptor().getTableName().getName());
|
||||
MetaTableAccessor.putToMetaTable(getConnection(), finalBarrier);
|
||||
}
|
||||
// Offline the region
|
||||
|
|
|
@ -98,6 +98,7 @@ import com.google.common.collect.ImmutableCollection;
|
|||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
|
||||
/**
|
||||
* A Store holds a column family in a Region. Its a memstore and a set of zero
|
||||
|
@ -129,7 +130,7 @@ public class HStore implements Store {
|
|||
protected final MemStore memstore;
|
||||
// This stores directory in the filesystem.
|
||||
protected final HRegion region;
|
||||
private final HColumnDescriptor family;
|
||||
private final ColumnFamilyDescriptor family;
|
||||
private final HRegionFileSystem fs;
|
||||
protected Configuration conf;
|
||||
protected CacheConfig cacheConf;
|
||||
|
@ -213,7 +214,7 @@ public class HStore implements Store {
|
|||
* failed. Can be null.
|
||||
* @throws IOException
|
||||
*/
|
||||
protected HStore(final HRegion region, final HColumnDescriptor family,
|
||||
protected HStore(final HRegion region, final ColumnFamilyDescriptor family,
|
||||
final Configuration confParam) throws IOException {
|
||||
|
||||
this.fs = region.getRegionFileSystem();
|
||||
|
@ -227,7 +228,7 @@ public class HStore implements Store {
|
|||
// add global config first, then table and cf overrides, then cf metadata.
|
||||
this.conf = new CompoundConfiguration()
|
||||
.add(confParam)
|
||||
.addStringMap(region.getTableDesc().getConfiguration())
|
||||
.addStringMap(region.getTableDescriptor().getConfiguration())
|
||||
.addStringMap(family.getConfiguration())
|
||||
.addBytesMap(family.getValues());
|
||||
this.blocksize = family.getBlocksize();
|
||||
|
@ -320,7 +321,7 @@ public class HStore implements Store {
|
|||
* Creates the cache config.
|
||||
* @param family The current column family.
|
||||
*/
|
||||
protected void createCacheConf(final HColumnDescriptor family) {
|
||||
protected void createCacheConf(final ColumnFamilyDescriptor family) {
|
||||
this.cacheConf = new CacheConfig(conf, family);
|
||||
}
|
||||
|
||||
|
@ -341,7 +342,7 @@ public class HStore implements Store {
|
|||
* @param family
|
||||
* @return TTL in seconds of the specified family
|
||||
*/
|
||||
public static long determineTTLFromFamily(final HColumnDescriptor family) {
|
||||
public static long determineTTLFromFamily(final ColumnFamilyDescriptor family) {
|
||||
// HCD.getTimeToLive returns ttl in seconds. Convert to milliseconds.
|
||||
long ttl = family.getTimeToLive();
|
||||
if (ttl == HConstants.FOREVER) {
|
||||
|
@ -455,7 +456,7 @@ public class HStore implements Store {
|
|||
}
|
||||
|
||||
@Override
|
||||
public HColumnDescriptor getFamily() {
|
||||
public ColumnFamilyDescriptor getColumnFamilyDescriptor() {
|
||||
return this.family;
|
||||
}
|
||||
|
||||
|
@ -1418,7 +1419,7 @@ public class HStore implements Store {
|
|||
}
|
||||
HRegionInfo info = this.region.getRegionInfo();
|
||||
CompactionDescriptor compactionDescriptor = ProtobufUtil.toCompactionDescriptor(info,
|
||||
family.getName(), inputPaths, outputPaths, fs.getStoreDir(getFamily().getNameAsString()));
|
||||
family.getName(), inputPaths, outputPaths, fs.getStoreDir(getColumnFamilyDescriptor().getNameAsString()));
|
||||
// Fix reaching into Region to get the maxWaitForSeqId.
|
||||
// Does this method belong in Region altogether given it is making so many references up there?
|
||||
// Could be Region#writeCompactionMarker(compactionDescriptor);
|
||||
|
@ -1736,9 +1737,9 @@ public class HStore implements Store {
|
|||
|
||||
private void removeUnneededFiles() throws IOException {
|
||||
if (!conf.getBoolean("hbase.store.delete.expired.storefile", true)) return;
|
||||
if (getFamily().getMinVersions() > 0) {
|
||||
if (getColumnFamilyDescriptor().getMinVersions() > 0) {
|
||||
LOG.debug("Skipping expired store file removal due to min version being " +
|
||||
getFamily().getMinVersions());
|
||||
getColumnFamilyDescriptor().getMinVersions());
|
||||
return;
|
||||
}
|
||||
this.lock.readLock().lock();
|
||||
|
@ -2546,7 +2547,7 @@ public class HStore implements Store {
|
|||
}
|
||||
// Only if this is successful it has to be removed
|
||||
try {
|
||||
this.fs.removeStoreFiles(this.getFamily().getNameAsString(), filesToRemove);
|
||||
this.fs.removeStoreFiles(this.getColumnFamilyDescriptor().getNameAsString(), filesToRemove);
|
||||
} catch (FailedArchiveException fae) {
|
||||
// Even if archiving some files failed, we still need to clear out any of the
|
||||
// files which were successfully archived. Otherwise we will receive a
|
||||
|
|
|
@ -24,9 +24,10 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.procedure2.util.StringUtils;
|
||||
|
||||
/**
|
||||
|
@ -56,13 +57,13 @@ public class IncreasingToUpperBoundRegionSplitPolicy extends ConstantSizeRegionS
|
|||
if (initialSize > 0) {
|
||||
return;
|
||||
}
|
||||
HTableDescriptor desc = region.getTableDesc();
|
||||
TableDescriptor desc = region.getTableDescriptor();
|
||||
if (desc != null) {
|
||||
initialSize = 2 * desc.getMemStoreFlushSize();
|
||||
}
|
||||
if (initialSize <= 0) {
|
||||
initialSize = 2 * conf.getLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE,
|
||||
HTableDescriptor.DEFAULT_MEMSTORE_FLUSH_SIZE);
|
||||
TableDescriptorBuilder.DEFAULT_MEMSTORE_FLUSH_SIZE);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -106,7 +107,7 @@ public class IncreasingToUpperBoundRegionSplitPolicy extends ConstantSizeRegionS
|
|||
if (rss == null) {
|
||||
return 0;
|
||||
}
|
||||
TableName tablename = region.getTableDesc().getTableName();
|
||||
TableName tablename = region.getTableDescriptor().getTableName();
|
||||
int tableRegionsCount = 0;
|
||||
try {
|
||||
List<Region> hri = rss.getOnlineRegions(tablename);
|
||||
|
|
|
@ -46,14 +46,14 @@ public class KeyPrefixRegionSplitPolicy extends IncreasingToUpperBoundRegionSpli
|
|||
prefixLength = 0;
|
||||
|
||||
// read the prefix length from the table descriptor
|
||||
String prefixLengthString = region.getTableDesc().getValue(
|
||||
String prefixLengthString = region.getTableDescriptor().getValue(
|
||||
PREFIX_LENGTH_KEY);
|
||||
if (prefixLengthString == null) {
|
||||
//read the deprecated value
|
||||
prefixLengthString = region.getTableDesc().getValue(PREFIX_LENGTH_KEY_DEPRECATED);
|
||||
prefixLengthString = region.getTableDescriptor().getValue(PREFIX_LENGTH_KEY_DEPRECATED);
|
||||
if (prefixLengthString == null) {
|
||||
LOG.error(PREFIX_LENGTH_KEY + " not specified for table "
|
||||
+ region.getTableDesc().getTableName()
|
||||
+ region.getTableDescriptor().getTableName()
|
||||
+ ". Using default RegionSplitPolicy");
|
||||
return;
|
||||
}
|
||||
|
@ -63,13 +63,13 @@ public class KeyPrefixRegionSplitPolicy extends IncreasingToUpperBoundRegionSpli
|
|||
} catch (NumberFormatException nfe) {
|
||||
/* Differentiate NumberFormatException from an invalid value range reported below. */
|
||||
LOG.error("Number format exception when parsing " + PREFIX_LENGTH_KEY + " for table "
|
||||
+ region.getTableDesc().getTableName() + ":"
|
||||
+ region.getTableDescriptor().getTableName() + ":"
|
||||
+ prefixLengthString + ". " + nfe);
|
||||
return;
|
||||
}
|
||||
if (prefixLength <= 0) {
|
||||
LOG.error("Invalid value for " + PREFIX_LENGTH_KEY + " for table "
|
||||
+ region.getTableDesc().getTableName() + ":"
|
||||
+ region.getTableDescriptor().getTableName() + ":"
|
||||
+ prefixLengthString + ". Using default RegionSplitPolicy");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -551,7 +551,7 @@ class MemStoreFlusher implements FlushRequester {
|
|||
private boolean isTooManyStoreFiles(Region region) {
|
||||
|
||||
// When compaction is disabled, the region is flushable
|
||||
if (!region.getTableDesc().isCompactionEnabled()) {
|
||||
if (!region.getTableDescriptor().isCompactionEnabled()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
|
|
|
@ -30,7 +30,7 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.metrics2.MetricsExecutor;
|
||||
|
||||
|
@ -65,7 +65,7 @@ public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable
|
|||
|
||||
@Override
|
||||
public String getTableName() {
|
||||
HTableDescriptor tableDesc = this.region.getTableDesc();
|
||||
TableDescriptor tableDesc = this.region.getTableDescriptor();
|
||||
if (tableDesc == null) {
|
||||
return UNKNOWN;
|
||||
}
|
||||
|
@ -74,7 +74,7 @@ public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable
|
|||
|
||||
@Override
|
||||
public String getNamespace() {
|
||||
HTableDescriptor tableDesc = this.region.getTableDesc();
|
||||
TableDescriptor tableDesc = this.region.getTableDescriptor();
|
||||
if (tableDesc == null) {
|
||||
return UNKNOWN;
|
||||
}
|
||||
|
|
|
@ -62,7 +62,7 @@ public class MetricsTableWrapperAggregateImpl implements MetricsTableWrapperAggr
|
|||
Map<TableName, MetricsTableValues> localMetricsTableMap = new HashMap<>();
|
||||
|
||||
for (Region r : regionServer.getOnlineRegionsLocalContext()) {
|
||||
TableName tbl= r.getTableDesc().getTableName();
|
||||
TableName tbl= r.getTableDescriptor().getTableName();
|
||||
MetricsTableValues metricsTable = localMetricsTableMap.get(tbl);
|
||||
if (metricsTable == null) {
|
||||
metricsTable = new MetricsTableValues();
|
||||
|
|
|
@ -110,7 +110,7 @@ public class RSDumpServlet extends StateDumpServlet {
|
|||
if (hRegion.getLockedRows().size() > 0) {
|
||||
for (HRegion.RowLockContext rowLockContext : hRegion.getLockedRows().values()) {
|
||||
sb.setLength(0);
|
||||
sb.append(hRegion.getTableDesc().getTableName()).append(",")
|
||||
sb.append(hRegion.getTableDescriptor().getTableName()).append(",")
|
||||
.append(hRegion.getRegionInfo().getEncodedName()).append(",");
|
||||
sb.append(rowLockContext.toString());
|
||||
out.println(sb.toString());
|
||||
|
|
|
@ -1548,7 +1548,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
if (QuotaUtil.isQuotaEnabled(getConfiguration()) &&
|
||||
!Superusers.isSuperUser(RpcServer.getRequestUser()) &&
|
||||
this.regionServer.getRegionServerSpaceQuotaManager().areCompactionsDisabled(
|
||||
region.getTableDesc().getTableName())) {
|
||||
region.getTableDescriptor().getTableName())) {
|
||||
throw new DoNotRetryIOException("Compactions on this region are "
|
||||
+ "disabled due to a space quota violation.");
|
||||
}
|
||||
|
@ -1784,7 +1784,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
requestCount.increment();
|
||||
Set<byte[]> columnFamilies;
|
||||
if (request.getFamilyCount() == 0) {
|
||||
columnFamilies = region.getTableDesc().getFamiliesKeys();
|
||||
columnFamilies = region.getTableDescriptor().getColumnFamilyNames();
|
||||
} else {
|
||||
columnFamilies = new TreeSet<>(Bytes.BYTES_RAWCOMPARATOR);
|
||||
for (ByteString cf: request.getFamilyList()) {
|
||||
|
@ -2890,7 +2890,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
|
||||
if (!scan.hasFamilies()) {
|
||||
// Adding all families to scanner
|
||||
for (byte[] family : region.getTableDesc().getFamiliesKeys()) {
|
||||
for (byte[] family : region.getTableDescriptor().getColumnFamilyNames()) {
|
||||
scan.addFamily(family);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.client.Put;
|
|||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.RowMutations;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.conf.ConfigurationObserver;
|
||||
import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
|
||||
import org.apache.hadoop.hbase.filter.ByteArrayComparable;
|
||||
|
@ -79,7 +80,7 @@ public interface Region extends ConfigurationObserver {
|
|||
HRegionInfo getRegionInfo();
|
||||
|
||||
/** @return table descriptor for this region */
|
||||
HTableDescriptor getTableDesc();
|
||||
TableDescriptor getTableDescriptor();
|
||||
|
||||
/** @return true if region is available (not closed and not closing) */
|
||||
boolean isAvailable();
|
||||
|
|
|
@ -55,6 +55,7 @@ import org.apache.hadoop.hbase.client.Mutation;
|
|||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
|
||||
import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
|
||||
import org.apache.hadoop.hbase.coprocessor.EndpointObserver;
|
||||
|
@ -260,7 +261,7 @@ public class RegionCoprocessorHost
|
|||
}
|
||||
|
||||
static List<TableCoprocessorAttribute> getTableCoprocessorAttrsFromSchema(Configuration conf,
|
||||
HTableDescriptor htd) {
|
||||
TableDescriptor htd) {
|
||||
List<TableCoprocessorAttribute> result = Lists.newArrayList();
|
||||
for (Map.Entry<Bytes, Bytes> e: htd.getValues().entrySet()) {
|
||||
String key = Bytes.toString(e.getKey().get()).trim();
|
||||
|
@ -324,7 +325,7 @@ public class RegionCoprocessorHost
|
|||
* @throws IOException
|
||||
*/
|
||||
public static void testTableCoprocessorAttrs(final Configuration conf,
|
||||
final HTableDescriptor htd) throws IOException {
|
||||
final TableDescriptor htd) throws IOException {
|
||||
String pathPrefix = UUID.randomUUID().toString();
|
||||
for (TableCoprocessorAttribute attr: getTableCoprocessorAttrsFromSchema(conf, htd)) {
|
||||
if (attr.getPriority() < 0) {
|
||||
|
@ -362,15 +363,15 @@ public class RegionCoprocessorHost
|
|||
// scan the table attributes for coprocessor load specifications
|
||||
// initialize the coprocessors
|
||||
List<RegionEnvironment> configured = new ArrayList<>();
|
||||
for (TableCoprocessorAttribute attr: getTableCoprocessorAttrsFromSchema(conf,
|
||||
region.getTableDesc())) {
|
||||
for (TableCoprocessorAttribute attr: getTableCoprocessorAttrsFromSchema(conf,
|
||||
region.getTableDescriptor())) {
|
||||
// Load encompasses classloading and coprocessor initialization
|
||||
try {
|
||||
RegionEnvironment env = load(attr.getPath(), attr.getClassName(), attr.getPriority(),
|
||||
attr.getConf());
|
||||
configured.add(env);
|
||||
LOG.info("Loaded coprocessor " + attr.getClassName() + " from HTD of " +
|
||||
region.getTableDesc().getTableName().getNameAsString() + " successfully.");
|
||||
region.getTableDescriptor().getTableName().getNameAsString() + " successfully.");
|
||||
} catch (Throwable t) {
|
||||
// Coprocessor failed to load, do we abort on error?
|
||||
if (conf.getBoolean(ABORT_ON_ERROR_KEY, DEFAULT_ABORT_ON_ERROR)) {
|
||||
|
|
|
@ -82,7 +82,7 @@ public class RegionServicesForStores {
|
|||
}
|
||||
|
||||
public int getNumStores() {
|
||||
return region.getTableDesc().getColumnFamilyCount();
|
||||
return region.getTableDescriptor().getColumnFamilyCount();
|
||||
}
|
||||
|
||||
// methods for tests
|
||||
|
|
|
@ -23,13 +23,14 @@ import java.util.List;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.conf.Configured;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.util.ReflectionUtils;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
|
||||
/**
|
||||
* A split policy determines when a region should be split.
|
||||
* @see IncreasingToUpperBoundRegionSplitPolicy Default split policy since
|
||||
|
@ -101,14 +102,14 @@ public abstract class RegionSplitPolicy extends Configured {
|
|||
public static RegionSplitPolicy create(HRegion region,
|
||||
Configuration conf) throws IOException {
|
||||
Class<? extends RegionSplitPolicy> clazz = getSplitPolicyClass(
|
||||
region.getTableDesc(), conf);
|
||||
region.getTableDescriptor(), conf);
|
||||
RegionSplitPolicy policy = ReflectionUtils.newInstance(clazz, conf);
|
||||
policy.configureForRegion(region);
|
||||
return policy;
|
||||
}
|
||||
|
||||
public static Class<? extends RegionSplitPolicy> getSplitPolicyClass(
|
||||
HTableDescriptor htd, Configuration conf) throws IOException {
|
||||
TableDescriptor htd, Configuration conf) throws IOException {
|
||||
String className = htd.getRegionSplitPolicyClassName();
|
||||
if (className == null) {
|
||||
className = conf.get(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,
|
||||
|
|
|
@ -21,7 +21,7 @@ package org.apache.hadoop.hbase.regionserver;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeepDeletedCells;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -57,13 +57,13 @@ public class ScanInfo {
|
|||
|
||||
/**
|
||||
* @param conf
|
||||
* @param family {@link HColumnDescriptor} describing the column family
|
||||
* @param family {@link ColumnFamilyDescriptor} describing the column family
|
||||
* @param ttl Store's TTL (in ms)
|
||||
* @param timeToPurgeDeletes duration in ms after which a delete marker can be purged during a
|
||||
* major compaction.
|
||||
* @param comparator The store's comparator
|
||||
*/
|
||||
public ScanInfo(final Configuration conf, final HColumnDescriptor family, final long ttl,
|
||||
public ScanInfo(final Configuration conf, final ColumnFamilyDescriptor family, final long ttl,
|
||||
final long timeToPurgeDeletes, final CellComparator comparator) {
|
||||
this(conf, family.getName(), family.getMinVersions(), family.getMaxVersions(), ttl,
|
||||
family.getKeepDeletedCells(), family.getBlocksize(), timeToPurgeDeletes, comparator);
|
||||
|
|
|
@ -150,7 +150,7 @@ public class SecureBulkLoadManager {
|
|||
}
|
||||
|
||||
String bulkToken =
|
||||
createStagingDir(baseStagingDir, getActiveUser(), region.getTableDesc().getTableName())
|
||||
createStagingDir(baseStagingDir, getActiveUser(), region.getTableDescriptor().getTableName())
|
||||
.toString();
|
||||
|
||||
return bulkToken;
|
||||
|
|
|
@ -25,11 +25,11 @@ import java.util.NavigableSet;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver;
|
||||
import org.apache.hadoop.hbase.io.HeapSize;
|
||||
|
@ -361,7 +361,7 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
|
|||
*/
|
||||
MemstoreSize getSizeOfSnapshot();
|
||||
|
||||
HColumnDescriptor getFamily();
|
||||
ColumnFamilyDescriptor getColumnFamilyDescriptor();
|
||||
|
||||
/**
|
||||
* @return The maximum sequence id in all store files.
|
||||
|
|
|
@ -472,7 +472,7 @@ public class StoreFileScanner implements KeyValueScanner {
|
|||
@Override
|
||||
public boolean shouldUseScanner(Scan scan, Store store, long oldestUnexpiredTS) {
|
||||
// if the file has no entries, no need to validate or create a scanner.
|
||||
byte[] cf = store.getFamily().getName();
|
||||
byte[] cf = store.getColumnFamilyDescriptor().getName();
|
||||
TimeRange timeRange = scan.getColumnFamilyTimeRange().get(cf);
|
||||
if (timeRange == null) {
|
||||
timeRange = scan.getTimeRange();
|
||||
|
|
|
@ -110,7 +110,7 @@ public class StripeStoreFlusher extends StoreFlusher {
|
|||
@Override
|
||||
public StoreFileWriter createWriter() throws IOException {
|
||||
StoreFileWriter writer = store.createWriterInTmp(
|
||||
kvCount, store.getFamily().getCompressionType(),
|
||||
kvCount, store.getColumnFamilyDescriptor().getCompressionType(),
|
||||
/* isCompaction = */ false,
|
||||
/* includeMVCCReadpoint = */ true,
|
||||
/* includesTags = */ true,
|
||||
|
|
|
@ -93,9 +93,9 @@ public abstract class Compactor<T extends CellSink> {
|
|||
this.store = store;
|
||||
this.compactionKVMax =
|
||||
this.conf.getInt(HConstants.COMPACTION_KV_MAX, HConstants.COMPACTION_KV_MAX_DEFAULT);
|
||||
this.compactionCompression = (this.store.getFamily() == null) ?
|
||||
Compression.Algorithm.NONE : this.store.getFamily().getCompactionCompressionType();
|
||||
this.keepSeqIdPeriod = Math.max(this.conf.getInt(HConstants.KEEP_SEQID_PERIOD,
|
||||
this.compactionCompression = (this.store.getColumnFamilyDescriptor() == null) ?
|
||||
Compression.Algorithm.NONE : this.store.getColumnFamilyDescriptor().getCompactionCompressionType();
|
||||
this.keepSeqIdPeriod = Math.max(this.conf.getInt(HConstants.KEEP_SEQID_PERIOD,
|
||||
HConstants.MIN_KEEP_SEQID_PERIOD), HConstants.MIN_KEEP_SEQID_PERIOD);
|
||||
this.dropCacheMajor = conf.getBoolean(MAJOR_COMPACTION_DROP_CACHE, true);
|
||||
this.dropCacheMinor = conf.getBoolean(MINOR_COMPACTION_DROP_CACHE, true);
|
||||
|
@ -397,7 +397,7 @@ public abstract class Compactor<T extends CellSink> {
|
|||
|
||||
throughputController.start(compactionName);
|
||||
KeyValueScanner kvs = (scanner instanceof KeyValueScanner)? (KeyValueScanner)scanner : null;
|
||||
long shippedCallSizeLimit = (long) numofFilesToCompact * this.store.getFamily().getBlocksize();
|
||||
long shippedCallSizeLimit = (long) numofFilesToCompact * this.store.getColumnFamilyDescriptor().getBlocksize();
|
||||
try {
|
||||
do {
|
||||
hasMore = scanner.next(cells, scannerContext);
|
||||
|
@ -499,7 +499,7 @@ public abstract class Compactor<T extends CellSink> {
|
|||
protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
|
||||
ScanType scanType, long smallestReadPoint, long earliestPutTs) throws IOException {
|
||||
Scan scan = new Scan();
|
||||
scan.setMaxVersions(store.getFamily().getMaxVersions());
|
||||
scan.setMaxVersions(store.getColumnFamilyDescriptor().getMaxVersions());
|
||||
return new StoreScanner(store, store.getScanInfo(), scan, scanners,
|
||||
scanType, smallestReadPoint, earliestPutTs);
|
||||
}
|
||||
|
@ -517,7 +517,7 @@ public abstract class Compactor<T extends CellSink> {
|
|||
long smallestReadPoint, long earliestPutTs, byte[] dropDeletesFromRow,
|
||||
byte[] dropDeletesToRow) throws IOException {
|
||||
Scan scan = new Scan();
|
||||
scan.setMaxVersions(store.getFamily().getMaxVersions());
|
||||
scan.setMaxVersions(store.getColumnFamilyDescriptor().getMaxVersions());
|
||||
return new StoreScanner(store, store.getScanInfo(), scan, scanners, smallestReadPoint,
|
||||
earliestPutTs, dropDeletesFromRow, dropDeletesToRow);
|
||||
}
|
||||
|
|
|
@ -50,6 +50,7 @@ public final class ThroughputControlUtil {
|
|||
}
|
||||
}
|
||||
return store.getRegionInfo().getRegionNameAsString() + NAME_DELIMITER
|
||||
+ store.getFamily().getNameAsString() + NAME_DELIMITER + opName + NAME_DELIMITER + counter;
|
||||
+ store.getColumnFamilyDescriptor().getNameAsString()
|
||||
+ NAME_DELIMITER + opName + NAME_DELIMITER + counter;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -359,7 +359,7 @@ public class AccessControlLists {
|
|||
* metadata table.
|
||||
*/
|
||||
static boolean isAclRegion(Region region) {
|
||||
return ACL_TABLE_NAME.equals(region.getTableDesc().getTableName());
|
||||
return ACL_TABLE_NAME.equals(region.getTableDescriptor().getTableName());
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -134,6 +134,7 @@ import com.google.protobuf.Message;
|
|||
import com.google.protobuf.RpcCallback;
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.Service;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
|
||||
/**
|
||||
* Provides basic authorization checks for data access and administrative
|
||||
|
@ -954,7 +955,7 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
|
|||
} else if (env instanceof RegionCoprocessorEnvironment) {
|
||||
// if running at region
|
||||
regionEnv = (RegionCoprocessorEnvironment) env;
|
||||
conf.addStringMap(regionEnv.getRegion().getTableDesc().getConfiguration());
|
||||
conf.addStringMap(regionEnv.getRegion().getTableDescriptor().getConfiguration());
|
||||
zk = regionEnv.getRegionServerServices().getZooKeeper();
|
||||
compatibleEarlyTermination = conf.getBoolean(AccessControlConstants.CF_ATTRIBUTE_EARLY_OUT,
|
||||
AccessControlConstants.DEFAULT_ATTRIBUTE_EARLY_OUT);
|
||||
|
@ -1551,7 +1552,7 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
|
|||
Region region = getRegion(env);
|
||||
TableName table = getTableName(region);
|
||||
Map<ByteRange, Integer> cfVsMaxVersions = Maps.newHashMap();
|
||||
for (HColumnDescriptor hcd : region.getTableDesc().getFamilies()) {
|
||||
for (ColumnFamilyDescriptor hcd : region.getTableDescriptor().getColumnFamilies()) {
|
||||
cfVsMaxVersions.put(new SimpleMutableByteRange(hcd.getName()), hcd.getMaxVersions());
|
||||
}
|
||||
if (!authResult.isAllowed()) {
|
||||
|
@ -2155,7 +2156,7 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
|
|||
User user = getActiveUser(ctx);
|
||||
for(Pair<byte[],String> el : familyPaths) {
|
||||
requirePermission(user, "preBulkLoadHFile",
|
||||
ctx.getEnvironment().getRegion().getTableDesc().getTableName(),
|
||||
ctx.getEnvironment().getRegion().getTableDescriptor().getTableName(),
|
||||
el.getFirst(),
|
||||
null,
|
||||
Action.CREATE);
|
||||
|
@ -2173,7 +2174,7 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
|
|||
public void prePrepareBulkLoad(ObserverContext<RegionCoprocessorEnvironment> ctx,
|
||||
PrepareBulkLoadRequest request) throws IOException {
|
||||
requireAccess(getActiveUser(ctx), "prePrepareBulkLoad",
|
||||
ctx.getEnvironment().getRegion().getTableDesc().getTableName(), Action.CREATE);
|
||||
ctx.getEnvironment().getRegion().getTableDescriptor().getTableName(), Action.CREATE);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -2187,7 +2188,7 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
|
|||
public void preCleanupBulkLoad(ObserverContext<RegionCoprocessorEnvironment> ctx,
|
||||
CleanupBulkLoadRequest request) throws IOException {
|
||||
requireAccess(getActiveUser(ctx), "preCleanupBulkLoad",
|
||||
ctx.getEnvironment().getRegion().getTableDesc().getTableName(), Action.CREATE);
|
||||
ctx.getEnvironment().getRegion().getTableDescriptor().getTableName(), Action.CREATE);
|
||||
}
|
||||
|
||||
/* ---- EndpointObserver implementation ---- */
|
||||
|
@ -2392,7 +2393,7 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
|
|||
AccessControlProtos.CheckPermissionsResponse response = null;
|
||||
try {
|
||||
User user = RpcServer.getRequestUser();
|
||||
TableName tableName = regionEnv.getRegion().getTableDesc().getTableName();
|
||||
TableName tableName = regionEnv.getRegion().getTableDescriptor().getTableName();
|
||||
for (Permission permission : permissions) {
|
||||
if (permission instanceof TablePermission) {
|
||||
// Check table permissions
|
||||
|
@ -2586,7 +2587,7 @@ public class AccessController implements MasterObserver, RegionObserver, RegionS
|
|||
@Override
|
||||
public void preMerge(ObserverContext<RegionServerCoprocessorEnvironment> ctx, Region regionA,
|
||||
Region regionB) throws IOException {
|
||||
requirePermission(getActiveUser(ctx), "mergeRegions", regionA.getTableDesc().getTableName(),
|
||||
requirePermission(getActiveUser(ctx), "mergeRegions", regionA.getTableDescriptor().getTableName(),
|
||||
null, null, Action.ADMIN);
|
||||
}
|
||||
|
||||
|
|
|
@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.Tag;
|
|||
import org.apache.hadoop.hbase.TagType;
|
||||
import org.apache.hadoop.hbase.TagUtil;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.filter.Filter;
|
||||
import org.apache.hadoop.hbase.io.util.StreamUtils;
|
||||
|
@ -267,7 +268,7 @@ public class VisibilityUtils {
|
|||
public static Filter createVisibilityLabelFilter(Region region, Authorizations authorizations)
|
||||
throws IOException {
|
||||
Map<ByteRange, Integer> cfVsMaxVersions = new HashMap<>();
|
||||
for (HColumnDescriptor hcd : region.getTableDesc().getFamilies()) {
|
||||
for (ColumnFamilyDescriptor hcd : region.getTableDescriptor().getColumnFamilies()) {
|
||||
cfVsMaxVersions.put(new SimpleMutableByteRange(hcd.getName()), hcd.getMaxVersions());
|
||||
}
|
||||
VisibilityLabelService vls = VisibilityLabelServiceManager.getInstance()
|
||||
|
|
|
@ -216,7 +216,7 @@ public final class SnapshotManifest {
|
|||
|
||||
for (Store store : region.getStores()) {
|
||||
// 2.1. build the snapshot reference for the store
|
||||
Object familyData = visitor.familyOpen(regionData, store.getFamily().getName());
|
||||
Object familyData = visitor.familyOpen(regionData, store.getColumnFamilyDescriptor().getName());
|
||||
monitor.rethrowException();
|
||||
|
||||
List<StoreFile> storeFiles = new ArrayList<>(store.getStorefiles());
|
||||
|
|
|
@ -71,6 +71,7 @@ import org.apache.hadoop.hbase.TableNotEnabledException;
|
|||
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
|
@ -79,6 +80,7 @@ import org.apache.hadoop.hbase.client.RegionLocator;
|
|||
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
|
||||
import org.apache.hadoop.hbase.tool.Canary.RegionTask.TaskType;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -117,14 +119,14 @@ public final class Canary implements Tool {
|
|||
public long getReadFailureCount();
|
||||
public long incReadFailureCount();
|
||||
public void publishReadFailure(ServerName serverName, HRegionInfo region, Exception e);
|
||||
public void publishReadFailure(ServerName serverName, HRegionInfo region, HColumnDescriptor column, Exception e);
|
||||
public void publishReadFailure(ServerName serverName, HRegionInfo region, ColumnFamilyDescriptor column, Exception e);
|
||||
public void updateReadFailedHostList(HRegionInfo region, String serverName);
|
||||
public Map<String,String> getReadFailures();
|
||||
public void publishReadTiming(ServerName serverName, HRegionInfo region, HColumnDescriptor column, long msTime);
|
||||
public void publishReadTiming(ServerName serverName, HRegionInfo region, ColumnFamilyDescriptor column, long msTime);
|
||||
public long getWriteFailureCount();
|
||||
public void publishWriteFailure(ServerName serverName, HRegionInfo region, Exception e);
|
||||
public void publishWriteFailure(ServerName serverName, HRegionInfo region, HColumnDescriptor column, Exception e);
|
||||
public void publishWriteTiming(ServerName serverName, HRegionInfo region, HColumnDescriptor column, long msTime);
|
||||
public void publishWriteFailure(ServerName serverName, HRegionInfo region, ColumnFamilyDescriptor column, Exception e);
|
||||
public void publishWriteTiming(ServerName serverName, HRegionInfo region, ColumnFamilyDescriptor column, long msTime);
|
||||
public void updateWriteFailedHostList(HRegionInfo region, String serverName);
|
||||
public Map<String,String> getWriteFailures();
|
||||
}
|
||||
|
@ -161,7 +163,7 @@ public final class Canary implements Tool {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void publishReadFailure(ServerName serverName, HRegionInfo region, HColumnDescriptor column, Exception e) {
|
||||
public void publishReadFailure(ServerName serverName, HRegionInfo region, ColumnFamilyDescriptor column, Exception e) {
|
||||
readFailureCount.incrementAndGet();
|
||||
LOG.error(String.format("read from region %s on regionserver %s column family %s failed",
|
||||
region.getRegionNameAsString(), serverName, column.getNameAsString()), e);
|
||||
|
@ -173,7 +175,7 @@ public final class Canary implements Tool {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void publishReadTiming(ServerName serverName, HRegionInfo region, HColumnDescriptor column, long msTime) {
|
||||
public void publishReadTiming(ServerName serverName, HRegionInfo region, ColumnFamilyDescriptor column, long msTime) {
|
||||
LOG.info(String.format("read from region %s on regionserver %s column family %s in %dms",
|
||||
region.getRegionNameAsString(), serverName, column.getNameAsString(), msTime));
|
||||
}
|
||||
|
@ -200,14 +202,14 @@ public final class Canary implements Tool {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void publishWriteFailure(ServerName serverName, HRegionInfo region, HColumnDescriptor column, Exception e) {
|
||||
public void publishWriteFailure(ServerName serverName, HRegionInfo region, ColumnFamilyDescriptor column, Exception e) {
|
||||
writeFailureCount.incrementAndGet();
|
||||
LOG.error(String.format("write to region %s on regionserver %s column family %s failed",
|
||||
region.getRegionNameAsString(), serverName, column.getNameAsString()), e);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void publishWriteTiming(ServerName serverName, HRegionInfo region, HColumnDescriptor column, long msTime) {
|
||||
public void publishWriteTiming(ServerName serverName, HRegionInfo region, ColumnFamilyDescriptor column, long msTime) {
|
||||
LOG.info(String.format("write to region %s on regionserver %s column family %s in %dms",
|
||||
region.getRegionNameAsString(), serverName, column.getNameAsString(), msTime));
|
||||
}
|
||||
|
@ -348,14 +350,14 @@ public final class Canary implements Tool {
|
|||
|
||||
public Void read() {
|
||||
Table table = null;
|
||||
HTableDescriptor tableDesc = null;
|
||||
TableDescriptor tableDesc = null;
|
||||
try {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(String.format("reading table descriptor for table %s",
|
||||
region.getTable()));
|
||||
}
|
||||
table = connection.getTable(region.getTable());
|
||||
tableDesc = table.getTableDescriptor();
|
||||
tableDesc = table.getDescriptor();
|
||||
} catch (IOException e) {
|
||||
LOG.debug("sniffRegion failed", e);
|
||||
sink.publishReadFailure(serverName, region, e);
|
||||
|
@ -374,7 +376,7 @@ public final class Canary implements Tool {
|
|||
Scan scan = null;
|
||||
ResultScanner rs = null;
|
||||
StopWatch stopWatch = new StopWatch();
|
||||
for (HColumnDescriptor column : tableDesc.getColumnFamilies()) {
|
||||
for (ColumnFamilyDescriptor column : tableDesc.getColumnFamilies()) {
|
||||
stopWatch.reset();
|
||||
startKey = region.getStartKey();
|
||||
// Can't do a get on empty start row so do a Scan of first element if any instead.
|
||||
|
@ -439,17 +441,17 @@ public final class Canary implements Tool {
|
|||
*/
|
||||
private Void write() {
|
||||
Table table = null;
|
||||
HTableDescriptor tableDesc = null;
|
||||
TableDescriptor tableDesc = null;
|
||||
try {
|
||||
table = connection.getTable(region.getTable());
|
||||
tableDesc = table.getTableDescriptor();
|
||||
tableDesc = table.getDescriptor();
|
||||
byte[] rowToCheck = region.getStartKey();
|
||||
if (rowToCheck.length == 0) {
|
||||
rowToCheck = new byte[]{0x0};
|
||||
}
|
||||
int writeValueSize =
|
||||
connection.getConfiguration().getInt(HConstants.HBASE_CANARY_WRITE_VALUE_SIZE_KEY, 10);
|
||||
for (HColumnDescriptor column : tableDesc.getColumnFamilies()) {
|
||||
for (ColumnFamilyDescriptor column : tableDesc.getColumnFamilies()) {
|
||||
Put put = new Put(rowToCheck);
|
||||
byte[] value = new byte[writeValueSize];
|
||||
Bytes.random(value);
|
||||
|
|
|
@ -43,12 +43,12 @@ import org.apache.hadoop.fs.FSDataInputStream;
|
|||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.ClusterStatus;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
|
@ -66,6 +66,8 @@ import com.google.common.base.Preconditions;
|
|||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
|
||||
/**
|
||||
* The {@link RegionSplitter} class provides several utilities to help in the
|
||||
|
@ -378,16 +380,16 @@ public class RegionSplitter {
|
|||
LOG.debug("Creating table " + tableName + " with " + columnFamilies.length
|
||||
+ " column families. Presplitting to " + splitCount + " regions");
|
||||
|
||||
HTableDescriptor desc = new HTableDescriptor(tableName);
|
||||
TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
|
||||
for (String cf : columnFamilies) {
|
||||
desc.addFamily(new HColumnDescriptor(Bytes.toBytes(cf)));
|
||||
builder.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(cf)).build());
|
||||
}
|
||||
try (Connection connection = ConnectionFactory.createConnection(conf)) {
|
||||
Admin admin = connection.getAdmin();
|
||||
try {
|
||||
Preconditions.checkArgument(!admin.tableExists(tableName),
|
||||
"Table already exists: " + tableName);
|
||||
admin.createTable(desc, splitAlgo.split(splitCount));
|
||||
admin.createTable(builder.build(), splitAlgo.split(splitCount));
|
||||
} finally {
|
||||
admin.close();
|
||||
}
|
||||
|
@ -684,9 +686,9 @@ public class RegionSplitter {
|
|||
FileSystem fs = tableDir.getFileSystem(connection.getConfiguration());
|
||||
// Clear the cache to forcibly refresh region information
|
||||
((ClusterConnection)connection).clearRegionCache();
|
||||
HTableDescriptor htd = null;
|
||||
TableDescriptor htd = null;
|
||||
try (Table table = connection.getTable(tableName)) {
|
||||
htd = table.getTableDescriptor();
|
||||
htd = table.getDescriptor();
|
||||
}
|
||||
try (RegionLocator regionLocator = connection.getRegionLocator(tableName)) {
|
||||
|
||||
|
@ -725,7 +727,7 @@ public class RegionSplitter {
|
|||
|
||||
// Check every Column Family for that region -- check does not have references.
|
||||
boolean refFound = false;
|
||||
for (HColumnDescriptor c : htd.getFamilies()) {
|
||||
for (ColumnFamilyDescriptor c : htd.getColumnFamilies()) {
|
||||
if ((refFound = regionFs.hasReferences(c.getNameAsString()))) {
|
||||
break;
|
||||
}
|
||||
|
|
|
@ -97,7 +97,7 @@ public class ServerRegionReplicaUtil extends RegionReplicaUtil {
|
|||
* @return whether the replica is read only
|
||||
*/
|
||||
public static boolean isReadOnly(HRegion region) {
|
||||
return region.getTableDesc().isReadOnly()
|
||||
return region.getTableDescriptor().isReadOnly()
|
||||
|| !isDefaultReplica(region.getRegionInfo());
|
||||
}
|
||||
|
||||
|
|
|
@ -76,6 +76,7 @@ import org.apache.hadoop.hbase.client.Result;
|
|||
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.fs.HFileSystem;
|
||||
import org.apache.hadoop.hbase.io.compress.Compression;
|
||||
|
@ -137,6 +138,9 @@ import org.apache.zookeeper.ZooKeeper;
|
|||
import org.apache.zookeeper.ZooKeeper.States;
|
||||
|
||||
import edu.umd.cs.findbugs.annotations.Nullable;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
|
||||
/**
|
||||
* Facility for testing HBase. Replacement for
|
||||
|
@ -1360,7 +1364,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
* Create a table.
|
||||
* @param tableName
|
||||
* @param family
|
||||
* @return An HTable instance for the created table.
|
||||
* @return A Table instance for the created table.
|
||||
* @throws IOException
|
||||
*/
|
||||
public Table createTable(TableName tableName, String family)
|
||||
|
@ -1372,7 +1376,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
* Create a table.
|
||||
* @param tableName
|
||||
* @param families
|
||||
* @return An HTable instance for the created table.
|
||||
* @return A Table instance for the created table.
|
||||
* @throws IOException
|
||||
*/
|
||||
public Table createTable(TableName tableName, String[] families)
|
||||
|
@ -1388,7 +1392,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
* Create a table.
|
||||
* @param tableName
|
||||
* @param family
|
||||
* @return An HTable instance for the created table.
|
||||
* @return A Table instance for the created table.
|
||||
* @throws IOException
|
||||
*/
|
||||
public Table createTable(TableName tableName, byte[] family)
|
||||
|
@ -1401,7 +1405,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
* @param tableName
|
||||
* @param family
|
||||
* @param numRegions
|
||||
* @return An HTable instance for the created table.
|
||||
* @return A Table instance for the created table.
|
||||
* @throws IOException
|
||||
*/
|
||||
public Table createMultiRegionTable(TableName tableName, byte[] family, int numRegions)
|
||||
|
@ -1418,7 +1422,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
* Create a table.
|
||||
* @param tableName
|
||||
* @param families
|
||||
* @return An HTable instance for the created table.
|
||||
* @return A Table instance for the created table.
|
||||
* @throws IOException
|
||||
*/
|
||||
public Table createTable(TableName tableName, byte[][] families)
|
||||
|
@ -1430,7 +1434,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
* Create a table with multiple regions.
|
||||
* @param tableName
|
||||
* @param families
|
||||
* @return An HTable instance for the created table.
|
||||
* @return A Table instance for the created table.
|
||||
* @throws IOException
|
||||
*/
|
||||
public Table createMultiRegionTable(TableName tableName, byte[][] families) throws IOException {
|
||||
|
@ -1442,7 +1446,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
* @param tableName
|
||||
* @param families
|
||||
* @param splitKeys
|
||||
* @return An HTable instance for the created table.
|
||||
* @return A Table instance for the created table.
|
||||
* @throws IOException
|
||||
*/
|
||||
public Table createTable(TableName tableName, byte[][] families, byte[][] splitKeys)
|
||||
|
@ -1467,10 +1471,26 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
* @param htd
|
||||
* @param families
|
||||
* @param c Configuration to use
|
||||
* @return An HTable instance for the created table.
|
||||
* @return A Table instance for the created table.
|
||||
* @throws IOException
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version.
|
||||
* use {@link #createTable(TableDescriptor, byte[][], Configuration)}
|
||||
*/
|
||||
@Deprecated
|
||||
public Table createTable(HTableDescriptor htd, byte[][] families, Configuration c)
|
||||
throws IOException {
|
||||
return createTable((TableDescriptor) htd, families, c);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a table.
|
||||
* @param htd
|
||||
* @param families
|
||||
* @param c Configuration to use
|
||||
* @return A Table instance for the created table.
|
||||
* @throws IOException
|
||||
*/
|
||||
public Table createTable(HTableDescriptor htd, byte[][] families, Configuration c)
|
||||
public Table createTable(TableDescriptor htd, byte[][] families, Configuration c)
|
||||
throws IOException {
|
||||
return createTable(htd, families, null, c);
|
||||
}
|
||||
|
@ -1481,34 +1501,68 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
* @param families
|
||||
* @param splitKeys
|
||||
* @param c Configuration to use
|
||||
* @return An HTable instance for the created table.
|
||||
* @return A Table instance for the created table.
|
||||
* @throws IOException
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version.
|
||||
* use {@link #createTable(TableDescriptor, byte[][], byte[][], Configuration)}
|
||||
*/
|
||||
@Deprecated
|
||||
public Table createTable(HTableDescriptor htd, byte[][] families, byte[][] splitKeys,
|
||||
Configuration c) throws IOException {
|
||||
return createTable((TableDescriptor) htd, families, splitKeys, c);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a table.
|
||||
* @param htd
|
||||
* @param families
|
||||
* @param splitKeys
|
||||
* @param c Configuration to use
|
||||
* @return A Table instance for the created table.
|
||||
* @throws IOException
|
||||
*/
|
||||
public Table createTable(TableDescriptor htd, byte[][] families, byte[][] splitKeys,
|
||||
Configuration c) throws IOException {
|
||||
TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(htd);
|
||||
for (byte[] family : families) {
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(family);
|
||||
// Disable blooms (they are on by default as of 0.95) but we disable them here because
|
||||
// tests have hard coded counts of what to expect in block cache, etc., and blooms being
|
||||
// on is interfering.
|
||||
hcd.setBloomFilterType(BloomType.NONE);
|
||||
htd.addFamily(hcd);
|
||||
builder.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(family)
|
||||
.setBloomFilterType(BloomType.NONE)
|
||||
.build());
|
||||
}
|
||||
getAdmin().createTable(htd, splitKeys);
|
||||
TableDescriptor td = builder.build();
|
||||
getAdmin().createTable(td, splitKeys);
|
||||
// HBaseAdmin only waits for regions to appear in hbase:meta
|
||||
// we should wait until they are assigned
|
||||
waitUntilAllRegionsAssigned(htd.getTableName());
|
||||
return getConnection().getTable(htd.getTableName());
|
||||
waitUntilAllRegionsAssigned(td.getTableName());
|
||||
return getConnection().getTable(td.getTableName());
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a table.
|
||||
* @param htd
|
||||
* @param splitRows
|
||||
* @return An HTable instance for the created table.
|
||||
* @return A Table instance for the created table.
|
||||
* @throws IOException
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version.
|
||||
* use {@link #createTable(TableDescriptor, byte[][])}
|
||||
*/
|
||||
@Deprecated
|
||||
public Table createTable(HTableDescriptor htd, byte[][] splitRows)
|
||||
throws IOException {
|
||||
return createTable((TableDescriptor) htd, splitRows);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a table.
|
||||
* @param htd
|
||||
* @param splitRows
|
||||
* @return A Table instance for the created table.
|
||||
* @throws IOException
|
||||
*/
|
||||
public Table createTable(HTableDescriptor htd, byte[][] splitRows)
|
||||
public Table createTable(TableDescriptor htd, byte[][] splitRows)
|
||||
throws IOException {
|
||||
getAdmin().createTable(htd, splitRows);
|
||||
// HBaseAdmin only waits for regions to appear in hbase:meta
|
||||
|
@ -1523,7 +1577,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
* @param families
|
||||
* @param splitKeys
|
||||
* @param c Configuration to use
|
||||
* @return An HTable instance for the created table.
|
||||
* @return A Table instance for the created table.
|
||||
* @throws IOException
|
||||
*/
|
||||
public Table createTable(TableName tableName, byte[][] families, byte[][] splitKeys,
|
||||
|
@ -1536,7 +1590,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
* @param tableName
|
||||
* @param family
|
||||
* @param numVersions
|
||||
* @return An HTable instance for the created table.
|
||||
* @return A Table instance for the created table.
|
||||
* @throws IOException
|
||||
*/
|
||||
public Table createTable(TableName tableName, byte[] family, int numVersions)
|
||||
|
@ -1549,7 +1603,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
* @param tableName
|
||||
* @param families
|
||||
* @param numVersions
|
||||
* @return An HTable instance for the created table.
|
||||
* @return A Table instance for the created table.
|
||||
* @throws IOException
|
||||
*/
|
||||
public Table createTable(TableName tableName, byte[][] families, int numVersions)
|
||||
|
@ -1563,7 +1617,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
* @param families
|
||||
* @param numVersions
|
||||
* @param splitKeys
|
||||
* @return An HTable instance for the created table.
|
||||
* @return A Table instance for the created table.
|
||||
* @throws IOException
|
||||
*/
|
||||
public Table createTable(TableName tableName, byte[][] families, int numVersions,
|
||||
|
@ -1585,7 +1639,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
* @param tableName
|
||||
* @param families
|
||||
* @param numVersions
|
||||
* @return An HTable instance for the created table.
|
||||
* @return A Table instance for the created table.
|
||||
* @throws IOException
|
||||
*/
|
||||
public Table createMultiRegionTable(TableName tableName, byte[][] families, int numVersions)
|
||||
|
@ -1599,7 +1653,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
* @param families
|
||||
* @param numVersions
|
||||
* @param blockSize
|
||||
* @return An HTable instance for the created table.
|
||||
* @return A Table instance for the created table.
|
||||
* @throws IOException
|
||||
*/
|
||||
public Table createTable(TableName tableName, byte[][] families,
|
||||
|
@ -1642,7 +1696,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
* @param tableName
|
||||
* @param families
|
||||
* @param numVersions
|
||||
* @return An HTable instance for the created table.
|
||||
* @return A Table instance for the created table.
|
||||
* @throws IOException
|
||||
*/
|
||||
public Table createTable(TableName tableName, byte[][] families,
|
||||
|
@ -1668,7 +1722,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
* @param tableName
|
||||
* @param family
|
||||
* @param splitRows
|
||||
* @return An HTable instance for the created table.
|
||||
* @return A Table instance for the created table.
|
||||
* @throws IOException
|
||||
*/
|
||||
public Table createTable(TableName tableName, byte[] family, byte[][] splitRows)
|
||||
|
@ -1687,7 +1741,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
* Create a table with multiple regions.
|
||||
* @param tableName
|
||||
* @param family
|
||||
* @return An HTable instance for the created table.
|
||||
* @return A Table instance for the created table.
|
||||
* @throws IOException
|
||||
*/
|
||||
public Table createMultiRegionTable(TableName tableName, byte[] family) throws IOException {
|
||||
|
@ -1696,11 +1750,22 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
|
||||
/**
|
||||
* Modify a table, synchronous. Waiting logic similar to that of {@code admin.rb#alter_status}.
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version.
|
||||
* use {@link #modifyTableSync(Admin, TableDescriptor)}
|
||||
*/
|
||||
@Deprecated
|
||||
@SuppressWarnings("serial")
|
||||
public static void modifyTableSync(Admin admin, HTableDescriptor desc)
|
||||
throws IOException, InterruptedException {
|
||||
admin.modifyTable(desc.getTableName(), desc);
|
||||
modifyTableSync(admin, (TableDescriptor) desc);
|
||||
}
|
||||
/**
|
||||
* Modify a table, synchronous. Waiting logic similar to that of {@code admin.rb#alter_status}.
|
||||
*/
|
||||
@SuppressWarnings("serial")
|
||||
public static void modifyTableSync(Admin admin, TableDescriptor desc)
|
||||
throws IOException, InterruptedException {
|
||||
admin.modifyTable(desc);
|
||||
Pair<Integer, Integer> status = new Pair<Integer, Integer>() {{
|
||||
setFirst(0);
|
||||
setSecond(0);
|
||||
|
@ -1840,8 +1905,24 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
* @param endKey
|
||||
* @return
|
||||
* @throws IOException
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version.
|
||||
* use {@link #createLocalHRegion(TableDescriptor, byte[], byte[])}
|
||||
*/
|
||||
@Deprecated
|
||||
public HRegion createLocalHRegion(HTableDescriptor desc, byte [] startKey,
|
||||
byte [] endKey) throws IOException {
|
||||
return createLocalHRegion((TableDescriptor) desc, startKey, endKey);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create an HRegion that writes to the local tmp dirs
|
||||
* @param desc
|
||||
* @param startKey
|
||||
* @param endKey
|
||||
* @return
|
||||
* @throws IOException
|
||||
*/
|
||||
public HRegion createLocalHRegion(TableDescriptor desc, byte [] startKey,
|
||||
byte [] endKey)
|
||||
throws IOException {
|
||||
HRegionInfo hri = new HRegionInfo(desc.getTableName(), startKey, endKey);
|
||||
|
@ -1851,8 +1932,19 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
/**
|
||||
* Create an HRegion that writes to the local tmp dirs. Creates the WAL for you. Be sure to call
|
||||
* {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} when you're finished with it.
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version.
|
||||
* use {@link #createLocalHRegion(HRegionInfo, TableDescriptor)}
|
||||
*/
|
||||
@Deprecated
|
||||
public HRegion createLocalHRegion(HRegionInfo info, HTableDescriptor desc) throws IOException {
|
||||
return createLocalHRegion(info, (TableDescriptor) desc);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create an HRegion that writes to the local tmp dirs. Creates the WAL for you. Be sure to call
|
||||
* {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} when you're finished with it.
|
||||
*/
|
||||
public HRegion createLocalHRegion(HRegionInfo info, TableDescriptor desc) throws IOException {
|
||||
return createRegionAndWAL(info, getDataTestDir(), getConfiguration(), desc);
|
||||
}
|
||||
|
||||
|
@ -1863,9 +1955,25 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
* @param wal wal for this region.
|
||||
* @return created hregion
|
||||
* @throws IOException
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version.
|
||||
* use {@link #createLocalHRegion(HRegionInfo, TableDescriptor, WAL)}
|
||||
*/
|
||||
@Deprecated
|
||||
public HRegion createLocalHRegion(HRegionInfo info, HTableDescriptor desc, WAL wal)
|
||||
throws IOException {
|
||||
return createLocalHRegion(info, (TableDescriptor) desc, wal);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create an HRegion that writes to the local tmp dirs with specified wal
|
||||
* @param info regioninfo
|
||||
* @param desc table descriptor
|
||||
* @param wal wal for this region.
|
||||
* @return created hregion
|
||||
* @throws IOException
|
||||
*/
|
||||
public HRegion createLocalHRegion(HRegionInfo info, TableDescriptor desc, WAL wal)
|
||||
throws IOException {
|
||||
return HRegion.createHRegion(info, getDataTestDir(), getConfiguration(), desc, wal);
|
||||
}
|
||||
|
||||
|
@ -2366,6 +2474,23 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
Bytes.toBytes("xxx"), Bytes.toBytes("yyy"), Bytes.toBytes("zzz")
|
||||
};
|
||||
|
||||
/**
|
||||
* Create rows in hbase:meta for regions of the specified table with the specified
|
||||
* start keys. The first startKey should be a 0 length byte array if you
|
||||
* want to form a proper range of regions.
|
||||
* @param conf
|
||||
* @param htd
|
||||
* @param startKeys
|
||||
* @return list of region info for regions added to meta
|
||||
* @throws IOException
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version.
|
||||
* use {@link #createMultiRegionsInMeta(Configuration, TableDescriptor, byte[][])}
|
||||
*/
|
||||
@Deprecated
|
||||
public List<HRegionInfo> createMultiRegionsInMeta(final Configuration conf,
|
||||
final HTableDescriptor htd, byte [][] startKeys) throws IOException {
|
||||
return createMultiRegionsInMeta(conf, (TableDescriptor) htd, startKeys);
|
||||
}
|
||||
/**
|
||||
* Create rows in hbase:meta for regions of the specified table with the specified
|
||||
* start keys. The first startKey should be a 0 length byte array if you
|
||||
|
@ -2377,7 +2502,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
* @throws IOException
|
||||
*/
|
||||
public List<HRegionInfo> createMultiRegionsInMeta(final Configuration conf,
|
||||
final HTableDescriptor htd, byte [][] startKeys)
|
||||
final TableDescriptor htd, byte [][] startKeys)
|
||||
throws IOException {
|
||||
Table meta = getConnection().getTable(TableName.META_TABLE_NAME);
|
||||
Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR);
|
||||
|
@ -2415,19 +2540,44 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
/**
|
||||
* Create a region with it's own WAL. Be sure to call
|
||||
* {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} to clean up all resources.
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version.
|
||||
* use {@link #createRegionAndWAL(HRegionInfo, Path, Configuration, TableDescriptor)}
|
||||
*/
|
||||
@Deprecated
|
||||
public static HRegion createRegionAndWAL(final HRegionInfo info, final Path rootDir,
|
||||
final Configuration conf, final HTableDescriptor htd) throws IOException {
|
||||
return createRegionAndWAL(info, rootDir, conf, htd, true);
|
||||
return createRegionAndWAL(info, rootDir, conf, (TableDescriptor) htd);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a region with it's own WAL. Be sure to call
|
||||
* {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} to clean up all resources.
|
||||
*/
|
||||
public static HRegion createRegionAndWAL(final HRegionInfo info, final Path rootDir,
|
||||
final Configuration conf, final TableDescriptor htd) throws IOException {
|
||||
return createRegionAndWAL(info, rootDir, conf, htd, true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a region with it's own WAL. Be sure to call
|
||||
* {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} to clean up all resources.
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version.
|
||||
* use {@link #createRegionAndWAL(HRegionInfo, Path, Configuration, TableDescriptor, boolean)}
|
||||
*/
|
||||
@Deprecated
|
||||
public static HRegion createRegionAndWAL(final HRegionInfo info, final Path rootDir,
|
||||
final Configuration conf, final HTableDescriptor htd, boolean initialize)
|
||||
throws IOException {
|
||||
return createRegionAndWAL(info, rootDir, conf, (TableDescriptor) htd, initialize);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a region with it's own WAL. Be sure to call
|
||||
* {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} to clean up all resources.
|
||||
*/
|
||||
public static HRegion createRegionAndWAL(final HRegionInfo info, final Path rootDir,
|
||||
final Configuration conf, final TableDescriptor htd, boolean initialize)
|
||||
throws IOException {
|
||||
ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
|
||||
WAL wal = createWal(conf, rootDir, info);
|
||||
return HRegion.createHRegion(info, rootDir, conf, htd, wal, initialize);
|
||||
|
@ -2738,7 +2888,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
* [2] https://issues.apache.org/jira/browse/ZOOKEEPER-1105
|
||||
*
|
||||
* @param nodeZK - the ZK watcher to expire
|
||||
* @param checkStatus - true to check if we can create an HTable with the
|
||||
* @param checkStatus - true to check if we can create a Table with the
|
||||
* current configuration.
|
||||
*/
|
||||
public void expireSession(ZooKeeperWatcher nodeZK, boolean checkStatus)
|
||||
|
@ -3435,7 +3585,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
Get get) throws IOException {
|
||||
Scan scan = new Scan(get);
|
||||
InternalScanner scanner = (InternalScanner) store.getScanner(scan,
|
||||
scan.getFamilyMap().get(store.getFamily().getName()),
|
||||
scan.getFamilyMap().get(store.getColumnFamilyDescriptor().getName()),
|
||||
// originally MultiVersionConcurrencyControl.resetThreadReadPoint() was called to set
|
||||
// readpoint 0.
|
||||
0);
|
||||
|
@ -3480,7 +3630,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
) throws IOException {
|
||||
Get get = new Get(row);
|
||||
Map<byte[], NavigableSet<byte[]>> s = get.getFamilyMap();
|
||||
s.put(store.getFamily().getName(), columns);
|
||||
s.put(store.getColumnFamilyDescriptor().getName(), columns);
|
||||
|
||||
return getFromStoreFile(store,get);
|
||||
}
|
||||
|
@ -3802,9 +3952,24 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
* Creates a pre-split table for load testing. If the table already exists,
|
||||
* logs a warning and continues.
|
||||
* @return the number of regions the table was split into
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version.
|
||||
* use {@link #createPreSplitLoadTestTable(Configuration,
|
||||
* TableDescriptor, ColumnFamilyDescriptor)}
|
||||
*/
|
||||
@Deprecated
|
||||
public static int createPreSplitLoadTestTable(Configuration conf,
|
||||
HTableDescriptor desc, HColumnDescriptor hcd) throws IOException {
|
||||
return createPreSplitLoadTestTable(conf, (TableDescriptor) desc,
|
||||
(ColumnFamilyDescriptor) hcd);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a pre-split table for load testing. If the table already exists,
|
||||
* logs a warning and continues.
|
||||
* @return the number of regions the table was split into
|
||||
*/
|
||||
public static int createPreSplitLoadTestTable(Configuration conf,
|
||||
TableDescriptor desc, ColumnFamilyDescriptor hcd) throws IOException {
|
||||
return createPreSplitLoadTestTable(conf, desc, hcd, DEFAULT_REGIONS_PER_SERVER);
|
||||
}
|
||||
|
||||
|
@ -3812,11 +3977,15 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
* Creates a pre-split table for load testing. If the table already exists,
|
||||
* logs a warning and continues.
|
||||
* @return the number of regions the table was split into
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version.
|
||||
* use {@link #createPreSplitLoadTestTable(Configuration,
|
||||
* TableDescriptor, ColumnFamilyDescriptor, int)}
|
||||
*/
|
||||
@Deprecated
|
||||
public static int createPreSplitLoadTestTable(Configuration conf,
|
||||
HTableDescriptor desc, HColumnDescriptor hcd, int numRegionsPerServer) throws IOException {
|
||||
return createPreSplitLoadTestTable(conf, desc, new HColumnDescriptor[] {hcd},
|
||||
numRegionsPerServer);
|
||||
return createPreSplitLoadTestTable(conf, (TableDescriptor) desc,
|
||||
(ColumnFamilyDescriptor) hcd, numRegionsPerServer);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -3825,8 +3994,35 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
* @return the number of regions the table was split into
|
||||
*/
|
||||
public static int createPreSplitLoadTestTable(Configuration conf,
|
||||
HTableDescriptor desc, HColumnDescriptor[] hcds, int numRegionsPerServer) throws IOException {
|
||||
TableDescriptor desc, ColumnFamilyDescriptor hcd, int numRegionsPerServer) throws IOException {
|
||||
return createPreSplitLoadTestTable(conf, desc, new ColumnFamilyDescriptor[] {hcd},
|
||||
numRegionsPerServer);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a pre-split table for load testing. If the table already exists,
|
||||
* logs a warning and continues.
|
||||
* @return the number of regions the table was split into
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version.
|
||||
* use {@link #createPreSplitLoadTestTable(Configuration,
|
||||
* TableDescriptor, ColumnFamilyDescriptor[], int)}
|
||||
*/
|
||||
@Deprecated
|
||||
public static int createPreSplitLoadTestTable(Configuration conf,
|
||||
HTableDescriptor desc, HColumnDescriptor[] hcds,
|
||||
int numRegionsPerServer) throws IOException {
|
||||
return createPreSplitLoadTestTable(conf, (TableDescriptor) desc,
|
||||
(ColumnFamilyDescriptor[]) hcds, numRegionsPerServer);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a pre-split table for load testing. If the table already exists,
|
||||
* logs a warning and continues.
|
||||
* @return the number of regions the table was split into
|
||||
*/
|
||||
public static int createPreSplitLoadTestTable(Configuration conf,
|
||||
TableDescriptor desc, ColumnFamilyDescriptor[] hcds,
|
||||
int numRegionsPerServer) throws IOException {
|
||||
return createPreSplitLoadTestTable(conf, desc, hcds,
|
||||
new RegionSplitter.HexStringSplit(), numRegionsPerServer);
|
||||
}
|
||||
|
@ -3837,13 +4033,15 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
* @return the number of regions the table was split into
|
||||
*/
|
||||
public static int createPreSplitLoadTestTable(Configuration conf,
|
||||
HTableDescriptor desc, HColumnDescriptor[] hcds,
|
||||
TableDescriptor td, ColumnFamilyDescriptor[] cds,
|
||||
SplitAlgorithm splitter, int numRegionsPerServer) throws IOException {
|
||||
for (HColumnDescriptor hcd : hcds) {
|
||||
if (!desc.hasFamily(hcd.getName())) {
|
||||
desc.addFamily(hcd);
|
||||
TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(td);
|
||||
for (ColumnFamilyDescriptor cd : cds) {
|
||||
if (!td.hasColumnFamily(cd.getName())) {
|
||||
builder.addColumnFamily(cd);
|
||||
}
|
||||
}
|
||||
td = builder.build();
|
||||
int totalNumberOfRegions = 0;
|
||||
Connection unmanagedConnection = ConnectionFactory.createConnection(conf);
|
||||
Admin admin = unmanagedConnection.getAdmin();
|
||||
|
@ -3865,12 +4063,12 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
byte[][] splits = splitter.split(
|
||||
totalNumberOfRegions);
|
||||
|
||||
admin.createTable(desc, splits);
|
||||
admin.createTable(td, splits);
|
||||
} catch (MasterNotRunningException e) {
|
||||
LOG.error("Master not running", e);
|
||||
throw new IOException(e);
|
||||
} catch (TableExistsException e) {
|
||||
LOG.warn("Table " + desc.getTableName() +
|
||||
LOG.warn("Table " + td.getTableName() +
|
||||
" already exists, continuing");
|
||||
} finally {
|
||||
admin.close();
|
||||
|
@ -3940,13 +4138,25 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
+ " on server " + server);
|
||||
}
|
||||
|
||||
public HRegion createTestRegion(String tableName, HColumnDescriptor hcd)
|
||||
/**
|
||||
* @deprecated since 2.0 version and will be removed in 3.0 version.
|
||||
* use {@link #createTestRegion(String, ColumnFamilyDescriptor)}
|
||||
*/
|
||||
@Deprecated
|
||||
public HRegion createTestRegion(String tableName, HColumnDescriptor cd)
|
||||
throws IOException {
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
|
||||
htd.addFamily(hcd);
|
||||
return createTestRegion(tableName, (ColumnFamilyDescriptor) cd);
|
||||
}
|
||||
|
||||
public HRegion createTestRegion(String tableName, ColumnFamilyDescriptor cd)
|
||||
throws IOException {
|
||||
TableDescriptor td
|
||||
= TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName))
|
||||
.addColumnFamily(cd)
|
||||
.build();
|
||||
HRegionInfo info =
|
||||
new HRegionInfo(TableName.valueOf(tableName), null, null, false);
|
||||
return createRegionAndWAL(info, getDataTestDir(), getConfiguration(), htd);
|
||||
return createRegionAndWAL(info, getDataTestDir(), getConfiguration(), td);
|
||||
}
|
||||
|
||||
public void setFileSystemURI(String fsURI) {
|
||||
|
@ -4049,13 +4259,13 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
|||
boolean tableAvailable = getAdmin().isTableAvailable(tableName);
|
||||
if (tableAvailable) {
|
||||
try (Table table = getConnection().getTable(tableName)) {
|
||||
HTableDescriptor htd = table.getTableDescriptor();
|
||||
TableDescriptor htd = table.getDescriptor();
|
||||
for (HRegionLocation loc : getConnection().getRegionLocator(tableName)
|
||||
.getAllRegionLocations()) {
|
||||
Scan scan = new Scan().withStartRow(loc.getRegionInfo().getStartKey())
|
||||
.withStopRow(loc.getRegionInfo().getEndKey()).setOneRowLimit()
|
||||
.setMaxResultsPerColumnFamily(1).setCacheBlocks(false);
|
||||
for (byte[] family : htd.getFamiliesKeys()) {
|
||||
for (byte[] family : htd.getColumnFamilyNames()) {
|
||||
scan.addFamily(family);
|
||||
}
|
||||
try (ResultScanner scanner = table.getScanner(scan)) {
|
||||
|
|
|
@ -629,7 +629,7 @@ public class MiniHBaseCluster extends HBaseCluster {
|
|||
for (JVMClusterUtil.RegionServerThread t:
|
||||
this.hbaseCluster.getRegionServers()) {
|
||||
for(Region r: t.getRegionServer().getOnlineRegionsLocalContext()) {
|
||||
if(r.getTableDesc().getTableName().equals(tableName)) {
|
||||
if(r.getTableDescriptor().getTableName().equals(tableName)) {
|
||||
r.flush(true);
|
||||
}
|
||||
}
|
||||
|
@ -657,7 +657,7 @@ public class MiniHBaseCluster extends HBaseCluster {
|
|||
for (JVMClusterUtil.RegionServerThread t:
|
||||
this.hbaseCluster.getRegionServers()) {
|
||||
for(Region r: t.getRegionServer().getOnlineRegionsLocalContext()) {
|
||||
if(r.getTableDesc().getTableName().equals(tableName)) {
|
||||
if(r.getTableDescriptor().getTableName().equals(tableName)) {
|
||||
r.compact(major);
|
||||
}
|
||||
}
|
||||
|
@ -703,7 +703,7 @@ public class MiniHBaseCluster extends HBaseCluster {
|
|||
for (JVMClusterUtil.RegionServerThread rst : getRegionServerThreads()) {
|
||||
HRegionServer hrs = rst.getRegionServer();
|
||||
for (Region region : hrs.getOnlineRegionsLocalContext()) {
|
||||
if (region.getTableDesc().getTableName().equals(tableName)) {
|
||||
if (region.getTableDescriptor().getTableName().equals(tableName)) {
|
||||
ret.add((HRegion)region);
|
||||
}
|
||||
}
|
||||
|
@ -801,7 +801,7 @@ public class MiniHBaseCluster extends HBaseCluster {
|
|||
for (JVMClusterUtil.RegionServerThread rst : getRegionServerThreads()) {
|
||||
HRegionServer hrs = rst.getRegionServer();
|
||||
for (Region region : hrs.getOnlineRegions(tableName)) {
|
||||
if (region.getTableDesc().getTableName().equals(tableName)) {
|
||||
if (region.getTableDescriptor().getTableName().equals(tableName)) {
|
||||
ret.add((HRegion)region);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -33,7 +33,9 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.regionserver.CompactingMemStore;
|
||||
import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
|
@ -98,7 +100,7 @@ public class TestIOFencing {
|
|||
@SuppressWarnings("deprecation")
|
||||
public CompactionBlockerRegion(Path tableDir, WAL log,
|
||||
FileSystem fs, Configuration confParam, HRegionInfo info,
|
||||
HTableDescriptor htd, RegionServerServices rsServices) {
|
||||
TableDescriptor htd, RegionServerServices rsServices) {
|
||||
super(tableDir, log, fs, confParam, info, htd, rsServices);
|
||||
}
|
||||
|
||||
|
@ -158,7 +160,7 @@ public class TestIOFencing {
|
|||
|
||||
public BlockCompactionsInPrepRegion(Path tableDir, WAL log,
|
||||
FileSystem fs, Configuration confParam, HRegionInfo info,
|
||||
HTableDescriptor htd, RegionServerServices rsServices) {
|
||||
TableDescriptor htd, RegionServerServices rsServices) {
|
||||
super(tableDir, log, fs, confParam, info, htd, rsServices);
|
||||
}
|
||||
@Override
|
||||
|
@ -181,18 +183,18 @@ public class TestIOFencing {
|
|||
public static class BlockCompactionsInCompletionRegion extends CompactionBlockerRegion {
|
||||
public BlockCompactionsInCompletionRegion(Path tableDir, WAL log,
|
||||
FileSystem fs, Configuration confParam, HRegionInfo info,
|
||||
HTableDescriptor htd, RegionServerServices rsServices) {
|
||||
TableDescriptor htd, RegionServerServices rsServices) {
|
||||
super(tableDir, log, fs, confParam, info, htd, rsServices);
|
||||
}
|
||||
@Override
|
||||
protected HStore instantiateHStore(final HColumnDescriptor family) throws IOException {
|
||||
protected HStore instantiateHStore(final ColumnFamilyDescriptor family) throws IOException {
|
||||
return new BlockCompactionsInCompletionHStore(this, family, this.conf);
|
||||
}
|
||||
}
|
||||
|
||||
public static class BlockCompactionsInCompletionHStore extends HStore {
|
||||
CompactionBlockerRegion r;
|
||||
protected BlockCompactionsInCompletionHStore(HRegion region, HColumnDescriptor family,
|
||||
protected BlockCompactionsInCompletionHStore(HRegion region, ColumnFamilyDescriptor family,
|
||||
Configuration confParam) throws IOException {
|
||||
super(region, family, confParam);
|
||||
r = (CompactionBlockerRegion) region;
|
||||
|
|
|
@ -256,7 +256,7 @@ public class TestHFileArchiving {
|
|||
clearArchiveDirectory();
|
||||
|
||||
// then get the current store files
|
||||
byte[][]columns = region.getTableDesc().getFamiliesKeys().toArray(new byte[0][]);
|
||||
byte[][]columns = region.getTableDescriptor().getColumnFamilyNames().toArray(new byte[0][]);
|
||||
List<String> storeFiles = region.getStoreFileList(columns);
|
||||
|
||||
// then delete the table so the hfiles get archived
|
||||
|
@ -335,7 +335,7 @@ public class TestHFileArchiving {
|
|||
clearArchiveDirectory();
|
||||
|
||||
// then get the current store files
|
||||
byte[][]columns = region.getTableDesc().getFamiliesKeys().toArray(new byte[0][]);
|
||||
byte[][]columns = region.getTableDescriptor().getColumnFamilyNames().toArray(new byte[0][]);
|
||||
List<String> storeFiles = region.getStoreFileList(columns);
|
||||
|
||||
// then delete the table so the hfiles get archived
|
||||
|
|
|
@ -181,7 +181,7 @@ public class TestIncrementalBackup extends TestBackupBase {
|
|||
false, tablesRestoreIncMultiple, tablesMapIncMultiple, true));
|
||||
|
||||
hTable = (HTable) conn.getTable(table1_restore);
|
||||
LOG.debug("After incremental restore: " + hTable.getTableDescriptor());
|
||||
LOG.debug("After incremental restore: " + hTable.getDescriptor());
|
||||
LOG.debug("f1 has " + TEST_UTIL.countRows(hTable, famName) + " rows");
|
||||
Assert.assertEquals(TEST_UTIL.countRows(hTable, famName), NB_ROWS_IN_BATCH + ADD_ROWS);
|
||||
LOG.debug("f2 has " + TEST_UTIL.countRows(hTable, fam2Name) + " rows");
|
||||
|
|
|
@ -420,8 +420,8 @@ public class TestAdmin1 {
|
|||
htd.addFamily(fam3);
|
||||
this.admin.createTable(htd);
|
||||
Table table = TEST_UTIL.getConnection().getTable(htd.getTableName());
|
||||
HTableDescriptor confirmedHtd = table.getTableDescriptor();
|
||||
assertEquals(htd.compareTo(confirmedHtd), 0);
|
||||
TableDescriptor confirmedHtd = table.getDescriptor();
|
||||
assertEquals(TableDescriptor.COMPARATOR.compare(htd, confirmedHtd), 0);
|
||||
MetaTableAccessor.fullScanMetaAndPrint(TEST_UTIL.getConnection());
|
||||
table.close();
|
||||
}
|
||||
|
|
|
@ -270,7 +270,7 @@ public class TestAvoidCellReferencesIntoShippedBlocks {
|
|||
List<? extends KeyValueScanner> scanners, ScanType scanType, long earliestPutTs)
|
||||
throws IOException {
|
||||
Scan scan = new Scan();
|
||||
scan.setMaxVersions(store.getFamily().getMaxVersions());
|
||||
scan.setMaxVersions(store.getColumnFamilyDescriptor().getMaxVersions());
|
||||
return new CompactorStoreScanner(store, store.getScanInfo(), scan, scanners, scanType,
|
||||
store.getSmallestReadPoint(), earliestPutTs);
|
||||
}
|
||||
|
|
|
@ -260,7 +260,7 @@ public class TestMetaCache {
|
|||
HBaseProtos.RegionSpecifier regionSpec) throws ServiceException {
|
||||
try {
|
||||
return TABLE_NAME.equals(
|
||||
rpcServices.getRegion(regionSpec).getTableDesc().getTableName());
|
||||
rpcServices.getRegion(regionSpec).getTableDescriptor().getTableName());
|
||||
} catch (IOException ioe) {
|
||||
throw new ServiceException(ioe);
|
||||
}
|
||||
|
|
|
@ -302,11 +302,13 @@ public class TestReplicaWithCluster {
|
|||
|
||||
@Test (timeout=120000)
|
||||
public void testChangeTable() throws Exception {
|
||||
HTableDescriptor hdt = HTU.createTableDescriptor("testChangeTable");
|
||||
hdt.setRegionReplication(NB_SERVERS);
|
||||
hdt.addCoprocessor(SlowMeCopro.class.getName());
|
||||
Table table = HTU.createTable(hdt, new byte[][]{f}, null);
|
||||
|
||||
TableDescriptor td = TableDescriptorBuilder.newBuilder(TableName.valueOf("testChangeTable"))
|
||||
.setRegionReplication(NB_SERVERS)
|
||||
.addCoprocessor(SlowMeCopro.class.getName())
|
||||
.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(f).build())
|
||||
.build();
|
||||
HTU.getAdmin().createTable(td);
|
||||
Table table = HTU.getConnection().getTable(td.getTableName());
|
||||
// basic test: it should work.
|
||||
Put p = new Put(row);
|
||||
p.addColumn(f, row, row);
|
||||
|
@ -317,13 +319,14 @@ public class TestReplicaWithCluster {
|
|||
Assert.assertFalse(r.isStale());
|
||||
|
||||
// Add a CF, it should work.
|
||||
HTableDescriptor bHdt = HTU.getAdmin().getTableDescriptor(hdt.getTableName());
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(row);
|
||||
hdt.addFamily(hcd);
|
||||
HTU.getAdmin().disableTable(hdt.getTableName());
|
||||
HTU.getAdmin().modifyTable(hdt.getTableName(), hdt);
|
||||
HTU.getAdmin().enableTable(hdt.getTableName());
|
||||
HTableDescriptor nHdt = HTU.getAdmin().getTableDescriptor(hdt.getTableName());
|
||||
TableDescriptor bHdt = HTU.getAdmin().listTableDescriptor(td.getTableName());
|
||||
td = TableDescriptorBuilder.newBuilder(td)
|
||||
.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(row).build())
|
||||
.build();
|
||||
HTU.getAdmin().disableTable(td.getTableName());
|
||||
HTU.getAdmin().modifyTable(td);
|
||||
HTU.getAdmin().enableTable(td.getTableName());
|
||||
TableDescriptor nHdt = HTU.getAdmin().listTableDescriptor(td.getTableName());
|
||||
Assert.assertEquals("fams=" + Arrays.toString(nHdt.getColumnFamilies()),
|
||||
bHdt.getColumnFamilyCount() + 1, nHdt.getColumnFamilyCount());
|
||||
|
||||
|
@ -347,12 +350,12 @@ public class TestReplicaWithCluster {
|
|||
}
|
||||
|
||||
Admin admin = HTU.getAdmin();
|
||||
nHdt =admin.getTableDescriptor(hdt.getTableName());
|
||||
nHdt =admin.listTableDescriptor(td.getTableName());
|
||||
Assert.assertEquals("fams=" + Arrays.toString(nHdt.getColumnFamilies()),
|
||||
bHdt.getColumnFamilyCount() + 1, nHdt.getColumnFamilyCount());
|
||||
|
||||
admin.disableTable(hdt.getTableName());
|
||||
admin.deleteTable(hdt.getTableName());
|
||||
admin.disableTable(td.getTableName());
|
||||
admin.deleteTable(td.getTableName());
|
||||
admin.close();
|
||||
}
|
||||
|
||||
|
|
|
@ -348,7 +348,7 @@ public class SimpleRegionObserver implements RegionObserver {
|
|||
assertNotNull(e.getRegion());
|
||||
assertNotNull(get);
|
||||
assertNotNull(results);
|
||||
if (e.getRegion().getTableDesc().getTableName().equals(
|
||||
if (e.getRegion().getTableDescriptor().getTableName().equals(
|
||||
TestRegionObserverInterface.TEST_TABLE)) {
|
||||
boolean foundA = false;
|
||||
boolean foundB = false;
|
||||
|
@ -380,7 +380,7 @@ public class SimpleRegionObserver implements RegionObserver {
|
|||
assertNotNull(e);
|
||||
assertNotNull(e.getRegion());
|
||||
assertNotNull(familyMap);
|
||||
if (e.getRegion().getTableDesc().getTableName().equals(
|
||||
if (e.getRegion().getTableDescriptor().getTableName().equals(
|
||||
TestRegionObserverInterface.TEST_TABLE)) {
|
||||
List<Cell> cells = familyMap.get(TestRegionObserverInterface.A);
|
||||
assertNotNull(cells);
|
||||
|
@ -417,7 +417,7 @@ public class SimpleRegionObserver implements RegionObserver {
|
|||
assertNotNull(e.getRegion());
|
||||
assertNotNull(familyMap);
|
||||
List<Cell> cells = familyMap.get(TestRegionObserverInterface.A);
|
||||
if (e.getRegion().getTableDesc().getTableName().equals(
|
||||
if (e.getRegion().getTableDescriptor().getTableName().equals(
|
||||
TestRegionObserverInterface.TEST_TABLE)) {
|
||||
assertNotNull(cells);
|
||||
assertNotNull(cells.get(0));
|
||||
|
@ -615,7 +615,7 @@ public class SimpleRegionObserver implements RegionObserver {
|
|||
RegionCoprocessorEnvironment e = ctx.getEnvironment();
|
||||
assertNotNull(e);
|
||||
assertNotNull(e.getRegion());
|
||||
if (e.getRegion().getTableDesc().getTableName().equals(
|
||||
if (e.getRegion().getTableDescriptor().getTableName().equals(
|
||||
TestRegionObserverInterface.TEST_TABLE)) {
|
||||
assertNotNull(familyPaths);
|
||||
assertEquals(1,familyPaths.size());
|
||||
|
@ -634,7 +634,7 @@ public class SimpleRegionObserver implements RegionObserver {
|
|||
RegionCoprocessorEnvironment e = ctx.getEnvironment();
|
||||
assertNotNull(e);
|
||||
assertNotNull(e.getRegion());
|
||||
if (e.getRegion().getTableDesc().getTableName().equals(
|
||||
if (e.getRegion().getTableDescriptor().getTableName().equals(
|
||||
TestRegionObserverInterface.TEST_TABLE)) {
|
||||
assertNotNull(familyPaths);
|
||||
assertEquals(1,familyPaths.size());
|
||||
|
|
|
@ -99,7 +99,7 @@ public class TestCoprocessorConfiguration {
|
|||
Configuration conf = new Configuration(CONF);
|
||||
HRegion region = mock(HRegion.class);
|
||||
when(region.getRegionInfo()).thenReturn(REGIONINFO);
|
||||
when(region.getTableDesc()).thenReturn(TABLEDESC);
|
||||
when(region.getTableDescriptor()).thenReturn(TABLEDESC);
|
||||
RegionServerServices rsServices = mock(RegionServerServices.class);
|
||||
systemCoprocessorLoaded.set(false);
|
||||
tableCoprocessorLoaded.set(false);
|
||||
|
@ -141,7 +141,7 @@ public class TestCoprocessorConfiguration {
|
|||
conf.setBoolean(CoprocessorHost.COPROCESSORS_ENABLED_CONF_KEY, false);
|
||||
HRegion region = mock(HRegion.class);
|
||||
when(region.getRegionInfo()).thenReturn(REGIONINFO);
|
||||
when(region.getTableDesc()).thenReturn(TABLEDESC);
|
||||
when(region.getTableDescriptor()).thenReturn(TABLEDESC);
|
||||
RegionServerServices rsServices = mock(RegionServerServices.class);
|
||||
systemCoprocessorLoaded.set(false);
|
||||
tableCoprocessorLoaded.set(false);
|
||||
|
@ -159,7 +159,7 @@ public class TestCoprocessorConfiguration {
|
|||
conf.setBoolean(CoprocessorHost.USER_COPROCESSORS_ENABLED_CONF_KEY, false);
|
||||
HRegion region = mock(HRegion.class);
|
||||
when(region.getRegionInfo()).thenReturn(REGIONINFO);
|
||||
when(region.getTableDesc()).thenReturn(TABLEDESC);
|
||||
when(region.getTableDescriptor()).thenReturn(TABLEDESC);
|
||||
RegionServerServices rsServices = mock(RegionServerServices.class);
|
||||
systemCoprocessorLoaded.set(false);
|
||||
tableCoprocessorLoaded.set(false);
|
||||
|
|
|
@ -23,7 +23,6 @@ import static org.junit.Assert.assertEquals;
|
|||
import static org.junit.Assert.assertNull;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.NavigableSet;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
|
@ -46,6 +45,7 @@ import org.apache.hadoop.hbase.client.Put;
|
|||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.filter.FilterBase;
|
||||
import org.apache.hadoop.hbase.regionserver.ChunkCreator;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
|
@ -228,7 +228,7 @@ public class TestRegionObserverScannerOpenHook {
|
|||
@SuppressWarnings("deprecation")
|
||||
public CompactionCompletionNotifyingRegion(Path tableDir, WAL log,
|
||||
FileSystem fs, Configuration confParam, HRegionInfo info,
|
||||
HTableDescriptor htd, RegionServerServices rsServices) {
|
||||
TableDescriptor htd, RegionServerServices rsServices) {
|
||||
super(tableDir, log, fs, confParam, info, htd, rsServices);
|
||||
}
|
||||
|
||||
|
|
|
@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.client.Admin;
|
|||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.RegionLocator;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper;
|
||||
import org.apache.hadoop.hbase.favored.FavoredNodeLoadBalancer;
|
||||
import org.apache.hadoop.hbase.favored.FavoredNodesPlan;
|
||||
|
@ -424,13 +425,12 @@ public class TestRegionPlacement {
|
|||
// All regions are supposed to have favored nodes,
|
||||
// except for hbase:meta and ROOT
|
||||
if (favoredServerList == null) {
|
||||
HTableDescriptor desc = region.getTableDesc();
|
||||
TableDescriptor desc = region.getTableDescriptor();
|
||||
// Verify they are ROOT and hbase:meta regions since no favored nodes
|
||||
assertNull(favoredSocketAddress);
|
||||
assertTrue("User region " +
|
||||
region.getTableDesc().getTableName() +
|
||||
" should have favored nodes",
|
||||
(desc.isRootRegion() || desc.isMetaRegion()));
|
||||
region.getTableDescriptor().getTableName() +
|
||||
" should have favored nodes", desc.isMetaRegion());
|
||||
} else {
|
||||
// For user region, the favored nodes in the region server should be
|
||||
// identical to favored nodes in the assignmentPlan
|
||||
|
|
|
@ -69,6 +69,9 @@ import org.junit.experimental.categories.Category;
|
|||
import org.mockito.Mockito;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
|
||||
/**
|
||||
* Test the master-related aspects of a snapshot
|
||||
|
@ -280,9 +283,11 @@ public class TestSnapshotFromMaster {
|
|||
// recreate test table with disabled compactions; otherwise compaction may happen before
|
||||
// snapshot, the call after snapshot will be a no-op and checks will fail
|
||||
UTIL.deleteTable(TABLE_NAME);
|
||||
HTableDescriptor htd = new HTableDescriptor(TABLE_NAME);
|
||||
htd.setCompactionEnabled(false);
|
||||
UTIL.createTable(htd, new byte[][] { TEST_FAM }, null);
|
||||
TableDescriptor td = TableDescriptorBuilder.newBuilder(TABLE_NAME)
|
||||
.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(TEST_FAM).build())
|
||||
.setCompactionEnabled(false)
|
||||
.build();
|
||||
UTIL.getAdmin().createTable(td);
|
||||
|
||||
// load the table
|
||||
for (int i = 0; i < blockingStoreFiles / 2; i ++) {
|
||||
|
@ -292,7 +297,6 @@ public class TestSnapshotFromMaster {
|
|||
|
||||
// disable the table so we can take a snapshot
|
||||
admin.disableTable(TABLE_NAME);
|
||||
htd.setCompactionEnabled(true);
|
||||
|
||||
// take a snapshot of the table
|
||||
String snapshotName = "snapshot";
|
||||
|
@ -305,8 +309,11 @@ public class TestSnapshotFromMaster {
|
|||
// ensure we only have one snapshot
|
||||
SnapshotTestingUtils.assertOneSnapshotThatMatches(admin, snapshotNameBytes, TABLE_NAME);
|
||||
|
||||
td = TableDescriptorBuilder.newBuilder(td)
|
||||
.setCompactionEnabled(true)
|
||||
.build();
|
||||
// enable compactions now
|
||||
admin.modifyTable(TABLE_NAME, htd);
|
||||
admin.modifyTable(td);
|
||||
|
||||
// renable the table so we can compact the regions
|
||||
admin.enableTable(TABLE_NAME);
|
||||
|
|
|
@ -45,7 +45,7 @@ public class NoOpScanPolicyObserver implements RegionObserver {
|
|||
public InternalScanner preFlushScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
|
||||
Store store, List<KeyValueScanner> scanners, InternalScanner s) throws IOException {
|
||||
ScanInfo oldSI = store.getScanInfo();
|
||||
ScanInfo scanInfo = new ScanInfo(oldSI.getConfiguration(), store.getFamily(), oldSI.getTtl(),
|
||||
ScanInfo scanInfo = new ScanInfo(oldSI.getConfiguration(), store.getColumnFamilyDescriptor(), oldSI.getTtl(),
|
||||
oldSI.getTimeToPurgeDeletes(), oldSI.getComparator());
|
||||
Scan scan = new Scan();
|
||||
scan.setMaxVersions(oldSI.getMaxVersions());
|
||||
|
@ -62,7 +62,7 @@ public class NoOpScanPolicyObserver implements RegionObserver {
|
|||
InternalScanner s) throws IOException {
|
||||
// this demonstrates how to override the scanners default behavior
|
||||
ScanInfo oldSI = store.getScanInfo();
|
||||
ScanInfo scanInfo = new ScanInfo(oldSI.getConfiguration(), store.getFamily(), oldSI.getTtl(),
|
||||
ScanInfo scanInfo = new ScanInfo(oldSI.getConfiguration(), store.getColumnFamilyDescriptor(), oldSI.getTtl(),
|
||||
oldSI.getTimeToPurgeDeletes(), oldSI.getComparator());
|
||||
Scan scan = new Scan();
|
||||
scan.setMaxVersions(oldSI.getMaxVersions());
|
||||
|
|
|
@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.client.Row;
|
|||
import org.apache.hadoop.hbase.client.RowMutations;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.coprocessor.Batch.Call;
|
||||
import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
|
||||
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
|
||||
|
@ -73,7 +74,7 @@ public class RegionAsTable implements Table {
|
|||
|
||||
@Override
|
||||
public TableName getName() {
|
||||
return this.region.getTableDesc().getTableName();
|
||||
return this.region.getTableDescriptor().getTableName();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -83,7 +84,12 @@ public class RegionAsTable implements Table {
|
|||
|
||||
@Override
|
||||
public HTableDescriptor getTableDescriptor() throws IOException {
|
||||
return this.region.getTableDesc();
|
||||
return new HTableDescriptor(this.region.getTableDescriptor());
|
||||
}
|
||||
|
||||
@Override
|
||||
public TableDescriptor getDescriptor() throws IOException {
|
||||
return this.region.getTableDescriptor();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -59,6 +59,7 @@ import org.apache.hadoop.hbase.client.Put;
|
|||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.RowMutations;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.filter.BinaryComparator;
|
||||
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
|
||||
import org.apache.hadoop.hbase.io.HeapSize;
|
||||
|
@ -660,7 +661,7 @@ public class TestAtomicOperation {
|
|||
public static class MockHRegion extends HRegion {
|
||||
|
||||
public MockHRegion(Path tableDir, WAL log, FileSystem fs, Configuration conf,
|
||||
final HRegionInfo regionInfo, final HTableDescriptor htd, RegionServerServices rsServices) {
|
||||
final HRegionInfo regionInfo, final TableDescriptor htd, RegionServerServices rsServices) {
|
||||
super(tableDir, log, fs, conf, regionInfo, htd, rsServices);
|
||||
}
|
||||
|
||||
|
|
|
@ -291,7 +291,7 @@ public class TestCompoundBloomFilter {
|
|||
Store store = mock(Store.class);
|
||||
HColumnDescriptor hcd = mock(HColumnDescriptor.class);
|
||||
when(hcd.getName()).thenReturn(Bytes.toBytes(RandomKeyValueUtil.COLUMN_FAMILY_NAME));
|
||||
when(store.getFamily()).thenReturn(hcd);
|
||||
when(store.getColumnFamilyDescriptor()).thenReturn(hcd);
|
||||
return scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
|
||||
}
|
||||
|
||||
|
|
|
@ -219,7 +219,7 @@ public class TestHMobStore {
|
|||
|
||||
Scan scan = new Scan(get);
|
||||
InternalScanner scanner = (InternalScanner) store.getScanner(scan,
|
||||
scan.getFamilyMap().get(store.getFamily().getName()),
|
||||
scan.getFamilyMap().get(store.getColumnFamilyDescriptor().getName()),
|
||||
0);
|
||||
|
||||
List<Cell> results = new ArrayList<>();
|
||||
|
@ -264,7 +264,7 @@ public class TestHMobStore {
|
|||
|
||||
Scan scan = new Scan(get);
|
||||
InternalScanner scanner = (InternalScanner) store.getScanner(scan,
|
||||
scan.getFamilyMap().get(store.getFamily().getName()),
|
||||
scan.getFamilyMap().get(store.getColumnFamilyDescriptor().getName()),
|
||||
0);
|
||||
|
||||
List<Cell> results = new ArrayList<>();
|
||||
|
@ -309,7 +309,7 @@ public class TestHMobStore {
|
|||
Scan scan = new Scan(get);
|
||||
scan.setAttribute(MobConstants.MOB_SCAN_RAW, Bytes.toBytes(Boolean.TRUE));
|
||||
InternalScanner scanner = (InternalScanner) store.getScanner(scan,
|
||||
scan.getFamilyMap().get(store.getFamily().getName()),
|
||||
scan.getFamilyMap().get(store.getColumnFamilyDescriptor().getName()),
|
||||
0);
|
||||
|
||||
List<Cell> results = new ArrayList<>();
|
||||
|
@ -354,7 +354,7 @@ public class TestHMobStore {
|
|||
|
||||
Scan scan = new Scan(get);
|
||||
InternalScanner scanner = (InternalScanner) store.getScanner(scan,
|
||||
scan.getFamilyMap().get(store.getFamily().getName()),
|
||||
scan.getFamilyMap().get(store.getColumnFamilyDescriptor().getName()),
|
||||
0);
|
||||
|
||||
List<Cell> results = new ArrayList<>();
|
||||
|
@ -406,7 +406,7 @@ public class TestHMobStore {
|
|||
Scan scan = new Scan(get);
|
||||
scan.setAttribute(MobConstants.MOB_SCAN_RAW, Bytes.toBytes(Boolean.TRUE));
|
||||
InternalScanner scanner = (InternalScanner) store.getScanner(scan,
|
||||
scan.getFamilyMap().get(store.getFamily().getName()),
|
||||
scan.getFamilyMap().get(store.getColumnFamilyDescriptor().getName()),
|
||||
0);
|
||||
|
||||
List<Cell> results = new ArrayList<>();
|
||||
|
@ -421,7 +421,7 @@ public class TestHMobStore {
|
|||
//this is not mob reference cell.
|
||||
Assert.assertFalse(MobUtils.isMobReferenceCell(cell));
|
||||
Assert.assertEquals(expected.get(i), results.get(i));
|
||||
Assert.assertEquals(100, store.getFamily().getMobThreshold());
|
||||
Assert.assertEquals(100, store.getColumnFamilyDescriptor().getMobThreshold());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -522,7 +522,7 @@ public class TestHMobStore {
|
|||
// Scan the values
|
||||
Scan scan = new Scan(get);
|
||||
InternalScanner scanner = (InternalScanner) store.getScanner(scan,
|
||||
scan.getFamilyMap().get(store.getFamily().getName()),
|
||||
scan.getFamilyMap().get(store.getColumnFamilyDescriptor().getName()),
|
||||
0);
|
||||
|
||||
List<Cell> results = new ArrayList<>();
|
||||
|
|
|
@ -111,6 +111,7 @@ import org.apache.hadoop.hbase.client.Result;
|
|||
import org.apache.hadoop.hbase.client.RowMutations;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
|
||||
import org.apache.hadoop.hbase.filter.BinaryComparator;
|
||||
import org.apache.hadoop.hbase.filter.ColumnCountGetFilter;
|
||||
|
@ -812,7 +813,7 @@ public class TestHRegion {
|
|||
Path regiondir = region.getRegionFileSystem().getRegionDir();
|
||||
FileSystem fs = region.getRegionFileSystem().getFileSystem();
|
||||
byte[] regionName = region.getRegionInfo().getEncodedNameAsBytes();
|
||||
byte[][] columns = region.getTableDesc().getFamiliesKeys().toArray(new byte[0][]);
|
||||
byte[][] columns = region.getTableDescriptor().getColumnFamilyNames().toArray(new byte[0][]);
|
||||
|
||||
assertEquals(0, region.getStoreFileList(columns).size());
|
||||
|
||||
|
@ -945,7 +946,7 @@ public class TestHRegion {
|
|||
writer.close();
|
||||
|
||||
// close the region now, and reopen again
|
||||
region.getTableDesc();
|
||||
region.getTableDescriptor();
|
||||
region.getRegionInfo();
|
||||
region.close();
|
||||
try {
|
||||
|
@ -4157,7 +4158,7 @@ public class TestHRegion {
|
|||
// use the static method to compute the value, it should be the same.
|
||||
// static method is used by load balancer or other components
|
||||
HDFSBlocksDistribution blocksDistribution2 = HRegion.computeHDFSBlocksDistribution(
|
||||
htu.getConfiguration(), firstRegion.getTableDesc(), firstRegion.getRegionInfo());
|
||||
htu.getConfiguration(), firstRegion.getTableDescriptor(), firstRegion.getRegionInfo());
|
||||
long uniqueBlocksWeight2 = blocksDistribution2.getUniqueBlocksTotalWeight();
|
||||
|
||||
assertTrue(uniqueBlocksWeight1 == uniqueBlocksWeight2);
|
||||
|
@ -5757,7 +5758,7 @@ public class TestHRegion {
|
|||
static class HRegionWithSeqId extends HRegion {
|
||||
public HRegionWithSeqId(final Path tableDir, final WAL wal, final FileSystem fs,
|
||||
final Configuration confParam, final HRegionInfo regionInfo,
|
||||
final HTableDescriptor htd, final RegionServerServices rsServices) {
|
||||
final TableDescriptor htd, final RegionServerServices rsServices) {
|
||||
super(tableDir, wal, fs, confParam, regionInfo, htd, rsServices);
|
||||
}
|
||||
@Override
|
||||
|
|
|
@ -1134,7 +1134,7 @@ public class TestHRegionReplayEvents {
|
|||
secondaryRegion.replayWALFlushStartMarker(FlushDescriptor.newBuilder().
|
||||
setFlushSequenceNumber(10)
|
||||
.setTableName(UnsafeByteOperations.unsafeWrap(
|
||||
primaryRegion.getTableDesc().getTableName().getName()))
|
||||
primaryRegion.getTableDescriptor().getTableName().getName()))
|
||||
.setAction(FlushAction.START_FLUSH)
|
||||
.setEncodedRegionName(
|
||||
UnsafeByteOperations.unsafeWrap(primaryRegion.getRegionInfo().getEncodedNameAsBytes()))
|
||||
|
@ -1542,7 +1542,7 @@ public class TestHRegionReplayEvents {
|
|||
// from primary and also deleted from the archive directory
|
||||
secondaryRegion.replayWALFlushCommitMarker(FlushDescriptor.newBuilder().
|
||||
setFlushSequenceNumber(Long.MAX_VALUE)
|
||||
.setTableName(UnsafeByteOperations.unsafeWrap(primaryRegion.getTableDesc().getTableName().getName()))
|
||||
.setTableName(UnsafeByteOperations.unsafeWrap(primaryRegion.getTableDescriptor().getTableName().getName()))
|
||||
.setAction(FlushAction.COMMIT_FLUSH)
|
||||
.setEncodedRegionName(
|
||||
UnsafeByteOperations.unsafeWrap(primaryRegion.getRegionInfo().getEncodedNameAsBytes()))
|
||||
|
@ -1562,7 +1562,7 @@ public class TestHRegionReplayEvents {
|
|||
// from primary and also deleted from the archive directory
|
||||
secondaryRegion.replayWALCompactionMarker(CompactionDescriptor.newBuilder()
|
||||
.setTableName(UnsafeByteOperations.unsafeWrap(
|
||||
primaryRegion.getTableDesc().getTableName().getName()))
|
||||
primaryRegion.getTableDescriptor().getTableName().getName()))
|
||||
.setEncodedRegionName(
|
||||
UnsafeByteOperations.unsafeWrap(primaryRegion.getRegionInfo().getEncodedNameAsBytes()))
|
||||
.setFamilyName(UnsafeByteOperations.unsafeWrap(families[0]))
|
||||
|
@ -1580,7 +1580,7 @@ public class TestHRegionReplayEvents {
|
|||
// from primary and also deleted from the archive directory
|
||||
secondaryRegion.replayWALRegionEventMarker(RegionEventDescriptor.newBuilder()
|
||||
.setTableName(UnsafeByteOperations.unsafeWrap(
|
||||
primaryRegion.getTableDesc().getTableName().getName()))
|
||||
primaryRegion.getTableDescriptor().getTableName().getName()))
|
||||
.setEncodedRegionName(
|
||||
UnsafeByteOperations.unsafeWrap(primaryRegion.getRegionInfo().getEncodedNameAsBytes()))
|
||||
.setRegionName(UnsafeByteOperations.unsafeWrap(primaryRegion.getRegionInfo().getRegionName()))
|
||||
|
@ -1600,7 +1600,7 @@ public class TestHRegionReplayEvents {
|
|||
// tests replaying bulk load event marker, but the bulk load files have already been compacted
|
||||
// from primary and also deleted from the archive directory
|
||||
secondaryRegion.replayWALBulkLoadEventMarker(BulkLoadDescriptor.newBuilder()
|
||||
.setTableName(ProtobufUtil.toProtoTableName(primaryRegion.getTableDesc().getTableName()))
|
||||
.setTableName(ProtobufUtil.toProtoTableName(primaryRegion.getTableDescriptor().getTableName()))
|
||||
.setEncodedRegionName(
|
||||
UnsafeByteOperations.unsafeWrap(primaryRegion.getRegionInfo().getEncodedNameAsBytes()))
|
||||
.setBulkloadSeqNum(Long.MAX_VALUE)
|
||||
|
|
|
@ -210,7 +210,7 @@ public class TestHStoreFile extends HBaseTestCase {
|
|||
HColumnDescriptor hcd = mock(HColumnDescriptor.class);
|
||||
byte[] cf = Bytes.toBytes("ty");
|
||||
when(hcd.getName()).thenReturn(cf);
|
||||
when(store.getFamily()).thenReturn(hcd);
|
||||
when(store.getColumnFamilyDescriptor()).thenReturn(hcd);
|
||||
StoreFileScanner scanner =
|
||||
new StoreFileScanner(reader, mock(HFileScanner.class), false, false, 0, 0, true);
|
||||
Scan scan = new Scan();
|
||||
|
@ -530,7 +530,7 @@ public class TestHStoreFile extends HBaseTestCase {
|
|||
Store store = mock(Store.class);
|
||||
HColumnDescriptor hcd = mock(HColumnDescriptor.class);
|
||||
when(hcd.getName()).thenReturn(Bytes.toBytes("family"));
|
||||
when(store.getFamily()).thenReturn(hcd);
|
||||
when(store.getColumnFamilyDescriptor()).thenReturn(hcd);
|
||||
boolean exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
|
||||
if (i % 2 == 0) {
|
||||
if (!exists) falseNeg++;
|
||||
|
@ -717,7 +717,7 @@ public class TestHStoreFile extends HBaseTestCase {
|
|||
Store store = mock(Store.class);
|
||||
HColumnDescriptor hcd = mock(HColumnDescriptor.class);
|
||||
when(hcd.getName()).thenReturn(Bytes.toBytes("family"));
|
||||
when(store.getFamily()).thenReturn(hcd);
|
||||
when(store.getColumnFamilyDescriptor()).thenReturn(hcd);
|
||||
// check false positives rate
|
||||
int falsePos = 0;
|
||||
int falseNeg = 0;
|
||||
|
@ -861,7 +861,7 @@ public class TestHStoreFile extends HBaseTestCase {
|
|||
Store store = mock(Store.class);
|
||||
HColumnDescriptor hcd = mock(HColumnDescriptor.class);
|
||||
when(hcd.getName()).thenReturn(family);
|
||||
when(store.getFamily()).thenReturn(hcd);
|
||||
when(store.getColumnFamilyDescriptor()).thenReturn(hcd);
|
||||
hsf.initReader();
|
||||
StoreFileReader reader = hsf.getReader();
|
||||
StoreFileScanner scanner = getStoreFileScanner(reader, false, false);
|
||||
|
|
|
@ -45,11 +45,15 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
|
|||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.Durability;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFile;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileContext;
|
||||
|
@ -171,7 +175,7 @@ public class TestMobStoreCompaction {
|
|||
assertEquals("Before compaction: number of mob cells", compactionThreshold,
|
||||
countMobCellsInMetadata());
|
||||
// Change the threshold larger than the data size
|
||||
region.getTableDesc().getFamily(COLUMN_FAMILY).setMobThreshold(500);
|
||||
setMobThreshold(region, COLUMN_FAMILY, 500);
|
||||
region.initialize();
|
||||
region.compactStores();
|
||||
|
||||
|
@ -182,6 +186,20 @@ public class TestMobStoreCompaction {
|
|||
assertEquals("After compaction: mob rows", 0, countMobRows());
|
||||
}
|
||||
|
||||
private static HRegion setMobThreshold(HRegion region, byte[] cfName, long modThreshold) {
|
||||
ColumnFamilyDescriptor cfd = ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(region.getTableDescriptor().getColumnFamily(cfName))
|
||||
.setMobThreshold(modThreshold)
|
||||
.build();
|
||||
TableDescriptor td = TableDescriptorBuilder
|
||||
.newBuilder(region.getTableDescriptor())
|
||||
.removeColumnFamily(cfName)
|
||||
.addColumnFamily(cfd)
|
||||
.build();
|
||||
region.setTableDescriptor(td);
|
||||
return region;
|
||||
}
|
||||
|
||||
/**
|
||||
* This test will first generate store files, then bulk load them and trigger the compaction.
|
||||
* When compaction, the cell value will be larger than the threshold.
|
||||
|
|
|
@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.client.Admin;
|
|||
import org.apache.hadoop.hbase.client.Durability;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
|
||||
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
|
||||
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
|
||||
|
@ -191,7 +192,7 @@ public class TestRegionServerAbort {
|
|||
*/
|
||||
public static class ErrorThrowingHRegion extends HRegion {
|
||||
public ErrorThrowingHRegion(Path tableDir, WAL wal, FileSystem fs, Configuration confParam,
|
||||
HRegionInfo regionInfo, HTableDescriptor htd,
|
||||
HRegionInfo regionInfo, TableDescriptor htd,
|
||||
RegionServerServices rsServices) {
|
||||
super(tableDir, wal, fs, confParam, regionInfo, htd, rsServices);
|
||||
}
|
||||
|
|
|
@ -41,6 +41,8 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
|
|||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.Append;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.Durability;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
|
@ -51,6 +53,8 @@ import org.apache.hadoop.hbase.client.Result;
|
|||
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.test.MetricsAssertHelper;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
|
@ -452,7 +456,7 @@ public class TestRegionServerMetrics {
|
|||
metricsRegionServer.getRegionServerWrapper().forceRecompute();
|
||||
assertCounter("mobScanCellsCount", numHfiles);
|
||||
|
||||
region.getTableDesc().getFamily(cf).setMobThreshold(100);
|
||||
setMobThreshold(region, cf, 100);
|
||||
// metrics are reset by the region initialization
|
||||
((HRegion) region).initialize();
|
||||
region.compact(true);
|
||||
|
@ -472,7 +476,7 @@ public class TestRegionServerMetrics {
|
|||
table.put(p);
|
||||
admin.flush(tableName);
|
||||
}
|
||||
region.getTableDesc().getFamily(cf).setMobThreshold(0);
|
||||
setMobThreshold(region, cf, 0);
|
||||
|
||||
// closing the region forces the compaction.discharger to archive the compacted hfiles
|
||||
((HRegion) region).close();
|
||||
|
@ -490,6 +494,20 @@ public class TestRegionServerMetrics {
|
|||
}
|
||||
}
|
||||
|
||||
private static Region setMobThreshold(Region region, byte[] cfName, long modThreshold) {
|
||||
ColumnFamilyDescriptor cfd = ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(region.getTableDescriptor().getColumnFamily(cfName))
|
||||
.setMobThreshold(modThreshold)
|
||||
.build();
|
||||
TableDescriptor td = TableDescriptorBuilder
|
||||
.newBuilder(region.getTableDescriptor())
|
||||
.removeColumnFamily(cfName)
|
||||
.addColumnFamily(cfd)
|
||||
.build();
|
||||
((HRegion)region).setTableDescriptor(td);
|
||||
return region;
|
||||
}
|
||||
|
||||
@Test
|
||||
@Ignore
|
||||
public void testRangeCountMetrics() throws Exception {
|
||||
|
|
|
@ -61,7 +61,7 @@ public class TestRegionSplitPolicy {
|
|||
HRegionInfo hri = new HRegionInfo(TABLENAME);
|
||||
htd = new HTableDescriptor(TABLENAME);
|
||||
mockRegion = Mockito.mock(HRegion.class);
|
||||
Mockito.doReturn(htd).when(mockRegion).getTableDesc();
|
||||
Mockito.doReturn(htd).when(mockRegion).getTableDescriptor();
|
||||
Mockito.doReturn(hri).when(mockRegion).getRegionInfo();
|
||||
stores = new ArrayList<>();
|
||||
Mockito.doReturn(stores).when(mockRegion).getStores();
|
||||
|
@ -240,7 +240,7 @@ public class TestRegionSplitPolicy {
|
|||
myHtd.setValue(KeyPrefixRegionSplitPolicy.PREFIX_LENGTH_KEY, String.valueOf(2));
|
||||
|
||||
HRegion myMockRegion = Mockito.mock(HRegion.class);
|
||||
Mockito.doReturn(myHtd).when(myMockRegion).getTableDesc();
|
||||
Mockito.doReturn(myHtd).when(myMockRegion).getTableDescriptor();
|
||||
Mockito.doReturn(stores).when(myMockRegion).getStores();
|
||||
|
||||
HStore mockStore = Mockito.mock(HStore.class);
|
||||
|
@ -349,7 +349,7 @@ public class TestRegionSplitPolicy {
|
|||
myHtd.setValue(DelimitedKeyPrefixRegionSplitPolicy.DELIMITER_KEY, ",");
|
||||
|
||||
HRegion myMockRegion = Mockito.mock(HRegion.class);
|
||||
Mockito.doReturn(myHtd).when(myMockRegion).getTableDesc();
|
||||
Mockito.doReturn(myHtd).when(myMockRegion).getTableDescriptor();
|
||||
Mockito.doReturn(stores).when(myMockRegion).getStores();
|
||||
|
||||
HStore mockStore = Mockito.mock(HStore.class);
|
||||
|
|
|
@ -51,6 +51,7 @@ import org.apache.hadoop.hbase.client.ResultScanner;
|
|||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.ScannerCallable;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.filter.Filter;
|
||||
import org.apache.hadoop.hbase.filter.FilterBase;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest;
|
||||
|
@ -445,7 +446,7 @@ public class TestScannerHeartbeatMessages {
|
|||
private static volatile boolean sleepBetweenColumnFamilies = false;
|
||||
|
||||
public HeartbeatHRegion(Path tableDir, WAL wal, FileSystem fs, Configuration confParam,
|
||||
HRegionInfo regionInfo, HTableDescriptor htd, RegionServerServices rsServices) {
|
||||
HRegionInfo regionInfo, TableDescriptor htd, RegionServerServices rsServices) {
|
||||
super(tableDir, wal, fs, confParam, regionInfo, htd, rsServices);
|
||||
}
|
||||
|
||||
|
|
|
@ -474,7 +474,7 @@ public class TestStore {
|
|||
w.close();
|
||||
this.store.close();
|
||||
// Reopen it... should pick up two files
|
||||
this.store = new HStore(this.store.getHRegion(), this.store.getFamily(), c);
|
||||
this.store = new HStore(this.store.getHRegion(), this.store.getColumnFamilyDescriptor(), c);
|
||||
Assert.assertEquals(2, this.store.getStorefilesCount());
|
||||
|
||||
result = HBaseTestingUtility.getFromStoreFile(store,
|
||||
|
|
|
@ -96,7 +96,7 @@ public class TestDateTieredCompactor {
|
|||
ScanInfo si = new ScanInfo(conf, col, Long.MAX_VALUE, 0, CellComparator.COMPARATOR);
|
||||
final Store store = mock(Store.class);
|
||||
when(store.getStorefiles()).thenReturn(storefiles);
|
||||
when(store.getFamily()).thenReturn(col);
|
||||
when(store.getColumnFamilyDescriptor()).thenReturn(col);
|
||||
when(store.getScanInfo()).thenReturn(si);
|
||||
when(store.areWritesEnabled()).thenReturn(true);
|
||||
when(store.getFileSystem()).thenReturn(mock(FileSystem.class));
|
||||
|
|
|
@ -773,7 +773,7 @@ public class TestStripeCompactionPolicy {
|
|||
Store store = mock(Store.class);
|
||||
HRegionInfo info = mock(HRegionInfo.class);
|
||||
when(info.getRegionNameAsString()).thenReturn("testRegion");
|
||||
when(store.getFamily()).thenReturn(col);
|
||||
when(store.getColumnFamilyDescriptor()).thenReturn(col);
|
||||
when(store.getRegionInfo()).thenReturn(info);
|
||||
when(
|
||||
store.createWriterInTmp(anyLong(), any(Compression.Algorithm.class), anyBoolean(),
|
||||
|
|
|
@ -196,7 +196,7 @@ public class TestStripeCompactor {
|
|||
HColumnDescriptor col = new HColumnDescriptor(NAME_OF_THINGS);
|
||||
ScanInfo si = new ScanInfo(conf, col, Long.MAX_VALUE, 0, CellComparator.COMPARATOR);
|
||||
Store store = mock(Store.class);
|
||||
when(store.getFamily()).thenReturn(col);
|
||||
when(store.getColumnFamilyDescriptor()).thenReturn(col);
|
||||
when(store.getScanInfo()).thenReturn(si);
|
||||
when(store.areWritesEnabled()).thenReturn(true);
|
||||
when(store.getFileSystem()).thenReturn(mock(FileSystem.class));
|
||||
|
|
|
@ -694,7 +694,7 @@ public class TestMasterReplication {
|
|||
region.getWAL().registerWALActionsListener(listener);
|
||||
|
||||
// request a roll
|
||||
admin.rollWALWriter(cluster.getServerHoldingRegion(region.getTableDesc().getTableName(),
|
||||
admin.rollWALWriter(cluster.getServerHoldingRegion(region.getTableDescriptor().getTableName(),
|
||||
region.getRegionInfo().getRegionName()));
|
||||
|
||||
// wait
|
||||
|
|
|
@ -233,7 +233,7 @@ public class TestMultiSlaveReplication {
|
|||
region.getWAL().registerWALActionsListener(listener);
|
||||
|
||||
// request a roll
|
||||
admin.rollWALWriter(cluster.getServerHoldingRegion(region.getTableDesc().getTableName(),
|
||||
admin.rollWALWriter(cluster.getServerHoldingRegion(region.getTableDescriptor().getTableName(),
|
||||
region.getRegionInfo().getRegionName()));
|
||||
|
||||
// wait
|
||||
|
|
|
@ -842,9 +842,9 @@ public class TestWithDisabledAuthorization extends SecureTestUtil {
|
|||
public Object run() throws Exception {
|
||||
HTableDescriptor htd = new HTableDescriptor(TEST_TABLE.getTableName());
|
||||
Region region_a = mock(Region.class);
|
||||
when(region_a.getTableDesc()).thenReturn(htd);
|
||||
when(region_a.getTableDescriptor()).thenReturn(htd);
|
||||
Region region_b = mock(Region.class);
|
||||
when(region_b.getTableDesc()).thenReturn(htd);
|
||||
when(region_b.getTableDescriptor()).thenReturn(htd);
|
||||
ACCESS_CONTROLLER.preMerge(ObserverContext.createAndPrepare(RSCP_ENV, null), region_a,
|
||||
region_b);
|
||||
return null;
|
||||
|
|
|
@ -213,7 +213,7 @@ public class HFileArchiveTestingUtil {
|
|||
public static Path getRegionArchiveDir(Configuration conf, HRegion region) throws IOException {
|
||||
return HFileArchiveUtil.getRegionArchiveDir(
|
||||
FSUtils.getRootDir(conf),
|
||||
region.getTableDesc().getTableName(),
|
||||
region.getTableDescriptor().getTableName(),
|
||||
region.getRegionInfo().getEncodedName());
|
||||
}
|
||||
|
||||
|
@ -227,7 +227,7 @@ public class HFileArchiveTestingUtil {
|
|||
public static Path getStoreArchivePath(Configuration conf, HRegion region, Store store)
|
||||
throws IOException {
|
||||
return HFileArchiveUtil.getStoreArchivePath(conf, region.getRegionInfo(),
|
||||
region.getRegionFileSystem().getTableDir(), store.getFamily().getName());
|
||||
region.getRegionFileSystem().getTableDir(), store.getColumnFamilyDescriptor().getName());
|
||||
}
|
||||
|
||||
public static Path getStoreArchivePath(HBaseTestingUtility util, String tableName,
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue