HBASE-12422 Use ConnectionFactory in HTable constructors Fix method signature in test

This commit is contained in:
stack 2014-12-13 21:38:34 -08:00
parent 743d16138e
commit 029768287a
7 changed files with 173 additions and 164 deletions

View File

@ -178,7 +178,7 @@ public class HTable implements HTableInterface, RegionLocator {
this.connection = null;
return;
}
this.connection = ConnectionManager.getConnectionInternal(conf);
this.connection = (ClusterConnection) ConnectionFactory.createConnection(conf);
this.configuration = conf;
this.pool = getDefaultExecutor(conf);
@ -250,7 +250,7 @@ public class HTable implements HTableInterface, RegionLocator {
@Deprecated
public HTable(Configuration conf, final TableName tableName, final ExecutorService pool)
throws IOException {
this.connection = ConnectionManager.getConnectionInternal(conf);
this.connection = (ClusterConnection) ConnectionFactory.createConnection(conf);
this.configuration = conf;
this.pool = pool;
if (pool == null) {

View File

@ -18,9 +18,6 @@
package org.apache.hadoop.hbase.client.coprocessor;
import static org.apache.hadoop.hbase.HConstants.EMPTY_START_ROW;
import static org.apache.hadoop.hbase.HConstants.LAST_ROW;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;

View File

@ -646,10 +646,11 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
if (!isSecureBulkLoadEndpointAvailable()) {
success = ProtobufUtil.bulkLoadHFile(getStub(), famPaths, regionName, assignSeqIds);
} else {
Table table = new HTable(conn.getConfiguration(), getTableName());
secureClient = new SecureBulkLoadClient(table);
success = secureClient.bulkLoadHFiles(famPaths, fsDelegationToken.getUserToken(),
bulkToken, getLocation().getRegionInfo().getStartKey());
try (Table table = conn.getTable(getTableName())) {
secureClient = new SecureBulkLoadClient(table);
success = secureClient.bulkLoadHFiles(famPaths, fsDelegationToken.getUserToken(),
bulkToken, getLocation().getRegionInfo().getStartKey());
}
}
return success;
} finally {

View File

@ -80,19 +80,22 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
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.ClusterConnection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnectable;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.MetaScanner;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.RowMutations;
import org.apache.hadoop.hbase.client.Table;
@ -106,7 +109,7 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService.Block
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
import org.apache.hadoop.hbase.wal.WALSplitter;
import org.apache.hadoop.hbase.security.AccessDeniedException;
import org.apache.hadoop.hbase.security.UserProvider;
import org.apache.hadoop.hbase.util.Bytes.ByteArrayComparator;
import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
@ -114,9 +117,9 @@ import org.apache.hadoop.hbase.util.hbck.HFileCorruptionChecker;
import org.apache.hadoop.hbase.util.hbck.TableIntegrityErrorHandler;
import org.apache.hadoop.hbase.util.hbck.TableIntegrityErrorHandlerImpl;
import org.apache.hadoop.hbase.util.hbck.TableLockChecker;
import org.apache.hadoop.hbase.wal.WALSplitter;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.hbase.security.AccessDeniedException;
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ipc.RemoteException;
@ -196,8 +199,8 @@ public class HBaseFsck extends Configured {
**********************/
private static final Log LOG = LogFactory.getLog(HBaseFsck.class.getName());
private ClusterStatus status;
private HConnection connection;
private HBaseAdmin admin;
private ClusterConnection connection;
private Admin admin;
private Table meta;
// threads to do ||izable tasks: retrieve data from regionservers, handle overlapping regions
protected ExecutorService executor;
@ -388,9 +391,9 @@ public class HBaseFsck extends Configured {
});
LOG.debug("Launching hbck");
connection = HConnectionManager.createConnection(getConf());
admin = new HBaseAdmin(connection);
meta = new HTable(TableName.META_TABLE_NAME, connection);
connection = (ClusterConnection)ConnectionFactory.createConnection(getConf());
admin = connection.getAdmin();
meta = connection.getTable(TableName.META_TABLE_NAME);
status = admin.getClusterStatus();
}

View File

@ -35,20 +35,20 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
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.ServerName;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.MetaTableAccessor;
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.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
@ -123,15 +123,17 @@ public class TestLoadIncrementalHFilesSplitRecovery {
* Creates a table with given table name and specified number of column
* families if the table does not already exist.
*/
private void setupTable(TableName table, int cfs) throws IOException {
private void setupTable(final Connection connection, TableName table, int cfs)
throws IOException {
try {
LOG.info("Creating table " + table);
HTableDescriptor htd = new HTableDescriptor(table);
for (int i = 0; i < cfs; i++) {
htd.addFamily(new HColumnDescriptor(family(i)));
}
util.getHBaseAdmin().createTable(htd);
try (Admin admin = connection.getAdmin()) {
admin.createTable(htd);
}
} catch (TableExistsException tee) {
LOG.info("Table " + table + " already exists");
}
@ -170,12 +172,14 @@ public class TestLoadIncrementalHFilesSplitRecovery {
/**
* Populate table with known values.
*/
private void populateTable(TableName table, int value) throws Exception {
private void populateTable(final Connection connection, TableName table, int value)
throws Exception {
// create HFiles for different column families
LoadIncrementalHFiles lih = new LoadIncrementalHFiles(util.getConfiguration());
Path bulk1 = buildBulkFiles(table, value);
HTable t = new HTable(util.getConfiguration(), table);
lih.doBulkLoad(bulk1, t);
try (Table t = connection.getTable(table)) {
lih.doBulkLoad(bulk1, (HTable)t);
}
}
/**
@ -265,20 +269,18 @@ public class TestLoadIncrementalHFilesSplitRecovery {
@Test(expected=IOException.class, timeout=120000)
public void testBulkLoadPhaseFailure() throws Exception {
TableName table = TableName.valueOf("bulkLoadPhaseFailure");
setupTable(table, 10);
final AtomicInteger attmptedCalls = new AtomicInteger();
final AtomicInteger failedCalls = new AtomicInteger();
util.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2);
try {
try (Connection connection = ConnectionFactory.createConnection(this.util.getConfiguration())) {
setupTable(connection, table, 10);
LoadIncrementalHFiles lih = new LoadIncrementalHFiles(util.getConfiguration()) {
protected List<LoadQueueItem> tryAtomicRegionLoad(final HConnection conn,
TableName tableName, final byte[] first, Collection<LoadQueueItem> lqis)
throws IOException {
throws IOException {
int i = attmptedCalls.incrementAndGet();
if (i == 1) {
HConnection errConn = null;
Connection errConn = null;
try {
errConn = getMockedConnection(util.getConfiguration());
} catch (Exception e) {
@ -286,23 +288,24 @@ public class TestLoadIncrementalHFilesSplitRecovery {
throw new RuntimeException("mocking cruft, should never happen");
}
failedCalls.incrementAndGet();
return super.tryAtomicRegionLoad(errConn, tableName, first, lqis);
return super.tryAtomicRegionLoad((HConnection)errConn, tableName, first, lqis);
}
return super.tryAtomicRegionLoad(conn, tableName, first, lqis);
return super.tryAtomicRegionLoad((HConnection)conn, tableName, first, lqis);
}
};
// create HFiles for different column families
Path dir = buildBulkFiles(table, 1);
HTable t = new HTable(util.getConfiguration(), table);
lih.doBulkLoad(dir, t);
} finally {
util.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
try {
// create HFiles for different column families
Path dir = buildBulkFiles(table, 1);
try (Table t = connection.getTable(table)) {
lih.doBulkLoad(dir, (HTable)t);
}
} finally {
util.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
}
fail("doBulkLoad should have thrown an exception");
}
fail("doBulkLoad should have thrown an exception");
}
@SuppressWarnings("deprecation")
@ -337,39 +340,39 @@ public class TestLoadIncrementalHFilesSplitRecovery {
@Test (timeout=120000)
public void testSplitWhileBulkLoadPhase() throws Exception {
final TableName table = TableName.valueOf("splitWhileBulkloadPhase");
setupTable(table, 10);
populateTable(table,1);
assertExpectedTable(table, ROWCOUNT, 1);
try (Connection connection = ConnectionFactory.createConnection(util.getConfiguration())) {
setupTable(connection, table, 10);
populateTable(connection, table,1);
assertExpectedTable(table, ROWCOUNT, 1);
// Now let's cause trouble. This will occur after checks and cause bulk
// files to fail when attempt to atomically import. This is recoverable.
final AtomicInteger attemptedCalls = new AtomicInteger();
LoadIncrementalHFiles lih2 = new LoadIncrementalHFiles(
util.getConfiguration()) {
protected void bulkLoadPhase(final Table htable, final HConnection conn,
ExecutorService pool, Deque<LoadQueueItem> queue,
final Multimap<ByteBuffer, LoadQueueItem> regionGroups) throws IOException {
int i = attemptedCalls.incrementAndGet();
if (i == 1) {
// On first attempt force a split.
forceSplit(table);
// Now let's cause trouble. This will occur after checks and cause bulk
// files to fail when attempt to atomically import. This is recoverable.
final AtomicInteger attemptedCalls = new AtomicInteger();
LoadIncrementalHFiles lih2 = new LoadIncrementalHFiles(util.getConfiguration()) {
protected void bulkLoadPhase(final Table htable, final HConnection conn,
ExecutorService pool, Deque<LoadQueueItem> queue,
final Multimap<ByteBuffer, LoadQueueItem> regionGroups) throws IOException {
int i = attemptedCalls.incrementAndGet();
if (i == 1) {
// On first attempt force a split.
forceSplit(table);
}
super.bulkLoadPhase(htable, conn, pool, queue, regionGroups);
}
};
super.bulkLoadPhase(htable, conn, pool, queue, regionGroups);
// create HFiles for different column families
try (Table t = connection.getTable(table)) {
Path bulk = buildBulkFiles(table, 2);
lih2.doBulkLoad(bulk, (HTable)t);
}
};
// create HFiles for different column families
HTable t = new HTable(util.getConfiguration(), table);
Path bulk = buildBulkFiles(table, 2);
lih2.doBulkLoad(bulk, t);
// check that data was loaded
// The three expected attempts are 1) failure because need to split, 2)
// load of split top 3) load of split bottom
assertEquals(attemptedCalls.get(), 3);
assertExpectedTable(table, ROWCOUNT, 2);
// check that data was loaded
// The three expected attempts are 1) failure because need to split, 2)
// load of split top 3) load of split bottom
assertEquals(attemptedCalls.get(), 3);
assertExpectedTable(table, ROWCOUNT, 2);
}
}
/**
@ -379,33 +382,35 @@ public class TestLoadIncrementalHFilesSplitRecovery {
@Test (timeout=120000)
public void testGroupOrSplitPresplit() throws Exception {
final TableName table = TableName.valueOf("groupOrSplitPresplit");
setupTable(table, 10);
populateTable(table, 1);
assertExpectedTable(table, ROWCOUNT, 1);
forceSplit(table);
try (Connection connection = ConnectionFactory.createConnection(util.getConfiguration())) {
setupTable(connection, table, 10);
populateTable(connection, table, 1);
assertExpectedTable(connection, table, ROWCOUNT, 1);
forceSplit(table);
final AtomicInteger countedLqis= new AtomicInteger();
LoadIncrementalHFiles lih = new LoadIncrementalHFiles(
util.getConfiguration()) {
protected List<LoadQueueItem> groupOrSplit(
Multimap<ByteBuffer, LoadQueueItem> regionGroups,
final LoadQueueItem item, final HTable htable,
final Pair<byte[][], byte[][]> startEndKeys) throws IOException {
List<LoadQueueItem> lqis = super.groupOrSplit(regionGroups, item, htable, startEndKeys);
if (lqis != null) {
countedLqis.addAndGet(lqis.size());
final AtomicInteger countedLqis= new AtomicInteger();
LoadIncrementalHFiles lih = new LoadIncrementalHFiles(
util.getConfiguration()) {
protected List<LoadQueueItem> groupOrSplit(
Multimap<ByteBuffer, LoadQueueItem> regionGroups,
final LoadQueueItem item, final HTable htable,
final Pair<byte[][], byte[][]> startEndKeys) throws IOException {
List<LoadQueueItem> lqis = super.groupOrSplit(regionGroups, item, htable, startEndKeys);
if (lqis != null) {
countedLqis.addAndGet(lqis.size());
}
return lqis;
}
return lqis;
};
// create HFiles for different column families
Path bulk = buildBulkFiles(table, 2);
try (Table t = connection.getTable(table)) {
lih.doBulkLoad(bulk, (HTable)t);
}
};
// create HFiles for different column families
Path bulk = buildBulkFiles(table, 2);
HTable ht = new HTable(util.getConfiguration(), table);
lih.doBulkLoad(bulk, ht);
assertExpectedTable(table, ROWCOUNT, 2);
assertEquals(20, countedLqis.get());
assertExpectedTable(connection, table, ROWCOUNT, 2);
assertEquals(20, countedLqis.get());
}
}
/**
@ -415,29 +420,32 @@ public class TestLoadIncrementalHFilesSplitRecovery {
@Test(expected = IOException.class, timeout=120000)
public void testGroupOrSplitFailure() throws Exception {
TableName table = TableName.valueOf("groupOrSplitFailure");
setupTable(table, 10);
try (Connection connection = ConnectionFactory.createConnection(util.getConfiguration())) {
setupTable(connection, table, 10);
LoadIncrementalHFiles lih = new LoadIncrementalHFiles(
util.getConfiguration()) {
int i = 0;
LoadIncrementalHFiles lih = new LoadIncrementalHFiles(
util.getConfiguration()) {
int i = 0;
protected List<LoadQueueItem> groupOrSplit(
Multimap<ByteBuffer, LoadQueueItem> regionGroups,
final LoadQueueItem item, final HTable table,
final Pair<byte[][], byte[][]> startEndKeys) throws IOException {
i++;
protected List<LoadQueueItem> groupOrSplit(
Multimap<ByteBuffer, LoadQueueItem> regionGroups,
final LoadQueueItem item, final HTable table,
final Pair<byte[][], byte[][]> startEndKeys) throws IOException {
i++;
if (i == 5) {
throw new IOException("failure");
if (i == 5) {
throw new IOException("failure");
}
return super.groupOrSplit(regionGroups, item, table, startEndKeys);
}
return super.groupOrSplit(regionGroups, item, table, startEndKeys);
}
};
};
// create HFiles for different column families
Path dir = buildBulkFiles(table,1);
HTable t = new HTable(util.getConfiguration(), table);
lih.doBulkLoad(dir, t);
// create HFiles for different column families
Path dir = buildBulkFiles(table,1);
try (Table t = connection.getTable(table)) {
lih.doBulkLoad(dir, (HTable)t);
}
}
fail("doBulkLoad should have thrown an exception");
}

View File

@ -32,7 +32,7 @@ import org.junit.experimental.categories.Category;
/**
* Reruns TestSecureLoadIncrementalHFilesSplitRecovery
* using LoadIncrementalHFiles in secure mode.
* This suite is unable to verify the security handoff/turnover
* This suite is unable to verify the security handoff/turnove
* as miniCluster is running as system user thus has root privileges
* and delegation tokens don't seem to work on miniDFS.
*
@ -62,9 +62,8 @@ public class TestSecureLoadIncrementalHFilesSplitRecovery extends TestLoadIncrem
}
//Disabling this test as it does not work in secure mode
@Test
@Test (timeout=180000)
@Override
public void testBulkLoadPhaseFailure() {
}
}
}

View File

@ -162,7 +162,7 @@ public class TestHBaseFsck {
TEST_UTIL.shutdownMiniCluster();
}
@Test
@Test (timeout=180000)
public void testHBaseFsck() throws Exception {
assertNoErrors(doFsck(conf, false));
TableName table = TableName.valueOf("tableBadMetaAssign");
@ -350,9 +350,10 @@ public class TestHBaseFsck {
}
if (metaRow) {
Table meta = new HTable(conf, TableName.META_TABLE_NAME, executorService);
Delete delete = new Delete(deleteRow);
meta.delete(delete);
try (Table meta = conn.getTable(TableName.META_TABLE_NAME, executorService)) {
Delete delete = new Delete(deleteRow);
meta.delete(delete);
}
}
}
LOG.info(hri.toString() + hsa.toString());
@ -449,7 +450,7 @@ public class TestHBaseFsck {
/**
* This creates a clean table and confirms that the table is clean.
*/
@Test
@Test (timeout=180000)
public void testHBaseFsckClean() throws Exception {
assertNoErrors(doFsck(conf, false));
TableName table = TableName.valueOf("tableClean");
@ -473,7 +474,7 @@ public class TestHBaseFsck {
/**
* Test thread pooling in the case where there are more regions than threads
*/
@Test
@Test (timeout=180000)
public void testHbckThreadpooling() throws Exception {
TableName table =
TableName.valueOf("tableDupeStartKey");
@ -492,7 +493,7 @@ public class TestHBaseFsck {
}
}
@Test
@Test (timeout=180000)
public void testHbckFixOrphanTable() throws Exception {
TableName table = TableName.valueOf("tableInfo");
FileSystem fs = null;
@ -546,7 +547,7 @@ public class TestHBaseFsck {
*
* @throws Exception
*/
@Test
@Test (timeout=180000)
public void testParallelHbck() throws Exception {
final ExecutorService service;
final Future<HBaseFsck> hbck1,hbck2;
@ -589,7 +590,7 @@ public class TestHBaseFsck {
* This create and fixes a bad table with regions that have a duplicate
* start key
*/
@Test
@Test (timeout=180000)
public void testDupeStartKey() throws Exception {
TableName table =
TableName.valueOf("tableDupeStartKey");
@ -630,7 +631,7 @@ public class TestHBaseFsck {
* This creates a table with region_replica > 1 and verifies hbck runs
* successfully
*/
@Test
@Test (timeout=180000)
public void testHbckWithRegionReplica() throws Exception {
TableName table =
TableName.valueOf("testHbckWithRegionReplica");
@ -643,7 +644,7 @@ public class TestHBaseFsck {
}
}
@Test
@Test (timeout=180000)
public void testHbckWithFewerReplica() throws Exception {
TableName table =
TableName.valueOf("testHbckWithFewerReplica");
@ -667,7 +668,7 @@ public class TestHBaseFsck {
}
}
@Test
@Test (timeout=180000)
public void testHbckWithExcessReplica() throws Exception {
TableName table =
TableName.valueOf("testHbckWithExcessReplica");
@ -765,7 +766,7 @@ public class TestHBaseFsck {
* This create and fixes a bad table with regions that have a duplicate
* start key
*/
@Test
@Test (timeout=180000)
public void testDupeRegion() throws Exception {
TableName table =
TableName.valueOf("tableDupeRegion");
@ -818,7 +819,7 @@ public class TestHBaseFsck {
/**
* This creates and fixes a bad table with regions that has startkey == endkey
*/
@Test
@Test (timeout=180000)
public void testDegenerateRegions() throws Exception {
TableName table = TableName.valueOf("tableDegenerateRegions");
try {
@ -858,7 +859,7 @@ public class TestHBaseFsck {
* This creates and fixes a bad table where a region is completely contained
* by another region.
*/
@Test
@Test (timeout=180000)
public void testContainedRegionOverlap() throws Exception {
TableName table =
TableName.valueOf("tableContainedRegionOverlap");
@ -900,7 +901,7 @@ public class TestHBaseFsck {
* region. Mess around the meta data so that closeRegion/offlineRegion
* throws exceptions.
*/
@Test
@Test (timeout=180000)
public void testSidelineOverlapRegion() throws Exception {
TableName table =
TableName.valueOf("testSidelineOverlapRegion");
@ -991,7 +992,7 @@ public class TestHBaseFsck {
* This creates and fixes a bad table where a region is completely contained
* by another region, and there is a hole (sort of like a bad split)
*/
@Test
@Test (timeout=180000)
public void testOverlapAndOrphan() throws Exception {
TableName table =
TableName.valueOf("tableOverlapAndOrphan");
@ -1036,7 +1037,7 @@ public class TestHBaseFsck {
* a start key contained in another region and its end key is contained in
* yet another region.
*/
@Test
@Test (timeout=180000)
public void testCoveredStartKey() throws Exception {
TableName table =
TableName.valueOf("tableCoveredStartKey");
@ -1077,7 +1078,7 @@ public class TestHBaseFsck {
* This creates and fixes a bad table with a missing region -- hole in meta
* and data missing in the fs.
*/
@Test
@Test (timeout=180000)
public void testRegionHole() throws Exception {
TableName table =
TableName.valueOf("tableRegionHole");
@ -1112,7 +1113,7 @@ public class TestHBaseFsck {
* This creates and fixes a bad table with a missing region -- hole in meta
* and data present but .regioinfino missing (an orphan hdfs region)in the fs.
*/
@Test
@Test (timeout=180000)
public void testHDFSRegioninfoMissing() throws Exception {
TableName table =
TableName.valueOf("tableHDFSRegioininfoMissing");
@ -1149,7 +1150,7 @@ public class TestHBaseFsck {
* This creates and fixes a bad table with a region that is missing meta and
* not assigned to a region server.
*/
@Test
@Test (timeout=180000)
public void testNotInMetaOrDeployedHole() throws Exception {
TableName table =
TableName.valueOf("tableNotInMetaOrDeployedHole");
@ -1184,7 +1185,7 @@ public class TestHBaseFsck {
/**
* This creates fixes a bad table with a hole in meta.
*/
@Test
@Test (timeout=180000)
public void testNotInMetaHole() throws Exception {
TableName table =
TableName.valueOf("tableNotInMetaHole");
@ -1220,7 +1221,7 @@ public class TestHBaseFsck {
* This creates and fixes a bad table with a region that is in meta but has
* no deployment or data hdfs
*/
@Test
@Test (timeout=180000)
public void testNotInHdfs() throws Exception {
TableName table =
TableName.valueOf("tableNotInHdfs");
@ -1255,7 +1256,7 @@ public class TestHBaseFsck {
* This creates and fixes a bad table with a region that is in meta but has
* no deployment or data hdfs. The table has region_replication set to 2.
*/
@Test
@Test (timeout=180000)
public void testNotInHdfsWithReplicas() throws Exception {
TableName table =
TableName.valueOf("tableNotInHdfs");
@ -1328,7 +1329,7 @@ public class TestHBaseFsck {
* This creates entries in hbase:meta with no hdfs data. This should cleanly
* remove the table.
*/
@Test
@Test (timeout=180000)
public void testNoHdfsTable() throws Exception {
TableName table = TableName.valueOf("NoHdfsTable");
setupTable(table);
@ -1378,7 +1379,7 @@ public class TestHBaseFsck {
/**
* when the hbase.version file missing, It is fix the fault.
*/
@Test
@Test (timeout=180000)
public void testNoVersionFile() throws Exception {
// delete the hbase.version file
Path rootDir = FSUtils.getRootDir(conf);
@ -1399,7 +1400,7 @@ public class TestHBaseFsck {
/**
* The region is not deployed when the table is disabled.
*/
@Test
@Test (timeout=180000)
public void testRegionShouldNotBeDeployed() throws Exception {
TableName table =
TableName.valueOf("tableRegionShouldNotBeDeployed");
@ -1460,7 +1461,7 @@ public class TestHBaseFsck {
/**
* This creates two tables and mess both of them and fix them one by one
*/
@Test
@Test (timeout=180000)
public void testFixByTable() throws Exception {
TableName table1 =
TableName.valueOf("testFixByTable1");
@ -1506,7 +1507,7 @@ public class TestHBaseFsck {
/**
* A split parent in meta, in hdfs, and not deployed
*/
@Test
@Test (timeout=180000)
public void testLingeringSplitParent() throws Exception {
TableName table =
TableName.valueOf("testLingeringSplitParent");
@ -1586,7 +1587,7 @@ public class TestHBaseFsck {
* Tests that LINGERING_SPLIT_PARENT is not erroneously reported for
* valid cases where the daughters are there.
*/
@Test
@Test (timeout=180000)
public void testValidLingeringSplitParent() throws Exception {
TableName table =
TableName.valueOf("testLingeringSplitParent");
@ -1705,8 +1706,7 @@ public class TestHBaseFsck {
*/
@Test(timeout=120000)
public void testMissingFirstRegion() throws Exception {
TableName table =
TableName.valueOf("testMissingFirstRegion");
TableName table = TableName.valueOf("testMissingFirstRegion");
try {
setupTable(table);
assertEquals(ROWKEYS.length, countRows());
@ -1788,7 +1788,7 @@ public class TestHBaseFsck {
/**
* Test -noHdfsChecking option can detect and fix assignments issue.
*/
@Test
@Test (timeout=180000)
public void testFixAssignmentsAndNoHdfsChecking() throws Exception {
TableName table =
TableName.valueOf("testFixAssignmentsAndNoHdfsChecking");
@ -1838,7 +1838,7 @@ public class TestHBaseFsck {
* However, it can not fix it without checking Hdfs because we need to get
* the region info from Hdfs in this case, then to patch the meta.
*/
@Test
@Test (timeout=180000)
public void testFixMetaNotWorkingWithNoHdfsChecking() throws Exception {
TableName table =
TableName.valueOf("testFixMetaNotWorkingWithNoHdfsChecking");
@ -1892,7 +1892,7 @@ public class TestHBaseFsck {
* Test -fixHdfsHoles doesn't work with -noHdfsChecking option,
* and -noHdfsChecking can't detect orphan Hdfs region.
*/
@Test
@Test (timeout=180000)
public void testFixHdfsHolesNotWorkingWithNoHdfsChecking() throws Exception {
TableName table =
TableName.valueOf("testFixHdfsHolesNotWorkingWithNoHdfsChecking");
@ -2128,7 +2128,8 @@ public class TestHBaseFsck {
final FileSystem fs = FileSystem.get(conf);
HBaseFsck hbck = new HBaseFsck(conf, exec) {
@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
@ -2147,7 +2148,7 @@ public class TestHBaseFsck {
/**
* Test fixing lingering reference file.
*/
@Test
@Test (timeout=180000)
public void testLingeringReferenceFile() throws Exception {
TableName table =
TableName.valueOf("testLingeringReferenceFile");
@ -2177,7 +2178,7 @@ public class TestHBaseFsck {
/**
* Test mission REGIONINFO_QUALIFIER in hbase:meta
*/
@Test
@Test (timeout=180000)
public void testMissingRegionInfoQualifier() throws Exception {
TableName table =
TableName.valueOf("testMissingRegionInfoQualifier");
@ -2231,7 +2232,7 @@ public class TestHBaseFsck {
* Test pluggable error reporter. It can be plugged in
* from system property or configuration.
*/
@Test
@Test (timeout=180000)
public void testErrorReporter() throws Exception {
try {
MockErrorReporter.calledCount = 0;
@ -2395,7 +2396,7 @@ public class TestHBaseFsck {
writeLock.release(); // release for clean state
}
@Test
@Test (timeout=180000)
public void testMetaOffline() throws Exception {
// check no errors
HBaseFsck hbck = doFsck(conf, false);
@ -2449,7 +2450,7 @@ public class TestHBaseFsck {
}
}
@Test
@Test (timeout=180000)
public void testTableWithNoRegions() throws Exception {
// We might end up with empty regions in a table
// see also testNoHdfsTable()
@ -2483,7 +2484,7 @@ public class TestHBaseFsck {
}
@Test
@Test (timeout=180000)
public void testHbckAfterRegionMerge() throws Exception {
TableName table = TableName.valueOf("testMergeRegionFilesInHdfs");
Table meta = null;
@ -2531,7 +2532,7 @@ public class TestHBaseFsck {
}
}
@Test
@Test (timeout=180000)
public void testRegionBoundariesCheck() throws Exception {
HBaseFsck hbck = doFsck(conf, false);
assertNoErrors(hbck); // no errors
@ -2547,7 +2548,7 @@ public class TestHBaseFsck {
@org.junit.Rule
public TestName name = new TestName();
@Test
@Test (timeout=180000)
public void testReadOnlyProperty() throws Exception {
HBaseFsck hbck = doFsck(conf, false);
Assert.assertEquals("shouldIgnorePreCheckPermission", true,