HBASE-13893 Replace HTable with Table in client tests (Jurriaan Mous)

This commit is contained in:
stack 2015-06-25 14:42:33 -07:00
parent e6ed792199
commit b5b5853043
96 changed files with 1996 additions and 1749 deletions

View File

@ -23,7 +23,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
/**
* A Get, Put, Increment, Append, or Delete associated with it's region. Used internally by
* {@link HTable#batch} to associate the action with it's region and maintain
* {@link Table#batch} to associate the action with it's region and maintain
* the index from the original request.
*/
@InterfaceAudience.Private

View File

@ -32,7 +32,7 @@ import java.util.concurrent.TimeUnit;
/**
* <p>
* Used to communicate with a single HBase table similar to {@link HTable}
* Used to communicate with a single HBase table similar to {@link Table}
* but meant for batched, potentially asynchronous puts. Obtain an instance from
* a {@link Connection} and call {@link #close()} afterwards.
* </p>

View File

@ -57,7 +57,7 @@ import com.google.protobuf.TextFormat;
/**
* Scanner operations such as create, next, etc.
* Used by {@link ResultScanner}s made by {@link HTable}. Passed to a retrying caller such as
* Used by {@link ResultScanner}s made by {@link Table}. Passed to a retrying caller such as
* {@link RpcRetryingCaller} so fails are retried.
*/
@InterfaceAudience.Private

View File

@ -36,13 +36,15 @@ public abstract class Batch {
*
* <p>
* When used with
* {@link org.apache.hadoop.hbase.client.HTable#coprocessorService(Class, byte[], byte[], org.apache.hadoop.hbase.client.coprocessor.Batch.Call)}
* {@link org.apache.hadoop.hbase.client.Table#coprocessorService(Class, byte[], byte[],
* org.apache.hadoop.hbase.client.coprocessor.Batch.Call)}
* the implementations {@link Batch.Call#call(Object)} method will be invoked
* with a proxy to each region's coprocessor {@link com.google.protobuf.Service} implementation.
* </p>
* @see org.apache.hadoop.hbase.client.coprocessor
* @see org.apache.hadoop.hbase.client.HTable#coprocessorService(byte[])
* @see org.apache.hadoop.hbase.client.HTable#coprocessorService(Class, byte[], byte[], org.apache.hadoop.hbase.client.coprocessor.Batch.Call)
* @see org.apache.hadoop.hbase.client.Table#coprocessorService(byte[])
* @see org.apache.hadoop.hbase.client.Table#coprocessorService(Class, byte[], byte[],
* org.apache.hadoop.hbase.client.coprocessor.Batch.Call)
* @param <T> the instance type to be passed to
* {@link Batch.Call#call(Object)}
* @param <R> the return type from {@link Batch.Call#call(Object)}
@ -59,13 +61,15 @@ public abstract class Batch {
*
* <p>
* When used with
* {@link org.apache.hadoop.hbase.client.HTable#coprocessorService(Class, byte[], byte[], org.apache.hadoop.hbase.client.coprocessor.Batch.Call)}
* {@link org.apache.hadoop.hbase.client.Table#coprocessorService(Class, byte[], byte[],
* org.apache.hadoop.hbase.client.coprocessor.Batch.Call)}
* the implementation's {@link Batch.Callback#update(byte[], byte[], Object)}
* method will be called with the {@link Batch.Call#call(Object)} return value
* from each region in the selected range.
* </p>
* @param <R> the return type from the associated {@link Batch.Call#call(Object)}
* @see org.apache.hadoop.hbase.client.HTable#coprocessorService(Class, byte[], byte[], org.apache.hadoop.hbase.client.coprocessor.Batch.Call)
* @see org.apache.hadoop.hbase.client.Table#coprocessorService(Class, byte[], byte[],
* org.apache.hadoop.hbase.client.coprocessor.Batch.Call)
*/
@InterfaceAudience.Public
@InterfaceStability.Stable

View File

@ -40,10 +40,10 @@ import com.google.protobuf.Message;
/**
* Provides clients with an RPC connection to call coprocessor endpoint {@link com.google.protobuf.Service}s
* against a given table region. An instance of this class may be obtained
* by calling {@link org.apache.hadoop.hbase.client.HTable#coprocessorService(byte[])},
* by calling {@link org.apache.hadoop.hbase.client.Table#coprocessorService(byte[])},
* but should normally only be used in creating a new {@link com.google.protobuf.Service} stub to call the endpoint
* methods.
* @see org.apache.hadoop.hbase.client.HTable#coprocessorService(byte[])
* @see org.apache.hadoop.hbase.client.Table#coprocessorService(byte[])
*/
@InterfaceAudience.Private
public class RegionCoprocessorRpcChannel extends CoprocessorRpcChannel{

View File

@ -382,7 +382,7 @@ public class TestOperation {
Assert.assertEquals(0, c.size());
Assert.assertEquals(HConstants.LATEST_TIMESTAMP, p.getTimeStamp());
p.add(FAMILY, ByteBuffer.wrap(QUALIFIER), 1984L, ByteBuffer.wrap(VALUE));
p.addColumn(FAMILY, ByteBuffer.wrap(QUALIFIER), 1984L, ByteBuffer.wrap(VALUE));
c = p.get(FAMILY, QUALIFIER);
Assert.assertEquals(1, c.size());
Assert.assertEquals(1984L, c.get(0).getTimestamp());
@ -391,7 +391,7 @@ public class TestOperation {
Assert.assertEquals(0, CellComparator.COMPARATOR.compare(c.get(0), new KeyValue(c.get(0))));
p = new Put(ROW);
p.add(FAMILY, ByteBuffer.wrap(QUALIFIER), 2013L, null);
p.addColumn(FAMILY, ByteBuffer.wrap(QUALIFIER), 2013L, null);
c = p.get(FAMILY, QUALIFIER);
Assert.assertEquals(1, c.size());
Assert.assertEquals(2013L, c.get(0).getTimestamp());
@ -400,7 +400,7 @@ public class TestOperation {
Assert.assertEquals(0, CellComparator.COMPARATOR.compare(c.get(0), new KeyValue(c.get(0))));
p = new Put(ByteBuffer.wrap(ROW));
p.add(FAMILY, ByteBuffer.wrap(QUALIFIER), 2001L, null);
p.addColumn(FAMILY, ByteBuffer.wrap(QUALIFIER), 2001L, null);
c = p.get(FAMILY, QUALIFIER);
Assert.assertEquals(1, c.size());
Assert.assertEquals(2001L, c.get(0).getTimestamp());
@ -410,7 +410,7 @@ public class TestOperation {
Assert.assertEquals(0, CellComparator.COMPARATOR.compare(c.get(0), new KeyValue(c.get(0))));
p = new Put(ByteBuffer.wrap(ROW), 1970L);
p.add(FAMILY, ByteBuffer.wrap(QUALIFIER), 2001L, null);
p.addColumn(FAMILY, ByteBuffer.wrap(QUALIFIER), 2001L, null);
c = p.get(FAMILY, QUALIFIER);
Assert.assertEquals(1, c.size());
Assert.assertEquals(2001L, c.get(0).getTimestamp());

View File

@ -237,7 +237,7 @@ public class IntegrationTestBulkLoad extends IntegrationTestBase {
}
util.createTable(
getTablename().getName(),
getTablename(),
new byte[][]{CHAIN_FAM, SORT_FAM, DATA_FAM},
getSplits(16)
);

View File

@ -377,7 +377,7 @@ public class IntegrationTestWithCellVisibilityLoadAndVerify extends IntegrationT
}
doLoad(getConf(), htd);
doVerify(getConf(), htd);
getTestingUtil(getConf()).deleteTable(htd.getName());
getTestingUtil(getConf()).deleteTable(getTablename());
return 0;
}

View File

@ -52,6 +52,9 @@ import org.junit.BeforeClass;
public class RowResourceBase {
protected static final String TABLE = "TestRowResource";
protected static final TableName TABLE_NAME = TableName.valueOf(TABLE);
protected static final String CFA = "a";
protected static final String CFB = "b";
protected static final String COLUMN_1 = CFA + ":1";
@ -76,6 +79,7 @@ public class RowResourceBase {
protected static Configuration conf;
protected static ObjectMapper jsonMapper;
@BeforeClass
public static void setUpBeforeClass() throws Exception {
conf = TEST_UTIL.getConfiguration();
@ -102,10 +106,10 @@ public class RowResourceBase {
@Before
public void beforeMethod() throws Exception {
Admin admin = TEST_UTIL.getHBaseAdmin();
if (admin.tableExists(TableName.valueOf(TABLE))) {
TEST_UTIL.deleteTable(Bytes.toBytes(TABLE));
if (admin.tableExists(TABLE_NAME)) {
TEST_UTIL.deleteTable(TABLE_NAME);
}
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(TABLE));
HTableDescriptor htd = new HTableDescriptor(TABLE);
htd.addFamily(new HColumnDescriptor(CFA));
htd.addFamily(new HColumnDescriptor(CFB));
admin.createTable(htd);
@ -114,8 +118,8 @@ public class RowResourceBase {
@After
public void afterMethod() throws Exception {
Admin admin = TEST_UTIL.getHBaseAdmin();
if (admin.tableExists(TableName.valueOf(TABLE))) {
TEST_UTIL.deleteTable(Bytes.toBytes(TABLE));
if (admin.tableExists(TABLE_NAME)) {
TEST_UTIL.deleteTable(TABLE_NAME);
}
}

View File

@ -84,8 +84,8 @@ public class TestStatusResource {
public static void setUpBeforeClass() throws Exception {
conf = TEST_UTIL.getConfiguration();
TEST_UTIL.startMiniCluster(1, 1);
TEST_UTIL.createTable(Bytes.toBytes("TestStatusResource"), Bytes.toBytes("D"));
TEST_UTIL.createTable(Bytes.toBytes("TestStatusResource2"), Bytes.toBytes("D"));
TEST_UTIL.createTable(TableName.valueOf("TestStatusResource"), Bytes.toBytes("D"));
TEST_UTIL.createTable(TableName.valueOf("TestStatusResource2"), Bytes.toBytes("D"));
REST_TEST_UTIL.startServletContainer(conf);
Cluster cluster = new Cluster();
cluster.add("localhost", REST_TEST_UTIL.getServletPort());

View File

@ -41,7 +41,7 @@ import org.apache.hadoop.mapred.RecordReader;
import org.apache.hadoop.mapred.Reporter;
/**
* A Base for {@link TableInputFormat}s. Receives a {@link HTable}, a
* A Base for {@link TableInputFormat}s. Receives a {@link Table}, a
* byte[] of input columns and optionally a {@link Filter}.
* Subclasses may use other TableRecordReader implementations.
*

View File

@ -433,7 +433,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
final Multimap<ByteBuffer, LoadQueueItem> regionGroups) throws IOException {
// atomically bulk load the groups.
Set<Future<List<LoadQueueItem>>> loadingFutures = new HashSet<Future<List<LoadQueueItem>>>();
for (Entry<ByteBuffer, ? extends Collection<LoadQueueItem>> e: regionGroups.asMap().entrySet()) {
for (Entry<ByteBuffer, ? extends Collection<LoadQueueItem>> e: regionGroups.asMap().entrySet()){
final byte[] first = e.getKey().array();
final Collection<LoadQueueItem> lqis = e.getValue();
@ -854,7 +854,8 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
* Algo:
* 1) Poll on the keys in order:
* a) Keep adding the mapped values to these keys (runningSum)
* b) Each time runningSum reaches 0, add the start Key from when the runningSum had started to a boundary list.
* b) Each time runningSum reaches 0, add the start Key from when the runningSum had started to
* a boundary list.
* 2) Return the boundary list.
*/
public static byte[][] inferBoundaries(TreeMap<byte[], Integer> bdryMap) {
@ -958,8 +959,9 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
Path hfofDir = new Path(dirPath);
try (HTable table = (HTable) connection.getTable(tableName);) {
doBulkLoad(hfofDir, table);
try (Table table = connection.getTable(tableName);
RegionLocator locator = connection.getRegionLocator(tableName)) {
doBulkLoad(hfofDir, admin, table, locator);
}
}

View File

@ -526,7 +526,7 @@ extends InputFormat<ImmutableBytesWritable, Result> {
}
}
//transform the List of bytes to byte[]
byte result[] = new byte[resultBytesList.size()];
byte[] result = new byte[resultBytesList.size()];
for (int k = 0; k < resultBytesList.size(); k++) {
result[k] = (byte) resultBytesList.get(k);
}
@ -558,7 +558,7 @@ extends InputFormat<ImmutableBytesWritable, Result> {
}
/**
* Allows subclasses to get the {@link HTable}.
* Allows subclasses to get the {@link Table}.
*
* @deprecated use {@link #getTable()}
*/

View File

@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Table;
/**
* Computes size of each region for given table and given column families.
@ -66,10 +67,11 @@ public class RegionSizeCalculator {
* @deprecated Use {@link #RegionSizeCalculator(RegionLocator, Admin)} instead.
*/
@Deprecated
public RegionSizeCalculator(HTable table) throws IOException {
public RegionSizeCalculator(Table table) throws IOException {
try (Connection conn = ConnectionFactory.createConnection(table.getConfiguration());
RegionLocator locator = conn.getRegionLocator(table.getName());
Admin admin = conn.getAdmin()) {
init(table.getRegionLocator(), admin);
init(locator, admin);
}
}

View File

@ -478,7 +478,7 @@ public abstract class HBaseTestCase extends TestCase {
}
/**
* A class that makes a {@link Incommon} out of a {@link HTable}
* A class that makes a {@link Incommon} out of a {@link Table}
*/
public static class HTableIncommon implements Incommon {
final Table table;

View File

@ -56,6 +56,7 @@ import org.apache.hadoop.hbase.Waiter.Predicate;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.BufferedMutator;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Consistency;
@ -64,6 +65,7 @@ import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HRegionLocator;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionLocator;
@ -1262,10 +1264,11 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @param family
* @return An HTable instance for the created table.
* @throws IOException
* @deprecated use {@link #createTable(TableName, byte[])}
*/
public HTable createTable(byte[] tableName, byte[] family)
throws IOException{
return createTable(TableName.valueOf(tableName), new byte[][]{family});
@Deprecated
public HTable createTable(byte[] tableName, byte[] family) throws IOException {
return createTable(TableName.valueOf(tableName), new byte[][] { family });
}
/**
@ -1321,7 +1324,9 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @param families
* @return An HTable instance for the created table.
* @throws IOException
* @deprecated use {@link #createTable(TableName, byte[][])}
*/
@Deprecated
public HTable createTable(byte[] tableName, byte[][] families)
throws IOException {
return createTable(tableName, families,
@ -1364,16 +1369,18 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
return createTable(tableName, families, splitKeys, new Configuration(getConfiguration()));
}
public HTable createTable(byte[] tableName, byte[][] families,
int numVersions, byte[] startKey, byte[] endKey, int numRegions) throws IOException {
return createTable(TableName.valueOf(tableName), families, numVersions,
startKey, endKey, numRegions);
@Deprecated
public HTable createTable(byte[] tableName, byte[][] families, int numVersions, byte[] startKey,
byte[] endKey, int numRegions) throws IOException {
return createTable(TableName.valueOf(tableName), families, numVersions, startKey, endKey,
numRegions);
}
public HTable createTable(String tableName, byte[][] families,
int numVersions, byte[] startKey, byte[] endKey, int numRegions) throws IOException {
return createTable(TableName.valueOf(tableName), families, numVersions,
startKey, endKey, numRegions);
@Deprecated
public HTable createTable(String tableName, byte[][] families, int numVersions, byte[] startKey,
byte[] endKey, int numRegions) throws IOException {
return createTable(TableName.valueOf(tableName), families, numVersions, startKey, endKey,
numRegions);
}
public HTable createTable(TableName tableName, byte[][] families,
@ -1452,10 +1459,11 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @param c Configuration to use
* @return An HTable instance for the created table.
* @throws IOException
* @deprecated use {@link #createTable(TableName, byte[][])}
*/
public HTable createTable(TableName tableName, byte[][] families,
final Configuration c)
throws IOException {
@Deprecated
public HTable createTable(TableName tableName, byte[][] families, final Configuration c)
throws IOException {
return createTable(tableName, families, (byte[][]) null, c);
}
@ -1480,10 +1488,11 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @param c Configuration to use
* @return An HTable instance for the created table.
* @throws IOException
* @deprecated use {@link #createTable(TableName, byte[][])}
*/
public HTable createTable(byte[] tableName, byte[][] families,
final Configuration c)
throws IOException {
@Deprecated
public HTable createTable(byte[] tableName, byte[][] families, final Configuration c)
throws IOException {
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
for(byte[] family : families) {
HColumnDescriptor hcd = new HColumnDescriptor(family);
@ -1505,7 +1514,9 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @param numVersions
* @return An HTable instance for the created table.
* @throws IOException
* @deprecated use {@link #createTable(TableName, byte[][], int)}
*/
@Deprecated
public HTable createTable(TableName tableName, byte[][] families,
final Configuration c, int numVersions)
throws IOException {
@ -1529,14 +1540,15 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @param numVersions
* @return An HTable instance for the created table.
* @throws IOException
* @deprecated use {@link #createTable(TableName, byte[][], int)}
*/
@Deprecated
public HTable createTable(byte[] tableName, byte[][] families,
final Configuration c, int numVersions)
throws IOException {
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
for(byte[] family : families) {
HColumnDescriptor hcd = new HColumnDescriptor(family)
.setMaxVersions(numVersions);
for (byte[] family : families) {
HColumnDescriptor hcd = new HColumnDescriptor(family).setMaxVersions(numVersions);
desc.addFamily(hcd);
}
getHBaseAdmin().createTable(desc);
@ -1550,7 +1562,9 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @param numVersions
* @return An HTable instance for the created table.
* @throws IOException
* @deprecated use {@link #createTable(TableName, byte[], int)}
*/
@Deprecated
public HTable createTable(byte[] tableName, byte[] family, int numVersions)
throws IOException {
return createTable(tableName, new byte[][]{family}, numVersions);
@ -1576,10 +1590,11 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @param numVersions
* @return An HTable instance for the created table.
* @throws IOException
* @deprecated use {@link #createTable(TableName, byte[][], int)}
*/
public HTable createTable(byte[] tableName, byte[][] families,
int numVersions)
throws IOException {
@Deprecated
public HTable createTable(byte[] tableName, byte[][] families, int numVersions)
throws IOException {
return createTable(TableName.valueOf(tableName), families, numVersions);
}
@ -1591,9 +1606,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @return An HTable instance for the created table.
* @throws IOException
*/
public HTable createTable(TableName tableName, byte[][] families,
int numVersions)
throws IOException {
public HTable createTable(TableName tableName, byte[][] families, int numVersions)
throws IOException {
return createTable(tableName, families, numVersions, (byte[][]) null);
}
@ -1640,7 +1654,9 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @param blockSize
* @return An HTable instance for the created table.
* @throws IOException
* @deprecated use {@link #createTable(TableName, byte[][], int, int)}
*/
@Deprecated
public HTable createTable(byte[] tableName, byte[][] families,
int numVersions, int blockSize) throws IOException {
return createTable(TableName.valueOf(tableName),
@ -1678,7 +1694,9 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @param numVersions
* @return An HTable instance for the created table.
* @throws IOException
* @deprecated use {@link #createTable(TableName, byte[][], int)}
*/
@Deprecated
public HTable createTable(byte[] tableName, byte[][] families,
int[] numVersions)
throws IOException {
@ -1717,7 +1735,9 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @param splitRows
* @return An HTable instance for the created table.
* @throws IOException
* @deprecated use {@link #createTable(TableName, byte[], byte[][])}
*/
@Deprecated
public HTable createTable(byte[] tableName, byte[] family, byte[][] splitRows)
throws IOException{
return createTable(TableName.valueOf(tableName), family, splitRows);
@ -1760,7 +1780,9 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @param splitRows
* @return An HTable instance for the created table.
* @throws IOException
* @deprecated use {@link #createTable(TableName, byte[][], byte[][])}
*/
@Deprecated
public HTable createTable(byte[] tableName, byte[][] families, byte[][] splitRows)
throws IOException {
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
@ -1817,7 +1839,9 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
/**
* Drop an existing table
* @param tableName existing table
* @deprecated use {@link #deleteTable(TableName)}
*/
@Deprecated
public void deleteTable(String tableName) throws IOException {
deleteTable(TableName.valueOf(tableName));
}
@ -1825,7 +1849,9 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
/**
* Drop an existing table
* @param tableName existing table
* @deprecated use {@link #deleteTable(TableName)}
*/
@Deprecated
public void deleteTable(byte[] tableName) throws IOException {
deleteTable(TableName.valueOf(tableName));
}
@ -1870,9 +1896,27 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
public static final byte [] START_KEY_BYTES = {FIRST_CHAR, FIRST_CHAR, FIRST_CHAR};
public static final String START_KEY = new String(START_KEY_BYTES, HConstants.UTF8_CHARSET);
@Deprecated
public HTableDescriptor createTableDescriptor(final String name,
final int minVersions, final int versions, final int ttl, KeepDeletedCells keepDeleted) {
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
return this.createTableDescriptor(TableName.valueOf(name), minVersions, versions, ttl,
keepDeleted);
}
/**
* Create a table of name <code>name</code>.
* @param name Name to give table.
* @return Column descriptor.
*/
@Deprecated
public HTableDescriptor createTableDescriptor(final String name) {
return createTableDescriptor(TableName.valueOf(name), HColumnDescriptor.DEFAULT_MIN_VERSIONS,
MAXVERSIONS, HConstants.FOREVER, HColumnDescriptor.DEFAULT_KEEP_DELETED);
}
public HTableDescriptor createTableDescriptor(final TableName name,
final int minVersions, final int versions, final int ttl, KeepDeletedCells keepDeleted) {
HTableDescriptor htd = new HTableDescriptor(name);
for (byte[] cfName : new byte[][]{ fam1, fam2, fam3 }) {
htd.addFamily(new HColumnDescriptor(cfName)
.setMinVersions(minVersions)
@ -1890,7 +1934,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @param name Name to give table.
* @return Column descriptor.
*/
public HTableDescriptor createTableDescriptor(final String name) {
public HTableDescriptor createTableDescriptor(final TableName name) {
return createTableDescriptor(name, HColumnDescriptor.DEFAULT_MIN_VERSIONS,
MAXVERSIONS, HConstants.FOREVER, HColumnDescriptor.DEFAULT_KEEP_DELETED);
}
@ -1942,11 +1986,33 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @throws IOException
* @return A region on which you must call
{@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} when done.
* @deprecated use
* {@link #createLocalHRegion(TableName, byte[], byte[], boolean, Durability, WAL, byte[]...)}
*/
@Deprecated
public HRegion createLocalHRegion(byte[] tableName, byte[] startKey, byte[] stopKey,
String callingMethod, Configuration conf, boolean isReadOnly, Durability durability,
WAL wal, byte[]... families) throws IOException {
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
return this
.createLocalHRegion(TableName.valueOf(tableName), startKey, stopKey, isReadOnly, durability,
wal, families);
}
/**
* @param tableName
* @param startKey
* @param stopKey
* @param callingMethod
* @param conf
* @param isReadOnly
* @param families
* @return A region on which you must call
* {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} when done.
* @throws IOException
*/
public HRegion createLocalHRegion(TableName tableName, byte[] startKey, byte[] stopKey,
boolean isReadOnly, Durability durability, WAL wal, byte[]... families) throws IOException {
HTableDescriptor htd = new HTableDescriptor(tableName);
htd.setReadOnly(isReadOnly);
for (byte[] family : families) {
HColumnDescriptor hcd = new HColumnDescriptor(family);
@ -1958,6 +2024,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
HRegionInfo info = new HRegionInfo(htd.getTableName(), startKey, stopKey, false);
return createLocalHRegion(info, htd, wal);
}
//
// ==========================================================================
@ -1967,7 +2034,9 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* @param tableName existing table
* @return HTable to that new table
* @throws IOException
* @deprecated use {@link #deleteTableData(TableName)}
*/
@Deprecated
public HTable deleteTableData(byte[] tableName) throws IOException {
return deleteTableData(TableName.valueOf(tableName));
}
@ -2021,11 +2090,15 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
/**
* Truncate a table using the admin command.
* Effectively disables, deletes, and recreates the table.
* @param tableName table which must exist.
*
* @param tableName table which must exist.
* @param preserveRegions keep the existing split points
* @return HTable for the new table
* @deprecated use {@link #truncateTable(TableName, boolean)}
*/
public HTable truncateTable(final byte[] tableName, final boolean preserveRegions) throws IOException {
@Deprecated
public HTable truncateTable(final byte[] tableName, final boolean preserveRegions)
throws IOException {
return truncateTable(TableName.valueOf(tableName), preserveRegions);
}
@ -2035,11 +2108,14 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
* For previous behavior of issuing row deletes, see
* deleteTableData.
* Expressly does not preserve regions of existing table.
*
* @param tableName table which must exist.
* @return HTable for the new table
* @deprecated use {@link #truncateTable(TableName)}
*/
@Deprecated
public HTable truncateTable(final byte[] tableName) throws IOException {
return truncateTable(tableName, false);
return truncateTable(TableName.valueOf(tableName), false);
}
/**
@ -2395,7 +2471,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
public List<HRegionInfo> createMultiRegionsInMeta(final Configuration conf,
final HTableDescriptor htd, byte [][] startKeys)
throws IOException {
Table meta = (HTable) getConnection().getTable(TableName.META_TABLE_NAME);
Table meta = getConnection().getTable(TableName.META_TABLE_NAME);
Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR);
List<HRegionInfo> newRegions = new ArrayList<HRegionInfo>(startKeys.length);
MetaTableAccessor
@ -2455,7 +2531,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
*/
public List<byte[]> getMetaTableRows() throws IOException {
// TODO: Redo using MetaTableAccessor class
Table t = (HTable) getConnection().getTable(TableName.META_TABLE_NAME);
Table t = getConnection().getTable(TableName.META_TABLE_NAME);
List<byte[]> rows = new ArrayList<byte[]>();
ResultScanner s = t.getScanner(new Scan());
for (Result result : s) {
@ -3510,8 +3586,17 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
}
@Deprecated
public HTable createRandomTable(String tableName, final Collection<String> families,
final int maxVersions, final int numColsPerRow, final int numFlushes, final int numRegions,
final int numRowsPerFlush) throws IOException, InterruptedException {
return (HTable) this
.createRandomTable(TableName.valueOf(tableName), families, maxVersions, numColsPerRow,
numFlushes, numRegions, numRowsPerFlush);
}
/** Creates a random table with the given parameters */
public HTable createRandomTable(String tableName,
public Table createRandomTable(TableName tableName,
final Collection<String> families,
final int maxVersions,
final int numColsPerRow,
@ -3541,7 +3626,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
final int splitStartKey = actualStartKey + keysPerRegion;
final int splitEndKey = actualEndKey - keysPerRegion;
final String keyFormat = "%08x";
final HTable table = createTable(tableName, cfBytes,
final Table table = createTable(tableName, cfBytes,
maxVersions,
Bytes.toBytes(String.format(keyFormat, splitStartKey)),
Bytes.toBytes(String.format(keyFormat, splitEndKey)),
@ -3551,6 +3636,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
getMiniHBaseCluster().flushcache(TableName.META_TABLE_NAME);
}
BufferedMutator mutator = getConnection().getBufferedMutator(tableName);
for (int iFlush = 0; iFlush < numFlushes; ++iFlush) {
for (int iRow = 0; iRow < numRowsPerFlush; ++iRow) {
final byte[] row = Bytes.toBytes(String.format(keyFormat,
@ -3575,19 +3662,20 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
}
if (!put.isEmpty()) {
table.put(put);
mutator.mutate(put);
}
if (!del.isEmpty()) {
table.delete(del);
mutator.mutate(del);
}
}
LOG.info("Initiating flush #" + iFlush + " for table " + tableName);
table.flushCommits();
mutator.flush();
if (hbaseCluster != null) {
getMiniHBaseCluster().flushcache(table.getName());
}
}
mutator.close();
return table;
}

View File

@ -23,6 +23,7 @@ import static org.junit.Assert.assertEquals;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MiscTests;
import org.apache.hadoop.hbase.util.Bytes;
@ -91,7 +92,7 @@ public class TestFullLogReconstruction {
*/
@Test (timeout=300000)
public void testReconstruction() throws Exception {
HTable table = TEST_UTIL.createMultiRegionTable(TABLE_NAME, FAMILY);
Table table = TEST_UTIL.createMultiRegionTable(TABLE_NAME, FAMILY);
// Load up the table with simple rows and count them
int initialCount = TEST_UTIL.loadTable(table, FAMILY);

View File

@ -30,6 +30,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region;
@ -73,12 +74,12 @@ public class TestGlobalMemStoreSize {
cluster.waitForActiveAndReadyMaster();
// Create a table with regions
byte [] table = Bytes.toBytes("TestGlobalMemStoreSize");
TableName table = TableName.valueOf("TestGlobalMemStoreSize");
byte [] family = Bytes.toBytes("family");
LOG.info("Creating table with " + regionNum + " regions");
HTable ht = TEST_UTIL.createMultiRegionTable(TableName.valueOf(table), family, regionNum);
Table ht = TEST_UTIL.createMultiRegionTable(table, family, regionNum);
int numRegions = -1;
try (RegionLocator r = ht.getRegionLocator()) {
try (RegionLocator r = TEST_UTIL.getConnection().getRegionLocator(table)) {
numRegions = r.getStartKeys().length;
}
assertEquals(regionNum,numRegions);

View File

@ -21,6 +21,7 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hdfs.MiniDFSCluster;
@ -65,12 +66,11 @@ public class TestHBaseOnOtherDfsCluster {
// do a simple create/write to ensure the cluster works as expected
byte[] family = Bytes.toBytes("testfamily");
byte[] tablename = Bytes.toBytes("testtable");
HTable table = util2.createTable(tablename, family);
TableName tablename = TableName.valueOf("testtable");
Table table = util2.createTable(tablename, family);
Put p = new Put(new byte[] { 1, 2, 3 });
p.add(family, null, new byte[] { 1 });
p.addColumn(family, null, new byte[] { 1 });
table.put(p);
table.flushCommits();
// shutdown and make sure cleanly shutting down
util2.shutdownMiniCluster();

View File

@ -98,9 +98,9 @@ public class TestMetaTableAccessor {
final TableName name =
TableName.valueOf("testRetrying");
LOG.info("Started " + name);
HTable t = UTIL.createMultiRegionTable(name, HConstants.CATALOG_FAMILY);
Table t = UTIL.createMultiRegionTable(name, HConstants.CATALOG_FAMILY);
int regionCount = -1;
try (RegionLocator r = t.getRegionLocator()) {
try (RegionLocator r = UTIL.getConnection().getRegionLocator(name)) {
regionCount = r.getStartKeys().length;
}
// Test it works getting a region from just made user table.
@ -494,7 +494,7 @@ public class TestMetaTableAccessor {
new byte[][] { Bytes.toBytes("region_a"), Bytes.toBytes("region_b") };
UTIL.createTable(TABLENAME, FAMILY, SPLIT_KEYS);
HTable table = (HTable) connection.getTable(TABLENAME);
Table table = connection.getTable(TABLENAME);
// Make sure all the regions are deployed
UTIL.countRows(table);

View File

@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.MasterNotRunningException;
@ -86,8 +87,7 @@ public class TestAdmin1 {
TEST_UTIL.getConfiguration().setInt("hbase.regionserver.msginterval", 100);
TEST_UTIL.getConfiguration().setInt("hbase.client.pause", 250);
TEST_UTIL.getConfiguration().setInt("hbase.client.retries.number", 6);
TEST_UTIL.getConfiguration().setBoolean(
"hbase.master.enabletable.roundrobin", true);
TEST_UTIL.getConfiguration().setBoolean("hbase.master.enabletable.roundrobin", true);
TEST_UTIL.startMiniCluster(3);
}
@ -104,7 +104,7 @@ public class TestAdmin1 {
@After
public void tearDown() throws Exception {
for (HTableDescriptor htd : this.admin.listTables()) {
TEST_UTIL.deleteTable(htd.getName());
TEST_UTIL.deleteTable(htd.getTableName());
}
}
@ -362,9 +362,9 @@ public class TestAdmin1 {
TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close();
tables = this.admin.listTables();
assertEquals(numTables + 1, tables.length);
assertTrue("Table must be enabled.", TEST_UTIL.getHBaseCluster()
.getMaster().getAssignmentManager().getTableStateManager().isTableState(
tableName, TableState.State.ENABLED));
assertTrue("Table must be enabled.",
TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().getTableStateManager()
.isTableState(tableName, TableState.State.ENABLED));
assertEquals(TableState.State.ENABLED, getStateFromMeta(tableName));
}
@ -434,11 +434,11 @@ public class TestAdmin1 {
HTableDescriptor htd = new HTableDescriptor(tableName);
htd.addFamily(fam1);
this.admin.createTable(htd);
HTable table = (HTable)TEST_UTIL.getConnection().getTable(htd.getTableName());
Table table = TEST_UTIL.getConnection().getTable(htd.getTableName());
long ts = this.admin.getLastMajorCompactionTimestamp(tableName);
assertEquals(0, ts);
Put p = new Put(Bytes.toBytes("row1"));
p.add(Bytes.toBytes("fam1"), Bytes.toBytes("fam1"), Bytes.toBytes("fam1"));
p.addColumn(Bytes.toBytes("fam1"), Bytes.toBytes("fam1"), Bytes.toBytes("fam1"));
table.put(p);
ts = this.admin.getLastMajorCompactionTimestamp(tableName);
// no files written -> no data
@ -449,12 +449,14 @@ public class TestAdmin1 {
// still 0, we flushed a file, but no major compaction happened
assertEquals(0, ts);
byte[] regionName =
table.getRegionLocator().getAllRegionLocations().get(0).getRegionInfo().getRegionName();
byte[] regionName;
try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
regionName = l.getAllRegionLocations().get(0).getRegionInfo().getRegionName();
}
long ts1 = this.admin.getLastMajorCompactionTimestampForRegion(regionName);
assertEquals(ts, ts1);
p = new Put(Bytes.toBytes("row2"));
p.add(Bytes.toBytes("fam1"), Bytes.toBytes("fam1"), Bytes.toBytes("fam1"));
p.addColumn(Bytes.toBytes("fam1"), Bytes.toBytes("fam1"), Bytes.toBytes("fam1"));
table.put(p);
this.admin.flush(tableName);
ts = this.admin.getLastMajorCompactionTimestamp(tableName);
@ -623,19 +625,19 @@ public class TestAdmin1 {
"hbase.online.schema.update.enable", true);
}
@SuppressWarnings("deprecation")
protected void verifyRoundRobinDistribution(HTable ht, int expectedRegions) throws IOException {
int numRS = ht.getConnection().getCurrentNrHRS();
Map<HRegionInfo, ServerName> regions = ht.getRegionLocations();
protected void verifyRoundRobinDistribution(ClusterConnection c, RegionLocator regionLocator, int
expectedRegions) throws IOException {
int numRS = c.getCurrentNrHRS();
List<HRegionLocation> regions = regionLocator.getAllRegionLocations();
Map<ServerName, List<HRegionInfo>> server2Regions = new HashMap<ServerName, List<HRegionInfo>>();
for (Map.Entry<HRegionInfo, ServerName> entry : regions.entrySet()) {
ServerName server = entry.getValue();
for (HRegionLocation loc : regions) {
ServerName server = loc.getServerName();
List<HRegionInfo> regs = server2Regions.get(server);
if (regs == null) {
regs = new ArrayList<HRegionInfo>();
server2Regions.put(server, regs);
}
regs.add(entry.getKey());
regs.add(loc.getRegionInfo());
}
if (numRS >= 2) {
// Ignore the master region server,
@ -656,28 +658,29 @@ public class TestAdmin1 {
HTableDescriptor desc = new HTableDescriptor(tableName);
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
admin.createTable(desc);
HTable ht = (HTable) TEST_UTIL.getConnection().getTable(tableName);
Map<HRegionInfo, ServerName> regions = ht.getRegionLocations();
assertEquals("Table should have only 1 region", 1, regions.size());
ht.close();
List<HRegionLocation> regions;
try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
regions = l.getAllRegionLocations();
assertEquals("Table should have only 1 region", 1, regions.size());
}
TableName TABLE_2 = TableName.valueOf(tableName.getNameAsString() + "_2");
desc = new HTableDescriptor(TABLE_2);
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
admin.createTable(desc, new byte[][]{new byte[]{42}});
HTable ht2 = (HTable) TEST_UTIL.getConnection().getTable(TABLE_2);
regions = ht2.getRegionLocations();
assertEquals("Table should have only 2 region", 2, regions.size());
ht2.close();
try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(TABLE_2)) {
regions = l.getAllRegionLocations();
assertEquals("Table should have only 2 region", 2, regions.size());
}
TableName TABLE_3 = TableName.valueOf(tableName.getNameAsString() + "_3");
desc = new HTableDescriptor(TABLE_3);
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
admin.createTable(desc, "a".getBytes(), "z".getBytes(), 3);
HTable ht3 = (HTable) TEST_UTIL.getConnection().getTable(TABLE_3);
regions = ht3.getRegionLocations();
assertEquals("Table should have only 3 region", 3, regions.size());
ht3.close();
try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(TABLE_3)) {
regions = l.getAllRegionLocations();
assertEquals("Table should have only 3 region", 3, regions.size());
}
TableName TABLE_4 = TableName.valueOf(tableName.getNameAsString() + "_4");
desc = new HTableDescriptor(TABLE_4);
@ -692,11 +695,11 @@ public class TestAdmin1 {
TableName TABLE_5 = TableName.valueOf(tableName.getNameAsString() + "_5");
desc = new HTableDescriptor(TABLE_5);
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
admin.createTable(desc, new byte[] {1}, new byte[] {127}, 16);
HTable ht5 = (HTable) TEST_UTIL.getConnection().getTable(TABLE_5);
regions = ht5.getRegionLocations();
assertEquals("Table should have 16 region", 16, regions.size());
ht5.close();
admin.createTable(desc, new byte[] { 1 }, new byte[] { 127 }, 16);
try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(TABLE_5)) {
regions = l.getAllRegionLocations();
assertEquals("Table should have 16 region", 16, regions.size());
}
}
@Test (timeout=300000)
@ -724,47 +727,53 @@ public class TestAdmin1 {
boolean tableAvailable = admin.isTableAvailable(tableName, splitKeys);
assertTrue("Table should be created with splitKyes + 1 rows in META", tableAvailable);
HTable ht = (HTable) TEST_UTIL.getConnection().getTable(tableName);
Map<HRegionInfo, ServerName> regions = ht.getRegionLocations();
assertEquals("Tried to create " + expectedRegions + " regions " +
"but only found " + regions.size(),
expectedRegions, regions.size());
System.err.println("Found " + regions.size() + " regions");
List<HRegionLocation> regions;
Iterator<HRegionLocation> hris;
HRegionInfo hri;
ClusterConnection conn = (ClusterConnection) TEST_UTIL.getConnection();
try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
regions = l.getAllRegionLocations();
assertEquals("Tried to create " + expectedRegions + " regions " +
"but only found " + regions.size(), expectedRegions, regions.size());
System.err.println("Found " + regions.size() + " regions");
hris = regions.iterator();
hri = hris.next().getRegionInfo();
assertTrue(hri.getStartKey() == null || hri.getStartKey().length == 0);
assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[0]));
hri = hris.next().getRegionInfo();
assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[0]));
assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[1]));
hri = hris.next().getRegionInfo();
assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[1]));
assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[2]));
hri = hris.next().getRegionInfo();
assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[2]));
assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[3]));
hri = hris.next().getRegionInfo();
assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[3]));
assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[4]));
hri = hris.next().getRegionInfo();
assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[4]));
assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[5]));
hri = hris.next().getRegionInfo();
assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[5]));
assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[6]));
hri = hris.next().getRegionInfo();
assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[6]));
assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[7]));
hri = hris.next().getRegionInfo();
assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[7]));
assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[8]));
hri = hris.next().getRegionInfo();
assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[8]));
assertTrue(hri.getEndKey() == null || hri.getEndKey().length == 0);
verifyRoundRobinDistribution(conn, l, expectedRegions);
}
Iterator<HRegionInfo> hris = regions.keySet().iterator();
HRegionInfo hri = hris.next();
assertTrue(hri.getStartKey() == null || hri.getStartKey().length == 0);
assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[0]));
hri = hris.next();
assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[0]));
assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[1]));
hri = hris.next();
assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[1]));
assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[2]));
hri = hris.next();
assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[2]));
assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[3]));
hri = hris.next();
assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[3]));
assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[4]));
hri = hris.next();
assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[4]));
assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[5]));
hri = hris.next();
assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[5]));
assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[6]));
hri = hris.next();
assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[6]));
assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[7]));
hri = hris.next();
assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[7]));
assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[8]));
hri = hris.next();
assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[8]));
assertTrue(hri.getEndKey() == null || hri.getEndKey().length == 0);
verifyRoundRobinDistribution(ht, expectedRegions);
ht.close();
// Now test using start/end with a number of regions
@ -784,47 +793,46 @@ public class TestAdmin1 {
admin = TEST_UTIL.getHBaseAdmin();
admin.createTable(desc, startKey, endKey, expectedRegions);
HTable ht2 = (HTable) TEST_UTIL.getConnection().getTable(TABLE_2);
regions = ht2.getRegionLocations();
assertEquals("Tried to create " + expectedRegions + " regions " +
"but only found " + regions.size(),
expectedRegions, regions.size());
System.err.println("Found " + regions.size() + " regions");
try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(TABLE_2)) {
regions = l.getAllRegionLocations();
assertEquals("Tried to create " + expectedRegions + " regions " +
"but only found " + regions.size(), expectedRegions, regions.size());
System.err.println("Found " + regions.size() + " regions");
hris = regions.keySet().iterator();
hri = hris.next();
assertTrue(hri.getStartKey() == null || hri.getStartKey().length == 0);
assertTrue(Bytes.equals(hri.getEndKey(), new byte [] {1,1,1,1,1,1,1,1,1,1}));
hri = hris.next();
assertTrue(Bytes.equals(hri.getStartKey(), new byte [] {1,1,1,1,1,1,1,1,1,1}));
assertTrue(Bytes.equals(hri.getEndKey(), new byte [] {2,2,2,2,2,2,2,2,2,2}));
hri = hris.next();
assertTrue(Bytes.equals(hri.getStartKey(), new byte [] {2,2,2,2,2,2,2,2,2,2}));
assertTrue(Bytes.equals(hri.getEndKey(), new byte [] {3,3,3,3,3,3,3,3,3,3}));
hri = hris.next();
assertTrue(Bytes.equals(hri.getStartKey(), new byte [] {3,3,3,3,3,3,3,3,3,3}));
assertTrue(Bytes.equals(hri.getEndKey(), new byte [] {4,4,4,4,4,4,4,4,4,4}));
hri = hris.next();
assertTrue(Bytes.equals(hri.getStartKey(), new byte [] {4,4,4,4,4,4,4,4,4,4}));
assertTrue(Bytes.equals(hri.getEndKey(), new byte [] {5,5,5,5,5,5,5,5,5,5}));
hri = hris.next();
assertTrue(Bytes.equals(hri.getStartKey(), new byte [] {5,5,5,5,5,5,5,5,5,5}));
assertTrue(Bytes.equals(hri.getEndKey(), new byte [] {6,6,6,6,6,6,6,6,6,6}));
hri = hris.next();
assertTrue(Bytes.equals(hri.getStartKey(), new byte [] {6,6,6,6,6,6,6,6,6,6}));
assertTrue(Bytes.equals(hri.getEndKey(), new byte [] {7,7,7,7,7,7,7,7,7,7}));
hri = hris.next();
assertTrue(Bytes.equals(hri.getStartKey(), new byte [] {7,7,7,7,7,7,7,7,7,7}));
assertTrue(Bytes.equals(hri.getEndKey(), new byte [] {8,8,8,8,8,8,8,8,8,8}));
hri = hris.next();
assertTrue(Bytes.equals(hri.getStartKey(), new byte [] {8,8,8,8,8,8,8,8,8,8}));
assertTrue(Bytes.equals(hri.getEndKey(), new byte [] {9,9,9,9,9,9,9,9,9,9}));
hri = hris.next();
assertTrue(Bytes.equals(hri.getStartKey(), new byte [] {9,9,9,9,9,9,9,9,9,9}));
assertTrue(hri.getEndKey() == null || hri.getEndKey().length == 0);
hris = regions.iterator();
hri = hris.next().getRegionInfo();
assertTrue(hri.getStartKey() == null || hri.getStartKey().length == 0);
assertTrue(Bytes.equals(hri.getEndKey(), new byte[] { 1, 1, 1, 1, 1, 1, 1, 1, 1, 1 }));
hri = hris.next().getRegionInfo();
assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 1, 1, 1, 1, 1, 1, 1, 1, 1, 1 }));
assertTrue(Bytes.equals(hri.getEndKey(), new byte[] { 2, 2, 2, 2, 2, 2, 2, 2, 2, 2 }));
hri = hris.next().getRegionInfo();
assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 2, 2, 2, 2, 2, 2, 2, 2, 2, 2 }));
assertTrue(Bytes.equals(hri.getEndKey(), new byte[] { 3, 3, 3, 3, 3, 3, 3, 3, 3, 3 }));
hri = hris.next().getRegionInfo();
assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 3, 3, 3, 3, 3, 3, 3, 3, 3, 3 }));
assertTrue(Bytes.equals(hri.getEndKey(), new byte[] { 4, 4, 4, 4, 4, 4, 4, 4, 4, 4 }));
hri = hris.next().getRegionInfo();
assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 4, 4, 4, 4, 4, 4, 4, 4, 4, 4 }));
assertTrue(Bytes.equals(hri.getEndKey(), new byte[] { 5, 5, 5, 5, 5, 5, 5, 5, 5, 5 }));
hri = hris.next().getRegionInfo();
assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 5, 5, 5, 5, 5, 5, 5, 5, 5, 5 }));
assertTrue(Bytes.equals(hri.getEndKey(), new byte[] { 6, 6, 6, 6, 6, 6, 6, 6, 6, 6 }));
hri = hris.next().getRegionInfo();
assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 6, 6, 6, 6, 6, 6, 6, 6, 6, 6 }));
assertTrue(Bytes.equals(hri.getEndKey(), new byte[] { 7, 7, 7, 7, 7, 7, 7, 7, 7, 7 }));
hri = hris.next().getRegionInfo();
assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 7, 7, 7, 7, 7, 7, 7, 7, 7, 7 }));
assertTrue(Bytes.equals(hri.getEndKey(), new byte[] { 8, 8, 8, 8, 8, 8, 8, 8, 8, 8 }));
hri = hris.next().getRegionInfo();
assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 8, 8, 8, 8, 8, 8, 8, 8, 8, 8 }));
assertTrue(Bytes.equals(hri.getEndKey(), new byte[] { 9, 9, 9, 9, 9, 9, 9, 9, 9, 9 }));
hri = hris.next().getRegionInfo();
assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 9, 9, 9, 9, 9, 9, 9, 9, 9, 9 }));
assertTrue(hri.getEndKey() == null || hri.getEndKey().length == 0);
verifyRoundRobinDistribution(ht2, expectedRegions);
ht2.close();
verifyRoundRobinDistribution(conn, l, expectedRegions);
}
// Try once more with something that divides into something infinite
@ -841,15 +849,14 @@ public class TestAdmin1 {
admin.createTable(desc, startKey, endKey, expectedRegions);
HTable ht3 = (HTable) TEST_UTIL.getConnection().getTable(TABLE_3);
regions = ht3.getRegionLocations();
assertEquals("Tried to create " + expectedRegions + " regions " +
"but only found " + regions.size(),
expectedRegions, regions.size());
System.err.println("Found " + regions.size() + " regions");
try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(TABLE_3)) {
regions = l.getAllRegionLocations();
assertEquals("Tried to create " + expectedRegions + " regions " +
"but only found " + regions.size(), expectedRegions, regions.size());
System.err.println("Found " + regions.size() + " regions");
verifyRoundRobinDistribution(ht3, expectedRegions);
ht3.close();
verifyRoundRobinDistribution(conn, l, expectedRegions);
}
// Try an invalid case where there are duplicate split keys
@ -962,20 +969,22 @@ public class TestAdmin1 {
HTableDescriptor desc = new HTableDescriptor(tableName);
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
admin.createTable(desc, splitKeys);
HTable ht = (HTable) TEST_UTIL.getConnection().getTable(tableName);
Map<HRegionInfo, ServerName> regions = ht.getRegionLocations();
assertEquals("Tried to create " + expectedRegions + " regions "
+ "but only found " + regions.size(), expectedRegions, regions.size());
// Disable table.
admin.disableTable(tableName);
// Enable table, use retain assignment to assign regions.
admin.enableTable(tableName);
Map<HRegionInfo, ServerName> regions2 = ht.getRegionLocations();
// Check the assignment.
assertEquals(regions.size(), regions2.size());
for (Map.Entry<HRegionInfo, ServerName> entry : regions.entrySet()) {
assertEquals(regions2.get(entry.getKey()), entry.getValue());
try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
List<HRegionLocation> regions = l.getAllRegionLocations();
assertEquals(
"Tried to create " + expectedRegions + " regions " + "but only found " + regions.size(),
expectedRegions, regions.size());
// Disable table.
admin.disableTable(tableName);
// Enable table, use retain assignment to assign regions.
admin.enableTable(tableName);
List<HRegionLocation> regions2 = l.getAllRegionLocations();
// Check the assignment.
assertEquals(regions.size(), regions2.size());
assertTrue(regions2.containsAll(regions));
}
}
@ -1022,133 +1031,132 @@ public class TestAdmin1 {
sb.append("_").append(Integer.toString(rowCounts[i]));
}
assertFalse(admin.tableExists(tableName));
final HTable table = TEST_UTIL.createTable(tableName, familyNames,
try(final Table table = TEST_UTIL.createTable(tableName, familyNames,
numVersions, blockSize);
final RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
int rowCount = 0;
byte[] q = new byte[0];
int rowCount = 0;
byte[] q = new byte[0];
// insert rows into column families. The number of rows that have values
// in a specific column family is decided by rowCounts[familyIndex]
for (int index = 0; index < familyNames.length; index++) {
ArrayList<Put> puts = new ArrayList<Put>(rowCounts[index]);
for (int i = 0; i < rowCounts[index]; i++) {
byte[] k = Bytes.toBytes(i);
Put put = new Put(k);
put.add(familyNames[index], q, k);
puts.add(put);
}
table.put(puts);
if ( rowCount < rowCounts[index] ) {
rowCount = rowCounts[index];
}
}
// get the initial layout (should just be one region)
Map<HRegionInfo, ServerName> m = table.getRegionLocations();
LOG.info("Initial regions (" + m.size() + "): " + m);
assertTrue(m.size() == 1);
// Verify row count
Scan scan = new Scan();
ResultScanner scanner = table.getScanner(scan);
int rows = 0;
for(@SuppressWarnings("unused") Result result : scanner) {
rows++;
}
scanner.close();
assertEquals(rowCount, rows);
// Have an outstanding scan going on to make sure we can scan over splits.
scan = new Scan();
scanner = table.getScanner(scan);
// Scan first row so we are into first region before split happens.
scanner.next();
// Split the table
this.admin.split(tableName, splitPoint);
final AtomicInteger count = new AtomicInteger(0);
Thread t = new Thread("CheckForSplit") {
@Override
public void run() {
for (int i = 0; i < 45; i++) {
try {
sleep(1000);
} catch (InterruptedException e) {
continue;
}
// check again table = new HTable(conf, tableName);
Map<HRegionInfo, ServerName> regions = null;
try {
regions = table.getRegionLocations();
} catch (IOException e) {
e.printStackTrace();
}
if (regions == null) continue;
count.set(regions.size());
if (count.get() >= 2) {
LOG.info("Found: " + regions);
break;
}
LOG.debug("Cycle waiting on split");
// insert rows into column families. The number of rows that have values
// in a specific column family is decided by rowCounts[familyIndex]
for (int index = 0; index < familyNames.length; index++) {
ArrayList<Put> puts = new ArrayList<Put>(rowCounts[index]);
for (int i = 0; i < rowCounts[index]; i++) {
byte[] k = Bytes.toBytes(i);
Put put = new Put(k);
put.addColumn(familyNames[index], q, k);
puts.add(put);
}
LOG.debug("CheckForSplit thread exited, current region count: " + count.get());
}
};
t.setPriority(Thread.NORM_PRIORITY - 2);
t.start();
t.join();
table.put(puts);
// Verify row count
rows = 1; // We counted one row above.
for (@SuppressWarnings("unused") Result result : scanner) {
rows++;
if (rows > rowCount) {
scanner.close();
assertTrue("Scanned more than expected (" + rowCount + ")", false);
if (rowCount < rowCounts[index]) {
rowCount = rowCounts[index];
}
}
}
scanner.close();
assertEquals(rowCount, rows);
Map<HRegionInfo, ServerName> regions = null;
try {
regions = table.getRegionLocations();
} catch (IOException e) {
e.printStackTrace();
}
assertEquals(2, regions.size());
Set<HRegionInfo> hRegionInfos = regions.keySet();
HRegionInfo[] r = hRegionInfos.toArray(new HRegionInfo[hRegionInfos.size()]);
if (splitPoint != null) {
// make sure the split point matches our explicit configuration
assertEquals(Bytes.toString(splitPoint),
Bytes.toString(r[0].getEndKey()));
assertEquals(Bytes.toString(splitPoint),
Bytes.toString(r[1].getStartKey()));
LOG.debug("Properly split on " + Bytes.toString(splitPoint));
} else {
if (familyNames.length > 1) {
int splitKey = Bytes.toInt(r[0].getEndKey());
// check if splitKey is based on the largest column family
// in terms of it store size
int deltaForLargestFamily = Math.abs(rowCount/2 - splitKey);
LOG.debug("SplitKey=" + splitKey + "&deltaForLargestFamily=" + deltaForLargestFamily +
", r=" + r[0]);
for (int index = 0; index < familyNames.length; index++) {
int delta = Math.abs(rowCounts[index]/2 - splitKey);
if (delta < deltaForLargestFamily) {
assertTrue("Delta " + delta + " for family " + index
+ " should be at least deltaForLargestFamily " + deltaForLargestFamily,
false);
// get the initial layout (should just be one region)
List<HRegionLocation> m = locator.getAllRegionLocations();
LOG.info("Initial regions (" + m.size() + "): " + m);
assertTrue(m.size() == 1);
// Verify row count
Scan scan = new Scan();
ResultScanner scanner = table.getScanner(scan);
int rows = 0;
for (@SuppressWarnings("unused") Result result : scanner) {
rows++;
}
scanner.close();
assertEquals(rowCount, rows);
// Have an outstanding scan going on to make sure we can scan over splits.
scan = new Scan();
scanner = table.getScanner(scan);
// Scan first row so we are into first region before split happens.
scanner.next();
// Split the table
this.admin.split(tableName, splitPoint);
final AtomicInteger count = new AtomicInteger(0);
Thread t = new Thread("CheckForSplit") {
@Override
public void run() {
for (int i = 0; i < 45; i++) {
try {
sleep(1000);
} catch (InterruptedException e) {
continue;
}
// check again
List<HRegionLocation> regions = null;
try {
regions = locator.getAllRegionLocations();
} catch (IOException e) {
e.printStackTrace();
}
if (regions == null)
continue;
count.set(regions.size());
if (count.get() >= 2) {
LOG.info("Found: " + regions);
break;
}
LOG.debug("Cycle waiting on split");
}
LOG.debug("CheckForSplit thread exited, current region count: " + count.get());
}
};
t.setPriority(Thread.NORM_PRIORITY - 2);
t.start();
t.join();
// Verify row count
rows = 1; // We counted one row above.
for (@SuppressWarnings("unused") Result result : scanner) {
rows++;
if (rows > rowCount) {
scanner.close();
assertTrue("Scanned more than expected (" + rowCount + ")", false);
}
}
scanner.close();
assertEquals(rowCount, rows);
List<HRegionLocation> regions = null;
try {
regions = locator.getAllRegionLocations();
} catch (IOException e) {
e.printStackTrace();
}
assertEquals(2, regions.size());
if (splitPoint != null) {
// make sure the split point matches our explicit configuration
assertEquals(Bytes.toString(splitPoint),
Bytes.toString(regions.get(0).getRegionInfo().getEndKey()));
assertEquals(Bytes.toString(splitPoint),
Bytes.toString(regions.get(1).getRegionInfo().getStartKey()));
LOG.debug("Properly split on " + Bytes.toString(splitPoint));
} else {
if (familyNames.length > 1) {
int splitKey = Bytes.toInt(regions.get(0).getRegionInfo().getEndKey());
// check if splitKey is based on the largest column family
// in terms of it store size
int deltaForLargestFamily = Math.abs(rowCount / 2 - splitKey);
LOG.debug("SplitKey=" + splitKey + "&deltaForLargestFamily=" + deltaForLargestFamily +
", r=" + regions.get(0).getRegionInfo());
for (int index = 0; index < familyNames.length; index++) {
int delta = Math.abs(rowCounts[index] / 2 - splitKey);
if (delta < deltaForLargestFamily) {
assertTrue("Delta " + delta + " for family " + index + " should be at least "
+ "deltaForLargestFamily " + deltaForLargestFamily, false);
}
}
}
}
TEST_UTIL.deleteTable(tableName);
}
TEST_UTIL.deleteTable(tableName);
table.close();
}
@Test
@ -1173,7 +1181,7 @@ public class TestAdmin1 {
Thread.sleep(10);
} while (oldRegions.size() != 9); //3 regions * 3 replicas
// write some data to the table
HTable ht = (HTable) TEST_UTIL.getConnection().getTable(tableName);
Table ht = TEST_UTIL.getConnection().getTable(tableName);
List<Put> puts = new ArrayList<Put>();
byte[] qualifier = "c".getBytes();
Put put = new Put(new byte[]{(byte)'1'});
@ -1186,7 +1194,6 @@ public class TestAdmin1 {
put.add(cf, qualifier, "100".getBytes());
puts.add(put);
ht.put(puts);
ht.flushCommits();
ht.close();
List<Pair<HRegionInfo, ServerName>> regions =
MetaTableAccessor.getTableRegionsAndLocations(TEST_UTIL.getConnection(), tableName);

View File

@ -104,7 +104,7 @@ public class TestAdmin2 {
@After
public void tearDown() throws Exception {
for (HTableDescriptor htd : this.admin.listTables()) {
TEST_UTIL.deleteTable(htd.getName());
TEST_UTIL.deleteTable(htd.getTableName());
}
}
@ -693,15 +693,17 @@ public class TestAdmin2 {
final TableName tableName = TableName.valueOf("testGetRegion");
LOG.info("Started " + tableName);
HTable t = TEST_UTIL.createMultiRegionTable(tableName, HConstants.CATALOG_FAMILY);
Table t = TEST_UTIL.createMultiRegionTable(tableName, HConstants.CATALOG_FAMILY);
HRegionLocation regionLocation = t.getRegionLocation("mmm");
HRegionInfo region = regionLocation.getRegionInfo();
byte[] regionName = region.getRegionName();
Pair<HRegionInfo, ServerName> pair = rawAdmin.getRegion(regionName);
assertTrue(Bytes.equals(regionName, pair.getFirst().getRegionName()));
pair = rawAdmin.getRegion(region.getEncodedNameAsBytes());
assertTrue(Bytes.equals(regionName, pair.getFirst().getRegionName()));
try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
HRegionLocation regionLocation = locator.getRegionLocation(Bytes.toBytes("mmm"));
HRegionInfo region = regionLocation.getRegionInfo();
byte[] regionName = region.getRegionName();
Pair<HRegionInfo, ServerName> pair = rawAdmin.getRegion(regionName);
assertTrue(Bytes.equals(regionName, pair.getFirst().getRegionName()));
pair = rawAdmin.getRegion(region.getEncodedNameAsBytes());
assertTrue(Bytes.equals(regionName, pair.getFirst().getRegionName()));
}
}
@Test(timeout = 30000)

View File

@ -57,7 +57,7 @@ public class TestCheckAndMutate {
final TableName tableName = TableName.valueOf("TestPutWithDelete");
final byte[] rowKey = Bytes.toBytes("12345");
final byte[] family = Bytes.toBytes("cf");
HTable table = TEST_UTIL.createTable(tableName, family);
Table table = TEST_UTIL.createTable(tableName, family);
TEST_UTIL.waitTableAvailable(tableName.getName(), 5000);
try {
// put one row
@ -79,11 +79,11 @@ public class TestCheckAndMutate {
// put the same row again with C column deleted
RowMutations rm = new RowMutations(rowKey);
put = new Put(rowKey);
put.add(family, Bytes.toBytes("A"), Bytes.toBytes("a"));
put.add(family, Bytes.toBytes("B"), Bytes.toBytes("b"));
put.addColumn(family, Bytes.toBytes("A"), Bytes.toBytes("a"));
put.addColumn(family, Bytes.toBytes("B"), Bytes.toBytes("b"));
rm.add(put);
Delete del = new Delete(rowKey);
del.deleteColumn(family, Bytes.toBytes("C"));
del.addColumn(family, Bytes.toBytes("C"));
rm.add(del);
boolean res = table.checkAndMutate(rowKey, family, Bytes.toBytes("A"), CompareFilter.CompareOp.EQUAL,
Bytes.toBytes("a"), rm);

View File

@ -56,7 +56,7 @@ public class TestClientPushback {
private static final Log LOG = LogFactory.getLog(TestClientPushback.class);
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static final byte[] tableName = Bytes.toBytes("client-pushback");
private static final TableName tableName = TableName.valueOf("client-pushback");
private static final byte[] family = Bytes.toBytes("f");
private static final byte[] qualifier = Bytes.toBytes("q");
private static long flushSizeBytes = 1024;
@ -87,31 +87,28 @@ public class TestClientPushback {
@Test(timeout=60000)
public void testClientTracksServerPushback() throws Exception{
Configuration conf = UTIL.getConfiguration();
TableName tablename = TableName.valueOf(tableName);
Connection conn = ConnectionFactory.createConnection(conf);
HTable table = (HTable) conn.getTable(tablename);
ClusterConnection conn = (ClusterConnection) ConnectionFactory.createConnection(conf);
HTable table = (HTable) conn.getTable(tableName);
HRegionServer rs = UTIL.getHBaseCluster().getRegionServer(0);
Region region = rs.getOnlineRegions(tablename).get(0);
Region region = rs.getOnlineRegions(tableName).get(0);
LOG.debug("Writing some data to "+tablename);
LOG.debug("Writing some data to "+tableName);
// write some data
Put p = new Put(Bytes.toBytes("row"));
p.add(family, qualifier, Bytes.toBytes("value1"));
p.addColumn(family, qualifier, Bytes.toBytes("value1"));
table.put(p);
table.flushCommits();
// get the current load on RS. Hopefully memstore isn't flushed since we wrote the the data
int load = (int)((((HRegion)region).addAndGetGlobalMemstoreSize(0) * 100) / flushSizeBytes);
LOG.debug("Done writing some data to "+tablename);
LOG.debug("Done writing some data to "+tableName);
// get the stats for the region hosting our table
ClusterConnection connection = table.connection;
ClientBackoffPolicy backoffPolicy = connection.getBackoffPolicy();
ClientBackoffPolicy backoffPolicy = conn.getBackoffPolicy();
assertTrue("Backoff policy is not correctly configured",
backoffPolicy instanceof ExponentialClientBackoffPolicy);
ServerStatisticTracker stats = connection.getStatisticsTracker();
ServerStatisticTracker stats = conn.getStatisticsTracker();
assertNotNull( "No stats configured for the client!", stats);
// get the names so we can query the stats
ServerName server = rs.getServerName();
@ -135,8 +132,9 @@ public class TestClientPushback {
ops.add(p);
final CountDownLatch latch = new CountDownLatch(1);
final AtomicLong endTime = new AtomicLong();
long startTime = EnvironmentEdgeManager.currentTime();
table.mutator.ap.submit(tablename, ops, true, new Batch.Callback<Result>() {
long startTime = EnvironmentEdgeManager.currentTime();
table.mutator.ap.submit(tableName, ops, true, new Batch.Callback<Result>() {
@Override
public void update(byte[] region, byte[] row, Result result) {
endTime.set(EnvironmentEdgeManager.currentTime());

View File

@ -100,7 +100,7 @@ public class TestFromClientSide3 {
for (HTableDescriptor htd: TEST_UTIL.getHBaseAdmin().listTables()) {
LOG.info("Tear down, remove table=" + htd.getTableName());
TEST_UTIL.deleteTable(htd.getTableName());
}
}
}
private void randomCFPuts(Table table, byte[] row, byte[] family, int nPuts)
@ -114,28 +114,23 @@ public class TestFromClientSide3 {
table.put(put);
}
private void performMultiplePutAndFlush(HBaseAdmin admin, HTable table,
private void performMultiplePutAndFlush(HBaseAdmin admin, Table table,
byte[] row, byte[] family, int nFlushes, int nPuts)
throws Exception {
// connection needed for poll-wait
HRegionLocation loc = table.getRegionLocation(row, true);
AdminProtos.AdminService.BlockingInterface server =
admin.getConnection().getAdmin(loc.getServerName());
byte[] regName = loc.getRegionInfo().getRegionName();
try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(table.getName())) {
// connection needed for poll-wait
HRegionLocation loc = locator.getRegionLocation(row, true);
AdminProtos.AdminService.BlockingInterface server =
admin.getConnection().getAdmin(loc.getServerName());
byte[] regName = loc.getRegionInfo().getRegionName();
for (int i = 0; i < nFlushes; i++) {
randomCFPuts(table, row, family, nPuts);
List<String> sf = ProtobufUtil.getStoreFiles(server, regName, FAMILY);
int sfCount = sf.size();
for (int i = 0; i < nFlushes; i++) {
randomCFPuts(table, row, family, nPuts);
List<String> sf = ProtobufUtil.getStoreFiles(server, regName, FAMILY);
int sfCount = sf.size();
// TODO: replace this api with a synchronous flush after HBASE-2949
admin.flush(table.getName());
// synchronously poll wait for a new storefile to appear (flush happened)
while (ProtobufUtil.getStoreFiles(
server, regName, FAMILY).size() == sfCount) {
Thread.sleep(40);
admin.flush(table.getName());
}
}
}
@ -156,151 +151,147 @@ public class TestFromClientSide3 {
String tableName = "testAdvancedConfigOverride";
TableName TABLE = TableName.valueOf(tableName);
HTable hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10);
Table hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10);
HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
ClusterConnection connection = (ClusterConnection)TEST_UTIL.getConnection();
ClusterConnection connection = (ClusterConnection) TEST_UTIL.getConnection();
// Create 3 store files.
byte[] row = Bytes.toBytes(random.nextInt());
performMultiplePutAndFlush(admin, hTable, row, FAMILY, 3, 100);
// Verify we have multiple store files.
HRegionLocation loc = hTable.getRegionLocation(row, true);
byte[] regionName = loc.getRegionInfo().getRegionName();
AdminProtos.AdminService.BlockingInterface server =
connection.getAdmin(loc.getServerName());
assertTrue(ProtobufUtil.getStoreFiles(
server, regionName, FAMILY).size() > 1);
try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(TABLE)) {
// Verify we have multiple store files.
HRegionLocation loc = locator.getRegionLocation(row, true);
byte[] regionName = loc.getRegionInfo().getRegionName();
AdminProtos.AdminService.BlockingInterface server = connection.getAdmin(loc.getServerName());
assertTrue(ProtobufUtil.getStoreFiles(server, regionName, FAMILY).size() > 1);
// Issue a compaction request
admin.compact(TABLE.getName());
// Issue a compaction request
admin.compact(TABLE.getName());
// poll wait for the compactions to happen
for (int i = 0; i < 10 * 1000 / 40; ++i) {
// The number of store files after compaction should be lesser.
loc = hTable.getRegionLocation(row, true);
if (!loc.getRegionInfo().isOffline()) {
regionName = loc.getRegionInfo().getRegionName();
server = connection.getAdmin(loc.getServerName());
if (ProtobufUtil.getStoreFiles(
server, regionName, FAMILY).size() <= 1) {
break;
// poll wait for the compactions to happen
for (int i = 0; i < 10 * 1000 / 40; ++i) {
// The number of store files after compaction should be lesser.
loc = locator.getRegionLocation(row, true);
if (!loc.getRegionInfo().isOffline()) {
regionName = loc.getRegionInfo().getRegionName();
server = connection.getAdmin(loc.getServerName());
if (ProtobufUtil.getStoreFiles(server, regionName, FAMILY).size() <= 1) {
break;
}
}
Thread.sleep(40);
}
Thread.sleep(40);
}
// verify the compactions took place and that we didn't just time out
assertTrue(ProtobufUtil.getStoreFiles(
server, regionName, FAMILY).size() <= 1);
// verify the compactions took place and that we didn't just time out
assertTrue(ProtobufUtil.getStoreFiles(server, regionName, FAMILY).size() <= 1);
// change the compaction.min config option for this table to 5
LOG.info("hbase.hstore.compaction.min should now be 5");
HTableDescriptor htd = new HTableDescriptor(hTable.getTableDescriptor());
htd.setValue("hbase.hstore.compaction.min", String.valueOf(5));
admin.modifyTable(TABLE, htd);
Pair<Integer, Integer> st;
while (null != (st = admin.getAlterStatus(TABLE)) && st.getFirst() > 0) {
LOG.debug(st.getFirst() + " regions left to update");
Thread.sleep(40);
}
LOG.info("alter status finished");
// change the compaction.min config option for this table to 5
LOG.info("hbase.hstore.compaction.min should now be 5");
HTableDescriptor htd = new HTableDescriptor(hTable.getTableDescriptor());
htd.setValue("hbase.hstore.compaction.min", String.valueOf(5));
admin.modifyTable(TABLE, htd);
Pair<Integer, Integer> st;
while (null != (st = admin.getAlterStatus(TABLE)) && st.getFirst() > 0) {
LOG.debug(st.getFirst() + " regions left to update");
Thread.sleep(40);
}
LOG.info("alter status finished");
// Create 3 more store files.
performMultiplePutAndFlush(admin, hTable, row, FAMILY, 3, 10);
// Create 3 more store files.
performMultiplePutAndFlush(admin, hTable, row, FAMILY, 3, 10);
// Issue a compaction request
admin.compact(TABLE.getName());
// Issue a compaction request
admin.compact(TABLE.getName());
// This time, the compaction request should not happen
Thread.sleep(10 * 1000);
loc = hTable.getRegionLocation(row, true);
regionName = loc.getRegionInfo().getRegionName();
server = connection.getAdmin(loc.getServerName());
int sfCount = ProtobufUtil.getStoreFiles(
server, regionName, FAMILY).size();
assertTrue(sfCount > 1);
// change an individual CF's config option to 2 & online schema update
LOG.info("hbase.hstore.compaction.min should now be 2");
HColumnDescriptor hcd = new HColumnDescriptor(htd.getFamily(FAMILY));
hcd.setValue("hbase.hstore.compaction.min", String.valueOf(2));
htd.modifyFamily(hcd);
admin.modifyTable(TABLE, htd);
while (null != (st = admin.getAlterStatus(TABLE)) && st.getFirst() > 0) {
LOG.debug(st.getFirst() + " regions left to update");
Thread.sleep(40);
}
LOG.info("alter status finished");
// Issue a compaction request
admin.compact(TABLE.getName());
// poll wait for the compactions to happen
for (int i = 0; i < 10 * 1000 / 40; ++i) {
loc = hTable.getRegionLocation(row, true);
// This time, the compaction request should not happen
Thread.sleep(10 * 1000);
loc = locator.getRegionLocation(row, true);
regionName = loc.getRegionInfo().getRegionName();
try {
server = connection.getAdmin(loc.getServerName());
if (ProtobufUtil.getStoreFiles(
server, regionName, FAMILY).size() < sfCount) {
break;
}
} catch (Exception e) {
LOG.debug("Waiting for region to come online: " + regionName);
}
Thread.sleep(40);
}
// verify the compaction took place and that we didn't just time out
assertTrue(ProtobufUtil.getStoreFiles(
server, regionName, FAMILY).size() < sfCount);
server = connection.getAdmin(loc.getServerName());
int sfCount = ProtobufUtil.getStoreFiles(server, regionName, FAMILY).size();
assertTrue(sfCount > 1);
// Finally, ensure that we can remove a custom config value after we made it
LOG.info("Removing CF config value");
LOG.info("hbase.hstore.compaction.min should now be 5");
hcd = new HColumnDescriptor(htd.getFamily(FAMILY));
hcd.setValue("hbase.hstore.compaction.min", null);
htd.modifyFamily(hcd);
admin.modifyTable(TABLE, htd);
while (null != (st = admin.getAlterStatus(TABLE)) && st.getFirst() > 0) {
LOG.debug(st.getFirst() + " regions left to update");
Thread.sleep(40);
// change an individual CF's config option to 2 & online schema update
LOG.info("hbase.hstore.compaction.min should now be 2");
HColumnDescriptor hcd = new HColumnDescriptor(htd.getFamily(FAMILY));
hcd.setValue("hbase.hstore.compaction.min", String.valueOf(2));
htd.modifyFamily(hcd);
admin.modifyTable(TABLE, htd);
while (null != (st = admin.getAlterStatus(TABLE)) && st.getFirst() > 0) {
LOG.debug(st.getFirst() + " regions left to update");
Thread.sleep(40);
}
LOG.info("alter status finished");
// Issue a compaction request
admin.compact(TABLE.getName());
// poll wait for the compactions to happen
for (int i = 0; i < 10 * 1000 / 40; ++i) {
loc = locator.getRegionLocation(row, true);
regionName = loc.getRegionInfo().getRegionName();
try {
server = connection.getAdmin(loc.getServerName());
if (ProtobufUtil.getStoreFiles(server, regionName, FAMILY).size() < sfCount) {
break;
}
} catch (Exception e) {
LOG.debug("Waiting for region to come online: " + regionName);
}
Thread.sleep(40);
}
// verify the compaction took place and that we didn't just time out
assertTrue(ProtobufUtil.getStoreFiles(
server, regionName, FAMILY).size() < sfCount);
// Finally, ensure that we can remove a custom config value after we made it
LOG.info("Removing CF config value");
LOG.info("hbase.hstore.compaction.min should now be 5");
hcd = new HColumnDescriptor(htd.getFamily(FAMILY));
hcd.setValue("hbase.hstore.compaction.min", null);
htd.modifyFamily(hcd);
admin.modifyTable(TABLE, htd);
while (null != (st = admin.getAlterStatus(TABLE)) && st.getFirst() > 0) {
LOG.debug(st.getFirst() + " regions left to update");
Thread.sleep(40);
}
LOG.info("alter status finished");
assertNull(hTable.getTableDescriptor().getFamily(FAMILY).getValue(
"hbase.hstore.compaction.min"));
}
LOG.info("alter status finished");
assertNull(hTable.getTableDescriptor().getFamily(FAMILY).getValue(
"hbase.hstore.compaction.min"));
}
@Test
public void testHTableBatchWithEmptyPut() throws Exception {
Table table = TEST_UTIL.createTable(
Bytes.toBytes("testHTableBatchWithEmptyPut"), new byte[][] { FAMILY });
public void testHTableBatchWithEmptyPut ()throws Exception {
Table table = TEST_UTIL.createTable(TableName.valueOf("testHTableBatchWithEmptyPut"),
new byte[][] { FAMILY });
try {
List actions = (List) new ArrayList();
Object[] results = new Object[2];
// create an empty Put
Put put1 = new Put(ROW);
actions.add(put1);
Put put2 = new Put(ANOTHERROW);
put2.add(FAMILY, QUALIFIER, VALUE);
actions.add(put2);
table.batch(actions, results);
fail("Empty Put should have failed the batch call");
} catch (IllegalArgumentException iae) {
} finally {
table.close();
}
}
@Test
public void testHTableExistsMethodSingleRegionSingleGet() throws Exception {
// Test with a single region table.
Table table = TEST_UTIL.createTable(
Bytes.toBytes("testHTableExistsMethodSingleRegionSingleGet"), new byte[][] { FAMILY });
// Test with a single region table.
Table table = TEST_UTIL.createTable(
TableName.valueOf("testHTableExistsMethodSingleRegionSingleGet"),
new byte[][] { FAMILY });
Put put = new Put(ROW);
put.add(FAMILY, QUALIFIER, VALUE);
@ -317,9 +308,8 @@ public class TestFromClientSide3 {
}
public void testHTableExistsMethodSingleRegionMultipleGets() throws Exception {
HTable table = TEST_UTIL.createTable(
Bytes.toBytes("testHTableExistsMethodSingleRegionMultipleGets"), new byte[][] { FAMILY });
Table table = TEST_UTIL.createTable(TableName.valueOf(
"testHTableExistsMethodSingleRegionMultipleGets"), new byte[][] { FAMILY });
Put put = new Put(ROW);
put.add(FAMILY, QUALIFIER, VALUE);
@ -330,7 +320,7 @@ public class TestFromClientSide3 {
gets.add(null);
gets.add(new Get(ANOTHERROW));
Boolean[] results = table.exists(gets);
boolean[] results = table.existsAll(gets);
assertEquals(results[0], true);
assertEquals(results[1], false);
assertEquals(results[2], false);
@ -338,11 +328,11 @@ public class TestFromClientSide3 {
@Test
public void testHTableExistsBeforeGet() throws Exception {
Table table = TEST_UTIL.createTable(
Bytes.toBytes("testHTableExistsBeforeGet"), new byte[][] { FAMILY });
Table table = TEST_UTIL.createTable(TableName.valueOf("testHTableExistsBeforeGet"),
new byte[][] { FAMILY });
try {
Put put = new Put(ROW);
put.add(FAMILY, QUALIFIER, VALUE);
put.addColumn(FAMILY, QUALIFIER, VALUE);
table.put(put);
Get get = new Get(ROW);
@ -362,13 +352,13 @@ public class TestFromClientSide3 {
public void testHTableExistsAllBeforeGet() throws Exception {
final byte[] ROW2 = Bytes.add(ROW, Bytes.toBytes("2"));
Table table = TEST_UTIL.createTable(
Bytes.toBytes("testHTableExistsAllBeforeGet"), new byte[][] { FAMILY });
TableName.valueOf("testHTableExistsAllBeforeGet"), new byte[][] { FAMILY });
try {
Put put = new Put(ROW);
put.add(FAMILY, QUALIFIER, VALUE);
put.addColumn(FAMILY, QUALIFIER, VALUE);
table.put(put);
put = new Put(ROW2);
put.add(FAMILY, QUALIFIER, VALUE);
put.addColumn(FAMILY, QUALIFIER, VALUE);
table.put(put);
Get get = new Get(ROW);
@ -393,12 +383,11 @@ public class TestFromClientSide3 {
@Test
public void testHTableExistsMethodMultipleRegionsSingleGet() throws Exception {
Table table = TEST_UTIL.createTable(
TableName.valueOf("testHTableExistsMethodMultipleRegionsSingleGet"), new byte[][] { FAMILY },
1, new byte[] { 0x00 }, new byte[] { (byte) 0xff }, 255);
Put put = new Put(ROW);
put.add(FAMILY, QUALIFIER, VALUE);
put.addColumn(FAMILY, QUALIFIER, VALUE);
Get get = new Get(ROW);
@ -413,8 +402,8 @@ public class TestFromClientSide3 {
@Test
public void testHTableExistsMethodMultipleRegionsMultipleGets() throws Exception {
HTable table = TEST_UTIL.createTable(
TableName.valueOf("testHTableExistsMethodMultipleRegionsMultipleGets"),
Table table = TEST_UTIL.createTable(
TableName.valueOf("testHTableExistsMethodMultipleRegionsMultipleGets"),
new byte[][] { FAMILY }, 1, new byte[] { 0x00 }, new byte[] { (byte) 0xff }, 255);
Put put = new Put(ROW);
put.add(FAMILY, QUALIFIER, VALUE);
@ -427,7 +416,7 @@ public class TestFromClientSide3 {
gets.add(new Get(Bytes.add(ANOTHERROW, new byte[] { 0x00 })));
LOG.info("Calling exists");
Boolean[] results = table.exists(gets);
boolean[] results = table.existsAll(gets);
assertEquals(results[0], false);
assertEquals(results[1], false);
assertEquals(results[2], true);
@ -441,7 +430,7 @@ public class TestFromClientSide3 {
gets = new ArrayList<Get>();
gets.add(new Get(new byte[] { 0x00 }));
gets.add(new Get(new byte[] { 0x00, 0x00 }));
results = table.exists(gets);
results = table.existsAll(gets);
assertEquals(results[0], true);
assertEquals(results[1], false);
@ -454,7 +443,7 @@ public class TestFromClientSide3 {
gets.add(new Get(new byte[] { (byte) 0xff }));
gets.add(new Get(new byte[] { (byte) 0xff, (byte) 0xff }));
gets.add(new Get(new byte[] { (byte) 0xff, (byte) 0xff, (byte) 0xff }));
results = table.exists(gets);
results = table.existsAll(gets);
assertEquals(results[0], false);
assertEquals(results[1], true);
assertEquals(results[2], false);
@ -491,8 +480,8 @@ public class TestFromClientSide3 {
@Test
public void testLeaseRenewal() throws Exception {
HTable table = TEST_UTIL.createTable(
Bytes.toBytes("testLeaseRenewal"), FAMILY);
Table table = TEST_UTIL.createTable(
TableName.valueOf("testLeaseRenewal"), FAMILY);
Put p = new Put(ROW_BYTES);
p.addColumn(FAMILY, COL_QUAL, VAL_BYTES);
table.put(p);

View File

@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ipc.AbstractRpcClient;
import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
@ -61,10 +62,10 @@ public class TestFromClientSideNoCodec {
@Test
public void testBasics() throws IOException {
final byte [] t = Bytes.toBytes("testBasics");
final TableName t = TableName.valueOf("testBasics");
final byte [][] fs = new byte[][] {Bytes.toBytes("cf1"), Bytes.toBytes("cf2"),
Bytes.toBytes("cf3") };
HTable ht = TEST_UTIL.createTable(t, fs);
Table ht = TEST_UTIL.createTable(t, fs);
// Check put and get.
final byte [] row = Bytes.toBytes("row");
Put p = new Put(row);
@ -79,10 +80,13 @@ public class TestFromClientSideNoCodec {
Bytes.equals(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength(),
f, 0, f.length));
}
// Check getRowOrBefore
byte [] f = fs[0];
r = ht.getRowOrBefore(row, f);
assertTrue(r.toString(), r.containsColumn(f, f));
if(ht instanceof HTableInterface) {
HTableInterface hti = (HTableInterface) ht;
// Check getRowOrBefore
byte[] f = fs[0];
r = hti.getRowOrBefore(row, f);
assertTrue(r.toString(), r.containsColumn(f, f));
}
// Check scan.
ResultScanner scanner = ht.getScanner(new Scan());
int count = 0;

View File

@ -139,33 +139,34 @@ public class TestHCM {
Connection con1 = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
Connection con2 = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration(), otherPool);
// make sure the internally created ExecutorService is the one passed
assertTrue(otherPool == ((ConnectionImplementation)con2).getCurrentBatchPool());
assertTrue(otherPool == ((ConnectionImplementation) con2).getCurrentBatchPool());
String tableName = "testClusterConnection";
TEST_UTIL.createTable(tableName.getBytes(), FAM_NAM).close();
HTable t = (HTable)con1.getTable(TableName.valueOf(tableName), otherPool);
TableName tableName = TableName.valueOf("testClusterConnection");
TEST_UTIL.createTable(tableName, FAM_NAM).close();
HTable t = (HTable)con1.getTable(tableName, otherPool);
// make sure passing a pool to the getTable does not trigger creation of an internal pool
assertNull("Internal Thread pool should be null", ((ConnectionImplementation)con1).getCurrentBatchPool());
assertNull("Internal Thread pool should be null",
((ConnectionImplementation) con1).getCurrentBatchPool());
// table should use the pool passed
assertTrue(otherPool == t.getPool());
t.close();
t = (HTable)con2.getTable(TableName.valueOf(tableName));
t = (HTable)con2.getTable(tableName);
// table should use the connectin's internal pool
assertTrue(otherPool == t.getPool());
t.close();
t = (HTable)con2.getTable(TableName.valueOf(tableName));
t = (HTable)con2.getTable(tableName);
// try other API too
assertTrue(otherPool == t.getPool());
t.close();
t = (HTable)con2.getTable(TableName.valueOf(tableName));
t = (HTable)con2.getTable(tableName);
// try other API too
assertTrue(otherPool == t.getPool());
t.close();
t = (HTable)con1.getTable(TableName.valueOf(tableName));
t = (HTable)con1.getTable(tableName);
ExecutorService pool = ((ConnectionImplementation)con1).getCurrentBatchPool();
// make sure an internal pool was created
assertNotNull("An internal Thread pool should have been created", pool);
@ -173,7 +174,7 @@ public class TestHCM {
assertTrue(t.getPool() == pool);
t.close();
t = (HTable)con1.getTable(TableName.valueOf(tableName));
t = (HTable)con1.getTable(tableName);
// still using the *same* internal pool
assertTrue(t.getPool() == pool);
t.close();
@ -214,25 +215,27 @@ public class TestHCM {
rs.waitForServerOnline();
final ServerName sn = rs.getRegionServer().getServerName();
HTable t = TEST_UTIL.createTable(tn, cf);
Table t = TEST_UTIL.createTable(tn, cf);
TEST_UTIL.waitTableAvailable(tn);
while(TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().
getRegionStates().isRegionsInTransition()){
Thread.sleep(1);
}
final ConnectionImplementation hci = (ConnectionImplementation)t.getConnection();
while (t.getRegionLocation(rk).getPort() != sn.getPort()){
TEST_UTIL.getHBaseAdmin().move(t.getRegionLocation(rk).getRegionInfo().
getEncodedNameAsBytes(), Bytes.toBytes(sn.toString()));
while(TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().
getRegionStates().isRegionsInTransition()){
Thread.sleep(1);
final ConnectionImplementation hci = (ConnectionImplementation)TEST_UTIL.getConnection();
try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(tn)) {
while (l.getRegionLocation(rk).getPort() != sn.getPort()) {
TEST_UTIL.getHBaseAdmin().move(l.getRegionLocation(rk).getRegionInfo().
getEncodedNameAsBytes(), Bytes.toBytes(sn.toString()));
while (TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().
getRegionStates().isRegionsInTransition()) {
Thread.sleep(1);
}
hci.clearRegionCache(tn);
}
hci.clearRegionCache(tn);
Assert.assertNotNull(hci.clusterStatusListener);
TEST_UTIL.assertRegionOnServer(l.getRegionLocation(rk).getRegionInfo(), sn, 20000);
}
Assert.assertNotNull(hci.clusterStatusListener);
TEST_UTIL.assertRegionOnServer(t.getRegionLocation(rk).getRegionInfo(), sn, 20000);
Put p1 = new Put(rk);
p1.add(cf, "qual".getBytes(), "val".getBytes());
@ -286,27 +289,31 @@ public class TestHCM {
public void testOperationTimeout() throws Exception {
HTableDescriptor hdt = TEST_UTIL.createTableDescriptor("HCM-testOperationTimeout");
hdt.addCoprocessor(SleepAndFailFirstTime.class.getName());
HTable table = TEST_UTIL.createTable(hdt, new byte[][]{FAM_NAM}, TEST_UTIL.getConfiguration());
Table t = TEST_UTIL.createTable(hdt, new byte[][]{FAM_NAM}, null);
// Check that it works if the timeout is big enough
table.setOperationTimeout(120 * 1000);
table.get(new Get(FAM_NAM));
if (t instanceof HTable) {
HTable table = (HTable) t;
// Resetting and retrying. Will fail this time, not enough time for the second try
SleepAndFailFirstTime.ct.set(0);
try {
table.setOperationTimeout(30 * 1000);
// Check that it works if the timeout is big enough
table.setOperationTimeout(120 * 1000);
table.get(new Get(FAM_NAM));
Assert.fail("We expect an exception here");
} catch (SocketTimeoutException e) {
// The client has a CallTimeout class, but it's not shared.We're not very clean today,
// in the general case you can expect the call to stop, but the exception may vary.
// In this test however, we're sure that it will be a socket timeout.
LOG.info("We received an exception, as expected ", e);
} catch (IOException e) {
Assert.fail("Wrong exception:" + e.getMessage());
} finally {
table.close();
// Resetting and retrying. Will fail this time, not enough time for the second try
SleepAndFailFirstTime.ct.set(0);
try {
table.setOperationTimeout(30 * 1000);
table.get(new Get(FAM_NAM));
Assert.fail("We expect an exception here");
} catch (SocketTimeoutException e) {
// The client has a CallTimeout class, but it's not shared.We're not very clean today,
// in the general case you can expect the call to stop, but the exception may vary.
// In this test however, we're sure that it will be a socket timeout.
LOG.info("We received an exception, as expected ", e);
} catch (IOException e) {
Assert.fail("Wrong exception:" + e.getMessage());
} finally {
table.close();
}
}
}
@ -326,10 +333,10 @@ public class TestHCM {
c2.setBoolean(RpcClient.SPECIFIC_WRITE_THREAD, allowsInterrupt);
Connection connection = ConnectionFactory.createConnection(c2);
final HTable table = (HTable) connection.getTable(tableName);
final Table table = connection.getTable(tableName);
Put put = new Put(ROW);
put.add(FAM_NAM, ROW, ROW);
put.addColumn(FAM_NAM, ROW, ROW);
table.put(put);
// 4 steps: ready=0; doGets=1; mustStop=2; stopped=3
@ -364,9 +371,12 @@ public class TestHCM {
}
});
ServerName sn = table.getRegionLocation(ROW).getServerName();
ServerName sn;
try(RegionLocator rl = connection.getRegionLocator(tableName)) {
sn = rl.getRegionLocation(ROW).getServerName();
}
ConnectionImplementation conn =
(ConnectionImplementation) table.getConnection();
(ConnectionImplementation) connection;
RpcClient rpcClient = conn.getRpcClient();
LOG.info("Going to cancel connections. connection=" + conn.toString() + ", sn=" + sn);
@ -469,14 +479,18 @@ public class TestHCM {
c2.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 30 * 1000);
final Connection connection = ConnectionFactory.createConnection(c2);
final HTable table = (HTable) connection.getTable(tableName);
final Table table = connection.getTable(tableName);
Put p = new Put(FAM_NAM);
p.add(FAM_NAM, FAM_NAM, FAM_NAM);
p.addColumn(FAM_NAM, FAM_NAM, FAM_NAM);
table.put(p);
final ConnectionImplementation hci = (ConnectionImplementation)table.getConnection();
final HRegionLocation loc = table.getRegionLocation(FAM_NAM);
final ConnectionImplementation hci = (ConnectionImplementation) connection;
final HRegionLocation loc;
try(RegionLocator rl = connection.getRegionLocator(tableName)) {
loc = rl.getRegionLocation(FAM_NAM);
}
Get get = new Get(FAM_NAM);
Assert.assertNotNull(table.get(get));
@ -553,14 +567,13 @@ public class TestHCM {
Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
Connection connection = ConnectionFactory.createConnection(conf);
final HTable table = (HTable) connection.getTable(TABLE_NAME);
final Table table = connection.getTable(TABLE_NAME);
TEST_UTIL.waitUntilAllRegionsAssigned(table.getName());
Put put = new Put(ROW);
put.add(FAM_NAM, ROW, ROW);
table.put(put);
ConnectionImplementation conn =
(ConnectionImplementation)table.getConnection();
ConnectionImplementation conn = (ConnectionImplementation) connection;
assertNotNull(conn.getCachedLocation(TABLE_NAME, ROW));
@ -756,12 +769,11 @@ public class TestHCM {
*/
@Test(timeout = 60000)
public void testCacheSeqNums() throws Exception{
HTable table = TEST_UTIL.createMultiRegionTable(TABLE_NAME2, FAM_NAM);
Table table = TEST_UTIL.createMultiRegionTable(TABLE_NAME2, FAM_NAM);
Put put = new Put(ROW);
put.add(FAM_NAM, ROW, ROW);
table.put(put);
ConnectionImplementation conn =
(ConnectionImplementation)table.getConnection();
ConnectionImplementation conn = (ConnectionImplementation) TEST_UTIL.getConnection();
HRegionLocation location = conn.getCachedLocation(TABLE_NAME2, ROW).getRegionLocation();
assertNotNull(location);
@ -870,10 +882,10 @@ public class TestHCM {
@Test (timeout=30000)
public void testMulti() throws Exception {
HTable table = TEST_UTIL.createMultiRegionTable(TABLE_NAME3, FAM_NAM);
Table table = TEST_UTIL.createMultiRegionTable(TABLE_NAME3, FAM_NAM);
try {
ConnectionImplementation conn =
(ConnectionImplementation)table.getConnection();
(ConnectionImplementation)TEST_UTIL.getConnection();
// We're now going to move the region and check that it works for the client
// First a new put to add the location in the cache
@ -1053,7 +1065,7 @@ public class TestHCM {
Configuration config = new Configuration(TEST_UTIL.getConfiguration());
TableName tableName = TableName.valueOf("testConnectionRideOverClusterRestart");
TEST_UTIL.createTable(tableName.getName(), new byte[][] {FAM_NAM}, config).close();
TEST_UTIL.createTable(tableName, new byte[][] {FAM_NAM}).close();
Connection connection = ConnectionFactory.createConnection(config);
Table table = connection.getTable(tableName);

View File

@ -64,7 +64,7 @@ public class TestHTableMultiplexer {
TEST_UTIL.shutdownMiniCluster();
}
private static void checkExistence(HTable htable, byte[] row, byte[] family, byte[] quality)
private static void checkExistence(Table htable, byte[] row, byte[] family, byte[] quality)
throws Exception {
// verify that the Get returns the correct result
Result r;
@ -93,63 +93,65 @@ public class TestHTableMultiplexer {
HTableMultiplexer multiplexer = new HTableMultiplexer(TEST_UTIL.getConfiguration(),
PER_REGIONSERVER_QUEUE_SIZE);
HTable htable1 =
Table htable1 =
TEST_UTIL.createTable(TABLE_1, new byte[][] { FAMILY }, VERSION,
Bytes.toBytes("aaaaa"), Bytes.toBytes("zzzzz"), NUM_REGIONS);
HTable htable2 =
Table htable2 =
TEST_UTIL.createTable(TABLE_2, new byte[][] { FAMILY }, VERSION, Bytes.toBytes("aaaaa"),
Bytes.toBytes("zzzzz"), NUM_REGIONS);
TEST_UTIL.waitUntilAllRegionsAssigned(TABLE_1);
TEST_UTIL.waitUntilAllRegionsAssigned(TABLE_2);
byte[][] startRows = htable1.getStartKeys();
byte[][] endRows = htable1.getEndKeys();
try (RegionLocator rl = TEST_UTIL.getConnection().getRegionLocator(TABLE_1)) {
byte[][] startRows = rl.getStartKeys();
byte[][] endRows = rl.getEndKeys();
// SinglePut case
for (int i = 0; i < NUM_REGIONS; i++) {
byte [] row = startRows[i];
if (row == null || row.length <= 0) continue;
Put put = new Put(row).add(FAMILY, QUALIFIER, VALUE1);
success = multiplexer.put(TABLE_1, put);
assertTrue("multiplexer.put returns", success);
// SinglePut case
for (int i = 0; i < NUM_REGIONS; i++) {
byte [] row = startRows[i];
if (row == null || row.length <= 0) continue;
Put put = new Put(row).add(FAMILY, QUALIFIER, VALUE1);
success = multiplexer.put(TABLE_1, put);
assertTrue("multiplexer.put returns", success);
put = new Put(row).add(FAMILY, QUALIFIER, VALUE1);
success = multiplexer.put(TABLE_2, put);
assertTrue("multiplexer.put failed", success);
put = new Put(row).add(FAMILY, QUALIFIER, VALUE1);
success = multiplexer.put(TABLE_2, put);
assertTrue("multiplexer.put failed", success);
LOG.info("Put for " + Bytes.toStringBinary(startRows[i]) + " @ iteration " + (i + 1));
LOG.info("Put for " + Bytes.toStringBinary(startRows[i]) + " @ iteration " + (i + 1));
// verify that the Get returns the correct result
checkExistence(htable1, startRows[i], FAMILY, QUALIFIER);
checkExistence(htable2, startRows[i], FAMILY, QUALIFIER);
}
// MultiPut case
List<Put> multiput = new ArrayList<Put>();
for (int i = 0; i < NUM_REGIONS; i++) {
byte [] row = endRows[i];
if (row == null || row.length <= 0) continue;
Put put = new Put(row);
put.add(FAMILY, QUALIFIER, VALUE2);
multiput.add(put);
}
failedPuts = multiplexer.put(TABLE_1, multiput);
assertTrue(failedPuts == null);
// verify that the Get returns the correct result
checkExistence(htable1, startRows[i], FAMILY, QUALIFIER);
checkExistence(htable2, startRows[i], FAMILY, QUALIFIER);
}
// MultiPut case
List<Put> multiput = new ArrayList<Put>();
for (int i = 0; i < NUM_REGIONS; i++) {
byte [] row = endRows[i];
if (row == null || row.length <= 0) continue;
Put put = new Put(row);
put.add(FAMILY, QUALIFIER, VALUE2);
multiput.add(put);
}
failedPuts = multiplexer.put(TABLE_1, multiput);
assertTrue(failedPuts == null);
// verify that the Get returns the correct result
for (int i = 0; i < NUM_REGIONS; i++) {
byte [] row = endRows[i];
if (row == null || row.length <= 0) continue;
Get get = new Get(row);
get.addColumn(FAMILY, QUALIFIER);
Result r;
int nbTry = 0;
do {
assertTrue(nbTry++ < 50);
Thread.sleep(100);
r = htable1.get(get);
} while (r == null || r.getValue(FAMILY, QUALIFIER) == null ||
Bytes.compareTo(VALUE2, r.getValue(FAMILY, QUALIFIER)) != 0);
for (int i = 0; i < NUM_REGIONS; i++) {
byte [] row = endRows[i];
if (row == null || row.length <= 0) continue;
Get get = new Get(row);
get.addColumn(FAMILY, QUALIFIER);
Result r;
int nbTry = 0;
do {
assertTrue(nbTry++ < 50);
Thread.sleep(100);
r = htable1.get(get);
} while (r == null || r.getValue(FAMILY, QUALIFIER) == null ||
Bytes.compareTo(VALUE2, r.getValue(FAMILY, QUALIFIER)) != 0);
}
}
}
}

View File

@ -64,7 +64,7 @@ public class TestHTableMultiplexerFlushCache {
TEST_UTIL.shutdownMiniCluster();
}
private static void checkExistence(final HTable htable, final byte[] row, final byte[] family,
private static void checkExistence(final Table htable, final byte[] row, final byte[] family,
final byte[] quality,
final byte[] value) throws Exception {
// verify that the Get returns the correct result
@ -86,31 +86,33 @@ public class TestHTableMultiplexerFlushCache {
public void testOnRegionChange() throws Exception {
TableName TABLE = TableName.valueOf("testOnRegionChange");
final int NUM_REGIONS = 10;
HTable htable = TEST_UTIL.createTable(TABLE, new byte[][] { FAMILY }, 3,
Table htable = TEST_UTIL.createTable(TABLE, new byte[][] { FAMILY }, 3,
Bytes.toBytes("aaaaa"), Bytes.toBytes("zzzzz"), NUM_REGIONS);
HTableMultiplexer multiplexer = new HTableMultiplexer(TEST_UTIL.getConfiguration(),
PER_REGIONSERVER_QUEUE_SIZE);
byte[][] startRows = htable.getStartKeys();
byte[] row = startRows[1];
assertTrue("2nd region should not start with empty row", row != null && row.length > 0);
try (RegionLocator r = TEST_UTIL.getConnection().getRegionLocator(TABLE)) {
byte[][] startRows = r.getStartKeys();
byte[] row = startRows[1];
assertTrue("2nd region should not start with empty row", row != null && row.length > 0);
Put put = new Put(row).add(FAMILY, QUALIFIER1, VALUE1);
assertTrue("multiplexer.put returns", multiplexer.put(TABLE, put));
checkExistence(htable, row, FAMILY, QUALIFIER1, VALUE1);
Put put = new Put(row).add(FAMILY, QUALIFIER1, VALUE1);
assertTrue("multiplexer.put returns", multiplexer.put(TABLE, put));
// Now let's shutdown the regionserver and let regions moved to other servers.
HRegionLocation loc = htable.getRegionLocation(row);
MiniHBaseCluster hbaseCluster = TEST_UTIL.getHBaseCluster();
hbaseCluster.stopRegionServer(loc.getServerName());
TEST_UTIL.waitUntilAllRegionsAssigned(TABLE);
checkExistence(htable, row, FAMILY, QUALIFIER1, VALUE1);
// put with multiplexer.
put = new Put(row).add(FAMILY, QUALIFIER2, VALUE2);
assertTrue("multiplexer.put returns", multiplexer.put(TABLE, put));
// Now let's shutdown the regionserver and let regions moved to other servers.
HRegionLocation loc = r.getRegionLocation(row);
MiniHBaseCluster hbaseCluster = TEST_UTIL.getHBaseCluster();
hbaseCluster.stopRegionServer(loc.getServerName());
TEST_UTIL.waitUntilAllRegionsAssigned(TABLE);
checkExistence(htable, row, FAMILY, QUALIFIER2, VALUE2);
// put with multiplexer.
put = new Put(row).addColumn(FAMILY, QUALIFIER2, VALUE2);
assertTrue("multiplexer.put returns", multiplexer.put(TABLE, put));
checkExistence(htable, row, FAMILY, QUALIFIER2, VALUE2);
}
}
}

View File

@ -157,19 +157,19 @@ public class TestMetaWithReplicas {
byte[] data = ZKUtil.getData(zkw, primaryMetaZnode);
ServerName primary = ServerName.parseFrom(data);
byte[] TABLE = Bytes.toBytes("testShutdownHandling");
TableName TABLE = TableName.valueOf("testShutdownHandling");
byte[][] FAMILIES = new byte[][] { Bytes.toBytes("foo") };
if (util.getHBaseAdmin().tableExists(TABLE)) {
util.getHBaseAdmin().disableTable(TABLE);
util.getHBaseAdmin().deleteTable(TABLE);
}
Table htable = util.createTable(TABLE, FAMILIES, conf);
Table htable = util.createTable(TABLE, FAMILIES);
util.getHBaseAdmin().flush(TableName.META_TABLE_NAME);
Thread.sleep(conf.getInt(StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD,
30000) * 6);
Connection c = ConnectionFactory.createConnection(util.getConfiguration());
List<HRegionInfo> regions = MetaTableAccessor.getTableRegions(c, TableName.valueOf(TABLE));
List<HRegionInfo> regions = MetaTableAccessor.getTableRegions(c, TABLE);
HRegionLocation hrl = MetaTableAccessor.getRegionLocation(c, regions.get(0));
// Ensure that the primary server for test table is not the same one as the primary
// of the meta region since we will be killing the srv holding the meta's primary...
@ -198,11 +198,11 @@ public class TestMetaWithReplicas {
}
((ClusterConnection)c).clearRegionCache();
htable.close();
htable = c.getTable(TableName.valueOf(TABLE));
htable = c.getTable(TABLE);
byte[] row = "test".getBytes();
Put put = new Put(row);
put.add("foo".getBytes(), row, row);
BufferedMutator m = c.getBufferedMutator(TableName.valueOf(TABLE));
BufferedMutator m = c.getBufferedMutator(TABLE);
m.mutate(put);
m.flush();
// Try to do a get of the row that was just put
@ -217,22 +217,22 @@ public class TestMetaWithReplicas {
((ClusterConnection)c).clearRegionCache();
htable.close();
conf.setBoolean(HConstants.USE_META_REPLICAS, false);
htable = c.getTable(TableName.valueOf(TABLE));
htable = c.getTable(TABLE);
r = htable.get(get);
assertTrue(Arrays.equals(r.getRow(), row));
}
@Test
public void testMetaLookupThreadPoolCreated() throws Exception {
byte[] TABLE = Bytes.toBytes("testMetaLookupThreadPoolCreated");
TableName TABLE = TableName.valueOf("testMetaLookupThreadPoolCreated");
byte[][] FAMILIES = new byte[][] { Bytes.toBytes("foo") };
if (TEST_UTIL.getHBaseAdmin().tableExists(TABLE)) {
TEST_UTIL.getHBaseAdmin().disableTable(TABLE);
TEST_UTIL.getHBaseAdmin().deleteTable(TABLE);
}
Table htable = TEST_UTIL.createTable(TABLE, FAMILIES, TEST_UTIL.getConfiguration());
Table htable = TEST_UTIL.createTable(TABLE, FAMILIES);
byte[] row = "test".getBytes();
ConnectionImplementation c = ((ConnectionImplementation)((HTable)htable).connection);
ConnectionImplementation c = ((ConnectionImplementation) TEST_UTIL.getConnection());
// check that metalookup pool would get created
c.relocateRegion(TABLE, row);
ExecutorService ex = c.getCurrentMetaLookupPool();

View File

@ -75,7 +75,7 @@ public class TestMultiParallel {
//((Log4JLogger)RpcClient.LOG).getLogger().setLevel(Level.ALL);
//((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL);
UTIL.startMiniCluster(slaves);
HTable t = UTIL.createMultiRegionTable(TEST_TABLE, Bytes.toBytes(FAMILY));
Table t = UTIL.createMultiRegionTable(TEST_TABLE, Bytes.toBytes(FAMILY));
UTIL.waitTableEnabled(TEST_TABLE);
t.close();
CONNECTION = ConnectionFactory.createConnection(UTIL.getConfiguration());

View File

@ -142,10 +142,10 @@ public class TestReplicaWithCluster {
HTableDescriptor hdt = HTU.createTableDescriptor("testCreateDeleteTable");
hdt.setRegionReplication(NB_SERVERS);
hdt.addCoprocessor(SlowMeCopro.class.getName());
Table table = HTU.createTable(hdt, new byte[][]{f}, HTU.getConfiguration());
Table table = HTU.createTable(hdt, new byte[][]{f}, null);
Put p = new Put(row);
p.add(f, row, row);
p.addColumn(f, row, row);
table.put(p);
Get g = new Get(row);
@ -174,11 +174,11 @@ public class TestReplicaWithCluster {
HTableDescriptor hdt = HTU.createTableDescriptor("testChangeTable");
hdt.setRegionReplication(NB_SERVERS);
hdt.addCoprocessor(SlowMeCopro.class.getName());
Table table = HTU.createTable(hdt, new byte[][]{f}, HTU.getConfiguration());
Table table = HTU.createTable(hdt, new byte[][]{f}, null);
// basic test: it should work.
Put p = new Put(row);
p.add(f, row, row);
p.addColumn(f, row, row);
table.put(p);
Get g = new Get(row);
@ -314,7 +314,7 @@ public class TestReplicaWithCluster {
HTableDescriptor hdt = HTU.createTableDescriptor("testBulkLoad");
hdt.setRegionReplication(NB_SERVERS);
hdt.addCoprocessor(SlowMeCopro.class.getName());
Table table = HTU.createTable(hdt, new byte[][]{f}, HTU.getConfiguration());
Table table = HTU.createTable(hdt, new byte[][]{f}, null);
// create hfiles to load.
LOG.debug("Creating test data");

View File

@ -85,7 +85,7 @@ public class TestReplicasClient {
}
private static final int NB_SERVERS = 1;
private static HTable table = null;
private static Table table = null;
private static final byte[] row = TestReplicasClient.class.getName().getBytes();
private static HRegionInfo hriPrimary;
@ -177,9 +177,11 @@ public class TestReplicasClient {
// Create table then get the single region for our new table.
HTableDescriptor hdt = HTU.createTableDescriptor(TestReplicasClient.class.getSimpleName());
hdt.addCoprocessor(SlowMeCopro.class.getName());
table = HTU.createTable(hdt, new byte[][]{f}, HTU.getConfiguration());
table = HTU.createTable(hdt, new byte[][]{f}, null);
hriPrimary = table.getRegionLocation(row, false).getRegionInfo();
try (RegionLocator locator = HTU.getConnection().getRegionLocator(hdt.getTableName())) {
hriPrimary = locator.getRegionLocation(row, false).getRegionInfo();
}
// mock a secondary region info to open
hriSecondary = new HRegionInfo(hriPrimary.getTable(), hriPrimary.getStartKey(),
@ -547,8 +549,7 @@ public class TestReplicasClient {
Thread.sleep(1000 + REFRESH_PERIOD * 2);
AsyncProcess ap = ((ClusterConnection) HTU.getHBaseAdmin().getConnection())
.getAsyncProcess();
AsyncProcess ap = ((ClusterConnection) HTU.getConnection()).getAsyncProcess();
// Make primary slowdown
SlowMeCopro.getCdl().set(new CountDownLatch(1));
@ -563,8 +564,10 @@ public class TestReplicasClient {
g.setConsistency(Consistency.TIMELINE);
gets.add(g);
Object[] results = new Object[2];
AsyncRequestFuture reqs = ap.submitAll(table.getPool(), table.getName(),
gets, null, results);
AsyncRequestFuture reqs = ap.submitAll(
HTable.getDefaultExecutor(HTU.getConfiguration()),
table.getName(), gets, null, results);
reqs.waitUntilDone();
// verify we got the right results back
for (Object r : results) {

View File

@ -71,8 +71,7 @@ public class TestResultSizeEstimation {
TableName TABLE = TableName.valueOf("testResultSizeEstimation");
byte[][] FAMILIES = new byte[][] { FAMILY };
Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
HTable table = TEST_UTIL.createTable(TABLE, FAMILIES, conf);
Table table = TEST_UTIL.createTable(TABLE, FAMILIES);
Put p = new Put(ROW1);
p.add(new KeyValue(ROW1, FAMILY, QUALIFIER, Long.MAX_VALUE, VALUE));
table.put(p);
@ -102,8 +101,7 @@ public class TestResultSizeEstimation {
TableName TABLE = TableName.valueOf("testResultSizeEstimationWithTags");
byte[][] FAMILIES = new byte[][] { FAMILY };
Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
HTable table = TEST_UTIL.createTable(TABLE, FAMILIES, conf);
Table table = TEST_UTIL.createTable(TABLE, FAMILIES);
Put p = new Put(ROW1);
p.add(new KeyValue(ROW1, FAMILY, QUALIFIER, Long.MAX_VALUE, VALUE,
new Tag[] { new Tag((byte)1, new byte[TAG_DATA_SIZE]) } ));

View File

@ -297,9 +297,9 @@ public class TestScannersFromClientSide {
}
assertTrue("Expected row count: " + expectedRowCount + " Actual row count: " + rowCount,
expectedRowCount == rowCount);
expectedRowCount == rowCount);
assertTrue("Expected cell count: " + expectedCellCount + " Actual cell count: " + cellCount,
expectedCellCount == cellCount);
expectedCellCount == cellCount);
scanner.close();
}
@ -310,7 +310,7 @@ public class TestScannersFromClientSide {
*/
@Test
public void testGetMaxResults() throws Exception {
byte [] TABLE = Bytes.toBytes("testGetMaxResults");
TableName TABLE = TableName.valueOf("testGetMaxResults");
byte [][] FAMILIES = HTestConst.makeNAscii(FAMILY, 3);
byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 20);
@ -430,7 +430,7 @@ public class TestScannersFromClientSide {
*/
@Test
public void testScanMaxResults() throws Exception {
byte [] TABLE = Bytes.toBytes("testScanLimit");
TableName TABLE = TableName.valueOf("testScanLimit");
byte [][] ROWS = HTestConst.makeNAscii(ROW, 2);
byte [][] FAMILIES = HTestConst.makeNAscii(FAMILY, 3);
byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 10);
@ -480,7 +480,7 @@ public class TestScannersFromClientSide {
*/
@Test
public void testGetRowOffset() throws Exception {
byte [] TABLE = Bytes.toBytes("testGetRowOffset");
TableName TABLE = TableName.valueOf("testGetRowOffset");
byte [][] FAMILIES = HTestConst.makeNAscii(FAMILY, 3);
byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 20);
@ -579,7 +579,7 @@ public class TestScannersFromClientSide {
TableName TABLE = TableName.valueOf("testScanOnReopenedRegion");
byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 2);
HTable ht = TEST_UTIL.createTable(TABLE, FAMILY);
Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
Put put;
Scan scan;
@ -599,7 +599,11 @@ public class TestScannersFromClientSide {
scan = new Scan(ROW);
scanner = ht.getScanner(scan);
HRegionLocation loc = ht.getRegionLocation(ROW);
HRegionLocation loc;
try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(TABLE)) {
loc = locator.getRegionLocation(ROW);
}
HRegionInfo hri = loc.getRegionInfo();
MiniHBaseCluster cluster = TEST_UTIL.getMiniHBaseCluster();
byte[] regionName = hri.getRegionName();
@ -649,12 +653,12 @@ public class TestScannersFromClientSide {
*/
@Test
public void testAsyncScanner() throws Exception {
byte [] TABLE = Bytes.toBytes("testAsyncScan");
TableName TABLE = TableName.valueOf("testAsyncScan");
byte [][] ROWS = HTestConst.makeNAscii(ROW, 2);
byte [][] FAMILIES = HTestConst.makeNAscii(FAMILY, 3);
byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 10);
HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES);
Table ht = TEST_UTIL.createTable(TABLE, FAMILIES);
Put put;
Scan scan;

View File

@ -182,7 +182,7 @@ public class TestSnapshotCloneIndependence {
runTestSnapshotDeleteIndependent(true);
}
private static void waitOnSplit(final HTable t, int originalCount) throws Exception {
private static void waitOnSplit(Connection c, final Table t, int originalCount) throws Exception {
for (int i = 0; i < 200; i++) {
try {
Thread.sleep(50);
@ -190,8 +190,10 @@ public class TestSnapshotCloneIndependence {
// Restore the interrupted status
Thread.currentThread().interrupt();
}
if (t.getAllRegionLocations().size() > originalCount) {
return;
try (RegionLocator locator = c.getRegionLocator(t.getName())) {
if (locator.getAllRegionLocations().size() > originalCount) {
return;
}
}
}
throw new Exception("Split did not increase the number of regions");
@ -276,7 +278,7 @@ public class TestSnapshotCloneIndependence {
final long startTime = System.currentTimeMillis();
final TableName localTableName =
TableName.valueOf(STRING_TABLE_NAME + startTime);
HTable original = UTIL.createTable(localTableName, TEST_FAM);
Table original = UTIL.createTable(localTableName, TEST_FAM);
UTIL.loadTable(original, TEST_FAM);
final int loadedTableCount = UTIL.countRows(original);
System.out.println("Original table has: " + loadedTableCount + " rows");
@ -298,7 +300,7 @@ public class TestSnapshotCloneIndependence {
admin.cloneSnapshot(snapshotName, cloneTableName);
// Verify that region information is the same pre-split
original.clearRegionCache();
((ClusterConnection) UTIL.getConnection()).clearRegionCache();
List<HRegionInfo> originalTableHRegions = admin.getTableRegions(localTableName);
final int originalRegionCount = originalTableHRegions.size();
@ -309,7 +311,7 @@ public class TestSnapshotCloneIndependence {
// Split a region on the parent table
admin.splitRegion(originalTableHRegions.get(0).getRegionName());
waitOnSplit(original, originalRegionCount);
waitOnSplit(UTIL.getConnection(), original, originalRegionCount);
// Verify that the cloned table region is not split
final int cloneTableRegionCount2 = admin.getTableRegions(cloneTableName).size();
@ -332,7 +334,7 @@ public class TestSnapshotCloneIndependence {
final long startTime = System.currentTimeMillis();
final TableName localTableName =
TableName.valueOf(STRING_TABLE_NAME + startTime);
HTable original = UTIL.createTable(localTableName, TEST_FAM);
Table original = UTIL.createTable(localTableName, TEST_FAM);
UTIL.loadTable(original, TEST_FAM);
final String snapshotNameAsString = "snapshot_" + localTableName;

View File

@ -97,7 +97,7 @@ public class TestSnapshotFromClient {
public void setup() throws Exception {
HTableDescriptor htd = new HTableDescriptor(TABLE_NAME);
htd.setRegionReplication(getNumReplicas());
UTIL.createTable(htd, new byte[][]{TEST_FAM}, UTIL.getConfiguration());
UTIL.createTable(htd, new byte[][]{TEST_FAM}, null);
}
protected int getNumReplicas() {

View File

@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.HTable;
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.coprocessor.AggregationClient;
import org.apache.hadoop.hbase.client.coprocessor.LongColumnInterpreter;
import org.apache.hadoop.hbase.filter.Filter;
@ -84,7 +85,7 @@ public class TestAggregateProtocol {
util.startMiniCluster(2);
final byte[][] SPLIT_KEYS = new byte[][] { ROWS[rowSeperator1],
ROWS[rowSeperator2] };
HTable table = util.createTable(TEST_TABLE, TEST_FAMILY, SPLIT_KEYS);
Table table = util.createTable(TEST_TABLE, TEST_FAMILY, SPLIT_KEYS);
/**
* The testtable has one CQ which is always populated and one variable CQ
* for each row rowkey1: CF:CQ CF:CQ1 rowKey2: CF:CQ CF:CQ2

View File

@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.HTable;
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.coprocessor.AggregationClient;
import org.apache.hadoop.hbase.client.coprocessor.BigDecimalColumnInterpreter;
import org.apache.hadoop.hbase.filter.Filter;
@ -83,7 +84,7 @@ public class TestBigDecimalColumnInterpreter {
util.startMiniCluster(2);
final byte[][] SPLIT_KEYS = new byte[][] { ROWS[rowSeperator1], ROWS[rowSeperator2] };
HTable table = util.createTable(TEST_TABLE, TEST_FAMILY, SPLIT_KEYS);
Table table = util.createTable(TEST_TABLE, TEST_FAMILY, SPLIT_KEYS);
/**
* The testtable has one CQ which is always populated and one variable CQ for each row rowkey1:
* CF:CQ CF:CQ1 rowKey2: CF:CQ CF:CQ2
@ -92,11 +93,11 @@ public class TestBigDecimalColumnInterpreter {
Put put = new Put(ROWS[i]);
put.setDurability(Durability.SKIP_WAL);
BigDecimal bd = new BigDecimal(i);
put.add(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes(bd));
put.addColumn(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes(bd));
table.put(put);
Put p2 = new Put(ROWS[i]);
put.setDurability(Durability.SKIP_WAL);
p2.add(TEST_FAMILY, Bytes.add(TEST_MULTI_CQ, Bytes.toBytes(bd)),
p2.addColumn(TEST_FAMILY, Bytes.add(TEST_MULTI_CQ, Bytes.toBytes(bd)),
Bytes.toBytes(bd.multiply(new BigDecimal("0.10"))));
table.put(p2);
}

View File

@ -26,11 +26,14 @@ import static org.junit.Assert.fail;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.NavigableMap;
import java.util.TreeMap;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
@ -173,9 +176,12 @@ public class TestCoprocessorEndpoint {
@Test
public void testCoprocessorService() throws Throwable {
HTable table = (HTable) util.getConnection().getTable(TEST_TABLE);
NavigableMap<HRegionInfo,ServerName> regions = table.getRegionLocations();
Table table = util.getConnection().getTable(TEST_TABLE);
List<HRegionLocation> regions;
try(RegionLocator rl = util.getConnection().getRegionLocator(TEST_TABLE)) {
regions = rl.getAllRegionLocations();
}
final TestProtos.EchoRequestProto request =
TestProtos.EchoRequestProto.newBuilder().setMessage("hello").build();
final Map<byte[], String> results = Collections.synchronizedMap(
@ -208,9 +214,9 @@ public class TestCoprocessorEndpoint {
LOG.info("Got value "+e.getValue()+" for region "+Bytes.toStringBinary(e.getKey()));
}
assertEquals(3, results.size());
for (HRegionInfo info : regions.navigableKeySet()) {
LOG.info("Region info is "+info.getRegionNameAsString());
assertTrue(results.containsKey(info.getRegionName()));
for (HRegionLocation info : regions) {
LOG.info("Region info is "+info.getRegionInfo().getRegionNameAsString());
assertTrue(results.containsKey(info.getRegionInfo().getRegionName()));
}
results.clear();
@ -247,8 +253,11 @@ public class TestCoprocessorEndpoint {
@Test
public void testCoprocessorServiceNullResponse() throws Throwable {
HTable table = (HTable) util.getConnection().getTable(TEST_TABLE);
NavigableMap<HRegionInfo,ServerName> regions = table.getRegionLocations();
Table table = util.getConnection().getTable(TEST_TABLE);
List<HRegionLocation> regions;
try(RegionLocator rl = util.getConnection().getRegionLocator(TEST_TABLE)) {
regions = rl.getAllRegionLocations();
}
final TestProtos.EchoRequestProto request =
TestProtos.EchoRequestProto.newBuilder().setMessage("hello").build();
@ -273,7 +282,8 @@ public class TestCoprocessorEndpoint {
LOG.info("Got value "+e.getValue()+" for region "+Bytes.toStringBinary(e.getKey()));
}
assertEquals(3, results.size());
for (HRegionInfo info : regions.navigableKeySet()) {
for (HRegionLocation region : regions) {
HRegionInfo info = region.getRegionInfo();
LOG.info("Region info is "+info.getRegionNameAsString());
assertTrue(results.containsKey(info.getRegionName()));
assertNull(results.get(info.getRegionName()));

View File

@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.HTable;
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.coprocessor.AggregationClient;
import org.apache.hadoop.hbase.client.coprocessor.DoubleColumnInterpreter;
import org.apache.hadoop.hbase.filter.Filter;
@ -81,7 +82,7 @@ public class TestDoubleColumnInterpreter {
util.startMiniCluster(2);
final byte[][] SPLIT_KEYS = new byte[][] { ROWS[rowSeperator1], ROWS[rowSeperator2] };
HTable table = util.createTable(TEST_TABLE, TEST_FAMILY, SPLIT_KEYS);
Table table = util.createTable(TEST_TABLE, TEST_FAMILY, SPLIT_KEYS);
/**
* The testtable has one CQ which is always populated and one variable CQ for each row rowkey1:
* CF:CQ CF:CQ1 rowKey2: CF:CQ CF:CQ2

View File

@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.HBaseTestingUtility;
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.MiniHBaseCluster;
import org.apache.hadoop.hbase.NamespaceDescriptor;
@ -45,6 +46,8 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
@ -1612,15 +1615,15 @@ public class TestMasterObserver {
cp.enableBypass(false);
cp.resetStates();
HTable table = UTIL.createMultiRegionTable(tableName, TEST_FAMILY);
Table table = UTIL.createMultiRegionTable(tableName, TEST_FAMILY);
try {
try (RegionLocator r = UTIL.getConnection().getRegionLocator(tableName)) {
UTIL.waitUntilAllRegionsAssigned(tableName);
NavigableMap<HRegionInfo, ServerName> regions = table.getRegionLocations();
Map.Entry<HRegionInfo, ServerName> firstGoodPair = null;
for (Map.Entry<HRegionInfo, ServerName> e: regions.entrySet()) {
if (e.getValue() != null) {
List<HRegionLocation> regions = r.getAllRegionLocations();
HRegionLocation firstGoodPair = null;
for (HRegionLocation e: regions) {
if (e.getServerName() != null) {
firstGoodPair = e;
break;
}
@ -1630,7 +1633,7 @@ public class TestMasterObserver {
// Try to force a move
Collection<ServerName> servers = master.getClusterStatus().getServers();
String destName = null;
String serverNameForFirstRegion = firstGoodPair.getValue().toString();
String serverNameForFirstRegion = firstGoodPair.getServerName().toString();
LOG.info("serverNameForFirstRegion=" + serverNameForFirstRegion);
ServerName masterServerName = master.getServerName();
boolean found = false;
@ -1647,7 +1650,7 @@ public class TestMasterObserver {
assertTrue("Found server", found);
LOG.info("Found " + destName);
master.getMasterRpcServices().moveRegion(null, RequestConverter.buildMoveRegionRequest(
firstGoodPair.getKey().getEncodedNameAsBytes(),Bytes.toBytes(destName)));
firstGoodPair.getRegionInfo().getEncodedNameAsBytes(),Bytes.toBytes(destName)));
assertTrue("Coprocessor should have been called on region move",
cp.wasMoveCalled());

View File

@ -53,6 +53,7 @@ import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.RowMutations;
@ -140,9 +141,8 @@ public class TestRegionObserverInterface {
true, true, true, false, true, true, true });
verifyMethodResult(SimpleRegionObserver.class,
new String[] {"getCtPreOpen", "getCtPostOpen", "getCtPreClose", "getCtPostClose"},
tableName,
new Integer[] {1, 1, 0, 0});
new String[] { "getCtPreOpen", "getCtPostOpen", "getCtPreClose", "getCtPostClose" },
tableName, new Integer[] { 1, 1, 0, 0 });
Get get = new Get(ROW);
get.addColumn(A, A);
@ -151,11 +151,9 @@ public class TestRegionObserverInterface {
table.get(get);
verifyMethodResult(SimpleRegionObserver.class,
new String[] {"hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut",
"hadDelete", "hadPrePreparedDeleteTS"},
tableName,
new Boolean[] {true, true, true, true, false, false}
);
new String[] { "hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut", "hadDelete",
"hadPrePreparedDeleteTS" }, tableName,
new Boolean[] { true, true, true, true, false, false });
Delete delete = new Delete(ROW);
delete.deleteColumn(A, A);
@ -182,7 +180,7 @@ public class TestRegionObserverInterface {
@Test (timeout=300000)
public void testRowMutation() throws IOException {
TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testRowMutation");
Table table = util.createTable(tableName, new byte[][] {A, B, C});
Table table = util.createTable(tableName, new byte[][] { A, B, C });
try {
verifyMethodResult(SimpleRegionObserver.class,
new String[] {"hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut",
@ -219,7 +217,7 @@ public class TestRegionObserverInterface {
@Test (timeout=300000)
public void testIncrementHook() throws IOException {
TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testIncrementHook");
Table table = util.createTable(tableName, new byte[][] {A, B, C});
Table table = util.createTable(tableName, new byte[][] { A, B, C });
try {
Increment inc = new Increment(Bytes.toBytes(0));
inc.addColumn(A, A, 1);
@ -254,11 +252,8 @@ public class TestRegionObserverInterface {
p = new Put(Bytes.toBytes(0));
p.add(A, A, A);
verifyMethodResult(SimpleRegionObserver.class,
new String[] {"hadPreCheckAndPut",
"hadPreCheckAndPutAfterRowLock", "hadPostCheckAndPut"},
tableName,
new Boolean[] {false, false, false}
);
new String[] { "hadPreCheckAndPut", "hadPreCheckAndPutAfterRowLock",
"hadPostCheckAndPut" }, tableName, new Boolean[] { false, false, false });
table.checkAndPut(Bytes.toBytes(0), A, A, A, p);
verifyMethodResult(SimpleRegionObserver.class,
new String[] {"hadPreCheckAndPut",
@ -304,7 +299,7 @@ public class TestRegionObserverInterface {
@Test (timeout=300000)
public void testAppendHook() throws IOException {
TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testAppendHook");
Table table = util.createTable(tableName, new byte[][] {A, B, C});
Table table = util.createTable(tableName, new byte[][] { A, B, C });
try {
Append app = new Append(Bytes.toBytes(0));
app.add(A, A, A);
@ -337,11 +332,8 @@ public class TestRegionObserverInterface {
util.waitUntilAllRegionsAssigned(tableName);
verifyMethodResult(SimpleRegionObserver.class,
new String[] {"hadPreGet", "hadPostGet", "wasScannerNextCalled",
"wasScannerCloseCalled"},
tableName,
new Boolean[] {false, false, false, false}
);
new String[] { "hadPreGet", "hadPostGet", "wasScannerNextCalled", "wasScannerCloseCalled" },
tableName, new Boolean[] { false, false, false, false });
Table table = util.getConnection().getTable(tableName);
Put put = new Put(ROW);
@ -562,8 +554,8 @@ public class TestRegionObserverInterface {
String testName = TestRegionObserverInterface.class.getName()+".bulkLoadHFileTest";
TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".bulkLoadHFileTest");
Configuration conf = util.getConfiguration();
HTable table = util.createTable(tableName, new byte[][] {A, B, C});
try {
Table table = util.createTable(tableName, new byte[][] {A, B, C});
try (RegionLocator locator = util.getConnection().getRegionLocator(tableName)) {
verifyMethodResult(SimpleRegionObserver.class,
new String[] {"hadPreBulkLoadHFile", "hadPostBulkLoadHFile"},
tableName,
@ -574,10 +566,10 @@ public class TestRegionObserverInterface {
final Path dir = util.getDataTestDirOnTestFS(testName).makeQualified(fs);
Path familyDir = new Path(dir, Bytes.toString(A));
createHFile(util.getConfiguration(), fs, new Path(familyDir,Bytes.toString(A)), A, A);
createHFile(util.getConfiguration(), fs, new Path(familyDir, Bytes.toString(A)), A, A);
// Bulk load
new LoadIncrementalHFiles(conf).doBulkLoad(dir, table);
new LoadIncrementalHFiles(conf).doBulkLoad(dir, util.getHBaseAdmin(), table, locator);
verifyMethodResult(SimpleRegionObserver.class,
new String[] {"hadPreBulkLoadHFile", "hadPostBulkLoadHFile"},
@ -595,21 +587,22 @@ public class TestRegionObserverInterface {
public void testRecovery() throws Exception {
LOG.info(TestRegionObserverInterface.class.getName() +".testRecovery");
TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testRecovery");
HTable table = util.createTable(tableName, new byte[][] {A, B, C});
try {
Table table = util.createTable(tableName, new byte[][] {A, B, C});
try (RegionLocator locator = util.getConnection().getRegionLocator(tableName)) {
JVMClusterUtil.RegionServerThread rs1 = cluster.startRegionServer();
ServerName sn2 = rs1.getRegionServer().getServerName();
String regEN = table.getRegionLocations().firstEntry().getKey().getEncodedName();
String regEN = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
util.getHBaseAdmin().move(regEN.getBytes(), sn2.getServerName().getBytes());
while (!sn2.equals(table.getRegionLocations().firstEntry().getValue() )){
while (!sn2.equals(locator.getAllRegionLocations().get(0).getServerName())){
Thread.sleep(100);
}
Put put = new Put(ROW);
put.add(A, A, A);
put.add(B, B, B);
put.add(C, C, C);
put.addColumn(A, A, A);
put.addColumn(B, B, B);
put.addColumn(C, C, C);
table.put(put);
verifyMethodResult(SimpleRegionObserver.class,
@ -646,46 +639,48 @@ public class TestRegionObserverInterface {
public void testLegacyRecovery() throws Exception {
LOG.info(TestRegionObserverInterface.class.getName() +".testLegacyRecovery");
TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testLegacyRecovery");
HTable table = util.createTable(tableName, new byte[][] {A, B, C});
Table table = util.createTable(tableName, new byte[][] {A, B, C});
try {
JVMClusterUtil.RegionServerThread rs1 = cluster.startRegionServer();
ServerName sn2 = rs1.getRegionServer().getServerName();
String regEN = table.getRegionLocations().firstEntry().getKey().getEncodedName();
try (RegionLocator locator = util.getConnection().getRegionLocator(tableName)) {
JVMClusterUtil.RegionServerThread rs1 = cluster.startRegionServer();
ServerName sn2 = rs1.getRegionServer().getServerName();
String regEN = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
util.getHBaseAdmin().move(regEN.getBytes(), sn2.getServerName().getBytes());
while (!sn2.equals(table.getRegionLocations().firstEntry().getValue() )){
Thread.sleep(100);
util.getHBaseAdmin().move(regEN.getBytes(), sn2.getServerName().getBytes());
while (!sn2.equals(locator.getAllRegionLocations().get(0).getServerName())) {
Thread.sleep(100);
}
Put put = new Put(ROW);
put.add(A, A, A);
put.add(B, B, B);
put.add(C, C, C);
table.put(put);
verifyMethodResult(SimpleRegionObserver.Legacy.class,
new String[] {"hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut",
"hadPreBatchMutate", "hadPostBatchMutate", "hadDelete"},
tableName,
new Boolean[] {false, false, true, true, true, true, false}
);
verifyMethodResult(SimpleRegionObserver.Legacy.class,
new String[] {"getCtPreWALRestore", "getCtPostWALRestore", "getCtPrePut",
"getCtPostPut", "getCtPreWALRestoreDeprecated", "getCtPostWALRestoreDeprecated"},
tableName,
new Integer[] {0, 0, 1, 1, 0, 0});
cluster.killRegionServer(rs1.getRegionServer().getServerName());
Threads.sleep(1000); // Let the kill soak in.
util.waitUntilAllRegionsAssigned(tableName);
LOG.info("All regions assigned");
verifyMethodResult(SimpleRegionObserver.Legacy.class,
new String[] {"getCtPreWALRestore", "getCtPostWALRestore", "getCtPrePut",
"getCtPostPut", "getCtPreWALRestoreDeprecated", "getCtPostWALRestoreDeprecated"},
tableName,
new Integer[]{1, 1, 0, 0, 1, 1});
}
Put put = new Put(ROW);
put.add(A, A, A);
put.add(B, B, B);
put.add(C, C, C);
table.put(put);
verifyMethodResult(SimpleRegionObserver.Legacy.class,
new String[] {"hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut",
"hadPreBatchMutate", "hadPostBatchMutate", "hadDelete"},
tableName,
new Boolean[] {false, false, true, true, true, true, false}
);
verifyMethodResult(SimpleRegionObserver.Legacy.class,
new String[] {"getCtPreWALRestore", "getCtPostWALRestore", "getCtPrePut", "getCtPostPut",
"getCtPreWALRestoreDeprecated", "getCtPostWALRestoreDeprecated"},
tableName,
new Integer[] {0, 0, 1, 1, 0, 0});
cluster.killRegionServer(rs1.getRegionServer().getServerName());
Threads.sleep(1000); // Let the kill soak in.
util.waitUntilAllRegionsAssigned(tableName);
LOG.info("All regions assigned");
verifyMethodResult(SimpleRegionObserver.Legacy.class,
new String[] {"getCtPreWALRestore", "getCtPostWALRestore", "getCtPrePut", "getCtPostPut",
"getCtPreWALRestoreDeprecated", "getCtPostWALRestoreDeprecated"},
tableName,
new Integer[]{1, 1, 0, 0, 1, 1});
} finally {
util.deleteTable(tableName);
table.close();
@ -696,28 +691,29 @@ public class TestRegionObserverInterface {
public void testPreWALRestoreSkip() throws Exception {
LOG.info(TestRegionObserverInterface.class.getName() + ".testPreWALRestoreSkip");
TableName tableName = TableName.valueOf(SimpleRegionObserver.TABLE_SKIPPED);
HTable table = util.createTable(tableName, new byte[][] { A, B, C });
Table table = util.createTable(tableName, new byte[][] { A, B, C });
JVMClusterUtil.RegionServerThread rs1 = cluster.startRegionServer();
ServerName sn2 = rs1.getRegionServer().getServerName();
String regEN = table.getRegionLocations().firstEntry().getKey().getEncodedName();
try (RegionLocator locator = util.getConnection().getRegionLocator(tableName)) {
JVMClusterUtil.RegionServerThread rs1 = cluster.startRegionServer();
ServerName sn2 = rs1.getRegionServer().getServerName();
String regEN = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
util.getHBaseAdmin().move(regEN.getBytes(), sn2.getServerName().getBytes());
while (!sn2.equals(table.getRegionLocations().firstEntry().getValue())) {
Thread.sleep(100);
util.getHBaseAdmin().move(regEN.getBytes(), sn2.getServerName().getBytes());
while (!sn2.equals(locator.getAllRegionLocations().get(0).getServerName())) {
Thread.sleep(100);
}
Put put = new Put(ROW);
put.add(A, A, A);
put.add(B, B, B);
put.add(C, C, C);
table.put(put);
cluster.killRegionServer(rs1.getRegionServer().getServerName());
Threads.sleep(20000); // just to be sure that the kill has fully started.
util.waitUntilAllRegionsAssigned(tableName);
}
Put put = new Put(ROW);
put.add(A, A, A);
put.add(B, B, B);
put.add(C, C, C);
table.put(put);
table.flushCommits();
cluster.killRegionServer(rs1.getRegionServer().getServerName());
Threads.sleep(20000); // just to be sure that the kill has fully started.
util.waitUntilAllRegionsAssigned(tableName);
verifyMethodResult(SimpleRegionObserver.class, new String[] { "getCtPreWALRestore",
"getCtPostWALRestore", "getCtPreWALRestoreDeprecated", "getCtPostWALRestoreDeprecated"},
tableName,

View File

@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.Waiter.Predicate;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
@ -98,7 +99,7 @@ public class TestRegionServerCoprocessorExceptionWithAbort {
// hosts the region we attempted to write to) to abort.
final byte[] TEST_FAMILY = Bytes.toBytes("aaa");
HTable table = TEST_UTIL.createMultiRegionTable(TABLE_NAME, TEST_FAMILY);
Table table = TEST_UTIL.createMultiRegionTable(TABLE_NAME, TEST_FAMILY);
TEST_UTIL.waitUntilAllRegionsAssigned(TABLE_NAME);
// Note which regionServer will abort (after put is attempted).
@ -109,7 +110,6 @@ public class TestRegionServerCoprocessorExceptionWithAbort {
Put put = new Put(ROW);
put.add(TEST_FAMILY, ROW, ROW);
table.put(put);
table.flushCommits();
} catch (IOException e) {
// The region server is going to be aborted.
// We may get an exception if we retry,

View File

@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
@ -97,7 +98,7 @@ public class TestRegionServerCoprocessorExceptionWithRemove {
TableName TEST_TABLE = TableName.valueOf("observed_table");
byte[] TEST_FAMILY = Bytes.toBytes("aaa");
HTable table = TEST_UTIL.createMultiRegionTable(TEST_TABLE, TEST_FAMILY);
Table table = TEST_UTIL.createMultiRegionTable(TEST_TABLE, TEST_FAMILY);
TEST_UTIL.waitUntilAllRegionsAssigned(TEST_TABLE);
// Note which regionServer that should survive the buggy coprocessor's
// prePut().
@ -108,12 +109,10 @@ public class TestRegionServerCoprocessorExceptionWithRemove {
try {
final byte[] ROW = Bytes.toBytes("aaa");
Put put = new Put(ROW);
put.add(TEST_FAMILY, ROW, ROW);
put.addColumn(TEST_FAMILY, ROW, ROW);
table.put(put);
table.flushCommits();
// We may need two puts to reliably get an exception
table.put(put);
table.flushCommits();
} catch (IOException e) {
threwIOE = true;
} finally {

View File

@ -29,11 +29,13 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
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.filter.MultiRowRangeFilter.RowRange;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
@ -51,7 +53,7 @@ public class TestMultiRowRangeFilter {
private byte[] family = Bytes.toBytes("family");
private byte[] qf = Bytes.toBytes("qf");
private byte[] value = Bytes.toBytes("val");
private byte[] tableName;
private TableName tableName;
private int numRows = 100;
/**
@ -218,8 +220,8 @@ public class TestMultiRowRangeFilter {
@Test
public void testMultiRowRangeFilterWithRangeOverlap() throws IOException {
tableName = Bytes.toBytes("testMultiRowRangeFilterWithRangeOverlap");
HTable ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE);
tableName = TableName.valueOf("testMultiRowRangeFilterWithRangeOverlap");
Table ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE);
generateRows(numRows, ht, family, qf, value);
Scan scan = new Scan();
@ -246,8 +248,8 @@ public class TestMultiRowRangeFilter {
@Test
public void testMultiRowRangeFilterWithoutRangeOverlap() throws IOException {
tableName = Bytes.toBytes("testMultiRowRangeFilterWithoutRangeOverlap");
HTable ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE);
tableName = TableName.valueOf("testMultiRowRangeFilterWithoutRangeOverlap");
Table ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE);
generateRows(numRows, ht, family, qf, value);
Scan scan = new Scan();
@ -273,8 +275,8 @@ public class TestMultiRowRangeFilter {
@Test
public void testMultiRowRangeFilterWithEmptyStartRow() throws IOException {
tableName = Bytes.toBytes("testMultiRowRangeFilterWithEmptyStartRow");
HTable ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE);
tableName = TableName.valueOf("testMultiRowRangeFilterWithEmptyStartRow");
Table ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE);
generateRows(numRows, ht, family, qf, value);
Scan scan = new Scan();
scan.setMaxVersions();
@ -295,8 +297,8 @@ public class TestMultiRowRangeFilter {
@Test
public void testMultiRowRangeFilterWithEmptyStopRow() throws IOException {
tableName = Bytes.toBytes("testMultiRowRangeFilterWithEmptyStopRow");
HTable ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE);
tableName = TableName.valueOf("testMultiRowRangeFilterWithEmptyStopRow");
Table ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE);
generateRows(numRows, ht, family, qf, value);
Scan scan = new Scan();
scan.setMaxVersions();
@ -316,8 +318,8 @@ public class TestMultiRowRangeFilter {
@Test
public void testMultiRowRangeFilterWithInclusive() throws IOException {
tableName = Bytes.toBytes("testMultiRowRangeFilterWithInclusive");
HTable ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE);
tableName = TableName.valueOf("testMultiRowRangeFilterWithInclusive");
Table ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE);
generateRows(numRows, ht, family, qf, value);
Scan scan = new Scan();
@ -344,8 +346,8 @@ public class TestMultiRowRangeFilter {
@Test
public void testMultiRowRangeFilterWithExclusive() throws IOException {
tableName = Bytes.toBytes("testMultiRowRangeFilterWithExclusive");
HTable ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE);
tableName = TableName.valueOf("testMultiRowRangeFilterWithExclusive");
Table ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE);
generateRows(numRows, ht, family, qf, value);
Scan scan = new Scan();
@ -370,8 +372,8 @@ public class TestMultiRowRangeFilter {
@Test
public void testMultiRowRangeWithFilterListAndOperator() throws IOException {
tableName = Bytes.toBytes("TestMultiRowRangeFilterWithFilterListAndOperator");
HTable ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE);
tableName = TableName.valueOf("TestMultiRowRangeFilterWithFilterListAndOperator");
Table ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE);
generateRows(numRows, ht, family, qf, value);
Scan scan = new Scan();
@ -405,8 +407,8 @@ public class TestMultiRowRangeFilter {
@Test
public void testMultiRowRangeWithFilterListOrOperator() throws IOException {
tableName = Bytes.toBytes("TestMultiRowRangeFilterWithFilterListOrOperator");
HTable ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE);
tableName = TableName.valueOf("TestMultiRowRangeFilterWithFilterListOrOperator");
Table ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE);
generateRows(numRows, ht, family, qf, value);
Scan scan = new Scan();
@ -440,18 +442,18 @@ public class TestMultiRowRangeFilter {
ht.close();
}
private void generateRows(int numberOfRows, HTable ht, byte[] family, byte[] qf, byte[] value)
private void generateRows(int numberOfRows, Table ht, byte[] family, byte[] qf, byte[] value)
throws IOException {
for (int i = 0; i < numberOfRows; i++) {
byte[] row = Bytes.toBytes(i);
Put p = new Put(row);
p.add(family, qf, value);
p.addColumn(family, qf, value);
ht.put(p);
}
TEST_UTIL.flush();
}
private List<Cell> getScanResult(byte[] startRow, byte[] stopRow, HTable ht) throws IOException {
private List<Cell> getScanResult(byte[] startRow, byte[] stopRow, Table ht) throws IOException {
Scan scan = new Scan();
scan.setMaxVersions();
if(!Bytes.toString(startRow).isEmpty()) {
@ -471,7 +473,7 @@ public class TestMultiRowRangeFilter {
return kvList;
}
private int getResultsSize(HTable ht, Scan scan) throws IOException {
private int getResultsSize(Table ht, Scan scan) throws IOException {
ResultScanner scanner = ht.getScanner(scan);
List<Cell> results = new ArrayList<Cell>();
Result r;

View File

@ -19,11 +19,15 @@ package org.apache.hadoop.hbase.io.encoding;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.NavigableMap;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.testclassification.IOTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.ServerName;
@ -75,7 +79,7 @@ public class TestLoadAndSwitchEncodeOnDisk extends
HColumnDescriptor hcd = getColumnDesc(admin);
System.err.println("\nDisabling encode-on-disk. Old column descriptor: " + hcd + "\n");
HTable t = (HTable) TEST_UTIL.getConnection().getTable(TABLE);
Table t = TEST_UTIL.getConnection().getTable(TABLE);
assertAllOnLine(t);
admin.disableTable(TABLE);
@ -92,7 +96,7 @@ public class TestLoadAndSwitchEncodeOnDisk extends
assertAllOnLine(t);
System.err.println("\nCompacting the table\n");
admin.majorCompact(TABLE.getName());
admin.majorCompact(TABLE);
// Wait until compaction completes
Threads.sleepWithoutInterrupt(5000);
HRegionServer rs = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0);
@ -103,10 +107,13 @@ public class TestLoadAndSwitchEncodeOnDisk extends
System.err.println("\nDone with the test, shutting down the cluster\n");
}
private void assertAllOnLine(final HTable t) throws IOException {
NavigableMap<HRegionInfo, ServerName> regions = t.getRegionLocations();
for (Map.Entry<HRegionInfo, ServerName> e: regions.entrySet()) {
byte [] startkey = e.getKey().getStartKey();
private void assertAllOnLine(final Table t) throws IOException {
List<HRegionLocation> regions;
try(RegionLocator rl = TEST_UTIL.getConnection().getRegionLocator(t.getName())) {
regions = rl.getAllRegionLocations();
}
for (HRegionLocation e: regions) {
byte [] startkey = e.getRegionInfo().getStartKey();
Scan s = new Scan(startkey);
ResultScanner scanner = t.getScanner(s);
Result r = scanner.next();

View File

@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.HTable;
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.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.NullWritable;
@ -77,7 +78,7 @@ public abstract class MultiTableInputFormatTestBase {
TEST_UTIL.startMiniCluster(3);
// create and fill table
for (String tableName : TABLES) {
try (HTable table =
try (Table table =
TEST_UTIL.createMultiRegionTable(TableName.valueOf(tableName),
INPUT_FAMILY, 4)) {
TEST_UTIL.loadTable(table, INPUT_FAMILY, false);

View File

@ -25,6 +25,7 @@ import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.testclassification.LargeTests;
@ -80,38 +81,37 @@ public class TestCellCounter {
*/
@Test (timeout=300000)
public void testCellCounter() throws Exception {
String sourceTable = "sourceTable";
TableName sourceTable = TableName.valueOf("sourceTable");
byte[][] families = { FAMILY_A, FAMILY_B };
Table t = UTIL.createTable(Bytes.toBytes(sourceTable), families);
Table t = UTIL.createTable(sourceTable, families);
try{
Put p = new Put(ROW1);
p.add(FAMILY_A, QUALIFIER, now, Bytes.toBytes("Data11"));
p.add(FAMILY_B, QUALIFIER, now + 1, Bytes.toBytes("Data12"));
p.add(FAMILY_A, QUALIFIER, now + 2, Bytes.toBytes("Data13"));
t.put(p);
p = new Put(ROW2);
p.add(FAMILY_B, QUALIFIER, now, Bytes.toBytes("Dat21"));
p.add(FAMILY_A, QUALIFIER, now + 1, Bytes.toBytes("Data22"));
p.add(FAMILY_B, QUALIFIER, now + 2, Bytes.toBytes("Data23"));
t.put(p);
String[] args = { sourceTable, FQ_OUTPUT_DIR.toString(), ";", "^row1" };
runCount(args);
FileInputStream inputStream = new FileInputStream(OUTPUT_DIR + File.separator +
"part-r-00000");
String data = IOUtils.toString(inputStream);
inputStream.close();
assertTrue(data.contains("Total Families Across all Rows" + "\t" + "2"));
assertTrue(data.contains("Total Qualifiers across all Rows" + "\t" + "2"));
assertTrue(data.contains("Total ROWS" + "\t" + "1"));
assertTrue(data.contains("b;q" + "\t" + "1"));
assertTrue(data.contains("a;q" + "\t" + "1"));
assertTrue(data.contains("row1;a;q_Versions" + "\t" + "1"));
assertTrue(data.contains("row1;b;q_Versions" + "\t" + "1"));
Put p = new Put(ROW1);
p.add(FAMILY_A, QUALIFIER, now, Bytes.toBytes("Data11"));
p.add(FAMILY_B, QUALIFIER, now + 1, Bytes.toBytes("Data12"));
p.add(FAMILY_A, QUALIFIER, now + 2, Bytes.toBytes("Data13"));
t.put(p);
p = new Put(ROW2);
p.add(FAMILY_B, QUALIFIER, now, Bytes.toBytes("Dat21"));
p.add(FAMILY_A, QUALIFIER, now + 1, Bytes.toBytes("Data22"));
p.add(FAMILY_B, QUALIFIER, now + 2, Bytes.toBytes("Data23"));
t.put(p);
String[] args = { sourceTable.getNameAsString(), FQ_OUTPUT_DIR.toString(), ";", "^row1" };
runCount(args);
FileInputStream inputStream = new FileInputStream(OUTPUT_DIR + File.separator +
"part-r-00000");
String data = IOUtils.toString(inputStream);
inputStream.close();
assertTrue(data.contains("Total Families Across all Rows" + "\t" + "2"));
assertTrue(data.contains("Total Qualifiers across all Rows" + "\t" + "2"));
assertTrue(data.contains("Total ROWS" + "\t" + "1"));
assertTrue(data.contains("b;q" + "\t" + "1"));
assertTrue(data.contains("a;q" + "\t" + "1"));
assertTrue(data.contains("row1;a;q_Versions" + "\t" + "1"));
assertTrue(data.contains("row1;b;q_Versions" + "\t" + "1"));
}finally{
t.close();
FileUtil.fullyDelete(new File(OUTPUT_DIR));
}
}
/**
@ -119,9 +119,9 @@ public class TestCellCounter {
*/
@Test (timeout=300000)
public void testCellCounterStartTimeRange() throws Exception {
String sourceTable = "testCellCounterStartTimeRange";
TableName sourceTable = TableName.valueOf("testCellCounterStartTimeRange");
byte[][] families = { FAMILY_A, FAMILY_B };
Table t = UTIL.createTable(Bytes.toBytes(sourceTable), families);
Table t = UTIL.createTable(sourceTable, families);
try{
Put p = new Put(ROW1);
p.add(FAMILY_A, QUALIFIER, now, Bytes.toBytes("Data11"));
@ -134,7 +134,7 @@ public class TestCellCounter {
p.add(FAMILY_B, QUALIFIER, now + 2, Bytes.toBytes("Data23"));
t.put(p);
String[] args = {
sourceTable, FQ_OUTPUT_DIR.toString(), ";", "^row1", "--starttime=" + now,
sourceTable.getNameAsString(), FQ_OUTPUT_DIR.toString(), ";", "^row1", "--starttime=" + now,
"--endtime=" + now + 2 };
runCount(args);
FileInputStream inputStream = new FileInputStream(OUTPUT_DIR + File.separator +
@ -159,9 +159,9 @@ public class TestCellCounter {
*/
@Test (timeout=300000)
public void testCellCounteEndTimeRange() throws Exception {
String sourceTable = "testCellCounterEndTimeRange";
TableName sourceTable = TableName.valueOf("testCellCounterEndTimeRange");
byte[][] families = { FAMILY_A, FAMILY_B };
Table t = UTIL.createTable(Bytes.toBytes(sourceTable), families);
Table t = UTIL.createTable(sourceTable, families);
try{
Put p = new Put(ROW1);
p.add(FAMILY_A, QUALIFIER, now, Bytes.toBytes("Data11"));
@ -174,7 +174,8 @@ public class TestCellCounter {
p.add(FAMILY_B, QUALIFIER, now + 2, Bytes.toBytes("Data23"));
t.put(p);
String[] args = {
sourceTable, FQ_OUTPUT_DIR.toString(), ";", "^row1", "--endtime=" + now + 1 };
sourceTable.getNameAsString(), FQ_OUTPUT_DIR.toString(), ";", "^row1",
"--endtime=" + now + 1 };
runCount(args);
FileInputStream inputStream = new FileInputStream(OUTPUT_DIR + File.separator +
"part-r-00000");
@ -198,9 +199,9 @@ public class TestCellCounter {
*/
@Test (timeout=300000)
public void testCellCounteOutOfTimeRange() throws Exception {
String sourceTable = "testCellCounterOutTimeRange";
TableName sourceTable = TableName.valueOf("testCellCounterOutTimeRange");
byte[][] families = { FAMILY_A, FAMILY_B };
Table t = UTIL.createTable(Bytes.toBytes(sourceTable), families);
Table t = UTIL.createTable(sourceTable, families);
try{
Put p = new Put(ROW1);
p.add(FAMILY_A, QUALIFIER, now, Bytes.toBytes("Data11"));
@ -213,7 +214,7 @@ public class TestCellCounter {
p.add(FAMILY_B, QUALIFIER, now + 2, Bytes.toBytes("Data23"));
t.put(p);
String[] args = {
sourceTable, FQ_OUTPUT_DIR.toString(), ";", "--starttime=" + now + 1,
sourceTable.getNameAsString(), FQ_OUTPUT_DIR.toString(), ";", "--starttime=" + now + 1,
"--endtime=" + now + 2 };
runCount(args);
@ -275,14 +276,14 @@ public class TestCellCounter {
*/
@Test(timeout = 300000)
public void testCellCounterForCompleteTable() throws Exception {
String sourceTable = "testCellCounterForCompleteTable";
TableName sourceTable = TableName.valueOf("testCellCounterForCompleteTable");
String outputPath = OUTPUT_DIR + sourceTable;
LocalFileSystem localFileSystem = new LocalFileSystem();
Path outputDir =
new Path(outputPath).makeQualified(localFileSystem.getUri(),
localFileSystem.getWorkingDirectory());
byte[][] families = { FAMILY_A, FAMILY_B };
Table t = UTIL.createTable(Bytes.toBytes(sourceTable), families);
Table t = UTIL.createTable(sourceTable, families);
try {
Put p = new Put(ROW1);
p.add(FAMILY_A, QUALIFIER, now, Bytes.toBytes("Data11"));
@ -294,7 +295,7 @@ public class TestCellCounter {
p.add(FAMILY_A, QUALIFIER, now + 1, Bytes.toBytes("Data22"));
p.add(FAMILY_B, QUALIFIER, now + 2, Bytes.toBytes("Data23"));
t.put(p);
String[] args = { sourceTable, outputDir.toString(), ";" };
String[] args = { sourceTable.getNameAsString(), outputDir.toString(), ";" };
runCount(args);
FileInputStream inputStream =
new FileInputStream(outputPath + File.separator + "part-r-00000");

View File

@ -189,13 +189,13 @@ public class TestCopyTable {
*/
@Test
public void testRenameFamily() throws Exception {
String sourceTable = "sourceTable";
String targetTable = "targetTable";
TableName sourceTable = TableName.valueOf("sourceTable");
TableName targetTable = TableName.valueOf("targetTable");
byte[][] families = { FAMILY_A, FAMILY_B };
Table t = TEST_UTIL.createTable(Bytes.toBytes(sourceTable), families);
Table t2 = TEST_UTIL.createTable(Bytes.toBytes(targetTable), families);
Table t = TEST_UTIL.createTable(sourceTable, families);
Table t2 = TEST_UTIL.createTable(targetTable, families);
Put p = new Put(ROW1);
p.add(FAMILY_A, QUALIFIER, Bytes.toBytes("Data11"));
p.add(FAMILY_B, QUALIFIER, Bytes.toBytes("Data12"));
@ -210,7 +210,7 @@ public class TestCopyTable {
long currentTime = System.currentTimeMillis();
String[] args = new String[] { "--new.name=" + targetTable, "--families=a:b", "--all.cells",
"--starttime=" + (currentTime - 100000), "--endtime=" + (currentTime + 100000),
"--versions=1", sourceTable };
"--versions=1", sourceTable.getNameAsString() };
assertNull(t2.get(new Get(ROW1)).getRow());
assertTrue(runCopy(args));

View File

@ -389,55 +389,57 @@ public class TestHFileOutputFormat {
util.startMiniCluster();
Path testDir = util.getDataTestDirOnTestFS("testLocalMRIncrementalLoad");
HBaseAdmin admin = util.getHBaseAdmin();
HTable table = util.createTable(TABLE_NAME, FAMILIES, splitKeys);
Table table = util.createTable(TABLE_NAME, FAMILIES, splitKeys);
assertEquals("Should start with empty table",
0, util.countRows(table));
int numRegions = -1;
try(RegionLocator r = table.getRegionLocator()) {
try(RegionLocator r = util.getConnection().getRegionLocator(TABLE_NAME)) {
numRegions = r.getStartKeys().length;
}
assertEquals("Should make 5 regions", numRegions, 5);
// Generate the bulk load files
util.startMiniMapReduceCluster();
runIncrementalPELoad(conf, table, testDir);
// This doesn't write into the table, just makes files
assertEquals("HFOF should not touch actual table",
0, util.countRows(table));
assertEquals("Should make 5 regions", numRegions, 5);
// Generate the bulk load files
util.startMiniMapReduceCluster();
runIncrementalPELoad(conf, table, r, testDir);
// This doesn't write into the table, just makes files
assertEquals("HFOF should not touch actual table",
0, util.countRows(table));
// Make sure that a directory was created for every CF
int dir = 0;
for (FileStatus f : testDir.getFileSystem(conf).listStatus(testDir)) {
for (byte[] family : FAMILIES) {
if (Bytes.toString(family).equals(f.getPath().getName())) {
++dir;
// Make sure that a directory was created for every CF
int dir = 0;
for (FileStatus f : testDir.getFileSystem(conf).listStatus(testDir)) {
for (byte[] family : FAMILIES) {
if (Bytes.toString(family).equals(f.getPath().getName())) {
++dir;
}
}
}
}
assertEquals("Column family not found in FS.", FAMILIES.length, dir);
assertEquals("Column family not found in FS.", FAMILIES.length, dir);
// handle the split case
if (shouldChangeRegions) {
LOG.info("Changing regions in table");
admin.disableTable(table.getName());
while(util.getMiniHBaseCluster().getMaster().getAssignmentManager().
getRegionStates().isRegionsInTransition()) {
Threads.sleep(200);
LOG.info("Waiting on table to finish disabling");
// handle the split case
if (shouldChangeRegions) {
LOG.info("Changing regions in table");
admin.disableTable(table.getName());
while(util.getMiniHBaseCluster().getMaster().getAssignmentManager().
getRegionStates().isRegionsInTransition()) {
Threads.sleep(200);
LOG.info("Waiting on table to finish disabling");
}
util.deleteTable(table.getName());
byte[][] newSplitKeys = generateRandomSplitKeys(14);
table = util.createTable(TABLE_NAME, FAMILIES, newSplitKeys);
while (r.getAllRegionLocations().size() != 15 ||
!admin.isTableAvailable(table.getName())) {
Thread.sleep(200);
LOG.info("Waiting for new region assignment to happen");
}
}
util.deleteTable(table.getName());
byte[][] newSplitKeys = generateRandomSplitKeys(14);
table = util.createTable(TABLE_NAME, FAMILIES, newSplitKeys);
while (table.getRegionLocations().size() != 15 ||
!admin.isTableAvailable(table.getName())) {
Thread.sleep(200);
LOG.info("Waiting for new region assignment to happen");
}
}
// Perform the actual load
new LoadIncrementalHFiles(conf).doBulkLoad(testDir, table);
// Perform the actual load
new LoadIncrementalHFiles(conf).doBulkLoad(testDir, admin, table, r);
}
// Ensure data shows up
int expectedRows = NMapInputFormat.getNumMapTasks(conf) * ROWSPERSPLIT;
@ -473,7 +475,7 @@ public class TestHFileOutputFormat {
}
private void runIncrementalPELoad(
Configuration conf, HTable table, Path outDir)
Configuration conf, Table table, RegionLocator regionLocator, Path outDir)
throws Exception {
Job job = new Job(conf, "testLocalMRIncrementalLoad");
job.setWorkingDirectory(util.getDataTestDirOnTestFS("runIncrementalPELoad"));
@ -482,12 +484,12 @@ public class TestHFileOutputFormat {
KeyValueSerialization.class.getName());
setupRandomGeneratorMapper(job);
HFileOutputFormat2.configureIncrementalLoad(job, table.getTableDescriptor(),
table.getRegionLocator());
regionLocator);
FileOutputFormat.setOutputPath(job, outDir);
Assert.assertFalse( util.getTestFileSystem().exists(outDir)) ;
assertEquals(table.getRegionLocator().getAllRegionLocations().size(), job.getNumReduceTasks());
assertEquals(regionLocator.getAllRegionLocations().size(), job.getNumReduceTasks());
assertTrue(job.waitForCompletion(true));
}
@ -910,7 +912,7 @@ public class TestHFileOutputFormat {
util.startMiniCluster();
final FileSystem fs = util.getDFSCluster().getFileSystem();
HBaseAdmin admin = util.getHBaseAdmin();
HTable table = util.createTable(TABLE_NAME, FAMILIES);
Table table = util.createTable(TABLE_NAME, FAMILIES);
assertEquals("Should start with empty table", 0, util.countRows(table));
// deep inspection: get the StoreFile dir
@ -925,11 +927,13 @@ public class TestHFileOutputFormat {
true);
util.startMiniMapReduceCluster();
for (int i = 0; i < 2; i++) {
Path testDir = util.getDataTestDirOnTestFS("testExcludeAllFromMinorCompaction_" + i);
runIncrementalPELoad(conf, table, testDir);
// Perform the actual load
new LoadIncrementalHFiles(conf).doBulkLoad(testDir, table);
try (RegionLocator regionLocator = util.getConnection().getRegionLocator(TABLE_NAME)) {
for (int i = 0; i < 2; i++) {
Path testDir = util.getDataTestDirOnTestFS("testExcludeAllFromMinorCompaction_" + i);
runIncrementalPELoad(conf, table, regionLocator, testDir);
// Perform the actual load
new LoadIncrementalHFiles(conf).doBulkLoad(testDir, admin, table, regionLocator);
}
}
// Ensure data shows up
@ -978,7 +982,7 @@ public class TestHFileOutputFormat {
Path testDir = util.getDataTestDirOnTestFS("testExcludeMinorCompaction");
final FileSystem fs = util.getDFSCluster().getFileSystem();
Admin admin = util.getHBaseAdmin();
HTable table = util.createTable(TABLE_NAME, FAMILIES);
Table table = util.createTable(TABLE_NAME, FAMILIES);
assertEquals("Should start with empty table", 0, util.countRows(table));
// deep inspection: get the StoreFile dir
@ -1004,10 +1008,13 @@ public class TestHFileOutputFormat {
conf.setBoolean("hbase.mapreduce.hfileoutputformat.compaction.exclude",
true);
util.startMiniMapReduceCluster();
runIncrementalPELoad(conf, table, testDir);
// Perform the actual load
new LoadIncrementalHFiles(conf).doBulkLoad(testDir, table);
try (RegionLocator locator = util.getConnection().getRegionLocator(TABLE_NAME)) {
runIncrementalPELoad(conf, table, locator, testDir);
// Perform the actual load
new LoadIncrementalHFiles(conf).doBulkLoad(testDir, admin, table, locator);
}
// Ensure data shows up
int expectedRows = NMapInputFormat.getNumMapTasks(conf) * ROWSPERSPLIT;
@ -1066,17 +1073,17 @@ public class TestHFileOutputFormat {
if ("newtable".equals(args[0])) {
TableName tname = TableName.valueOf(args[1]);
byte[][] splitKeys = generateRandomSplitKeys(4);
HTable table = util.createTable(tname, FAMILIES, splitKeys);
Table table = util.createTable(tname, FAMILIES, splitKeys);
} else if ("incremental".equals(args[0])) {
TableName tname = TableName.valueOf(args[1]);
HTable table = (HTable) util.getConnection().getTable(tname);
Table table = util.getConnection().getTable(tname);
Path outDir = new Path("incremental-out");
runIncrementalPELoad(conf, table, outDir);
try (RegionLocator locator = util.getConnection().getRegionLocator(tname)) {
runIncrementalPELoad(conf, table, locator, outDir);
}
} else {
throw new RuntimeException(
"usage: TestHFileOutputFormat newtable | incremental");
}
}
}

View File

@ -56,6 +56,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.HRegionLocator;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionLocator;
@ -388,25 +389,26 @@ public class TestHFileOutputFormat2 {
Configuration conf = util.getConfiguration();
byte[][] splitKeys = generateRandomSplitKeys(4);
util.startMiniCluster();
try {
HTable table = util.createTable(TABLE_NAME, FAMILIES, splitKeys);
Admin admin = table.getConnection().getAdmin();
Table table = util.createTable(TABLE_NAME, FAMILIES, splitKeys);
try (RegionLocator r = util.getConnection().getRegionLocator(TABLE_NAME)) {
Admin admin = util.getConnection().getAdmin();
Path testDir = util.getDataTestDirOnTestFS("testLocalMRIncrementalLoad");
assertEquals("Should start with empty table",
0, util.countRows(table));
int numRegions = -1;
try (RegionLocator r = table.getRegionLocator()) {
int numRegions;
numRegions = r.getStartKeys().length;
}
assertEquals("Should make 5 regions", numRegions, 5);
// Generate the bulk load files
util.startMiniMapReduceCluster();
runIncrementalPELoad(conf, table.getTableDescriptor(), table.getRegionLocator(), testDir);
// This doesn't write into the table, just makes files
assertEquals("HFOF should not touch actual table",
0, util.countRows(table));
assertEquals("Should make 5 regions", numRegions, 5);
// Generate the bulk load files
util.startMiniMapReduceCluster();
runIncrementalPELoad(conf, table.getTableDescriptor(), r, testDir);
// This doesn't write into the table, just makes files
assertEquals("HFOF should not touch actual table",
0, util.countRows(table));
// Make sure that a directory was created for every CF
int dir = 0;
@ -432,7 +434,8 @@ public class TestHFileOutputFormat2 {
byte[][] newSplitKeys = generateRandomSplitKeys(14);
table = util.createTable(TABLE_NAME, FAMILIES, newSplitKeys);
while (table.getRegionLocator().getAllRegionLocations().size() != 15 ||
while (util.getConnection().getRegionLocator(TABLE_NAME)
.getAllRegionLocations().size() != 15 ||
!admin.isTableAvailable(table.getName())) {
Thread.sleep(200);
LOG.info("Waiting for new region assignment to happen");
@ -440,7 +443,7 @@ public class TestHFileOutputFormat2 {
}
// Perform the actual load
new LoadIncrementalHFiles(conf).doBulkLoad(testDir, table);
new LoadIncrementalHFiles(conf).doBulkLoad(testDir, admin, table, r);
// Ensure data shows up
int expectedRows = NMapInputFormat.getNumMapTasks(conf) * ROWSPERSPLIT;
@ -911,9 +914,10 @@ public class TestHFileOutputFormat2 {
util.startMiniCluster();
try (Connection conn = ConnectionFactory.createConnection();
Admin admin = conn.getAdmin()) {
Admin admin = conn.getAdmin();
Table table = util.createTable(TABLE_NAME, FAMILIES);
RegionLocator locator = conn.getRegionLocator(TABLE_NAME)) {
final FileSystem fs = util.getDFSCluster().getFileSystem();
HTable table = util.createTable(TABLE_NAME, FAMILIES);
assertEquals("Should start with empty table", 0, util.countRows(table));
// deep inspection: get the StoreFile dir
@ -933,7 +937,7 @@ public class TestHFileOutputFormat2 {
runIncrementalPELoad(conf, table.getTableDescriptor(), conn.getRegionLocator(TABLE_NAME),
testDir);
// Perform the actual load
new LoadIncrementalHFiles(conf).doBulkLoad(testDir, table);
new LoadIncrementalHFiles(conf).doBulkLoad(testDir, admin, table, locator);
}
// Ensure data shows up
@ -1077,7 +1081,7 @@ public class TestHFileOutputFormat2 {
if ("newtable".equals(args[0])) {
TableName tname = TableName.valueOf(args[1]);
byte[][] splitKeys = generateRandomSplitKeys(4);
try (HTable table = util.createTable(tname, FAMILIES, splitKeys)) {
try (Table table = util.createTable(tname, FAMILIES, splitKeys)) {
}
} else if ("incremental".equals(args[0])) {
TableName tname = TableName.valueOf(args[1]);

View File

@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.Bytes;
@ -68,7 +69,7 @@ public class TestHashTable {
@Test
public void testHashTable() throws Exception {
final String tableName = "testHashTable";
final TableName tableName = TableName.valueOf("testHashTable");
final byte[] family = Bytes.toBytes("family");
final byte[] column1 = Bytes.toBytes("c1");
final byte[] column2 = Bytes.toBytes("c2");
@ -85,7 +86,7 @@ public class TestHashTable {
long timestamp = 1430764183454L;
// put rows into the first table
HTable t1 = TEST_UTIL.createTable(TableName.valueOf(tableName), family, splitRows);
Table t1 = TEST_UTIL.createTable(tableName, family, splitRows);
for (int i = 0; i < numRows; i++) {
Put p = new Put(Bytes.toBytes(i), timestamp);
p.addColumn(family, column1, column1);
@ -97,21 +98,21 @@ public class TestHashTable {
HashTable hashTable = new HashTable(TEST_UTIL.getConfiguration());
Path testDir = TEST_UTIL.getDataTestDirOnTestFS(tableName);
Path testDir = TEST_UTIL.getDataTestDirOnTestFS(tableName.getNameAsString());
long batchSize = 300;
int code = hashTable.run(new String[] {
"--batchsize=" + batchSize,
"--numhashfiles=" + numHashFiles,
"--scanbatch=2",
tableName,
tableName.getNameAsString(),
testDir.toString()});
assertEquals("test job failed", 0, code);
FileSystem fs = TEST_UTIL.getTestFileSystem();
HashTable.TableHash tableHash = HashTable.TableHash.read(fs.getConf(), testDir);
assertEquals(tableName, tableHash.tableName);
assertEquals(tableName.getNameAsString(), tableHash.tableName);
assertEquals(batchSize, tableHash.batchSize);
assertEquals(numHashFiles, tableHash.numHashFiles);
assertEquals(numHashFiles - 1, tableHash.partitions.size());

View File

@ -110,32 +110,32 @@ public class TestImportTSVWithOperationAttributes implements Configurable {
@Test
public void testMROnTable() throws Exception {
String tableName = "test-" + UUID.randomUUID();
TableName tableName = TableName.valueOf("test-" + UUID.randomUUID());
// Prepare the arguments required for the test.
String[] args = new String[] {
"-D" + ImportTsv.MAPPER_CONF_KEY
+ "=org.apache.hadoop.hbase.mapreduce.TsvImporterCustomTestMapperForOprAttr",
"-D" + ImportTsv.COLUMNS_CONF_KEY + "=HBASE_ROW_KEY,FAM:A,FAM:B,HBASE_ATTRIBUTES_KEY",
"-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b", tableName };
"-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b", tableName.getNameAsString() };
String data = "KEY\u001bVALUE1\u001bVALUE2\u001btest=>myvalue\n";
util.createTable(TableName.valueOf(tableName), FAMILY);
util.createTable(tableName, FAMILY);
doMROnTableTest(util, FAMILY, data, args, 1, true);
util.deleteTable(tableName);
}
@Test
public void testMROnTableWithInvalidOperationAttr() throws Exception {
String tableName = "test-" + UUID.randomUUID();
TableName tableName = TableName.valueOf("test-" + UUID.randomUUID());
// Prepare the arguments required for the test.
String[] args = new String[] {
"-D" + ImportTsv.MAPPER_CONF_KEY
+ "=org.apache.hadoop.hbase.mapreduce.TsvImporterCustomTestMapperForOprAttr",
"-D" + ImportTsv.COLUMNS_CONF_KEY + "=HBASE_ROW_KEY,FAM:A,FAM:B,HBASE_ATTRIBUTES_KEY",
"-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b", tableName };
"-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b", tableName.getNameAsString() };
String data = "KEY\u001bVALUE1\u001bVALUE2\u001btest1=>myvalue\n";
util.createTable(TableName.valueOf(tableName), FAMILY);
util.createTable(tableName, FAMILY);
doMROnTableTest(util, FAMILY, data, args, 1, false);
util.deleteTable(tableName);
}

View File

@ -101,16 +101,16 @@ public class TestImportTSVWithTTLs implements Configurable {
@Test
public void testMROnTable() throws Exception {
String tableName = "test-" + UUID.randomUUID();
TableName tableName = TableName.valueOf("test-" + UUID.randomUUID());
// Prepare the arguments required for the test.
String[] args = new String[] {
"-D" + ImportTsv.MAPPER_CONF_KEY
+ "=org.apache.hadoop.hbase.mapreduce.TsvImporterMapper",
"-D" + ImportTsv.COLUMNS_CONF_KEY + "=HBASE_ROW_KEY,FAM:A,FAM:B,HBASE_CELL_TTL",
"-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b", tableName };
"-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b", tableName.getNameAsString() };
String data = "KEY\u001bVALUE1\u001bVALUE2\u001b1000000\n";
util.createTable(TableName.valueOf(tableName), FAMILY);
util.createTable(tableName, FAMILY);
doMROnTableTest(util, FAMILY, data, args, 1);
util.deleteTable(tableName);
}

View File

@ -154,16 +154,16 @@ public class TestImportTSVWithVisibilityLabels implements Configurable {
@Test
public void testMROnTable() throws Exception {
String tableName = "test-" + UUID.randomUUID();
TableName tableName = TableName.valueOf("test-" + UUID.randomUUID());
// Prepare the arguments required for the test.
String[] args = new String[] {
"-D" + ImportTsv.MAPPER_CONF_KEY
+ "=org.apache.hadoop.hbase.mapreduce.TsvImporterMapper",
"-D" + ImportTsv.COLUMNS_CONF_KEY + "=HBASE_ROW_KEY,FAM:A,FAM:B,HBASE_CELL_VISIBILITY",
"-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b", tableName };
"-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b", tableName.getNameAsString() };
String data = "KEY\u001bVALUE1\u001bVALUE2\u001bsecret&private\n";
util.createTable(TableName.valueOf(tableName), FAMILY);
util.createTable(tableName, FAMILY);
doMROnTableTest(util, FAMILY, data, args, 1);
util.deleteTable(tableName);
}
@ -222,25 +222,25 @@ public class TestImportTSVWithVisibilityLabels implements Configurable {
@Test
public void testMROnTableWithBulkload() throws Exception {
String tableName = "test-" + UUID.randomUUID();
Path hfiles = new Path(util.getDataTestDirOnTestFS(tableName), "hfiles");
TableName tableName = TableName.valueOf("test-" + UUID.randomUUID());
Path hfiles = new Path(util.getDataTestDirOnTestFS(tableName.getNameAsString()), "hfiles");
// Prepare the arguments required for the test.
String[] args = new String[] {
"-D" + ImportTsv.BULK_OUTPUT_CONF_KEY + "=" + hfiles.toString(),
"-D" + ImportTsv.COLUMNS_CONF_KEY
+ "=HBASE_ROW_KEY,FAM:A,FAM:B,HBASE_CELL_VISIBILITY",
"-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b", tableName };
"-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b", tableName.getNameAsString() };
String data = "KEY\u001bVALUE1\u001bVALUE2\u001bsecret&private\n";
util.createTable(TableName.valueOf(tableName), FAMILY);
util.createTable(tableName, FAMILY);
doMROnTableTest(util, FAMILY, data, args, 1);
util.deleteTable(tableName);
}
@Test
public void testBulkOutputWithTsvImporterTextMapper() throws Exception {
String table = "test-" + UUID.randomUUID();
TableName table = TableName.valueOf("test-" + UUID.randomUUID());
String FAMILY = "FAM";
Path bulkOutputPath = new Path(util.getDataTestDirOnTestFS(table),"hfiles");
Path bulkOutputPath = new Path(util.getDataTestDirOnTestFS(table.getNameAsString()),"hfiles");
// Prepare the arguments required for the test.
String[] args =
new String[] {
@ -249,7 +249,8 @@ public class TestImportTSVWithVisibilityLabels implements Configurable {
"-D" + ImportTsv.COLUMNS_CONF_KEY
+ "=HBASE_ROW_KEY,FAM:A,FAM:B,HBASE_CELL_VISIBILITY",
"-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b",
"-D" + ImportTsv.BULK_OUTPUT_CONF_KEY + "=" + bulkOutputPath.toString(), table
"-D" + ImportTsv.BULK_OUTPUT_CONF_KEY + "=" + bulkOutputPath.toString(),
table.getNameAsString()
};
String data = "KEY\u001bVALUE4\u001bVALUE8\u001bsecret&private\n";
doMROnTableTest(util, FAMILY, data, args, 4);
@ -258,17 +259,17 @@ public class TestImportTSVWithVisibilityLabels implements Configurable {
@Test
public void testMRWithOutputFormat() throws Exception {
String tableName = "test-" + UUID.randomUUID();
Path hfiles = new Path(util.getDataTestDirOnTestFS(tableName), "hfiles");
TableName tableName = TableName.valueOf("test-" + UUID.randomUUID());
Path hfiles = new Path(util.getDataTestDirOnTestFS(tableName.getNameAsString()), "hfiles");
// Prepare the arguments required for the test.
String[] args = new String[] {
"-D" + ImportTsv.MAPPER_CONF_KEY
+ "=org.apache.hadoop.hbase.mapreduce.TsvImporterMapper",
"-D" + ImportTsv.BULK_OUTPUT_CONF_KEY + "=" + hfiles.toString(),
"-D" + ImportTsv.COLUMNS_CONF_KEY + "=HBASE_ROW_KEY,FAM:A,FAM:B,HBASE_CELL_VISIBILITY",
"-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b", tableName };
"-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b", tableName.getNameAsString() };
String data = "KEY\u001bVALUE4\u001bVALUE8\u001bsecret&private\n";
util.createTable(TableName.valueOf(tableName), FAMILY);
util.createTable(tableName, FAMILY);
doMROnTableTest(util, FAMILY, data, args, 1);
util.deleteTable(tableName);
}

View File

@ -125,7 +125,7 @@ public class TestImportTsv implements Configurable {
doMROnTableTest(null, 1);
util.deleteTable(table);
}
@Test
public void testMROnTableWithTimestamp() throws Exception {
util.createTable(TableName.valueOf(table), FAMILY);
@ -147,7 +147,7 @@ public class TestImportTsv implements Configurable {
doMROnTableTest(null, 3);
util.deleteTable(table);
}
@Test
public void testBulkOutputWithoutAnExistingTable() throws Exception {
// Prepare the arguments required for the test.
@ -169,7 +169,7 @@ public class TestImportTsv implements Configurable {
doMROnTableTest(null, 3);
util.deleteTable(table);
}
@Test
public void testBulkOutputWithAnExistingTableNoStrictTrue() throws Exception {
util.createTable(TableName.valueOf(table), FAMILY);

View File

@ -50,6 +50,7 @@ import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
@ -177,8 +178,10 @@ public class TestLoadIncrementalHFilesSplitRecovery {
// create HFiles for different column families
LoadIncrementalHFiles lih = new LoadIncrementalHFiles(util.getConfiguration());
Path bulk1 = buildBulkFiles(table, value);
try (Table t = connection.getTable(table)) {
lih.doBulkLoad(bulk1, (HTable)t);
try (Table t = connection.getTable(table);
RegionLocator locator = connection.getRegionLocator(table);
Admin admin = connection.getAdmin()) {
lih.doBulkLoad(bulk1, admin, t, locator);
}
}
@ -223,7 +226,7 @@ public class TestLoadIncrementalHFilesSplitRecovery {
@BeforeClass
public static void setupCluster() throws Exception {
util = new HBaseTestingUtility();
util.getConfiguration().set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,"");
util.getConfiguration().set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, "");
util.startMiniCluster(1);
}
@ -298,8 +301,10 @@ public class TestLoadIncrementalHFilesSplitRecovery {
try {
// create HFiles for different column families
Path dir = buildBulkFiles(table, 1);
try (Table t = connection.getTable(table)) {
lih.doBulkLoad(dir, (HTable)t);
try (Table t = connection.getTable(table);
RegionLocator locator = connection.getRegionLocator(table);
Admin admin = connection.getAdmin()) {
lih.doBulkLoad(dir, admin, t, locator);
}
} finally {
util.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
@ -364,9 +369,11 @@ public class TestLoadIncrementalHFilesSplitRecovery {
};
// create HFiles for different column families
try (Table t = connection.getTable(table)) {
try (Table t = connection.getTable(table);
RegionLocator locator = connection.getRegionLocator(table);
Admin admin = connection.getAdmin()) {
Path bulk = buildBulkFiles(table, 2);
lih2.doBulkLoad(bulk, (HTable)t);
lih2.doBulkLoad(bulk, admin, t, locator);
}
// check that data was loaded
@ -408,8 +415,10 @@ public class TestLoadIncrementalHFilesSplitRecovery {
// create HFiles for different column families
Path bulk = buildBulkFiles(table, 2);
try (Table t = connection.getTable(table)) {
lih.doBulkLoad(bulk, (HTable)t);
try (Table t = connection.getTable(table);
RegionLocator locator = connection.getRegionLocator(table);
Admin admin = connection.getAdmin()) {
lih.doBulkLoad(bulk, admin, t, locator);
}
assertExpectedTable(connection, table, ROWCOUNT, 2);
assertEquals(20, countedLqis.get());
@ -446,8 +455,10 @@ public class TestLoadIncrementalHFilesSplitRecovery {
// create HFiles for different column families
Path dir = buildBulkFiles(table,1);
try (Table t = connection.getTable(table)) {
lih.doBulkLoad(dir, (HTable)t);
try (Table t = connection.getTable(table);
RegionLocator locator = connection.getRegionLocator(table);
Admin admin = connection.getAdmin()) {
lih.doBulkLoad(dir, admin, t, locator);
}
}
@ -472,7 +483,7 @@ public class TestLoadIncrementalHFilesSplitRecovery {
protected List<LoadQueueItem> groupOrSplit(
Multimap<ByteBuffer, LoadQueueItem> regionGroups,
final LoadQueueItem item, final HTable htable,
final LoadQueueItem item, final Table htable,
final Pair<byte[][], byte[][]> startEndKeys) throws IOException {
List<LoadQueueItem> lqis = super.groupOrSplit(regionGroups, item, htable, startEndKeys);
if (lqis != null) {
@ -483,8 +494,10 @@ public class TestLoadIncrementalHFilesSplitRecovery {
};
// do bulkload when there is no region hole in hbase:meta.
try {
loader.doBulkLoad(dir, (HTable)table);
try (Table t = connection.getTable(tableName);
RegionLocator locator = connection.getRegionLocator(tableName);
Admin admin = connection.getAdmin()) {
loader.doBulkLoad(dir, admin, t, locator);
} catch (Exception e) {
LOG.error("exeception=", e);
}
@ -502,10 +515,12 @@ public class TestLoadIncrementalHFilesSplitRecovery {
}
}
try {
loader.doBulkLoad(dir, (HTable)table);
try (Table t = connection.getTable(tableName);
RegionLocator locator = connection.getRegionLocator(tableName);
Admin admin = connection.getAdmin()) {
loader.doBulkLoad(dir, admin, t, locator);
} catch (Exception e) {
LOG.error("exeception=", e);
LOG.error("exception=", e);
assertTrue("IOException expected", e instanceof IOException);
}

View File

@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.mapreduce;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests;
@ -40,9 +41,9 @@ public class TestMultiTableInputFormat extends MultiTableInputFormatTestBase {
@BeforeClass
public static void setupLogging() {
TEST_UTIL.enableDebug(MultiTableInputFormat.class);
}
}
@Override
@Override
protected void initJob(List<Scan> scans, Job job) throws IOException {
TableMapReduceUtil.initTableMapperJob(scans, ScanMapper.class,
ImmutableBytesWritable.class, ImmutableBytesWritable.class, job);

View File

@ -70,7 +70,7 @@ public class TestMultithreadedTableMapper {
@BeforeClass
public static void beforeClass() throws Exception {
UTIL.startMiniCluster();
HTable table =
Table table =
UTIL.createMultiRegionTable(MULTI_REGION_TABLE_NAME, new byte[][] { INPUT_FAMILY,
OUTPUT_FAMILY });
UTIL.loadTable(table, INPUT_FAMILY, false);

View File

@ -155,7 +155,7 @@ public class TestRowCounter {
long ts;
// clean up content of TABLE_NAME
HTable table = TEST_UTIL.deleteTableData(TableName.valueOf(TABLE_NAME));
Table table = TEST_UTIL.deleteTableData(TableName.valueOf(TABLE_NAME));
ts = System.currentTimeMillis();
put1.add(family, col1, ts, Bytes.toBytes("val1"));
table.put(put1);

View File

@ -80,8 +80,8 @@ public class TestSyncTable {
@Test
public void testSyncTable() throws Exception {
String sourceTableName = "testSourceTable";
String targetTableName = "testTargetTable";
TableName sourceTableName = TableName.valueOf("testSourceTable");
TableName targetTableName = TableName.valueOf("testTargetTable");
Path testDir = TEST_UTIL.getDataTestDirOnTestFS("testSyncTable");
writeTestData(sourceTableName, targetTableName);
@ -101,10 +101,10 @@ public class TestSyncTable {
TEST_UTIL.cleanupDataTestDirOnTestFS();
}
private void assertEqualTables(int expectedRows, String sourceTableName, String targetTableName)
throws Exception {
Table sourceTable = TEST_UTIL.getConnection().getTable(TableName.valueOf(sourceTableName));
Table targetTable = TEST_UTIL.getConnection().getTable(TableName.valueOf(targetTableName));
private void assertEqualTables(int expectedRows, TableName sourceTableName,
TableName targetTableName) throws Exception {
Table sourceTable = TEST_UTIL.getConnection().getTable(sourceTableName);
Table targetTable = TEST_UTIL.getConnection().getTable(targetTableName);
ResultScanner sourceScanner = sourceTable.getScanner(new Scan());
ResultScanner targetScanner = targetTable.getScanner(new Scan());
@ -177,13 +177,13 @@ public class TestSyncTable {
targetTable.close();
}
private Counters syncTables(String sourceTableName, String targetTableName,
private Counters syncTables(TableName sourceTableName, TableName targetTableName,
Path testDir) throws Exception {
SyncTable syncTable = new SyncTable(TEST_UTIL.getConfiguration());
int code = syncTable.run(new String[] {
testDir.toString(),
sourceTableName,
targetTableName
sourceTableName.getNameAsString(),
targetTableName.getNameAsString()
});
assertEquals("sync table job failed", 0, code);
@ -191,7 +191,7 @@ public class TestSyncTable {
return syncTable.counters;
}
private void hashSourceTable(String sourceTableName, Path testDir)
private void hashSourceTable(TableName sourceTableName, Path testDir)
throws Exception, IOException {
int numHashFiles = 3;
long batchSize = 100; // should be 2 batches per region
@ -201,14 +201,14 @@ public class TestSyncTable {
"--batchsize=" + batchSize,
"--numhashfiles=" + numHashFiles,
"--scanbatch=" + scanBatch,
sourceTableName,
sourceTableName.getNameAsString(),
testDir.toString()});
assertEquals("hash table job failed", 0, code);
FileSystem fs = TEST_UTIL.getTestFileSystem();
HashTable.TableHash tableHash = HashTable.TableHash.read(fs.getConf(), testDir);
assertEquals(sourceTableName, tableHash.tableName);
assertEquals(sourceTableName.getNameAsString(), tableHash.tableName);
assertEquals(batchSize, tableHash.batchSize);
assertEquals(numHashFiles, tableHash.numHashFiles);
assertEquals(numHashFiles - 1, tableHash.partitions.size());
@ -216,7 +216,7 @@ public class TestSyncTable {
LOG.info("Hash table completed");
}
private void writeTestData(String sourceTableName, String targetTableName)
private void writeTestData(TableName sourceTableName, TableName targetTableName)
throws Exception {
final byte[] family = Bytes.toBytes("family");
final byte[] column1 = Bytes.toBytes("c1");
@ -229,10 +229,10 @@ public class TestSyncTable {
int sourceRegions = 10;
int targetRegions = 6;
HTable sourceTable = TEST_UTIL.createTable(TableName.valueOf(sourceTableName),
Table sourceTable = TEST_UTIL.createTable(sourceTableName,
family, generateSplits(numRows, sourceRegions));
HTable targetTable = TEST_UTIL.createTable(TableName.valueOf(targetTableName),
Table targetTable = TEST_UTIL.createTable(targetTableName,
family, generateSplits(numRows, targetRegions));
long timestamp = 1430764183454L;

View File

@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.HTable;
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.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.NullWritable;
@ -63,12 +64,12 @@ public abstract class TestTableInputFormatScanBase {
private static final Log LOG = LogFactory.getLog(TestTableInputFormatScanBase.class);
static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
static final byte[] TABLE_NAME = Bytes.toBytes("scantest");
static final TableName TABLE_NAME = TableName.valueOf("scantest");
static final byte[] INPUT_FAMILY = Bytes.toBytes("contents");
static final String KEY_STARTROW = "startRow";
static final String KEY_LASTROW = "stpRow";
private static HTable table = null;
private static Table table = null;
@BeforeClass
public static void setUpBeforeClass() throws Exception {
@ -82,7 +83,7 @@ public abstract class TestTableInputFormatScanBase {
// start mini hbase cluster
TEST_UTIL.startMiniCluster(3);
// create and fill table
table = TEST_UTIL.createMultiRegionTable(TableName.valueOf(TABLE_NAME), INPUT_FAMILY);
table = TEST_UTIL.createMultiRegionTable(TABLE_NAME, INPUT_FAMILY);
TEST_UTIL.loadTable(table, INPUT_FAMILY, false);
// start MR cluster
TEST_UTIL.startMiniMapReduceCluster();
@ -182,7 +183,7 @@ public abstract class TestTableInputFormatScanBase {
String jobName = "ScanFromConfig" + (start != null ? start.toUpperCase() : "Empty") +
"To" + (stop != null ? stop.toUpperCase() : "Empty");
Configuration c = new Configuration(TEST_UTIL.getConfiguration());
c.set(TableInputFormat.INPUT_TABLE, Bytes.toString(TABLE_NAME));
c.set(TableInputFormat.INPUT_TABLE, TABLE_NAME.getNameAsString());
c.set(TableInputFormat.SCAN_COLUMN_FAMILY, Bytes.toString(INPUT_FAMILY));
c.set(KEY_STARTROW, start != null ? start : "");
c.set(KEY_LASTROW, last != null ? last : "");
@ -233,7 +234,7 @@ public abstract class TestTableInputFormatScanBase {
LOG.info("scan before: " + scan);
Job job = new Job(c, jobName);
TableMapReduceUtil.initTableMapperJob(
Bytes.toString(TABLE_NAME), scan, ScanMapper.class,
TABLE_NAME, scan, ScanMapper.class,
ImmutableBytesWritable.class, ImmutableBytesWritable.class, job);
job.setReducerClass(ScanReducer.class);
job.setNumReduceTasks(1); // one to get final "first" and "last" key
@ -264,11 +265,11 @@ public abstract class TestTableInputFormatScanBase {
c.set(KEY_STARTROW, "");
c.set(KEY_LASTROW, "");
Job job = new Job(c, jobName);
TableMapReduceUtil.initTableMapperJob(Bytes.toString(TABLE_NAME), scan, ScanMapper.class,
TableMapReduceUtil.initTableMapperJob(TABLE_NAME.getNameAsString(), scan, ScanMapper.class,
ImmutableBytesWritable.class, ImmutableBytesWritable.class, job);
TableInputFormat tif = new TableInputFormat();
tif.setConf(job.getConfiguration());
Assert.assertEquals(new String(TABLE_NAME), new String(table.getTableName()));
Assert.assertEquals(TABLE_NAME, table.getName());
List<InputSplit> splits = tif.getSplits(job);
Assert.assertEquals(expectedNumOfSplits, splits.size());
}

View File

@ -76,7 +76,7 @@ public abstract class TestTableMapReduceBase {
@BeforeClass
public static void beforeClass() throws Exception {
UTIL.startMiniCluster();
HTable table =
Table table =
UTIL.createMultiRegionTable(MULTI_REGION_TABLE_NAME, new byte[][] { INPUT_FAMILY,
OUTPUT_FAMILY });
UTIL.loadTable(table, INPUT_FAMILY, false);

View File

@ -176,9 +176,9 @@ public class TestAssignmentManagerOnCluster {
*/
@Test (timeout=60000)
public void testAssignRegion() throws Exception {
String table = "testAssignRegion";
TableName table = TableName.valueOf("testAssignRegion");
try {
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(table));
HTableDescriptor desc = new HTableDescriptor(table);
desc.addFamily(new HColumnDescriptor(FAMILY));
admin.createTable(desc);
@ -203,7 +203,7 @@ public class TestAssignmentManagerOnCluster {
RegionState newState = regionStates.getRegionState(hri);
assertTrue(newState.isOpened());
} finally {
TEST_UTIL.deleteTable(Bytes.toBytes(table));
TEST_UTIL.deleteTable(table);
}
}
@ -212,7 +212,7 @@ public class TestAssignmentManagerOnCluster {
*/
@Test (timeout=120000)
public void testAssignRegionOnRestartedServer() throws Exception {
String table = "testAssignRegionOnRestartedServer";
TableName table = TableName.valueOf("testAssignRegionOnRestartedServer");
TEST_UTIL.getMiniHBaseCluster().getConf().setInt("hbase.assignment.maximum.attempts", 20);
TEST_UTIL.getMiniHBaseCluster().stopMaster(0);
//restart the master so that conf take into affect
@ -221,7 +221,7 @@ public class TestAssignmentManagerOnCluster {
ServerName deadServer = null;
HMaster master = null;
try {
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(table));
HTableDescriptor desc = new HTableDescriptor(table);
desc.addFamily(new HColumnDescriptor(FAMILY));
admin.createTable(desc);
@ -260,7 +260,7 @@ public class TestAssignmentManagerOnCluster {
master.serverManager.expireServer(deadServer);
}
TEST_UTIL.deleteTable(Bytes.toBytes(table));
TEST_UTIL.deleteTable(table);
// reset the value for other tests
TEST_UTIL.getMiniHBaseCluster().getConf().setInt("hbase.assignment.maximum.attempts", 3);
@ -431,9 +431,9 @@ public class TestAssignmentManagerOnCluster {
*/
@Test (timeout=60000)
public void testAssignWhileClosing() throws Exception {
String table = "testAssignWhileClosing";
TableName table = TableName.valueOf("testAssignWhileClosing");
try {
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(table));
HTableDescriptor desc = new HTableDescriptor(table);
desc.addFamily(new HColumnDescriptor(FAMILY));
admin.createTable(desc);
@ -471,7 +471,7 @@ public class TestAssignmentManagerOnCluster {
TEST_UTIL.assertRegionOnlyOnServer(hri, serverName, 200);
} finally {
MyRegionObserver.preCloseEnabled.set(false);
TEST_UTIL.deleteTable(Bytes.toBytes(table));
TEST_UTIL.deleteTable(table);
}
}
@ -480,9 +480,9 @@ public class TestAssignmentManagerOnCluster {
*/
@Test (timeout=60000)
public void testCloseFailed() throws Exception {
String table = "testCloseFailed";
TableName table = TableName.valueOf("testCloseFailed");
try {
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(table));
HTableDescriptor desc = new HTableDescriptor(table);
desc.addFamily(new HColumnDescriptor(FAMILY));
admin.createTable(desc);
@ -517,7 +517,7 @@ public class TestAssignmentManagerOnCluster {
TEST_UTIL.assertRegionOnServer(hri, serverName, 200);
} finally {
MyRegionObserver.preCloseEnabled.set(false);
TEST_UTIL.deleteTable(Bytes.toBytes(table));
TEST_UTIL.deleteTable(table);
}
}
@ -526,9 +526,9 @@ public class TestAssignmentManagerOnCluster {
*/
@Test (timeout=60000)
public void testOpenFailed() throws Exception {
String table = "testOpenFailed";
TableName table = TableName.valueOf("testOpenFailed");
try {
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(table));
HTableDescriptor desc = new HTableDescriptor(table);
desc.addFamily(new HColumnDescriptor(FAMILY));
admin.createTable(desc);
@ -558,7 +558,7 @@ public class TestAssignmentManagerOnCluster {
TEST_UTIL.assertRegionOnServer(hri, serverName, 200);
} finally {
MyLoadBalancer.controledRegion = null;
TEST_UTIL.deleteTable(Bytes.toBytes(table));
TEST_UTIL.deleteTable(table);
}
}
@ -654,9 +654,9 @@ public class TestAssignmentManagerOnCluster {
*/
@Test (timeout=60000)
public void testCloseHang() throws Exception {
String table = "testCloseHang";
TableName table = TableName.valueOf("testCloseHang");
try {
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(table));
HTableDescriptor desc = new HTableDescriptor(table);
desc.addFamily(new HColumnDescriptor(FAMILY));
admin.createTable(desc);
@ -690,7 +690,7 @@ public class TestAssignmentManagerOnCluster {
TEST_UTIL.assertRegionOnServer(hri, serverName, 200);
} finally {
MyRegionObserver.postCloseEnabled.set(false);
TEST_UTIL.deleteTable(Bytes.toBytes(table));
TEST_UTIL.deleteTable(table);
}
}
@ -699,9 +699,9 @@ public class TestAssignmentManagerOnCluster {
*/
@Test (timeout=60000)
public void testOpenCloseRacing() throws Exception {
String table = "testOpenCloseRacing";
TableName table = TableName.valueOf("testOpenCloseRacing");
try {
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(table));
HTableDescriptor desc = new HTableDescriptor(table);
desc.addFamily(new HColumnDescriptor(FAMILY));
admin.createTable(desc);
@ -761,7 +761,7 @@ public class TestAssignmentManagerOnCluster {
TEST_UTIL.assertRegionOnlyOnServer(hri, serverName, 6000);
} finally {
MyRegionObserver.postOpenEnabled.set(false);
TEST_UTIL.deleteTable(Bytes.toBytes(table));
TEST_UTIL.deleteTable(table);
}
}
@ -770,11 +770,11 @@ public class TestAssignmentManagerOnCluster {
*/
@Test (timeout=60000)
public void testAssignRacingWithSSH() throws Exception {
String table = "testAssignRacingWithSSH";
TableName table = TableName.valueOf("testAssignRacingWithSSH");
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
MyMaster master = null;
try {
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(table));
HTableDescriptor desc = new HTableDescriptor(table);
desc.addFamily(new HColumnDescriptor(FAMILY));
admin.createTable(desc);
@ -836,7 +836,7 @@ public class TestAssignmentManagerOnCluster {
if (master != null) {
master.enableSSH(true);
}
TEST_UTIL.deleteTable(Bytes.toBytes(table));
TEST_UTIL.deleteTable(table);
cluster.startRegionServer();
}
}
@ -939,11 +939,11 @@ public class TestAssignmentManagerOnCluster {
*/
@Test (timeout=60000)
public void testAssignOfflinedRegionBySSH() throws Exception {
String table = "testAssignOfflinedRegionBySSH";
TableName table = TableName.valueOf("testAssignOfflinedRegionBySSH");
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
MyMaster master = null;
try {
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(table));
HTableDescriptor desc = new HTableDescriptor(table);
desc.addFamily(new HColumnDescriptor(FAMILY));
admin.createTable(desc);
@ -1008,7 +1008,7 @@ public class TestAssignmentManagerOnCluster {
TEST_UTIL.assertRegionOnlyOnServer(hri, serverName, 200);
} finally {
MyRegionServer.abortedServer = null;
TEST_UTIL.deleteTable(Bytes.toBytes(table));
TEST_UTIL.deleteTable(table);
cluster.startRegionServer();
}
}
@ -1018,11 +1018,11 @@ public class TestAssignmentManagerOnCluster {
*/
@Test (timeout=60000)
public void testAssignDisabledRegionBySSH() throws Exception {
String table = "testAssignDisabledRegionBySSH";
TableName table = TableName.valueOf("testAssignDisabledRegionBySSH");
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
MyMaster master = null;
MyMaster master;
try {
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(table));
HTableDescriptor desc = new HTableDescriptor(table);
desc.addFamily(new HColumnDescriptor(FAMILY));
admin.createTable(desc);
@ -1085,7 +1085,7 @@ public class TestAssignmentManagerOnCluster {
assertTrue(regionStates.isRegionOffline(hri));
} finally {
MyRegionServer.abortedServer = null;
TEST_UTIL.deleteTable(Bytes.toBytes(table));
TEST_UTIL.deleteTable(table);
cluster.startRegionServer();
}
}
@ -1095,10 +1095,10 @@ public class TestAssignmentManagerOnCluster {
*/
@Test(timeout = 60000)
public void testReportRegionStateTransition() throws Exception {
String table = "testReportRegionStateTransition";
TableName table = TableName.valueOf("testReportRegionStateTransition");
try {
MyRegionServer.simulateRetry = true;
HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(table));
HTableDescriptor desc = new HTableDescriptor(table);
desc.addFamily(new HColumnDescriptor(FAMILY));
admin.createTable(desc);
Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
@ -1114,13 +1114,13 @@ public class TestAssignmentManagerOnCluster {
// Assert the the region is actually open on the server
TEST_UTIL.assertRegionOnServer(hri, serverName, 200);
// Closing region should just work fine
admin.disableTable(TableName.valueOf(table));
admin.disableTable(table);
assertTrue(regionStates.isRegionOffline(hri));
List<HRegionInfo> regions = TEST_UTIL.getHBaseAdmin().getOnlineRegions(serverName);
assertTrue(!regions.contains(hri));
} finally {
MyRegionServer.simulateRetry = false;
TEST_UTIL.deleteTable(Bytes.toBytes(table));
TEST_UTIL.deleteTable(table);
}
}

View File

@ -343,10 +343,11 @@ public class TestDistributedLogSplitting {
master.balanceSwitch(false);
final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
HTable ht = installTable(zkw, TABLE_NAME, FAMILY_NAME, NUM_REGIONS_TO_CREATE);
Table ht = installTable(zkw, TABLE_NAME, FAMILY_NAME, NUM_REGIONS_TO_CREATE);
NonceGeneratorWithDups ng = new NonceGeneratorWithDups();
NonceGenerator oldNg =
ConnectionUtils.injectNonceGeneratorForTesting((ClusterConnection)ht.getConnection(), ng);
ConnectionUtils.injectNonceGeneratorForTesting(
(ClusterConnection)TEST_UTIL.getConnection(), ng);
try {
List<Increment> reqs = new ArrayList<Increment>();
@ -380,7 +381,8 @@ public class TestDistributedLogSplitting {
}
}
} finally {
ConnectionUtils.injectNonceGeneratorForTesting((ClusterConnection) ht.getConnection(), oldNg);
ConnectionUtils.injectNonceGeneratorForTesting((ClusterConnection)
TEST_UTIL.getConnection(), oldNg);
ht.close();
zkw.close();
}
@ -711,7 +713,7 @@ public class TestDistributedLogSplitting {
List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
List<HRegionInfo> regions = null;
HRegionServer hrs = null;
@ -901,7 +903,7 @@ public class TestDistributedLogSplitting {
List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
final SplitLogManager slm = master.getMasterFileSystem().splitLogManager;
Set<HRegionInfo> regionSet = new HashSet<HRegionInfo>();
@ -1399,7 +1401,7 @@ public class TestDistributedLogSplitting {
LOG.info("testReadWriteSeqIdFiles");
startCluster(2);
final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
HTable ht = installTable(zkw, "table", "family", 10);
Table ht = installTable(zkw, "table", "family", 10);
FileSystem fs = master.getMasterFileSystem().getFileSystem();
Path tableDir = FSUtils.getTableDir(FSUtils.getRootDir(conf), TableName.valueOf("table"));
List<Path> regionDirs = FSUtils.getRegionDirs(fs, tableDir);
@ -1425,19 +1427,19 @@ public class TestDistributedLogSplitting {
ht.close();
}
HTable installTable(ZooKeeperWatcher zkw, String tname, String fname, int nrs) throws Exception {
Table installTable(ZooKeeperWatcher zkw, String tname, String fname, int nrs) throws Exception {
return installTable(zkw, tname, fname, nrs, 0);
}
HTable installTable(ZooKeeperWatcher zkw, String tname, String fname, int nrs,
Table installTable(ZooKeeperWatcher zkw, String tname, String fname, int nrs,
int existingRegions) throws Exception {
// Create a table with regions
TableName table = TableName.valueOf(tname);
byte [] family = Bytes.toBytes(fname);
LOG.info("Creating table with " + nrs + " regions");
HTable ht = TEST_UTIL.createMultiRegionTable(table, family, nrs);
Table ht = TEST_UTIL.createMultiRegionTable(table, family, nrs);
int numRegions = -1;
try (RegionLocator r = ht.getRegionLocator()) {
try (RegionLocator r = TEST_UTIL.getConnection().getRegionLocator(table)) {
numRegions = r.getStartKeys().length;
}
assertEquals(nrs, numRegions);

View File

@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region;
@ -72,9 +73,8 @@ public class TestGetLastFlushedSequenceId {
public void test() throws IOException, InterruptedException {
testUtil.getHBaseAdmin().createNamespace(
NamespaceDescriptor.create(tableName.getNamespaceAsString()).build());
HTable table = testUtil.createTable(tableName, families);
Table table = testUtil.createTable(tableName, families);
table.put(new Put(Bytes.toBytes("k")).add(family, Bytes.toBytes("q"), Bytes.toBytes("v")));
table.flushCommits();
MiniHBaseCluster cluster = testUtil.getMiniHBaseCluster();
List<JVMClusterUtil.RegionServerThread> rsts = cluster.getRegionServerThreads();
Region region = null;

View File

@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
@ -83,7 +84,7 @@ public class TestMaster {
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
HMaster m = cluster.getMaster();
try (HTable ht = TEST_UTIL.createTable(TABLENAME, FAMILYNAME)) {
try (Table ht = TEST_UTIL.createTable(TABLENAME, FAMILYNAME)) {
assertTrue(m.assignmentManager.getTableStateManager().isTableState(TABLENAME,
TableState.State.ENABLED));
TEST_UTIL.loadTable(ht, FAMILYNAME, false);

View File

@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MasterTests;
@ -67,9 +68,9 @@ public class TestMasterRestartAfterDisablingTable {
TableName table = TableName.valueOf("tableRestart");
byte[] family = Bytes.toBytes("family");
log("Creating table with " + NUM_REGIONS_TO_CREATE + " regions");
HTable ht = TEST_UTIL.createMultiRegionTable(table, family, NUM_REGIONS_TO_CREATE);
Table ht = TEST_UTIL.createMultiRegionTable(table, family, NUM_REGIONS_TO_CREATE);
int numRegions = -1;
try (RegionLocator r = ht.getRegionLocator()) {
try (RegionLocator r = TEST_UTIL.getConnection().getRegionLocator(table)) {
numRegions = r.getStartKeys().length;
}
numRegions += 1; // catalogs

View File

@ -66,9 +66,9 @@ public class TestMasterTransitions {
TEST_UTIL.startMiniCluster(2);
// Create a table of three families. This will assign a region.
TEST_UTIL.createMultiRegionTable(TABLENAME, FAMILIES);
HTable t = (HTable) TEST_UTIL.getConnection().getTable(TABLENAME);
Table t = TEST_UTIL.getConnection().getTable(TABLENAME);
int countOfRegions = -1;
try (RegionLocator r = t.getRegionLocator()) {
try (RegionLocator r = TEST_UTIL.getConnection().getRegionLocator(TABLENAME)) {
countOfRegions = r.getStartKeys().length;
}
TEST_UTIL.waitUntilAllRegionsAssigned(TABLENAME);

View File

@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
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.MiniHBaseCluster;
@ -50,7 +51,9 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.master.balancer.FavoredNodeAssignmentHelper;
import org.apache.hadoop.hbase.master.balancer.FavoredNodeLoadBalancer;
import org.apache.hadoop.hbase.master.balancer.FavoredNodesPlan;
@ -546,11 +549,10 @@ public class TestRegionPlacement {
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
admin.createTable(desc, splitKeys);
HTable ht = (HTable) CONNECTION.getTable(tableName);
@SuppressWarnings("deprecation")
Map<HRegionInfo, ServerName> regions = ht.getRegionLocations();
assertEquals("Tried to create " + expectedRegions + " regions "
+ "but only found " + regions.size(), expectedRegions, regions.size());
ht.close();
try (RegionLocator r = CONNECTION.getRegionLocator(tableName)) {
List<HRegionLocation> regions = r.getAllRegionLocations();
assertEquals("Tried to create " + expectedRegions + " regions "
+ "but only found " + regions.size(), expectedRegions, regions.size());
}
}
}

View File

@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MasterTests;
@ -78,9 +79,9 @@ public class TestRollingRestart {
TableName table = TableName.valueOf("tableRestart");
byte [] family = Bytes.toBytes("family");
log("Creating table with " + NUM_REGIONS_TO_CREATE + " regions");
HTable ht = TEST_UTIL.createMultiRegionTable(table, family, NUM_REGIONS_TO_CREATE);
Table ht = TEST_UTIL.createMultiRegionTable(table, family, NUM_REGIONS_TO_CREATE);
int numRegions = -1;
try (RegionLocator r = ht.getRegionLocator()) {
try (RegionLocator r = TEST_UTIL.getConnection().getRegionLocator(table)) {
numRegions = r.getStartKeys().length;
}
numRegions += 1; // catalogs

View File

@ -288,12 +288,12 @@ public class TestSnapshotFromMaster {
UTIL.deleteTable(TABLE_NAME);
HTableDescriptor htd = new HTableDescriptor(TABLE_NAME);
htd.setCompactionEnabled(false);
UTIL.createTable(htd, new byte[][] { TEST_FAM }, UTIL.getConfiguration());
UTIL.createTable(htd, new byte[][] { TEST_FAM }, null);
// load the table (creates 4 hfiles)
UTIL.loadTable(UTIL.getConnection().getTable(TABLE_NAME), TEST_FAM);
UTIL.flush(TABLE_NAME);
// Put some more data into the table so for sure we get more storefiles.
UTIL.loadTable((HTable) UTIL.getConnection().getTable(TABLE_NAME), TEST_FAM);
UTIL.loadTable(UTIL.getConnection().getTable(TABLE_NAME), TEST_FAM);
// disable the table so we can take a snapshot
admin.disableTable(TABLE_NAME);

View File

@ -446,7 +446,7 @@ public class TestNamespaceAuditor {
// This call will pass.
ADMIN.createTable(tableDescOne);
Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
HTable htable = (HTable)connection.getTable(tableOne);
Table htable = connection.getTable(tableOne);
UTIL.loadNumericRows(htable, Bytes.toBytes("info"), 1, 1000);
ADMIN.flush(tableOne);
stateInfo = getNamespaceState(nsp1);

View File

@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@ -66,7 +67,7 @@ public class TestQuotaThrottle {
};
private static ManualEnvironmentEdge envEdge;
private static HTable[] tables;
private static Table[] tables;
@BeforeClass
public static void setUpBeforeClass() throws Exception {
@ -81,7 +82,7 @@ public class TestQuotaThrottle {
TEST_UTIL.waitTableAvailable(QuotaTableUtil.QUOTA_TABLE_NAME);
QuotaCache.TEST_FORCE_REFRESH = true;
tables = new HTable[TABLE_NAMES.length];
tables = new Table[TABLE_NAMES.length];
for (int i = 0; i < TABLE_NAMES.length; ++i) {
tables[i] = TEST_UTIL.createTable(TABLE_NAMES[i], FAMILY);
}
@ -505,13 +506,13 @@ public class TestQuotaThrottle {
assertEquals(30, doGets(30, tables[1]));
}
private int doPuts(int maxOps, final HTable... tables) throws Exception {
private int doPuts(int maxOps, final Table... tables) throws Exception {
int count = 0;
try {
while (count < maxOps) {
Put put = new Put(Bytes.toBytes("row-" + count));
put.add(FAMILY, QUALIFIER, Bytes.toBytes("data-" + count));
for (final HTable table: tables) {
for (final Table table: tables) {
table.put(put);
}
count += tables.length;
@ -527,12 +528,12 @@ public class TestQuotaThrottle {
return count;
}
private long doGets(int maxOps, final HTable... tables) throws Exception {
private long doGets(int maxOps, final Table... tables) throws Exception {
int count = 0;
try {
while (count < maxOps) {
Get get = new Get(Bytes.toBytes("row-" + count));
for (final HTable table: tables) {
for (final Table table: tables) {
table.get(get);
}
count += tables.length;

View File

@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
@ -94,7 +95,7 @@ public class TestEndToEndSplitTransaction {
public void testMasterOpsWhileSplitting() throws Exception {
TableName tableName = TableName.valueOf("TestSplit");
byte[] familyName = Bytes.toBytes("fam");
try (HTable ht = TEST_UTIL.createTable(tableName, familyName)) {
try (Table ht = TEST_UTIL.createTable(tableName, familyName)) {
TEST_UTIL.loadTable(ht, familyName, false);
}
HRegionServer server = TEST_UTIL.getHBaseCluster().getRegionServer(0);
@ -318,19 +319,23 @@ public class TestEndToEndSplitTransaction {
/** verify region boundaries obtained from HTable.getStartEndKeys() */
void verifyRegionsUsingHTable() throws IOException {
HTable table = null;
Table table = null;
try {
//HTable.getStartEndKeys()
table = (HTable) connection.getTable(tableName);
Pair<byte[][], byte[][]> keys = table.getRegionLocator().getStartEndKeys();
verifyStartEndKeys(keys);
table = connection.getTable(tableName);
//HTable.getRegionsInfo()
Set<HRegionInfo> regions = new TreeSet<HRegionInfo>();
for (HRegionLocation loc : table.getRegionLocator().getAllRegionLocations()) {
regions.add(loc.getRegionInfo());
try(RegionLocator rl = connection.getRegionLocator(tableName)) {
Pair<byte[][], byte[][]> keys = rl.getStartEndKeys();
verifyStartEndKeys(keys);
//HTable.getRegionsInfo()
Set<HRegionInfo> regions = new TreeSet<HRegionInfo>();
for (HRegionLocation loc : rl.getAllRegionLocations()) {
regions.add(loc.getRegionInfo());
}
verifyTableRegions(regions);
}
verifyTableRegions(regions);
} finally {
IOUtils.closeQuietly(table);
}

View File

@ -196,7 +196,7 @@ public class TestHRegion {
private final int MAX_VERSIONS = 2;
// Test names
protected byte[] tableName;
protected TableName tableName;
protected String method;
protected final byte[] qual1 = Bytes.toBytes("qual1");
protected final byte[] qual2 = Bytes.toBytes("qual2");
@ -216,7 +216,7 @@ public class TestHRegion {
CONF = TEST_UTIL.getConfiguration();
dir = TEST_UTIL.getDataTestDir("TestHRegion").toString();
method = name.getMethodName();
tableName = Bytes.toBytes(name.getMethodName());
tableName = TableName.valueOf(name.getMethodName());
}
@After
@ -351,13 +351,13 @@ public class TestHRegion {
* @param callingMethod a unique component for the path, probably the name of the test method.
*/
private static WAL createWALCompatibleWithFaultyFileSystem(String callingMethod,
Configuration conf, byte[] tableName) throws IOException {
Configuration conf, TableName tableName) throws IOException {
final Path logDir = TEST_UTIL.getDataTestDirOnTestFS(callingMethod + ".log");
final Configuration walConf = new Configuration(conf);
FSUtils.setRootDir(walConf, logDir);
return (new WALFactory(walConf,
Collections.<WALActionsListener>singletonList(new MetricsWAL()), callingMethod))
.getWAL(tableName);
.getWAL(tableName.toBytes());
}
/**
@ -888,7 +888,7 @@ public class TestHRegion {
final WALFactory wals = new WALFactory(walConf, null, method);
final WAL wal = wals.getWAL(tableName.getName());
this.region = initHRegion(tableName.getName(), HConstants.EMPTY_START_ROW,
this.region = initHRegion(tableName, HConstants.EMPTY_START_ROW,
HConstants.EMPTY_END_ROW, method, CONF, false, Durability.USE_DEFAULT, wal, family);
try {
Path regiondir = region.getRegionFileSystem().getRegionDir();
@ -1046,7 +1046,7 @@ public class TestHRegion {
final WALFactory wals = new WALFactory(walConf, null, method);
WAL wal = spy(wals.getWAL(tableName.getName()));
this.region = initHRegion(tableName.getName(), HConstants.EMPTY_START_ROW,
this.region = initHRegion(tableName, HConstants.EMPTY_START_ROW,
HConstants.EMPTY_END_ROW, method, CONF, false, Durability.USE_DEFAULT, wal, family);
try {
int i = 0;
@ -1089,7 +1089,7 @@ public class TestHRegion {
}
region.close();
this.region = initHRegion(tableName.getName(), HConstants.EMPTY_START_ROW,
this.region = initHRegion(tableName, HConstants.EMPTY_START_ROW,
HConstants.EMPTY_END_ROW, method, CONF, false, Durability.USE_DEFAULT, wal, family);
region.put(put);
@ -1207,7 +1207,7 @@ public class TestHRegion {
*/
@Test
public void testWeirdCacheBehaviour() throws Exception {
byte[] TABLE = Bytes.toBytes("testWeirdCacheBehaviour");
TableName TABLE = TableName.valueOf("testWeirdCacheBehaviour");
byte[][] FAMILIES = new byte[][] { Bytes.toBytes("trans-blob"), Bytes.toBytes("trans-type"),
Bytes.toBytes("trans-date"), Bytes.toBytes("trans-tags"), Bytes.toBytes("trans-group") };
this.region = initHRegion(TABLE, getName(), CONF, FAMILIES);
@ -1250,7 +1250,7 @@ public class TestHRegion {
@Test
public void testAppendWithReadOnlyTable() throws Exception {
byte[] TABLE = Bytes.toBytes("readOnlyTable");
TableName TABLE = TableName.valueOf("readOnlyTable");
this.region = initHRegion(TABLE, getName(), CONF, true, Bytes.toBytes("somefamily"));
boolean exceptionCaught = false;
Append append = new Append(Bytes.toBytes("somerow"));
@ -1270,7 +1270,7 @@ public class TestHRegion {
@Test
public void testIncrWithReadOnlyTable() throws Exception {
byte[] TABLE = Bytes.toBytes("readOnlyTable");
TableName TABLE = TableName.valueOf("readOnlyTable");
this.region = initHRegion(TABLE, getName(), CONF, true, Bytes.toBytes("somefamily"));
boolean exceptionCaught = false;
Increment inc = new Increment(Bytes.toBytes("somerow"));
@ -1362,11 +1362,11 @@ public class TestHRegion {
@Test
public void testFamilyWithAndWithoutColon() throws Exception {
byte[] b = Bytes.toBytes(getName());
TableName b = TableName.valueOf(getName());
byte[] cf = Bytes.toBytes(COLUMN_FAMILY);
this.region = initHRegion(b, getName(), CONF, cf);
try {
Put p = new Put(b);
Put p = new Put(b.toBytes());
byte[] cfwithcolon = Bytes.toBytes(COLUMN_FAMILY + ":");
p.add(cfwithcolon, cfwithcolon, cfwithcolon);
boolean exception = false;
@ -1387,7 +1387,7 @@ public class TestHRegion {
byte[] cf = Bytes.toBytes(COLUMN_FAMILY);
byte[] qual = Bytes.toBytes("qual");
byte[] val = Bytes.toBytes("val");
this.region = initHRegion(Bytes.toBytes(getName()), getName(), CONF, cf);
this.region = initHRegion(TableName.valueOf(getName()), getName(), CONF, cf);
MetricsWALSource source = CompatibilitySingletonFactory.getInstance(MetricsWALSource.class);
try {
long syncs = metricsAssertHelper.getCounter("syncTimeNumOps", source);
@ -1428,7 +1428,7 @@ public class TestHRegion {
byte[] cf = Bytes.toBytes(COLUMN_FAMILY);
byte[] qual = Bytes.toBytes("qual");
byte[] val = Bytes.toBytes("val");
this.region = initHRegion(Bytes.toBytes(getName()), getName(), CONF, cf);
this.region = initHRegion(TableName.valueOf(getName()), getName(), CONF, cf);
MetricsWALSource source = CompatibilitySingletonFactory.getInstance(MetricsWALSource.class);
try {
long syncs = metricsAssertHelper.getCounter("syncTimeNumOps", source);
@ -1526,7 +1526,7 @@ public class TestHRegion {
@Test
public void testBatchPutWithTsSlop() throws Exception {
byte[] b = Bytes.toBytes(getName());
TableName b = TableName.valueOf(getName());
byte[] cf = Bytes.toBytes(COLUMN_FAMILY);
byte[] qual = Bytes.toBytes("qual");
byte[] val = Bytes.toBytes("val");
@ -4049,7 +4049,7 @@ public class TestHRegion {
HColumnDescriptor hcd = new HColumnDescriptor(fam1).setMaxVersions(Integer.MAX_VALUE)
.setBloomFilterType(BloomType.ROWCOL);
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
HTableDescriptor htd = new HTableDescriptor(tableName);
htd.addFamily(hcd);
HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
this.region = TEST_UTIL.createLocalHRegion(info, htd);
@ -4154,7 +4154,7 @@ public class TestHRegion {
HColumnDescriptor hcd = new HColumnDescriptor(familyName).setMaxVersions(Integer.MAX_VALUE)
.setBloomFilterType(BloomType.ROWCOL);
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
HTableDescriptor htd = new HTableDescriptor(tableName);
htd.addFamily(hcd);
HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
this.region = TEST_UTIL.createLocalHRegion(info, htd);
@ -4204,7 +4204,7 @@ public class TestHRegion {
try {
cluster = htu.startMiniCluster(1, regionServersCount, dataNodeHosts);
byte[][] families = { fam1, fam2 };
Table ht = htu.createTable(Bytes.toBytes(this.getName()), families);
Table ht = htu.createTable(TableName.valueOf(this.getName()), families);
// Setting up region
byte row[] = Bytes.toBytes("row1");
@ -4646,7 +4646,7 @@ public class TestHRegion {
FSUtils.setRootDir(walConf, logDir);
final WALFactory wals = new WALFactory(walConf, null, UUID.randomUUID().toString());
final WAL wal = spy(wals.getWAL(tableName.getName()));
this.region = initHRegion(tableName.getName(), HConstants.EMPTY_START_ROW,
this.region = initHRegion(tableName, HConstants.EMPTY_START_ROW,
HConstants.EMPTY_END_ROW, method, conf, false, tableDurability, wal,
new byte[][] { family });
@ -4949,7 +4949,7 @@ public class TestHRegion {
@Test
public void testFlushResult() throws IOException {
String method = name.getMethodName();
byte[] tableName = Bytes.toBytes(method);
TableName tableName = TableName.valueOf(method);
byte[] family = Bytes.toBytes("family");
this.region = initHRegion(tableName, method, family);
@ -4961,7 +4961,7 @@ public class TestHRegion {
// Flush enough files to get up to the threshold, doesn't need compactions
for (int i = 0; i < 2; i++) {
Put put = new Put(tableName).add(family, family, tableName);
Put put = new Put(tableName.toBytes()).add(family, family, tableName.toBytes());
region.put(put);
fr = region.flush(true);
assertTrue(fr.isFlushSucceeded());
@ -4970,7 +4970,7 @@ public class TestHRegion {
// Two flushes after the threshold, compactions are needed
for (int i = 0; i < 2; i++) {
Put put = new Put(tableName).add(family, family, tableName);
Put put = new Put(tableName.toBytes()).add(family, family, tableName.toBytes());
region.put(put);
fr = region.flush(true);
assertTrue(fr.isFlushSucceeded());
@ -5007,20 +5007,6 @@ public class TestHRegion {
*/
private static HRegion initHRegion(TableName tableName, String callingMethod, Configuration conf,
byte[]... families) throws IOException {
return initHRegion(tableName.getName(), null, null, callingMethod, conf, false, families);
}
/**
* @param tableName
* @param callingMethod
* @param conf
* @param families
* @throws IOException
* @return A region on which you must call
* {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} when done.
*/
private static HRegion initHRegion(byte[] tableName, String callingMethod, Configuration conf,
byte[]... families) throws IOException {
return initHRegion(tableName, null, null, callingMethod, conf, false, families);
}
@ -5034,16 +5020,16 @@ public class TestHRegion {
* @return A region on which you must call
* {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} when done.
*/
private static HRegion initHRegion(byte[] tableName, String callingMethod, Configuration conf,
private static HRegion initHRegion(TableName tableName, String callingMethod, Configuration conf,
boolean isReadOnly, byte[]... families) throws IOException {
return initHRegion(tableName, null, null, callingMethod, conf, isReadOnly, families);
}
public static HRegion initHRegion(byte[] tableName, byte[] startKey, byte[] stopKey,
public static HRegion initHRegion(TableName tableName, byte[] startKey, byte[] stopKey,
String callingMethod, Configuration conf, boolean isReadOnly, byte[]... families)
throws IOException {
Path logDir = TEST_UTIL.getDataTestDirOnTestFS(callingMethod + ".log");
HRegionInfo hri = new HRegionInfo(TableName.valueOf(tableName), startKey, stopKey);
HRegionInfo hri = new HRegionInfo(tableName, startKey, stopKey);
final WAL wal = HBaseTestingUtility.createWal(conf, logDir, hri);
return initHRegion(tableName, startKey, stopKey, callingMethod, conf, isReadOnly,
Durability.SYNC_WAL, wal, families);
@ -5061,10 +5047,10 @@ public class TestHRegion {
* @return A region on which you must call
* {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} when done.
*/
public static HRegion initHRegion(byte[] tableName, byte[] startKey, byte[] stopKey,
public static HRegion initHRegion(TableName tableName, byte[] startKey, byte[] stopKey,
String callingMethod, Configuration conf, boolean isReadOnly, Durability durability,
WAL wal, byte[]... families) throws IOException {
return TEST_UTIL.createLocalHRegion(tableName, startKey, stopKey, callingMethod, conf,
return TEST_UTIL.createLocalHRegion(tableName, startKey, stopKey,
isReadOnly, durability, wal, families);
}
@ -5647,7 +5633,7 @@ public class TestHRegion {
@Test (timeout=60000)
public void testSplitRegionWithReverseScan() throws IOException {
byte [] tableName = Bytes.toBytes("testSplitRegionWithReverseScan");
TableName tableName = TableName.valueOf("testSplitRegionWithReverseScan");
byte [] qualifier = Bytes.toBytes("qualifier");
Configuration hc = initSplit();
int numRows = 3;
@ -6024,7 +6010,7 @@ public class TestHRegion {
@Test (timeout=24000)
public void testRegionTooBusy() throws IOException {
String method = "testRegionTooBusy";
byte[] tableName = Bytes.toBytes(method);
TableName tableName = TableName.valueOf(method);
byte[] family = Bytes.toBytes("family");
long defaultBusyWaitDuration = CONF.getLong("hbase.busy.wait.duration",
HRegion.DEFAULT_BUSY_WAIT_DURATION);
@ -6198,7 +6184,7 @@ public class TestHRegion {
}
}
static HRegion initHRegion(byte[] tableName, String callingMethod,
static HRegion initHRegion(TableName tableName, String callingMethod,
byte[]... families) throws IOException {
return initHRegion(tableName, callingMethod, HBaseConfiguration.create(),
families);

View File

@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
@ -76,12 +77,17 @@ public class TestHRegionOnCluster {
// Put data: r1->v1
Log.info("Loading r1 to v1 into " + TABLENAME);
HTable table = (HTable) TEST_UTIL.getConnection().getTable(TABLENAME);
Table table = TEST_UTIL.getConnection().getTable(TABLENAME);
putDataAndVerify(table, "r1", FAMILY, "v1", 1);
TEST_UTIL.waitUntilAllRegionsAssigned(table.getName());
// Move region to target server
HRegionInfo regionInfo = table.getRegionLocation("r1").getRegionInfo();
HRegionInfo regionInfo;
try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(TABLENAME)) {
regionInfo = locator.getRegionLocation(Bytes.toBytes("r1")).getRegionInfo();
}
int originServerNum = cluster.getServerWith(regionInfo.getRegionName());
HRegionServer originServer = cluster.getRegionServer(originServerNum);
int targetServerNum = (originServerNum + 1) % NUM_RS;

View File

@ -110,7 +110,7 @@ public class TestPerColumnFamilyFlush {
}
// A helper function to verify edits.
void verifyEdit(int familyNum, int putNum, HTable table) throws IOException {
void verifyEdit(int familyNum, int putNum, Table table) throws IOException {
Result r = table.get(createGet(familyNum, putNum));
byte[] family = FAMILIES[familyNum - 1];
byte[] qf = Bytes.toBytes("q" + familyNum);
@ -342,7 +342,7 @@ public class TestPerColumnFamilyFlush {
TEST_UTIL.startMiniCluster(numRegionServers);
TEST_UTIL.getHBaseAdmin().createNamespace(
NamespaceDescriptor.create(TABLENAME.getNamespaceAsString()).build());
HTable table = TEST_UTIL.createTable(TABLENAME, FAMILIES);
Table table = TEST_UTIL.createTable(TABLENAME, FAMILIES);
HTableDescriptor htd = table.getTableDescriptor();
for (byte[] family : FAMILIES) {
@ -360,7 +360,6 @@ public class TestPerColumnFamilyFlush {
table.put(createPut(3, i));
}
}
table.flushCommits();
Thread.sleep(1000);
Pair<Region, HRegionServer> desiredRegionAndServer = getRegionWithName(TABLENAME);
@ -466,8 +465,7 @@ public class TestPerColumnFamilyFlush {
final int numRegionServers = 1;
TEST_UTIL.startMiniCluster(numRegionServers);
try {
HTable table = null;
table = TEST_UTIL.createTable(tableName, FAMILIES);
Table table = TEST_UTIL.createTable(tableName, FAMILIES);
// Force flush the namespace table so edits to it are not hanging around as oldest
// edits. Otherwise, below, when we make maximum number of WAL files, then it will be
// the namespace region that is flushed and not the below 'desiredRegion'.
@ -489,7 +487,6 @@ public class TestPerColumnFamilyFlush {
for (int j = 0; j < 100; j++) {
table.put(createPut(1, i * 100 + j));
}
table.flushCommits();
// Roll the WAL. The log file count is less than maxLogs so no flush is triggered.
int currentNumRolledLogFiles = getNumRolledLogFiles(desiredRegion);
assertNull(getWAL(desiredRegion).rollWriter());
@ -503,7 +500,6 @@ public class TestPerColumnFamilyFlush {
assertTrue(desiredRegion.getStore(FAMILY2).getMemStoreSize() < cfFlushSizeLowerBound);
assertTrue(desiredRegion.getStore(FAMILY3).getMemStoreSize() < cfFlushSizeLowerBound);
table.put(createPut(1, 12345678));
table.flushCommits();
// Make numRolledLogFiles greater than maxLogs
desiredRegionAndServer.getSecond().walRoller.requestRollAll();
// Wait for some time till the flush caused by log rolling happens.

View File

@ -34,6 +34,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.util.Bytes;
@ -53,7 +54,7 @@ import org.junit.experimental.categories.Category;
public class TestRegionFavoredNodes {
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private static HTable table;
private static Table table;
private static final TableName TABLE_NAME =
TableName.valueOf("table");
private static final byte[] COLUMN_FAMILY = Bytes.toBytes("family");

View File

@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.TableName;
@ -68,7 +69,7 @@ public class TestRegionReplicas {
private static final Log LOG = LogFactory.getLog(TestRegionReplicas.class);
private static final int NB_SERVERS = 1;
private static HTable table;
private static Table table;
private static final byte[] row = "TestRegionReplicas".getBytes();
private static HRegionInfo hriPrimary;
@ -91,7 +92,9 @@ public class TestRegionReplicas {
// Create table then get the single region for our new table.
table = HTU.createTable(tableName, f);
hriPrimary = table.getRegionLocation(row, false).getRegionInfo();
try (RegionLocator locator = HTU.getConnection().getRegionLocator(tableName)) {
hriPrimary = locator.getRegionLocation(row, false).getRegionInfo();
}
// mock a secondary region info to open
hriSecondary = new HRegionInfo(hriPrimary.getTable(), hriPrimary.getStartKey(),

View File

@ -337,7 +337,7 @@ public class TestRegionServerMetrics {
p.add(cf, qualifier, val);
puts.add(p);
}
try (HTable t = TEST_UTIL.createTable(tableName, cf)) {
try (Table t = TEST_UTIL.createTable(tableName, cf)) {
t.put(puts);
Scan s = new Scan();
@ -387,7 +387,7 @@ public class TestRegionServerMetrics {
p.add(cf, qualifier, val);
puts.add(p);
}
try (HTable t = TEST_UTIL.createTable(tableName, cf)) {
try (Table t = TEST_UTIL.createTable(tableName, cf)) {
t.put(puts);
Scan s = new Scan();

View File

@ -30,6 +30,8 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.HTable;
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.master.HMaster;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.RequestConverter;
@ -58,7 +60,7 @@ import com.google.protobuf.ServiceException;
public class TestRegionServerNoMaster {
private static final int NB_SERVERS = 1;
private static HTable table;
private static Table table;
private static final byte[] row = "ee".getBytes();
private static HRegionInfo hri;
@ -78,7 +80,9 @@ public class TestRegionServerNoMaster {
p.add(HConstants.CATALOG_FAMILY, row, row);
table.put(p);
hri = table.getRegionLocation(row, false).getRegionInfo();
try (RegionLocator locator = HTU.getConnection().getRegionLocator(tableName)) {
hri = locator.getRegionLocation(row, false).getRegionInfo();
}
regionName = hri.getRegionName();
stopMasterAndAssignMeta(HTU);

View File

@ -28,7 +28,10 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
@ -52,14 +55,14 @@ public class TestRegionServerOnlineConfigChange {
private static HBaseTestingUtility hbaseTestingUtility = new HBaseTestingUtility();
private static Configuration conf = null;
private static HTable t1 = null;
private static Table t1 = null;
private static HRegionServer rs1 = null;
private static byte[] r1name = null;
private static Region r1 = null;
private final static String table1Str = "table1";
private final static String columnFamily1Str = "columnFamily1";
private final static byte[] TABLE1 = Bytes.toBytes(table1Str);
private final static TableName TABLE1 = TableName.valueOf(table1Str);
private final static byte[] COLUMN_FAMILY1 = Bytes.toBytes(columnFamily1Str);
@ -68,12 +71,13 @@ public class TestRegionServerOnlineConfigChange {
conf = hbaseTestingUtility.getConfiguration();
hbaseTestingUtility.startMiniCluster(1,1);
t1 = hbaseTestingUtility.createTable(TABLE1, COLUMN_FAMILY1);
@SuppressWarnings("deprecation")
HRegionInfo firstHRI = t1.getRegionLocations().keySet().iterator().next();
r1name = firstHRI.getRegionName();
rs1 = hbaseTestingUtility.getHBaseCluster().getRegionServer(
hbaseTestingUtility.getHBaseCluster().getServerWith(r1name));
r1 = rs1.getRegion(r1name);
try (RegionLocator locator = hbaseTestingUtility.getConnection().getRegionLocator(TABLE1)) {
HRegionInfo firstHRI = locator.getAllRegionLocations().get(0).getRegionInfo();
r1name = firstHRI.getRegionName();
rs1 = hbaseTestingUtility.getHBaseCluster().getRegionServer(
hbaseTestingUtility.getHBaseCluster().getServerWith(r1name));
r1 = rs1.getRegion(r1name);
}
}
@AfterClass

View File

@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@ -83,7 +84,9 @@ public class TestScannerWithBulkload {
Configuration conf = TEST_UTIL.getConfiguration();
conf.setBoolean("hbase.mapreduce.bulkload.assign.sequenceNumbers", true);
final LoadIncrementalHFiles bulkload = new LoadIncrementalHFiles(conf);
bulkload.doBulkLoad(hfilePath, (HTable) table);
try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
bulkload.doBulkLoad(hfilePath, admin, table, locator);
}
ResultScanner scanner = table.getScanner(scan);
Result result = scanner.next();
result = scanAfterBulkLoad(scanner, result, "version2");
@ -168,8 +171,8 @@ public class TestScannerWithBulkload {
private Table init(HBaseAdmin admin, long l, Scan scan, TableName tableName) throws Exception {
Table table = TEST_UTIL.getConnection().getTable(tableName);
Put put0 = new Put(Bytes.toBytes("row1"));
put0.add(new KeyValue(Bytes.toBytes("row1"), Bytes.toBytes("col"), Bytes.toBytes("q"), l, Bytes
.toBytes("version0")));
put0.add(new KeyValue(Bytes.toBytes("row1"), Bytes.toBytes("col"), Bytes.toBytes("q"), l,
Bytes.toBytes("version0")));
table.put(put0);
admin.flush(tableName);
Put put1 = new Put(Bytes.toBytes("row2"));
@ -195,9 +198,9 @@ public class TestScannerWithBulkload {
@Test
public void testBulkLoadWithParallelScan() throws Exception {
TableName tableName = TableName.valueOf("testBulkLoadWithParallelScan");
final TableName tableName = TableName.valueOf("testBulkLoadWithParallelScan");
final long l = System.currentTimeMillis();
HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
final HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
createTable(admin, tableName);
Scan scan = createScan();
final Table table = init(admin, l, scan, tableName);
@ -217,7 +220,9 @@ public class TestScannerWithBulkload {
put1.add(new KeyValue(Bytes.toBytes("row5"), Bytes.toBytes("col"), Bytes.toBytes("q"), l,
Bytes.toBytes("version0")));
table.put(put1);
bulkload.doBulkLoad(hfilePath, (HTable) table);
try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
bulkload.doBulkLoad(hfilePath, admin, table, locator);
}
latch.countDown();
} catch (TableNotFoundException e) {
} catch (IOException e) {
@ -231,7 +236,6 @@ public class TestScannerWithBulkload {
scanAfterBulkLoad(scanner, result, "version1");
scanner.close();
table.close();
}
@Test
@ -248,7 +252,9 @@ public class TestScannerWithBulkload {
Configuration conf = TEST_UTIL.getConfiguration();
conf.setBoolean("hbase.mapreduce.bulkload.assign.sequenceNumbers", true);
final LoadIncrementalHFiles bulkload = new LoadIncrementalHFiles(conf);
bulkload.doBulkLoad(hfilePath, (HTable) table);
try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
bulkload.doBulkLoad(hfilePath, admin, table, locator);
}
ResultScanner scanner = table.getScanner(scan);
Result result = scanner.next();
// We had 'version0', 'version1' for 'row1,col:q' in the table.

View File

@ -156,18 +156,18 @@ public class TestServerCustomProtocol {
@Before
public void before() throws Exception {
final byte[][] SPLIT_KEYS = new byte[][] { ROW_B, ROW_C };
HTable table = util.createTable(TEST_TABLE, TEST_FAMILY, SPLIT_KEYS);
Table table = util.createTable(TEST_TABLE, TEST_FAMILY, SPLIT_KEYS);
Put puta = new Put( ROW_A );
puta.add(TEST_FAMILY, Bytes.toBytes("col1"), Bytes.toBytes(1));
puta.addColumn(TEST_FAMILY, Bytes.toBytes("col1"), Bytes.toBytes(1));
table.put(puta);
Put putb = new Put( ROW_B );
putb.add(TEST_FAMILY, Bytes.toBytes("col1"), Bytes.toBytes(1));
putb.addColumn(TEST_FAMILY, Bytes.toBytes("col1"), Bytes.toBytes(1));
table.put(putb);
Put putc = new Put( ROW_C );
putc.add(TEST_FAMILY, Bytes.toBytes("col1"), Bytes.toBytes(1));
putc.addColumn(TEST_FAMILY, Bytes.toBytes("col1"), Bytes.toBytes(1));
table.put(putc);
}
@ -306,8 +306,8 @@ public class TestServerCustomProtocol {
@Test
public void testSingleMethod() throws Throwable {
try (HTable table = (HTable) util.getConnection().getTable(TEST_TABLE)) {
RegionLocator locator = table.getRegionLocator();
try (Table table = util.getConnection().getTable(TEST_TABLE);
RegionLocator locator = util.getConnection().getRegionLocator(TEST_TABLE)) {
Map<byte [], String> results = table.coprocessorService(PingProtos.PingService.class,
null, ROW_A,
new Batch.Call<PingProtos.PingService, String>() {
@ -335,10 +335,11 @@ public class TestServerCustomProtocol {
@Test
public void testRowRange() throws Throwable {
try (HTable table = (HTable) util.getConnection().getTable(TEST_TABLE)) {
RegionLocator locator = table.getRegionLocator();
for (Entry<HRegionInfo, ServerName> e: table.getRegionLocations().entrySet()) {
LOG.info("Region " + e.getKey().getRegionNameAsString() + ", servername=" + e.getValue());
try (Table table = util.getConnection().getTable(TEST_TABLE);
RegionLocator locator = util.getConnection().getRegionLocator(TEST_TABLE)) {
for (HRegionLocation e: locator.getAllRegionLocations()) {
LOG.info("Region " + e.getRegionInfo().getRegionNameAsString()
+ ", servername=" + e.getServerName());
}
// Here are what regions looked like on a run:
//
@ -355,7 +356,7 @@ public class TestServerCustomProtocol {
results = ping(table, ROW_BC, null);
assertEquals(2, results.size());
// should contain last 2 regions
HRegionLocation loc = table.getRegionLocation(ROW_A, true);
HRegionLocation loc = locator.getRegionLocation(ROW_A, true);
assertNull("Should be missing region for row aaa (prior to start row)",
results.get(loc.getRegionInfo().getRegionName()));
verifyRegionResults(locator, results, ROW_B);
@ -367,7 +368,7 @@ public class TestServerCustomProtocol {
assertEquals(2, results.size());
verifyRegionResults(locator, results, ROW_A);
verifyRegionResults(locator, results, ROW_B);
loc = table.getRegionLocation(ROW_C, true);
loc = locator.getRegionLocation(ROW_C, true);
assertNull("Should be missing region for row ccc (past stop row)",
results.get(loc.getRegionInfo().getRegionName()));
@ -377,7 +378,7 @@ public class TestServerCustomProtocol {
assertEquals(2, results.size());
verifyRegionResults(locator, results, ROW_A);
verifyRegionResults(locator, results, ROW_B);
loc = table.getRegionLocation(ROW_C, true);
loc = locator.getRegionLocation(ROW_C, true);
assertNull("Should be missing region for row ccc (past stop row)",
results.get(loc.getRegionInfo().getRegionName()));
@ -386,10 +387,10 @@ public class TestServerCustomProtocol {
// should only contain region bbb
assertEquals(1, results.size());
verifyRegionResults(locator, results, ROW_B);
loc = table.getRegionLocation(ROW_A, true);
loc = locator.getRegionLocation(ROW_A, true);
assertNull("Should be missing region for row aaa (prior to start)",
results.get(loc.getRegionInfo().getRegionName()));
loc = table.getRegionLocation(ROW_C, true);
loc = locator.getRegionLocation(ROW_C, true);
assertNull("Should be missing region for row ccc (past stop row)",
results.get(loc.getRegionInfo().getRegionName()));
}
@ -415,8 +416,8 @@ public class TestServerCustomProtocol {
@Test
public void testCompoundCall() throws Throwable {
try (HTable table = (HTable) util.getConnection().getTable(TEST_TABLE)) {
RegionLocator locator = table.getRegionLocator();
try (Table table = util.getConnection().getTable(TEST_TABLE);
RegionLocator locator = util.getConnection().getRegionLocator(TEST_TABLE)) {
Map<byte [], String> results = compoundOfHelloAndPing(table, ROW_A, ROW_C);
verifyRegionResults(locator, results, "Hello, pong", ROW_A);
verifyRegionResults(locator, results, "Hello, pong", ROW_B);
@ -426,8 +427,8 @@ public class TestServerCustomProtocol {
@Test
public void testNullCall() throws Throwable {
try(HTable table = (HTable) util.getConnection().getTable(TEST_TABLE)) {
RegionLocator locator = table.getRegionLocator();
try (Table table = util.getConnection().getTable(TEST_TABLE);
RegionLocator locator = util.getConnection().getRegionLocator(TEST_TABLE)) {
Map<byte[],String> results = hello(table, null, ROW_A, ROW_C);
verifyRegionResults(locator, results, "Who are you?", ROW_A);
verifyRegionResults(locator, results, "Who are you?", ROW_B);
@ -437,8 +438,8 @@ public class TestServerCustomProtocol {
@Test
public void testNullReturn() throws Throwable {
try (HTable table = (HTable) util.getConnection().getTable(TEST_TABLE)) {
RegionLocator locator = table.getRegionLocator();
try (Table table = util.getConnection().getTable(TEST_TABLE);
RegionLocator locator = util.getConnection().getRegionLocator(TEST_TABLE)) {
Map<byte[],String> results = hello(table, "nobody", ROW_A, ROW_C);
verifyRegionResults(locator, results, null, ROW_A);
verifyRegionResults(locator, results, null, ROW_B);
@ -448,7 +449,7 @@ public class TestServerCustomProtocol {
@Test
public void testEmptyReturnType() throws Throwable {
try (HTable table = (HTable) util.getConnection().getTable(TEST_TABLE)) {
try (Table table = util.getConnection().getTable(TEST_TABLE)) {
Map<byte[],String> results = noop(table, ROW_A, ROW_C);
assertEquals("Should have results from three regions", 3, results.size());
// all results should be null

View File

@ -298,7 +298,7 @@ public class TestSplitTransactionOnCluster {
TableName.valueOf("testRSSplitDaughtersAreOnlinedAfterShutdownHandling");
// Create table then get the single region for our new table.
HTable t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
Table t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
List<HRegion> regions = cluster.getRegions(tableName);
HRegionInfo hri = getAndCheckSingleTableRegion(regions);
@ -343,7 +343,7 @@ public class TestSplitTransactionOnCluster {
TableName.valueOf("testExistingZnodeBlocksSplitAndWeRollback");
// Create table then get the single region for our new table.
HTable t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
Table t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
List<HRegion> regions = cluster.getRegions(tableName);
HRegionInfo hri = getAndCheckSingleTableRegion(regions);
@ -401,7 +401,7 @@ public class TestSplitTransactionOnCluster {
TableName.valueOf("testShutdownFixupWhenDaughterHasSplit");
// Create table then get the single region for our new table.
HTable t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
Table t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
List<HRegion> regions = cluster.getRegions(tableName);
HRegionInfo hri = getAndCheckSingleTableRegion(regions);
@ -567,7 +567,7 @@ public class TestSplitTransactionOnCluster {
.valueOf("testMasterRestartAtRegionSplitPendingCatalogJanitor");
// Create table then get the single region for our new table.
HTable t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
Table t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
List<HRegion> regions = cluster.getRegions(tableName);
HRegionInfo hri = getAndCheckSingleTableRegion(regions);
@ -683,8 +683,7 @@ public class TestSplitTransactionOnCluster {
htd.setRegionReplication(2);
htd.addCoprocessor(SlowMeCopro.class.getName());
// Create table then get the single region for our new table.
Table t = TESTING_UTIL.createTable(htd, new byte[][]{Bytes.toBytes("cf")},
TESTING_UTIL.getConfiguration());
Table t = TESTING_UTIL.createTable(htd, new byte[][]{Bytes.toBytes("cf")}, null);
List<HRegion> oldRegions;
do {
oldRegions = cluster.getRegions(tableName);
@ -965,7 +964,7 @@ public class TestSplitTransactionOnCluster {
desc.addFamily(new HColumnDescriptor(Bytes.toBytes("f")));
admin.createTable(desc);
Connection connection = ConnectionFactory.createConnection(cluster.getConfiguration());
HTable hTable = (HTable) connection.getTable(desc.getTableName());
Table hTable = connection.getTable(desc.getTableName());
for(int i = 1; i < 5; i++) {
Put p1 = new Put(("r"+i).getBytes());
p1.add(Bytes.toBytes("f"), "q1".getBytes(), "v".getBytes());
@ -1266,9 +1265,9 @@ public class TestSplitTransactionOnCluster {
return regions;
}
private HTable createTableAndWait(TableName tableName, byte[] cf) throws IOException,
private Table createTableAndWait(TableName tableName, byte[] cf) throws IOException,
InterruptedException {
HTable t = TESTING_UTIL.createTable(tableName, cf);
Table t = TESTING_UTIL.createTable(tableName, cf);
awaitTableRegions(tableName);
assertTrue("Table not online: " + tableName,
cluster.getRegions(tableName).size() != 0);

View File

@ -87,7 +87,7 @@ public class TestCompactionWithThroughputController {
admin.disableTable(tableName);
admin.deleteTable(tableName);
}
HTable table = TEST_UTIL.createTable(tableName, family);
Table table = TEST_UTIL.createTable(tableName, family);
Random rand = new Random();
for (int i = 0; i < 10; i++) {
for (int j = 0; j < 10; j++) {

View File

@ -55,7 +55,7 @@ public class TestReplicationKillRS extends TestReplicationBase {
Thread killer = killARegionServer(util, 5000, rsToKill1);
LOG.info("Start loading table");
int initialCount = utility1.loadTable((HTable)htable1, famName);
int initialCount = utility1.loadTable(htable1, famName);
LOG.info("Done loading table");
killer.join(5000);
LOG.info("Done waiting for threads");

View File

@ -40,7 +40,9 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.coprocessor.BaseWALObserver;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
@ -86,7 +88,7 @@ public class TestRegionReplicaReplicationEndpointNoMaster {
private static final int NB_SERVERS = 2;
private static TableName tableName = TableName.valueOf(
TestRegionReplicaReplicationEndpointNoMaster.class.getSimpleName());
private static HTable table;
private static Table table;
private static final byte[] row = "TestRegionReplicaReplicator".getBytes();
private static HRegionServer rs0;
@ -117,10 +119,12 @@ public class TestRegionReplicaReplicationEndpointNoMaster {
HTU.startMiniCluster(NB_SERVERS);
// Create table then get the single region for our new table.
HTableDescriptor htd = HTU.createTableDescriptor(tableName.toString());
table = HTU.createTable(htd, new byte[][]{f}, HTU.getConfiguration());
HTableDescriptor htd = HTU.createTableDescriptor(tableName.getNameAsString());
table = HTU.createTable(htd, new byte[][]{f}, null);
hriPrimary = table.getRegionLocation(row, false).getRegionInfo();
try (RegionLocator locator = HTU.getConnection().getRegionLocator(tableName)) {
hriPrimary = locator.getRegionLocation(row, false).getRegionInfo();
}
// mock a secondary region info to open
hriSecondary = new HRegionInfo(hriPrimary.getTable(), hriPrimary.getStartKey(),

View File

@ -955,10 +955,12 @@ public class TestAccessController extends SecureTestUtil {
try (Connection conn = ConnectionFactory.createConnection(conf);
HTable table = (HTable)conn.getTable(tableName)) {
Admin admin = conn.getAdmin();
RegionLocator locator = conn.getRegionLocator(tableName);
Table table = conn.getTable(tableName)) {
TEST_UTIL.waitUntilAllRegionsAssigned(tableName);
LoadIncrementalHFiles loader = new LoadIncrementalHFiles(conf);
loader.doBulkLoad(loadPath, table);
loader.doBulkLoad(loadPath, admin, table, locator);
}
}
@ -1964,7 +1966,7 @@ public class TestAccessController extends SecureTestUtil {
HRegionLocation location = regions.get(0);
final HRegionInfo hri = location.getRegionInfo();
final ServerName server = location.getServerName();
try (HTable table = (HTable) systemUserConnection.getTable(TEST_TABLE2)) {
try (Table table = systemUserConnection.getTable(TEST_TABLE2)) {
AccessTestAction moveAction = new AccessTestAction() {
@Override
public Object run() throws Exception {

View File

@ -79,6 +79,7 @@ import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
@ -144,7 +145,7 @@ public class TestHBaseFsck {
private static Admin admin;
// for the instance, reset every test run
private HTable tbl;
private Table tbl;
private final static byte[][] SPLITS = new byte[][] { Bytes.toBytes("A"),
Bytes.toBytes("B"), Bytes.toBytes("C") };
// one row per region.
@ -155,7 +156,7 @@ public class TestHBaseFsck {
@BeforeClass
public static void setUpBeforeClass() throws Exception {
TEST_UTIL.getConfiguration().set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
MasterSyncObserver.class.getName());
MasterSyncObserver.class.getName());
conf.setInt("hbase.regionserver.handler.count", 2);
conf.setInt("hbase.regionserver.metahandler.count", 30);
@ -326,7 +327,8 @@ public class TestHBaseFsck {
private void deleteRegion(Configuration conf, final HTableDescriptor htd,
byte[] startKey, byte[] endKey, boolean unassign, boolean metaRow,
boolean hdfs) throws IOException, InterruptedException {
deleteRegion(conf, htd, startKey, endKey, unassign, metaRow, hdfs, false, HRegionInfo.DEFAULT_REPLICA_ID);
deleteRegion(conf, htd, startKey, endKey, unassign, metaRow, hdfs, false,
HRegionInfo.DEFAULT_REPLICA_ID);
}
/**
@ -344,7 +346,11 @@ public class TestHBaseFsck {
LOG.info("** Before delete:");
dumpMeta(htd.getTableName());
List<HRegionLocation> locations = tbl.getAllRegionLocations();
List<HRegionLocation> locations;
try(RegionLocator rl = connection.getRegionLocator(tbl.getName())) {
locations = rl.getAllRegionLocations();
}
for (HRegionLocation location : locations) {
HRegionInfo hri = location.getRegionInfo();
ServerName hsa = location.getServerName();
@ -426,15 +432,14 @@ public class TestHBaseFsck {
desc.addFamily(hcd); // If a table has no CF's it doesn't get checked
createTable(TEST_UTIL, desc, SPLITS);
tbl = (HTable) connection.getTable(tablename, tableExecutorService);
tbl = connection.getTable(tablename, tableExecutorService);
List<Put> puts = new ArrayList<Put>();
for (byte[] row : ROWKEYS) {
Put p = new Put(row);
p.add(FAM, Bytes.toBytes("val"), row);
p.addColumn(FAM, Bytes.toBytes("val"), row);
puts.add(p);
}
tbl.put(puts);
tbl.flushCommits();
}
/**
@ -683,13 +688,12 @@ public class TestHBaseFsck {
TEST_UTIL.assertRegionOnServer(hriDupe, server, REGION_ONLINE_TIMEOUT);
HBaseFsck hbck = doFsck(conf, false);
assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.DUPE_STARTKEYS,
ERROR_CODE.DUPE_STARTKEYS});
assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.DUPE_STARTKEYS, ERROR_CODE.DUPE_STARTKEYS });
assertEquals(2, hbck.getOverlapGroups(table).size());
assertEquals(ROWKEYS.length, countRows()); // seems like the "bigger" region won.
// fix the degenerate region.
doFsck(conf,true);
doFsck(conf, true);
// check that the degenerate region is gone and no data loss
HBaseFsck hbck2 = doFsck(conf,false);
@ -727,16 +731,16 @@ public class TestHBaseFsck {
admin.flush(table);
assertNoErrors(doFsck(conf, false));
assertEquals(ROWKEYS.length, countRows());
deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
Bytes.toBytes("C"), true, false, false, false, 1); // unassign one replica
deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"), Bytes.toBytes("C"), true,
false, false, false, 1); // unassign one replica
// check that problem exists
HBaseFsck hbck = doFsck(conf, false);
assertErrors(hbck, new ERROR_CODE[]{ERROR_CODE.NOT_DEPLOYED});
assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.NOT_DEPLOYED });
// fix the problem
hbck = doFsck(conf, true);
// run hbck again to make sure we don't see any errors
hbck = doFsck(conf, false);
assertErrors(hbck, new ERROR_CODE[]{});
assertErrors(hbck, new ERROR_CODE[] {});
} finally {
cleanupTable(table);
}
@ -868,13 +872,12 @@ public class TestHBaseFsck {
// TODO why is dupe region different from dupe start keys?
HBaseFsck hbck = doFsck(conf, false);
assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.DUPE_STARTKEYS,
ERROR_CODE.DUPE_STARTKEYS});
assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.DUPE_STARTKEYS, ERROR_CODE.DUPE_STARTKEYS });
assertEquals(2, hbck.getOverlapGroups(table).size());
assertEquals(ROWKEYS.length, countRows()); // seems like the "bigger" region won.
// fix the degenerate region.
doFsck(conf,true);
doFsck(conf, true);
// check that the degenerate region is gone and no data loss
HBaseFsck hbck2 = doFsck(conf,false);
@ -894,7 +897,7 @@ public class TestHBaseFsck {
TableName table = TableName.valueOf("tableDegenerateRegions");
try {
setupTable(table);
assertNoErrors(doFsck(conf,false));
assertNoErrors(doFsck(conf, false));
assertEquals(ROWKEYS.length, countRows());
// Now let's mess it up, by adding a region with a duplicate startkey
@ -913,7 +916,7 @@ public class TestHBaseFsck {
assertEquals(ROWKEYS.length, countRows());
// fix the degenerate region.
doFsck(conf,true);
doFsck(conf, true);
// check that the degenerate region is gone and no data loss
HBaseFsck hbck2 = doFsck(conf,false);
@ -947,8 +950,7 @@ public class TestHBaseFsck {
TEST_UTIL.assertRegionOnServer(hriOverlap, server, REGION_ONLINE_TIMEOUT);
HBaseFsck hbck = doFsck(conf, false);
assertErrors(hbck, new ERROR_CODE[] {
ERROR_CODE.OVERLAP_IN_REGION_CHAIN });
assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.OVERLAP_IN_REGION_CHAIN });
assertEquals(2, hbck.getOverlapGroups(table).size());
assertEquals(ROWKEYS.length, countRows());
@ -1073,8 +1075,8 @@ public class TestHBaseFsck {
// Mess it up by creating an overlap in the metadata
admin.disableTable(table);
deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("A"),
Bytes.toBytes("B"), true, true, false, true, HRegionInfo.DEFAULT_REPLICA_ID);
deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("A"), Bytes.toBytes("B"), true,
true, false, true, HRegionInfo.DEFAULT_REPLICA_ID);
admin.enableTable(table);
HRegionInfo hriOverlap =
@ -1086,9 +1088,9 @@ public class TestHBaseFsck {
TEST_UTIL.assertRegionOnServer(hriOverlap, server, REGION_ONLINE_TIMEOUT);
HBaseFsck hbck = doFsck(conf, false);
assertErrors(hbck, new ERROR_CODE[] {
ERROR_CODE.ORPHAN_HDFS_REGION, ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
ERROR_CODE.HOLE_IN_REGION_CHAIN});
assertErrors(hbck,
new ERROR_CODE[] { ERROR_CODE.ORPHAN_HDFS_REGION, ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
ERROR_CODE.HOLE_IN_REGION_CHAIN });
// fix the problem.
doFsck(conf, true);
@ -1158,13 +1160,12 @@ public class TestHBaseFsck {
// Mess it up by leaving a hole in the assignment, meta, and hdfs data
admin.disableTable(table);
deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
Bytes.toBytes("C"), true, true, true);
deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"), Bytes.toBytes("C"), true,
true, true);
admin.enableTable(table);
HBaseFsck hbck = doFsck(conf, false);
assertErrors(hbck, new ERROR_CODE[] {
ERROR_CODE.HOLE_IN_REGION_CHAIN});
assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.HOLE_IN_REGION_CHAIN });
// holes are separate from overlap groups
assertEquals(0, hbck.getOverlapGroups(table).size());
@ -1173,7 +1174,7 @@ public class TestHBaseFsck {
// check that hole fixed
assertNoErrors(doFsck(conf,false));
assertEquals(ROWKEYS.length - 2 , countRows()); // lost a region so lost a row
assertEquals(ROWKEYS.length - 2, countRows()); // lost a region so lost a row
} finally {
cleanupTable(table);
}
@ -1192,15 +1193,14 @@ public class TestHBaseFsck {
// Mess it up by leaving a hole in the meta data
admin.disableTable(table);
deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
Bytes.toBytes("C"), true, true, false, true, HRegionInfo.DEFAULT_REPLICA_ID);
deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"), Bytes.toBytes("C"), true,
true, false, true, HRegionInfo.DEFAULT_REPLICA_ID);
admin.enableTable(table);
HBaseFsck hbck = doFsck(conf, false);
assertErrors(hbck, new ERROR_CODE[] {
ERROR_CODE.ORPHAN_HDFS_REGION,
ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
ERROR_CODE.HOLE_IN_REGION_CHAIN});
assertErrors(hbck,
new ERROR_CODE[] { ERROR_CODE.ORPHAN_HDFS_REGION, ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
ERROR_CODE.HOLE_IN_REGION_CHAIN });
// holes are separate from overlap groups
assertEquals(0, hbck.getOverlapGroups(table).size());
@ -1229,22 +1229,22 @@ public class TestHBaseFsck {
// Mess it up by leaving a hole in the meta data
admin.disableTable(table);
deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
Bytes.toBytes("C"), true, true, false); // don't rm from fs
deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"), Bytes.toBytes("C"), true,
true, false); // don't rm from fs
admin.enableTable(table);
HBaseFsck hbck = doFsck(conf, false);
assertErrors(hbck, new ERROR_CODE[] {
ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN});
assertErrors(hbck,
new ERROR_CODE[] { ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN });
// holes are separate from overlap groups
assertEquals(0, hbck.getOverlapGroups(table).size());
// fix hole
assertErrors(doFsck(conf, true) , new ERROR_CODE[] {
ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN});
assertErrors(doFsck(conf, true),
new ERROR_CODE[] { ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN });
// check that hole fixed
assertNoErrors(doFsck(conf,false));
assertNoErrors(doFsck(conf, false));
assertEquals(ROWKEYS.length, countRows());
} finally {
cleanupTable(table);
@ -1260,7 +1260,7 @@ public class TestHBaseFsck {
desc.addFamily(new HColumnDescriptor(Bytes.toBytes("f")));
createTable(TEST_UTIL, desc, null);
tbl = (HTable) connection.getTable(desc.getTableName());
tbl = connection.getTable(desc.getTableName());
for (int i = 0; i < 5; i++) {
Put p1 = new Put(("r" + i).getBytes());
p1.add(Bytes.toBytes("f"), "q1".getBytes(), "v".getBytes());
@ -1293,9 +1293,9 @@ public class TestHBaseFsck {
// fix hole
assertErrors(
doFsck(conf, false, true, false, false, false, false, false, false, false, false, null),
new ERROR_CODE[] { ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
ERROR_CODE.NOT_IN_META_OR_DEPLOYED });
doFsck(conf, false, true, false, false, false, false, false, false, false, false, null),
new ERROR_CODE[] { ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
ERROR_CODE.NOT_IN_META_OR_DEPLOYED });
// check that hole fixed
assertNoErrors(doFsck(conf, false));
@ -1322,22 +1322,22 @@ public class TestHBaseFsck {
// Mess it up by leaving a hole in the meta data
admin.disableTable(table);
deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
Bytes.toBytes("C"), false, true, false); // don't rm from fs
deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"), Bytes.toBytes("C"), false,
true, false); // don't rm from fs
admin.enableTable(table);
HBaseFsck hbck = doFsck(conf, false);
assertErrors(hbck, new ERROR_CODE[] {
ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN});
assertErrors(hbck,
new ERROR_CODE[] { ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN });
// holes are separate from overlap groups
assertEquals(0, hbck.getOverlapGroups(table).size());
// fix hole
assertErrors(doFsck(conf, true) , new ERROR_CODE[] {
ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN});
assertErrors(doFsck(conf, true),
new ERROR_CODE[] { ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN });
// check that hole fixed
assertNoErrors(doFsck(conf,false));
assertNoErrors(doFsck(conf, false));
assertEquals(ROWKEYS.length, countRows());
} finally {
cleanupTable(table);
@ -1360,8 +1360,8 @@ public class TestHBaseFsck {
admin.flush(table);
// Mess it up by leaving a hole in the hdfs data
deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
Bytes.toBytes("C"), false, false, true); // don't rm meta
deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"), Bytes.toBytes("C"), false,
false, true); // don't rm meta
HBaseFsck hbck = doFsck(conf, false);
assertErrors(hbck, new ERROR_CODE[] {ERROR_CODE.NOT_IN_HDFS});
@ -1407,17 +1407,17 @@ public class TestHBaseFsck {
admin.flush(table);
// Mess it up by leaving a hole in the hdfs data
deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
Bytes.toBytes("C"), false, false, true); // don't rm meta
deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"), Bytes.toBytes("C"), false,
false, true); // don't rm meta
HBaseFsck hbck = doFsck(conf, false);
assertErrors(hbck, new ERROR_CODE[] {ERROR_CODE.NOT_IN_HDFS});
assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.NOT_IN_HDFS });
// fix hole
doFsck(conf, true);
// check that hole fixed
assertNoErrors(doFsck(conf,false));
assertNoErrors(doFsck(conf, false));
assertEquals(ROWKEYS.length - 2, countRows());
// the following code checks whether the old primary/secondary has
@ -1633,20 +1633,18 @@ public class TestHBaseFsck {
// make sure data in regions, if in wal only there is no data loss
admin.flush(table2);
// Mess them up by leaving a hole in the hdfs data
deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
Bytes.toBytes("C"), false, false, true); // don't rm meta
deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"), Bytes.toBytes("C"), false,
false, true); // don't rm meta
HBaseFsck hbck = doFsck(conf, false);
assertErrors(hbck, new ERROR_CODE[] {
ERROR_CODE.NOT_IN_HDFS, ERROR_CODE.NOT_IN_HDFS});
assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.NOT_IN_HDFS, ERROR_CODE.NOT_IN_HDFS });
// fix hole in table 1
doFsck(conf, true, table1);
// check that hole in table 1 fixed
assertNoErrors(doFsck(conf, false, table1));
// check that hole in table 2 still there
assertErrors(doFsck(conf, false, table2),
new ERROR_CODE[] {ERROR_CODE.NOT_IN_HDFS});
assertErrors(doFsck(conf, false, table2), new ERROR_CODE[] { ERROR_CODE.NOT_IN_HDFS });
// fix hole in table 2
doFsck(conf, true, table2);
@ -1672,7 +1670,11 @@ public class TestHBaseFsck {
// make sure data in regions, if in wal only there is no data loss
admin.flush(table);
HRegionLocation location = tbl.getRegionLocation("B");
HRegionLocation location;
try(RegionLocator rl = connection.getRegionLocator(tbl.getName())) {
location = rl.getRegionLocation(Bytes.toBytes("B"));
}
// Delete one region from meta, but not hdfs, unassign it.
deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
@ -1752,33 +1754,38 @@ public class TestHBaseFsck {
// make sure data in regions, if in wal only there is no data loss
admin.flush(table);
HRegionLocation location = tbl.getRegionLocation(Bytes.toBytes("B"));
meta = connection.getTable(TableName.META_TABLE_NAME, tableExecutorService);
HRegionInfo hri = location.getRegionInfo();
try(RegionLocator rl = connection.getRegionLocator(tbl.getName())) {
HRegionLocation location = rl.getRegionLocation(Bytes.toBytes("B"));
// do a regular split
byte[] regionName = location.getRegionInfo().getRegionName();
admin.splitRegion(location.getRegionInfo().getRegionName(), Bytes.toBytes("BM"));
TestEndToEndSplitTransaction.blockUntilRegionSplit(conf, 60000, regionName, true);
meta = connection.getTable(TableName.META_TABLE_NAME, tableExecutorService);
HRegionInfo hri = location.getRegionInfo();
// TODO: fixHdfsHoles does not work against splits, since the parent dir lingers on
// for some time until children references are deleted. HBCK erroneously sees this as
// overlapping regions
HBaseFsck hbck = doFsck(conf, true, true, false, false, false, true, true, true, false, false, null);
assertErrors(hbck, new ERROR_CODE[] {}); //no LINGERING_SPLIT_PARENT reported
// do a regular split
byte[] regionName = location.getRegionInfo().getRegionName();
admin.splitRegion(location.getRegionInfo().getRegionName(), Bytes.toBytes("BM"));
TestEndToEndSplitTransaction.blockUntilRegionSplit(conf, 60000, regionName, true);
// assert that the split hbase:meta entry is still there.
Get get = new Get(hri.getRegionName());
Result result = meta.get(get);
assertNotNull(result);
assertNotNull(MetaTableAccessor.getHRegionInfo(result));
// TODO: fixHdfsHoles does not work against splits, since the parent dir lingers on
// for some time until children references are deleted. HBCK erroneously sees this as
// overlapping regions
HBaseFsck hbck = doFsck(conf, true, true, false, false, false, true, true, true, false,
false, null);
assertErrors(hbck, new ERROR_CODE[] {}); //no LINGERING_SPLIT_PARENT reported
assertEquals(ROWKEYS.length, countRows());
// assert that the split hbase:meta entry is still there.
Get get = new Get(hri.getRegionName());
Result result = meta.get(get);
assertNotNull(result);
assertNotNull(MetaTableAccessor.getHRegionInfo(result));
// assert that we still have the split regions
assertEquals(tbl.getStartKeys().length, SPLITS.length + 1 + 1); //SPLITS + 1 is # regions pre-split.
assertNoErrors(doFsck(conf, false));
assertEquals(ROWKEYS.length, countRows());
// assert that we still have the split regions
assertEquals(rl.getStartKeys().length, SPLITS.length + 1 + 1); //SPLITS + 1 is # regions
// pre-split.
assertNoErrors(doFsck(conf, false));
}
} finally {
cleanupTable(table);
IOUtils.closeQuietly(meta);
@ -1799,56 +1806,68 @@ public class TestHBaseFsck {
// make sure data in regions, if in wal only there is no data loss
admin.flush(table);
HRegionLocation location = tbl.getRegionLocation(Bytes.toBytes("B"));
HRegionInfo hri = location.getRegionInfo();
try(RegionLocator rl = connection.getRegionLocator(tbl.getName())) {
HRegionLocation location = rl.getRegionLocation(Bytes.toBytes("B"));
// do a regular split
byte[] regionName = location.getRegionInfo().getRegionName();
admin.splitRegion(location.getRegionInfo().getRegionName(), Bytes.toBytes("BM"));
TestEndToEndSplitTransaction.blockUntilRegionSplit(conf, 60000, regionName, true);
HRegionInfo hri = location.getRegionInfo();
PairOfSameType<HRegionInfo> daughters =
MetaTableAccessor.getDaughterRegions(meta.get(new Get(regionName)));
// do a regular split
byte[] regionName = location.getRegionInfo().getRegionName();
admin.splitRegion(location.getRegionInfo().getRegionName(), Bytes.toBytes("BM"));
TestEndToEndSplitTransaction.blockUntilRegionSplit(conf, 60000, regionName, true);
// Delete daughter regions from meta, but not hdfs, unassign it.
Map<HRegionInfo, ServerName> hris = tbl.getRegionLocations();
undeployRegion(connection, hris.get(daughters.getFirst()), daughters.getFirst());
undeployRegion(connection, hris.get(daughters.getSecond()), daughters.getSecond());
PairOfSameType<HRegionInfo> daughters = MetaTableAccessor.getDaughterRegions(
meta.get(new Get(regionName)));
List<Delete> deletes = new ArrayList<>();
deletes.add(new Delete(daughters.getFirst().getRegionName()));
deletes.add(new Delete(daughters.getSecond().getRegionName()));
meta.delete(deletes);
// Delete daughter regions from meta, but not hdfs, unassign it.
// Remove daughters from regionStates
RegionStates regionStates = TEST_UTIL.getMiniHBaseCluster().getMaster().
getAssignmentManager().getRegionStates();
regionStates.deleteRegion(daughters.getFirst());
regionStates.deleteRegion(daughters.getSecond());
ServerName firstSN =
rl.getRegionLocation(daughters.getFirst().getStartKey()).getServerName();
ServerName secondSN =
rl.getRegionLocation(daughters.getSecond().getStartKey()).getServerName();
HBaseFsck hbck = doFsck(conf, false);
assertErrors(hbck,
new ERROR_CODE[] { ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
ERROR_CODE.HOLE_IN_REGION_CHAIN }); //no LINGERING_SPLIT_PARENT
undeployRegion(connection, firstSN, daughters.getFirst());
undeployRegion(connection, secondSN, daughters.getSecond());
// now fix it. The fix should not revert the region split, but add daughters to META
hbck = doFsck(conf, true, true, false, false, false, false, false, false, false, false, null);
assertErrors(hbck,
new ERROR_CODE[] { ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
ERROR_CODE.HOLE_IN_REGION_CHAIN });
List<Delete> deletes = new ArrayList<>();
deletes.add(new Delete(daughters.getFirst().getRegionName()));
deletes.add(new Delete(daughters.getSecond().getRegionName()));
meta.delete(deletes);
// assert that the split hbase:meta entry is still there.
Get get = new Get(hri.getRegionName());
Result result = meta.get(get);
assertNotNull(result);
assertNotNull(MetaTableAccessor.getHRegionInfo(result));
// Remove daughters from regionStates
RegionStates regionStates = TEST_UTIL.getMiniHBaseCluster().getMaster().
getAssignmentManager().getRegionStates();
regionStates.deleteRegion(daughters.getFirst());
regionStates.deleteRegion(daughters.getSecond());
assertEquals(ROWKEYS.length, countRows());
HBaseFsck hbck = doFsck(conf, false);
assertErrors(hbck, new ERROR_CODE[] {
ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
ERROR_CODE.HOLE_IN_REGION_CHAIN }); //no LINGERING_SPLIT_PARENT
// assert that we still have the split regions
assertEquals(tbl.getStartKeys().length, SPLITS.length + 1 + 1); //SPLITS + 1 is # regions pre-split.
assertNoErrors(doFsck(conf, false)); //should be fixed by now
// now fix it. The fix should not revert the region split, but add daughters to META
hbck = doFsck(conf, true, true, false, false, false, false, false, false, false,
false, null);
assertErrors(hbck, new ERROR_CODE[] {
ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
ERROR_CODE.HOLE_IN_REGION_CHAIN });
// assert that the split hbase:meta entry is still there.
Get get = new Get(hri.getRegionName());
Result result = meta.get(get);
assertNotNull(result);
assertNotNull(MetaTableAccessor.getHRegionInfo(result));
assertEquals(ROWKEYS.length, countRows());
// assert that we still have the split regions
assertEquals(rl.getStartKeys().length, SPLITS.length + 1 + 1); //SPLITS + 1 is # regions
// pre-split.
assertNoErrors(doFsck(conf, false)); //should be fixed by now
}
} finally {
meta.close();
cleanupTable(table);
@ -1952,13 +1971,13 @@ public class TestHBaseFsck {
assertEquals(ROWKEYS.length, countRows());
// Mess it up by closing a region
deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("A"),
Bytes.toBytes("B"), true, false, false, false, HRegionInfo.DEFAULT_REPLICA_ID);
deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("A"), Bytes.toBytes("B"), true,
false, false, false, HRegionInfo.DEFAULT_REPLICA_ID);
// verify there is no other errors
HBaseFsck hbck = doFsck(conf, false);
assertErrors(hbck, new ERROR_CODE[] {
ERROR_CODE.NOT_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN});
assertErrors(hbck,
new ERROR_CODE[] { ERROR_CODE.NOT_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN });
// verify that noHdfsChecking report the same errors
HBaseFsck fsck = new HBaseFsck(conf, hbfsckExecutorService);
@ -1967,8 +1986,8 @@ public class TestHBaseFsck {
fsck.setTimeLag(0);
fsck.setCheckHdfs(false);
fsck.onlineHbck();
assertErrors(fsck, new ERROR_CODE[] {
ERROR_CODE.NOT_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN});
assertErrors(fsck,
new ERROR_CODE[] { ERROR_CODE.NOT_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN });
fsck.close();
// verify that fixAssignments works fine with noHdfsChecking
@ -2062,8 +2081,8 @@ public class TestHBaseFsck {
// Mess it up by creating an overlap in the metadata
admin.disableTable(table);
deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("A"),
Bytes.toBytes("B"), true, true, false, true, HRegionInfo.DEFAULT_REPLICA_ID);
deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("A"), Bytes.toBytes("B"), true,
true, false, true, HRegionInfo.DEFAULT_REPLICA_ID);
admin.enableTable(table);
HRegionInfo hriOverlap =
@ -2086,8 +2105,7 @@ public class TestHBaseFsck {
fsck.setTimeLag(0);
fsck.setCheckHdfs(false);
fsck.onlineHbck();
assertErrors(fsck, new ERROR_CODE[] {
ERROR_CODE.HOLE_IN_REGION_CHAIN});
assertErrors(fsck, new ERROR_CODE[] { ERROR_CODE.HOLE_IN_REGION_CHAIN });
fsck.close();
// verify that fixHdfsHoles doesn't work with noHdfsChecking
@ -2101,7 +2119,7 @@ public class TestHBaseFsck {
fsck.setFixHdfsOrphans(true);
fsck.onlineHbck();
assertFalse(fsck.shouldRerun());
assertErrors(fsck, new ERROR_CODE[] { ERROR_CODE.HOLE_IN_REGION_CHAIN});
assertErrors(fsck, new ERROR_CODE[] { ERROR_CODE.HOLE_IN_REGION_CHAIN });
fsck.close();
} finally {
if (admin.isTableDisabled(table)) {
@ -2229,7 +2247,8 @@ public class TestHBaseFsck {
final FileSystem fs = FileSystem.get(conf);
HBaseFsck hbck = new HBaseFsck(conf, hbfsckExecutorService) {
@Override
public HFileCorruptionChecker createHFileCorruptionChecker(boolean sidelineCorruptHFiles) throws IOException {
public HFileCorruptionChecker createHFileCorruptionChecker(boolean sidelineCorruptHFiles)
throws IOException {
return new HFileCorruptionChecker(conf, executor, sidelineCorruptHFiles) {
AtomicBoolean attemptedFirstHFile = new AtomicBoolean(false);
@Override
@ -2259,7 +2278,8 @@ public class TestHBaseFsck {
final FileSystem fs = FileSystem.get(conf);
HBaseFsck hbck = new HBaseFsck(conf, hbfsckExecutorService) {
@Override
public HFileCorruptionChecker createHFileCorruptionChecker(boolean sidelineCorruptHFiles) throws IOException {
public HFileCorruptionChecker createHFileCorruptionChecker(boolean sidelineCorruptHFiles)
throws IOException {
return new HFileCorruptionChecker(conf, executor, sidelineCorruptHFiles) {
AtomicBoolean attemptedFirstHFile = new AtomicBoolean(false);
@Override
@ -2529,16 +2549,20 @@ public class TestHBaseFsck {
Threads.sleep(300); // wait some more to ensure writeLock.acquire() is called
hbck = doFsck(conf, false);
assertErrors(hbck, new ERROR_CODE[] {ERROR_CODE.EXPIRED_TABLE_LOCK}); // still one expired, one not-expired
assertErrors(hbck, new ERROR_CODE[] {
ERROR_CODE.EXPIRED_TABLE_LOCK}); // still one expired, one not-expired
edge.incrementTime(conf.getLong(TableLockManager.TABLE_LOCK_EXPIRE_TIMEOUT,
TableLockManager.DEFAULT_TABLE_LOCK_EXPIRE_TIMEOUT_MS)); // let table lock expire
hbck = doFsck(conf, false);
assertErrors(hbck, new ERROR_CODE[] {ERROR_CODE.EXPIRED_TABLE_LOCK, ERROR_CODE.EXPIRED_TABLE_LOCK}); // both are expired
assertErrors(hbck, new ERROR_CODE[] {ERROR_CODE.EXPIRED_TABLE_LOCK,
ERROR_CODE.EXPIRED_TABLE_LOCK}); // both are expired
conf.setLong(TableLockManager.TABLE_LOCK_EXPIRE_TIMEOUT, 1);
// reaping from ZKInterProcessWriteLock uses znode cTime,
// which is not injectable through EnvironmentEdge
conf.setLong(TableLockManager.TABLE_LOCK_EXPIRE_TIMEOUT, 1); // reaping from ZKInterProcessWriteLock uses znode cTime,
// which is not injectable through EnvironmentEdge
Threads.sleep(10);
hbck = doFsck(conf, true); // now fix both cases
@ -2616,7 +2640,7 @@ public class TestHBaseFsck {
HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toString(FAM));
desc.addFamily(hcd); // If a table has no CF's it doesn't get checked
createTable(TEST_UTIL, desc, null);
tbl = (HTable) connection.getTable(table, tableExecutorService);
tbl = connection.getTable(table, tableExecutorService);
// Mess it up by leaving a hole in the assignment, meta, and hdfs data
deleteRegion(conf, tbl.getTableDescriptor(), HConstants.EMPTY_START_ROW,
@ -2648,36 +2672,37 @@ public class TestHBaseFsck {
setupTable(table);
assertEquals(ROWKEYS.length, countRows());
// make sure data in regions, if in wal only there is no data loss
admin.flush(table);
HRegionInfo region1 = tbl.getRegionLocation(Bytes.toBytes("A")).getRegionInfo();
HRegionInfo region2 = tbl.getRegionLocation(Bytes.toBytes("B")).getRegionInfo();
try(RegionLocator rl = connection.getRegionLocator(tbl.getName())) {
// make sure data in regions, if in wal only there is no data loss
admin.flush(table);
HRegionInfo region1 = rl.getRegionLocation(Bytes.toBytes("A")).getRegionInfo();
HRegionInfo region2 = rl.getRegionLocation(Bytes.toBytes("B")).getRegionInfo();
int regionCountBeforeMerge = tbl.getRegionLocations().size();
int regionCountBeforeMerge = rl.getAllRegionLocations().size();
assertNotEquals(region1, region2);
assertNotEquals(region1, region2);
// do a region merge
admin.mergeRegions(region1.getEncodedNameAsBytes(),
region2.getEncodedNameAsBytes(), false);
// do a region merge
admin.mergeRegions(region1.getEncodedNameAsBytes(), region2.getEncodedNameAsBytes(), false);
// wait until region merged
long timeout = System.currentTimeMillis() + 30 * 1000;
while (true) {
if (tbl.getRegionLocations().size() < regionCountBeforeMerge) {
break;
} else if (System.currentTimeMillis() > timeout) {
fail("Time out waiting on region " + region1.getEncodedName()
+ " and " + region2.getEncodedName() + " be merged");
// wait until region merged
long timeout = System.currentTimeMillis() + 30 * 1000;
while (true) {
if (rl.getAllRegionLocations().size() < regionCountBeforeMerge) {
break;
} else if (System.currentTimeMillis() > timeout) {
fail("Time out waiting on region " + region1.getEncodedName() + " and " + region2
.getEncodedName() + " be merged");
}
Thread.sleep(10);
}
Thread.sleep(10);
assertEquals(ROWKEYS.length, countRows());
HBaseFsck hbck = doFsck(conf, false);
assertNoErrors(hbck); // no errors
}
assertEquals(ROWKEYS.length, countRows());
HBaseFsck hbck = doFsck(conf, false);
assertNoErrors(hbck); // no errors
} finally {
TEST_UTIL.getHBaseCluster().getMaster().setCatalogJanitorEnabled(true);
cleanupTable(table);

View File

@ -58,7 +58,7 @@ public class TestProcessBasedCluster {
cluster.startMiniDFS();
cluster.startHBase();
try {
TEST_UTIL.createRandomTable(HTestConst.DEFAULT_TABLE_STR,
TEST_UTIL.createRandomTable(HTestConst.DEFAULT_TABLE,
HTestConst.DEFAULT_CF_STR_SET,
HColumnDescriptor.DEFAULT_VERSIONS, COLS_PER_ROW, FLUSHES, NUM_REGIONS,
ROWS_PER_FLUSH);

View File

@ -34,9 +34,12 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.MiscTests;
import org.apache.hadoop.hbase.util.RegionSplitter.HexStringSplit;
@ -109,22 +112,22 @@ public class TestRegionSplitter {
expectedBounds.add(new byte[] { 0x20, 0, 0, 0, 0, 0, 0, 0});
expectedBounds.add(new byte[] { 0x30, 0, 0, 0, 0, 0, 0, 0});
expectedBounds.add(new byte[] { 0x40, 0, 0, 0, 0, 0, 0, 0});
expectedBounds.add(new byte[] { 0x50, 0, 0, 0, 0, 0, 0, 0});
expectedBounds.add(new byte[] { 0x60, 0, 0, 0, 0, 0, 0, 0});
expectedBounds.add(new byte[] { 0x70, 0, 0, 0, 0, 0, 0, 0});
expectedBounds.add(new byte[] {(byte)0x80, 0, 0, 0, 0, 0, 0, 0});
expectedBounds.add(new byte[] {(byte)0x90, 0, 0, 0, 0, 0, 0, 0});
expectedBounds.add(new byte[] { 0x50, 0, 0, 0, 0, 0, 0, 0 });
expectedBounds.add(new byte[] { 0x60, 0, 0, 0, 0, 0, 0, 0 });
expectedBounds.add(new byte[] { 0x70, 0, 0, 0, 0, 0, 0, 0 });
expectedBounds.add(new byte[] { (byte) 0x80, 0, 0, 0, 0, 0, 0, 0 });
expectedBounds.add(new byte[] { (byte) 0x90, 0, 0, 0, 0, 0, 0, 0 });
expectedBounds.add(new byte[] {(byte)0xa0, 0, 0, 0, 0, 0, 0, 0});
expectedBounds.add(new byte[] {(byte)0xb0, 0, 0, 0, 0, 0, 0, 0});
expectedBounds.add(new byte[] {(byte)0xc0, 0, 0, 0, 0, 0, 0, 0});
expectedBounds.add(new byte[] {(byte)0xd0, 0, 0, 0, 0, 0, 0, 0});
expectedBounds.add(new byte[] { (byte) 0xb0, 0, 0, 0, 0, 0, 0, 0 });
expectedBounds.add(new byte[] { (byte) 0xc0, 0, 0, 0, 0, 0, 0, 0 });
expectedBounds.add(new byte[] { (byte) 0xd0, 0, 0, 0, 0, 0, 0, 0 });
expectedBounds.add(new byte[] {(byte)0xe0, 0, 0, 0, 0, 0, 0, 0});
expectedBounds.add(new byte[] {(byte)0xf0, 0, 0, 0, 0, 0, 0, 0});
expectedBounds.add(new byte[] { (byte) 0xf0, 0, 0, 0, 0, 0, 0, 0 });
expectedBounds.add(ArrayUtils.EMPTY_BYTE_ARRAY);
// Do table creation/pre-splitting and verification of region boundaries
preSplitTableAndVerify(expectedBounds, UniformSplit.class.getSimpleName(),
TableName.valueOf("NewUniformPresplitTable"));
TableName.valueOf("NewUniformPresplitTable"));
}
/**
@ -181,8 +184,7 @@ public class TestRegionSplitter {
byte[][] twoRegionsSplits = splitter.split(2);
assertEquals(1, twoRegionsSplits.length);
assertArrayEquals(twoRegionsSplits[0],
new byte[] { (byte) 0x80, 0, 0, 0, 0, 0, 0, 0 });
assertArrayEquals(twoRegionsSplits[0], new byte[] { (byte) 0x80, 0, 0, 0, 0, 0, 0, 0 });
byte[][] threeRegionsSplits = splitter.split(3);
assertEquals(2, threeRegionsSplits.length);
@ -194,7 +196,7 @@ public class TestRegionSplitter {
// Check splitting existing regions that have start and end points
byte[] splitPoint = splitter.split(new byte[] {0x10}, new byte[] {0x30});
assertArrayEquals(new byte[] {0x20}, splitPoint);
assertArrayEquals(new byte[] { 0x20 }, splitPoint);
byte[] lastRow = new byte[] {xFF, xFF, xFF, xFF, xFF, xFF, xFF, xFF};
assertArrayEquals(lastRow, splitter.lastRow());
@ -202,15 +204,15 @@ public class TestRegionSplitter {
assertArrayEquals(firstRow, splitter.firstRow());
splitPoint = splitter.split(firstRow, new byte[] {0x20});
assertArrayEquals(splitPoint, new byte[] {0x10});
assertArrayEquals(splitPoint, new byte[] { 0x10 });
splitPoint = splitter.split(new byte[] {(byte)0xdf, xFF, xFF, xFF, xFF,
xFF, xFF, xFF}, lastRow);
assertArrayEquals(splitPoint,
new byte[] {(byte)0xef, xFF, xFF, xFF, xFF, xFF, xFF, xFF});
assertArrayEquals(splitPoint, new byte[] { (byte) 0xef, xFF, xFF, xFF, xFF, xFF, xFF, xFF
});
splitPoint = splitter.split(new byte[] {'a', 'a', 'a'}, new byte[] {'a', 'a', 'b'});
assertArrayEquals(splitPoint, new byte[] {'a', 'a', 'a', (byte)0x80 });
assertArrayEquals(splitPoint, new byte[] { 'a', 'a', 'a', (byte) 0x80 });
}
@Test
@ -283,7 +285,7 @@ public class TestRegionSplitter {
final Configuration conf = UTIL.getConfiguration();
conf.setInt("split.count", numRegions);
SplitAlgorithm splitAlgo = RegionSplitter.newSplitAlgoInstance(conf, splitClass);
RegionSplitter.createPresplitTable(tableName, splitAlgo, new String[] {CF_NAME}, conf);
RegionSplitter.createPresplitTable(tableName, splitAlgo, new String[] { CF_NAME }, conf);
verifyBounds(expectedBounds, tableName);
}
@ -291,8 +293,7 @@ public class TestRegionSplitter {
public void noopRollingSplit() throws Exception {
final List<byte[]> expectedBounds = new ArrayList<byte[]>();
expectedBounds.add(ArrayUtils.EMPTY_BYTE_ARRAY);
rollingSplitAndVerify(
TableName.valueOf(TestRegionSplitter.class.getSimpleName()),
rollingSplitAndVerify(TableName.valueOf(TestRegionSplitter.class.getSimpleName()),
"UniformSplit", expectedBounds);
}
@ -309,27 +310,27 @@ public class TestRegionSplitter {
private void verifyBounds(List<byte[]> expectedBounds, TableName tableName)
throws Exception {
// Get region boundaries from the cluster and verify their endpoints
final int numRegions = expectedBounds.size()-1;
final HTable hTable = (HTable) UTIL.getConnection().getTable(tableName);
final Map<HRegionInfo, ServerName> regionInfoMap = hTable.getRegionLocations();
// Get region boundaries from the cluster and verify their endpoints
final int numRegions = expectedBounds.size()-1;
try (Table table = UTIL.getConnection().getTable(tableName);
RegionLocator locator = UTIL.getConnection().getRegionLocator(tableName)) {
final List<HRegionLocation> regionInfoMap = locator.getAllRegionLocations();
assertEquals(numRegions, regionInfoMap.size());
for (Map.Entry<HRegionInfo, ServerName> entry: regionInfoMap.entrySet()) {
final HRegionInfo regionInfo = entry.getKey();
byte[] regionStart = regionInfo.getStartKey();
byte[] regionEnd = regionInfo.getEndKey();
for (HRegionLocation entry : regionInfoMap) {
final HRegionInfo regionInfo = entry.getRegionInfo();
byte[] regionStart = regionInfo.getStartKey();
byte[] regionEnd = regionInfo.getEndKey();
// This region's start key should be one of the region boundaries
int startBoundaryIndex = indexOfBytes(expectedBounds, regionStart);
assertNotSame(-1, startBoundaryIndex);
// This region's start key should be one of the region boundaries
int startBoundaryIndex = indexOfBytes(expectedBounds, regionStart);
assertNotSame(-1, startBoundaryIndex);
// This region's end key should be the region boundary that comes
// after the starting boundary.
byte[] expectedRegionEnd = expectedBounds.get(
startBoundaryIndex+1);
assertEquals(0, Bytes.compareTo(regionEnd, expectedRegionEnd));
// This region's end key should be the region boundary that comes
// after the starting boundary.
byte[] expectedRegionEnd = expectedBounds.get(startBoundaryIndex + 1);
assertEquals(0, Bytes.compareTo(regionEnd, expectedRegionEnd));
}
hTable.close();
}
}
/**

View File

@ -34,6 +34,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
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.NamespaceDescriptor;
@ -46,6 +47,7 @@ import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
@ -182,10 +184,14 @@ public class OfflineMetaRebuildTestCore {
HTableDescriptor htd = tbl.getTableDescriptor();
dumpMeta(htd);
Map<HRegionInfo, ServerName> hris = ((HTable)tbl).getRegionLocations();
for (Entry<HRegionInfo, ServerName> e : hris.entrySet()) {
HRegionInfo hri = e.getKey();
ServerName hsa = e.getValue();
List<HRegionLocation> regions;
try(RegionLocator rl = connection.getRegionLocator(tbl.getName())) {
regions = rl.getAllRegionLocations();
}
for (HRegionLocation e : regions) {
HRegionInfo hri = e.getRegionInfo();
ServerName hsa = e.getServerName();
if (Bytes.compareTo(hri.getStartKey(), startKey) == 0
&& Bytes.compareTo(hri.getEndKey(), endKey) == 0) {