HBASE-19951 Cleanup the explicit timeout value for test method

This commit is contained in:
Michael Stack 2018-02-07 11:55:18 -08:00
parent 7d449892af
commit 8f1e01b6e5
174 changed files with 674 additions and 671 deletions

View File

@ -586,7 +586,7 @@ public class TestAsyncProcess {
doSubmitRequest(maxHeapSizePerRequest, putsHeapSize);
}
@Test(timeout=300000)
@Test
public void testSubmitRandomSizeRequest() throws Exception {
Random rn = new Random();
final long limit = 10 * 1024 * 1024;
@ -611,7 +611,7 @@ public class TestAsyncProcess {
doSubmitRequest(maxHeapSizePerRequest, putsHeapSize);
}
@Test(timeout=120000)
@Test
public void testSubmitLargeRequest() throws Exception {
long maxHeapSizePerRequest = 2 * 1024 * 1024;
long putsHeapSize = maxHeapSizePerRequest * 2;

View File

@ -462,7 +462,7 @@ public class TestClientScanner {
* Tests the case where all replicas of a region throw an exception. It should not cause a hang
* but the exception should propagate to the client
*/
@Test (timeout = 30000)
@Test
public void testExceptionsFromReplicasArePropagated() throws IOException {
scan.setConsistency(Consistency.TIMELINE);

View File

@ -105,7 +105,7 @@ public class TestProcedureFuture {
* we are skipping the waitOperationResult() call,
* since we are getting the procedure result.
*/
@Test(timeout=60000)
@Test
public void testWithProcId() throws Exception {
HBaseAdmin admin = Mockito.mock(HBaseAdmin.class);
TestFuture f = new TestFuture(admin, 100L);
@ -120,7 +120,7 @@ public class TestProcedureFuture {
/**
* Verify that the spin loop for the procedure running works.
*/
@Test(timeout=60000)
@Test
public void testWithProcIdAndSpinning() throws Exception {
final AtomicInteger spinCount = new AtomicInteger(0);
HBaseAdmin admin = Mockito.mock(HBaseAdmin.class);
@ -147,7 +147,7 @@ public class TestProcedureFuture {
* When a master return a result without procId,
* we are skipping the getProcedureResult() call.
*/
@Test(timeout=60000)
@Test
public void testWithoutProcId() throws Exception {
HBaseAdmin admin = Mockito.mock(HBaseAdmin.class);
TestFuture f = new TestFuture(admin, null);
@ -167,7 +167,7 @@ public class TestProcedureFuture {
* This happens when the operation calls happens on a "new master" but while we are waiting
* the operation to be completed, we failover on an "old master".
*/
@Test(timeout=60000)
@Test
public void testOnServerWithNoProcedureSupport() throws Exception {
HBaseAdmin admin = Mockito.mock(HBaseAdmin.class);
TestFuture f = new TestFuture(admin, 100L) {

View File

@ -64,7 +64,7 @@ public class TestSnapshotFromAdmin {
* passed from the server ensures the correct overall waiting for the snapshot to finish.
* @throws Exception
*/
@Test(timeout = 60000)
@Test
public void testBackoffLogic() throws Exception {
final int pauseTime = 100;
final int maxWaitTime =

View File

@ -243,7 +243,7 @@ public class TestChoreService {
}
}
@Test (timeout=20000)
@Test
public void testInitialChorePrecedence() throws InterruptedException {
ChoreService service = new ChoreService("testInitialChorePrecedence");
@ -272,7 +272,7 @@ public class TestChoreService {
}
}
@Test (timeout=20000)
@Test
public void testCancelChore() throws InterruptedException {
final int period = 100;
ScheduledChore chore1 = new DoNothingChore("chore1", period);
@ -289,7 +289,7 @@ public class TestChoreService {
}
}
@Test (timeout=20000)
@Test
public void testScheduledChoreConstruction() {
final String NAME = "chore";
final int PERIOD = 100;
@ -322,7 +322,7 @@ public class TestChoreService {
invalidDelayChore.getInitialDelay());
}
@Test (timeout=20000)
@Test
public void testChoreServiceConstruction() throws InterruptedException {
final int corePoolSize = 10;
final int defaultCorePoolSize = ChoreService.MIN_CORE_POOL_SIZE;
@ -350,7 +350,7 @@ public class TestChoreService {
}
}
@Test (timeout=20000)
@Test
public void testFrequencyOfChores() throws InterruptedException {
final int period = 100;
// Small delta that acts as time buffer (allowing chores to complete if running slowly)
@ -377,7 +377,7 @@ public class TestChoreService {
}
}
@Test (timeout=20000)
@Test
public void testForceTrigger() throws InterruptedException {
final int period = 100;
final int delta = 10;
@ -413,7 +413,7 @@ public class TestChoreService {
}
}
@Test (timeout=20000)
@Test
public void testCorePoolIncrease() throws InterruptedException {
final int initialCorePoolSize = 3;
ChoreService service = new ChoreService("testCorePoolIncrease", initialCorePoolSize, false);
@ -453,7 +453,7 @@ public class TestChoreService {
}
}
@Test(timeout = 30000)
@Test
public void testCorePoolDecrease() throws InterruptedException {
final int initialCorePoolSize = 3;
ChoreService service = new ChoreService("testCorePoolDecrease", initialCorePoolSize, false);
@ -522,7 +522,7 @@ public class TestChoreService {
}
}
@Test (timeout=20000)
@Test
public void testNumberOfRunningChores() throws InterruptedException {
ChoreService service = new ChoreService("testNumberOfRunningChores");
@ -563,7 +563,7 @@ public class TestChoreService {
}
}
@Test (timeout=20000)
@Test
public void testNumberOfChoresMissingStartTime() throws InterruptedException {
ChoreService service = new ChoreService("testNumberOfChoresMissingStartTime");
@ -611,7 +611,7 @@ public class TestChoreService {
* been scheduled with the service. For example, if 4 ScheduledChores are scheduled with a
* ChoreService, the number of threads in the ChoreService's core pool should never exceed 4
*/
@Test (timeout=20000)
@Test
public void testMaximumChoreServiceThreads() throws InterruptedException {
ChoreService service = new ChoreService("testMaximumChoreServiceThreads");
@ -658,7 +658,7 @@ public class TestChoreService {
}
}
@Test (timeout=20000)
@Test
public void testChangingChoreServices() throws InterruptedException {
final int period = 100;
final int sleepTime = 10;
@ -697,7 +697,7 @@ public class TestChoreService {
}
}
@Test (timeout=20000)
@Test
public void testStopperForScheduledChores() throws InterruptedException {
ChoreService service = new ChoreService("testStopperForScheduledChores");
Stoppable stopperForGroup1 = new SampleStopper();
@ -752,7 +752,7 @@ public class TestChoreService {
}
}
@Test (timeout=20000)
@Test
public void testShutdownCancelsScheduledChores() throws InterruptedException {
final int period = 100;
ChoreService service = new ChoreService("testShutdownCancelsScheduledChores");
@ -776,7 +776,7 @@ public class TestChoreService {
assertFalse(successChore3.isScheduled());
}
@Test (timeout=20000)
@Test
public void testShutdownWorksWhileChoresAreExecuting() throws InterruptedException {
final int period = 100;
final int sleep = 5 * period;
@ -804,7 +804,7 @@ public class TestChoreService {
}
}
@Test (timeout=20000)
@Test
public void testShutdownRejectsNewSchedules() throws InterruptedException {
final int period = 100;
ChoreService service = new ChoreService("testShutdownRejectsNewSchedules");

View File

@ -88,7 +88,7 @@ public class TestKeyStoreKeyProvider {
}
}
@Test(timeout=30000)
@Test
public void testKeyStoreKeyProviderWithPassword() throws Exception {
KeyProvider provider = new KeyStoreKeyProvider();
provider.init("jceks://" + storeFile.toURI().getPath() + "?password=" + PASSWORD);
@ -101,7 +101,7 @@ public class TestKeyStoreKeyProvider {
}
}
@Test(timeout=30000)
@Test
public void testKeyStoreKeyProviderWithPasswordFile() throws Exception {
KeyProvider provider = new KeyStoreKeyProvider();
provider.init("jceks://" + storeFile.toURI().getPath() + "?passwordFile=" +

View File

@ -43,7 +43,7 @@ public class TestThreads {
private final AtomicBoolean wasInterrupted = new AtomicBoolean(false);
@Test(timeout=60000)
@Test
public void testSleepWithoutInterrupt() throws InterruptedException {
Thread sleeper = new Thread(new Runnable() {
@Override

View File

@ -52,13 +52,13 @@ public class TestHttpServerLifecycle extends HttpServerFunctionalTest {
*
* @throws Throwable on failure
*/
@Ignore ("Hangs on occasion; see HBASE-14430") @Test(timeout=60000)
@Ignore ("Hangs on occasion; see HBASE-14430") @Test
public void testCreatedServerIsNotAlive() throws Throwable {
HttpServer server = createTestServer();
assertNotLive(server);
}
@Ignore ("Hangs on occasion; see HBASE-14430") @Test(timeout=60000)
@Ignore ("Hangs on occasion; see HBASE-14430") @Test
public void testStopUnstartedServer() throws Throwable {
HttpServer server = createTestServer();
stop(server);
@ -69,7 +69,7 @@ public class TestHttpServerLifecycle extends HttpServerFunctionalTest {
*
* @throws Throwable on failure
*/
@Ignore ("Hangs on occasion; see HBASE-14430") @Test(timeout=60000)
@Ignore ("Hangs on occasion; see HBASE-14430") @Test
public void testStartedServerIsAlive() throws Throwable {
HttpServer server = null;
server = createTestServer();
@ -95,7 +95,7 @@ public class TestHttpServerLifecycle extends HttpServerFunctionalTest {
*
* @throws Throwable on failure
*/
@Ignore ("Hangs on occasion; see HBASE-14430") @Test(timeout=60000)
@Ignore ("Hangs on occasion; see HBASE-14430") @Test
public void testStoppedServerIsNotAlive() throws Throwable {
HttpServer server = createAndStartTestServer();
assertAlive(server);
@ -108,7 +108,7 @@ public class TestHttpServerLifecycle extends HttpServerFunctionalTest {
*
* @throws Throwable on failure
*/
@Ignore ("Hangs on occasion; see HBASE-14430") @Test(timeout=60000)
@Ignore ("Hangs on occasion; see HBASE-14430") @Test
public void testStoppingTwiceServerIsAllowed() throws Throwable {
HttpServer server = createAndStartTestServer();
assertAlive(server);
@ -124,7 +124,7 @@ public class TestHttpServerLifecycle extends HttpServerFunctionalTest {
* @throws Throwable
* on failure
*/
@Ignore ("Hangs on occasion; see HBASE-14430") @Test(timeout=60000)
@Ignore ("Hangs on occasion; see HBASE-14430") @Test
public void testWepAppContextAfterServerStop() throws Throwable {
HttpServer server = null;
String key = "test.attribute.key";

View File

@ -47,7 +47,7 @@ public class TestLogLevel {
static final PrintStream out = System.out;
@Test (timeout=60000)
@Test
@SuppressWarnings("deprecation")
public void testDynamicLogLevel() throws Exception {
String logName = TestLogLevel.class.getName();

View File

@ -293,7 +293,7 @@ public class IntegrationTestRpcClient {
Test that not started connections are successfully removed from connection pool when
rpc client is closing.
*/
@Test (timeout = 30000)
@Test
public void testRpcWithWriteThread() throws IOException, InterruptedException {
LOG.info("Starting test");
Cluster cluster = new Cluster(1, 1);
@ -312,7 +312,7 @@ public class IntegrationTestRpcClient {
}
@Test (timeout = 1800000)
@Test
public void testRpcWithChaosMonkeyWithSyncClient() throws Throwable {
for (int i = 0; i < numIterations; i++) {
TimeoutThread.runWithTimeout(new Callable<Void>() {
@ -333,7 +333,7 @@ public class IntegrationTestRpcClient {
}
}
@Test (timeout = 900000)
@Test
@Ignore // TODO: test fails with async client
public void testRpcWithChaosMonkeyWithAsyncClient() throws Throwable {
for (int i = 0; i < numIterations; i++) {

View File

@ -86,7 +86,7 @@ public class TestCellCounter {
* Test CellCounter all data should print to output
*
*/
@Test (timeout=300000)
@Test
public void testCellCounter() throws Exception {
final TableName sourceTable = TableName.valueOf(name.getMethodName());
byte[][] families = { FAMILY_A, FAMILY_B };
@ -124,7 +124,7 @@ public class TestCellCounter {
/**
* Test CellCounter all data should print to output
*/
@Test(timeout = 300000)
@Test
public void testCellCounterPrefix() throws Exception {
final TableName sourceTable = TableName.valueOf(name.getMethodName());
byte[][] families = { FAMILY_A, FAMILY_B };
@ -162,7 +162,7 @@ public class TestCellCounter {
/**
* Test CellCounter with time range all data should print to output
*/
@Test (timeout=300000)
@Test
public void testCellCounterStartTimeRange() throws Exception {
final TableName sourceTable = TableName.valueOf(name.getMethodName());
byte[][] families = { FAMILY_A, FAMILY_B };
@ -203,7 +203,7 @@ public class TestCellCounter {
/**
* Test CellCounter with time range all data should print to output
*/
@Test (timeout=300000)
@Test
public void testCellCounteEndTimeRange() throws Exception {
final TableName sourceTable = TableName.valueOf(name.getMethodName());
byte[][] families = { FAMILY_A, FAMILY_B };
@ -243,7 +243,7 @@ public class TestCellCounter {
/**
* Test CellCounter with time range all data should print to output
*/
@Test (timeout=300000)
@Test
public void testCellCounteOutOfTimeRange() throws Exception {
final TableName sourceTable = TableName.valueOf(name.getMethodName());
byte[][] families = { FAMILY_A, FAMILY_B };
@ -288,7 +288,7 @@ public class TestCellCounter {
/**
* Test main method of CellCounter
*/
@Test (timeout=300000)
@Test
public void testCellCounterMain() throws Exception {
PrintStream oldPrintStream = System.err;
@ -320,7 +320,7 @@ public class TestCellCounter {
/**
* Test CellCounter for complete table all data should print to output
*/
@Test(timeout = 600000)
@Test
public void testCellCounterForCompleteTable() throws Exception {
final TableName sourceTable = TableName.valueOf(name.getMethodName());
String outputPath = OUTPUT_DIR + sourceTable;

View File

@ -60,7 +60,7 @@ public class TestHRegionPartitioner {
/**
* Test HRegionPartitioner
*/
@Test (timeout=300000)
@Test
public void testHRegionPartitioner() throws Exception {
byte[][] families = { Bytes.toBytes("familyA"), Bytes.toBytes("familyB") };

View File

@ -80,7 +80,7 @@ public class TestProcedureEvents {
fs.delete(logDir, true);
}
@Test(timeout=30000)
@Test
public void testTimeoutEventProcedure() throws Exception {
final int NTIMEOUTS = 5;
@ -92,12 +92,12 @@ public class TestProcedureEvents {
assertEquals(NTIMEOUTS + 1, proc.getTimeoutsCount());
}
@Test(timeout=30000)
@Test
public void testTimeoutEventProcedureDoubleExecution() throws Exception {
testTimeoutEventProcedureDoubleExecution(false);
}
@Test(timeout=30000)
@Test
public void testTimeoutEventProcedureDoubleExecutionKillIfSuspended() throws Exception {
testTimeoutEventProcedureDoubleExecution(true);
}

View File

@ -133,7 +133,7 @@ public class TestProcedureExecution {
}
}
@Test(timeout=30000)
@Test
public void testBadSubprocList() {
List<String> state = new ArrayList<>();
Procedure subProc2 = new TestSequentialProcedure("subProc2", state);
@ -155,7 +155,7 @@ public class TestProcedureExecution {
assertEquals("rootProc-rollback", state.get(3));
}
@Test(timeout=30000)
@Test
public void testSingleSequentialProc() {
List<String> state = new ArrayList<>();
Procedure subProc2 = new TestSequentialProcedure("subProc2", state);
@ -170,7 +170,7 @@ public class TestProcedureExecution {
assertEquals(state.toString(), 3, state.size());
}
@Test(timeout=30000)
@Test
public void testSingleSequentialProcRollback() {
List<String> state = new ArrayList<>();
Procedure subProc2 = new TestSequentialProcedure("subProc2", state,
@ -221,7 +221,7 @@ public class TestProcedureExecution {
protected boolean abort(Void env) { return false; }
}
@Test(timeout=30000)
@Test
public void testRollbackRetriableFailure() {
long procId = ProcedureTestingUtility.submitAndWait(procExecutor, new TestFaultyRollback());
@ -298,7 +298,7 @@ public class TestProcedureExecution {
}
}
@Test(timeout=30000)
@Test
public void testAbortTimeout() {
final int PROC_TIMEOUT_MSEC = 2500;
List<String> state = new ArrayList<>();
@ -317,7 +317,7 @@ public class TestProcedureExecution {
assertEquals("wproc-rollback", state.get(1));
}
@Test(timeout=30000)
@Test
public void testAbortTimeoutWithChildren() {
List<String> state = new ArrayList<>();
Procedure proc = new TestWaitingProcedure("wproc", state, true);

View File

@ -75,7 +75,7 @@ public class TestProcedureExecutor {
procExecutor.start(numThreads, true);
}
@Test(timeout=60000)
@Test
public void testWorkerStuck() throws Exception {
// replace the executor
final Configuration conf = new Configuration(htu.getConfiguration());

View File

@ -85,7 +85,7 @@ public class TestProcedureNonce {
fs.delete(logDir, true);
}
@Test(timeout=30000)
@Test
public void testCompletedProcWithSameNonce() throws Exception {
final long nonceGroup = 123;
final long nonce = 2222;
@ -111,7 +111,7 @@ public class TestProcedureNonce {
ProcedureTestingUtility.assertProcNotFailed(result);
}
@Test(timeout=30000)
@Test
public void testRunningProcWithSameNonce() throws Exception {
final long nonceGroup = 456;
final long nonce = 33333;
@ -163,12 +163,12 @@ public class TestProcedureNonce {
ProcedureTestingUtility.assertProcFailed(result);
}
@Test(timeout=30000)
@Test
public void testConcurrentNonceRegistration() throws IOException {
testConcurrentNonceRegistration(true, 567, 44444);
}
@Test(timeout=30000)
@Test
public void testConcurrentNonceRegistrationWithRollback() throws IOException {
testConcurrentNonceRegistration(false, 890, 55555);
}

View File

@ -190,7 +190,7 @@ public class TestProcedureRecovery {
restart();
}
@Test(timeout=30000)
@Test
public void testSingleStepProcRecovery() throws Exception {
Procedure proc = new TestSingleStepProcedure();
procExecutor.testing.killBeforeStoreUpdate = true;
@ -216,7 +216,7 @@ public class TestProcedureRecovery {
assertEquals(1, Bytes.toInt(result.getResult()));
}
@Test(timeout=30000)
@Test
public void testMultiStepProcRecovery() throws Exception {
// Step 0 - kill
Procedure proc = new TestMultiStepProcedure();
@ -245,7 +245,7 @@ public class TestProcedureRecovery {
ProcedureTestingUtility.assertProcNotFailed(result);
}
@Test(timeout=30000)
@Test
public void testMultiStepRollbackRecovery() throws Exception {
// Step 0 - kill
Procedure proc = new TestMultiStepProcedure();
@ -403,7 +403,7 @@ public class TestProcedureRecovery {
}
}
@Test(timeout=30000)
@Test
public void testStateMachineMultipleLevel() throws Exception {
long procId = procExecutor.submitProcedure(new TestStateMachineProcedure(true));
// Wait the completion
@ -414,7 +414,7 @@ public class TestProcedureRecovery {
assertEquals(4, procExecutor.getLastProcId());
}
@Test(timeout=30000)
@Test
public void testStateMachineRecovery() throws Exception {
ProcedureTestingUtility.setToggleKillBeforeStoreUpdate(procExecutor, true);
ProcedureTestingUtility.setKillBeforeStoreUpdate(procExecutor, true);
@ -452,7 +452,7 @@ public class TestProcedureRecovery {
assertEquals(26, Bytes.toInt(result.getResult()));
}
@Test(timeout=30000)
@Test
public void testStateMachineRollbackRecovery() throws Exception {
ProcedureTestingUtility.setToggleKillBeforeStoreUpdate(procExecutor, true);
ProcedureTestingUtility.setKillBeforeStoreUpdate(procExecutor, true);

View File

@ -86,7 +86,7 @@ public class TestProcedureReplayOrder {
fs.delete(logDir, true);
}
@Test(timeout=90000)
@Test
public void testSingleStepReplayOrder() throws Exception {
final int NUM_PROC_XTHREAD = 32;
final int NUM_PROCS = NUM_THREADS * NUM_PROC_XTHREAD;
@ -107,7 +107,7 @@ public class TestProcedureReplayOrder {
procEnv.assertSortedExecList(NUM_PROCS);
}
@Test(timeout=90000)
@Test
public void testMultiStepReplayOrder() throws Exception {
final int NUM_PROC_XTHREAD = 24;
final int NUM_PROCS = NUM_THREADS * (NUM_PROC_XTHREAD * 2);

View File

@ -59,12 +59,12 @@ public class TestProcedureSchedulerConcurrency {
procSched.stop();
}
@Test(timeout=60000)
@Test
public void testConcurrentWaitWake() throws Exception {
testConcurrentWaitWake(false);
}
@Test(timeout=60000)
@Test
public void testConcurrentWaitWakeBatch() throws Exception {
testConcurrentWaitWake(true);
}

View File

@ -71,7 +71,7 @@ public class TestProcedureSuspended {
procStore.stop(false);
}
@Test(timeout=10000)
@Test
public void testSuspendWhileHoldingLocks() {
final AtomicBoolean lockA = new AtomicBoolean(false);
final AtomicBoolean lockB = new AtomicBoolean(false);
@ -125,7 +125,7 @@ public class TestProcedureSuspended {
assertEquals(false, lockB.get());
}
@Test(timeout=10000)
@Test
public void testYieldWhileHoldingLocks() {
final AtomicBoolean lock = new AtomicBoolean(false);

View File

@ -580,7 +580,7 @@ public class TestWALProcedureStore {
}
}
@Test(timeout=60000)
@Test
public void testWalReplayOrder_AB_A() throws Exception {
/*
* | A B | -> | A |
@ -623,7 +623,7 @@ public class TestWALProcedureStore {
});
}
@Test(timeout=60000)
@Test
public void testWalReplayOrder_ABC_BAD() throws Exception {
/*
* | A B C | -> | B A D |

View File

@ -78,7 +78,7 @@ public class TestHBaseTestingUtility {
* that what we insert in one place doesn't end up in the other.
* @throws Exception
*/
@Test (timeout=180000)
@Test
public void testMultiClusters() throws Exception {
// Create three clusters

View File

@ -79,7 +79,7 @@ public class TestJMXConnectorServer {
/**
* This tests to validate the HMaster's ConnectorServer after unauthorised stopMaster call.
*/
@Test(timeout = 180000)
@Test
public void testHMConnectorServerWhenStopMaster() throws Exception {
conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
JMXListener.class.getName() + "," + MyAccessController.class.getName());
@ -117,7 +117,7 @@ public class TestJMXConnectorServer {
* This tests to validate the RegionServer's ConnectorServer after unauthorised stopRegionServer
* call.
*/
@Test(timeout = 180000)
@Test
public void testRSConnectorServerWhenStopRegionServer() throws Exception {
conf.set(CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY,
JMXListener.class.getName() + "," + MyAccessController.class.getName());
@ -147,7 +147,7 @@ public class TestJMXConnectorServer {
/**
* This tests to validate the HMaster's ConnectorServer after unauthorised shutdown call.
*/
@Test(timeout = 180000)
@Test
public void testHMConnectorServerWhenShutdownCluster() throws Exception {
conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
JMXListener.class.getName() + "," + MyAccessController.class.getName());

View File

@ -288,7 +288,7 @@ public class TestNamespace {
admin.deleteTable(desc.getTableName());
}
@Test(timeout = 60000)
@Test
public void testNamespaceOperations() throws IOException {
admin.createNamespace(NamespaceDescriptor.create(prefix + "ns1").build());
admin.createNamespace(NamespaceDescriptor.create(prefix + "ns2").build());

View File

@ -62,19 +62,19 @@ public class TestNodeHealthCheckChore {
if (!fs.mkdirs(testDir)) throw new IOException("Failed mkdir " + testDir);
}
@Test(timeout=60000)
@Test
public void testHealthCheckerSuccess() throws Exception {
String normalScript = "echo \"I am all fine\"";
healthCheckerTest(normalScript, HealthCheckerExitStatus.SUCCESS);
}
@Test(timeout=60000)
@Test
public void testHealthCheckerFail() throws Exception {
String errorScript = "echo ERROR" + eol + "echo \"Node not healthy\"";
healthCheckerTest(errorScript, HealthCheckerExitStatus.FAILED);
}
@Test(timeout=60000)
@Test
public void testHealthCheckerTimeout() throws Exception {
String timeOutScript = "sleep 10" + eol + "echo \"I am fine\"";
healthCheckerTest(timeOutScript, HealthCheckerExitStatus.TIMED_OUT);
@ -99,7 +99,7 @@ public class TestNodeHealthCheckChore {
this.healthScriptFile.delete();
}
@Test(timeout=60000)
@Test
public void testRSHealthChore() throws Exception{
Stoppable stop = new StoppableImplementation();
Configuration conf = getConfForNodeHealthScript();

View File

@ -103,7 +103,7 @@ public class TestRegionRebalancing {
* @throws IOException
* @throws InterruptedException
*/
@Test (timeout=300000)
@Test
public void testRebalanceOnRegionServerNumberChange()
throws IOException, InterruptedException {
try(Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());

View File

@ -124,7 +124,7 @@ public class TestStochasticBalancerJmxMetrics extends BalancerTestBase {
/**
* In Ensemble mode, there should be only one ensemble table
*/
@Test (timeout=60000)
@Test
public void testJmxMetrics_EnsembleMode() throws Exception {
loadBalancer = new StochasticLoadBalancer();
@ -153,7 +153,7 @@ public class TestStochasticBalancerJmxMetrics extends BalancerTestBase {
/**
* In per-table mode, each table has a set of metrics
*/
@Test (timeout=60000)
@Test
public void testJmxMetrics_PerTableMode() throws Exception {
loadBalancer = new StochasticLoadBalancer();

View File

@ -140,7 +140,7 @@ public class TestZooKeeperTableArchiveClient {
/**
* Test turning on/off archiving
*/
@Test (timeout=300000)
@Test
public void testArchivingEnableDisable() throws Exception {
// 1. turn on hfile backups
LOG.debug("----Starting archiving");
@ -163,7 +163,7 @@ public class TestZooKeeperTableArchiveClient {
archivingClient.getArchivingEnabled(TABLE_NAME));
}
@Test (timeout=300000)
@Test
public void testArchivingOnSingleTable() throws Exception {
createArchiveDirectory();
FileSystem fs = UTIL.getTestFileSystem();
@ -211,7 +211,7 @@ public class TestZooKeeperTableArchiveClient {
* Test archiving/cleaning across multiple tables, where some are retained, and others aren't
* @throws Exception on failure
*/
@Test (timeout=300000)
@Test
public void testMultipleTables() throws Exception {
createArchiveDirectory();
String otherTable = "otherTable";

View File

@ -114,7 +114,7 @@ public class TestAdmin1 {
}
}
@Test (timeout=300000)
@Test
public void testSplitFlushCompactUnknownTable() throws InterruptedException {
final TableName unknowntable = TableName.valueOf(name.getMethodName());
Exception exception = null;
@ -142,7 +142,7 @@ public class TestAdmin1 {
assertTrue(exception instanceof TableNotFoundException);
}
@Test (timeout=300000)
@Test
public void testDeleteEditUnknownColumnFamilyAndOrTable() throws IOException {
// Test we get exception if we try to
final TableName nonexistentTable = TableName.valueOf("nonexistent");
@ -236,7 +236,7 @@ public class TestAdmin1 {
}
}
@Test (timeout=300000)
@Test
public void testDisableAndEnableTable() throws IOException {
final byte [] row = Bytes.toBytes("row");
final byte [] qualifier = Bytes.toBytes("qualifier");
@ -301,7 +301,7 @@ public class TestAdmin1 {
return state.getState();
}
@Test (timeout=300000)
@Test
public void testDisableAndEnableTables() throws IOException {
final byte [] row = Bytes.toBytes("row");
final byte [] qualifier = Bytes.toBytes("qualifier");
@ -359,7 +359,7 @@ public class TestAdmin1 {
assertEquals(TableState.State.ENABLED, getStateFromMeta(table2));
}
@Test (timeout=300000)
@Test
public void testCreateTable() throws IOException {
HTableDescriptor [] tables = admin.listTables();
int numTables = tables.length;
@ -373,12 +373,12 @@ public class TestAdmin1 {
assertEquals(TableState.State.ENABLED, getStateFromMeta(tableName));
}
@Test (timeout=300000)
@Test
public void testTruncateTable() throws IOException {
testTruncateTable(TableName.valueOf(name.getMethodName()), false);
}
@Test (timeout=300000)
@Test
public void testTruncateTablePreservingSplits() throws IOException {
testTruncateTable(TableName.valueOf(name.getMethodName()), true);
}
@ -415,7 +415,7 @@ public class TestAdmin1 {
}
}
@Test (timeout=300000)
@Test
public void testGetTableDescriptor() throws IOException {
HColumnDescriptor fam1 = new HColumnDescriptor("fam1");
HColumnDescriptor fam2 = new HColumnDescriptor("fam2");
@ -432,7 +432,7 @@ public class TestAdmin1 {
table.close();
}
@Test (timeout=300000)
@Test
public void testCompactionTimestamps() throws Exception {
HColumnDescriptor fam1 = new HColumnDescriptor("fam1");
final TableName tableName = TableName.valueOf(name.getMethodName());
@ -486,7 +486,7 @@ public class TestAdmin1 {
table.close();
}
@Test (timeout=300000)
@Test
public void testHColumnValidName() {
boolean exceptionThrown;
try {
@ -502,7 +502,7 @@ public class TestAdmin1 {
* @throws IOException
* @throws InterruptedException
*/
@Test (timeout=300000)
@Test
public void testOnlineChangeTableSchema() throws IOException, InterruptedException {
final TableName tableName = TableName.valueOf(name.getMethodName());
HTableDescriptor [] tables = admin.listTables();
@ -618,7 +618,7 @@ public class TestAdmin1 {
}
}
@Test (timeout=300000)
@Test
public void testCreateTableNumberOfRegions() throws IOException, InterruptedException {
final TableName tableName = TableName.valueOf(name.getMethodName());
HTableDescriptor desc = new HTableDescriptor(tableName);
@ -668,7 +668,7 @@ public class TestAdmin1 {
}
}
@Test (timeout=300000)
@Test
public void testCreateTableWithRegions() throws IOException, InterruptedException {
final TableName tableName = TableName.valueOf(name.getMethodName());
@ -844,7 +844,7 @@ public class TestAdmin1 {
}
}
@Test (timeout=300000)
@Test
public void testTableAvailableWithRandomSplitKeys() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
HTableDescriptor desc = new HTableDescriptor(tableName);
@ -859,7 +859,7 @@ public class TestAdmin1 {
assertFalse("Table should be created with 1 row in META", tableAvailable);
}
@Test (timeout=300000)
@Test
public void testCreateTableWithOnlyEmptyStartRow() throws IOException {
final byte[] tableName = Bytes.toBytes(name.getMethodName());
byte[][] splitKeys = new byte[1][];
@ -873,7 +873,7 @@ public class TestAdmin1 {
}
}
@Test (timeout=300000)
@Test
public void testCreateTableWithEmptyRowInTheSplitKeys() throws IOException{
final byte[] tableName = Bytes.toBytes(name.getMethodName());
byte[][] splitKeys = new byte[3][];
@ -890,7 +890,7 @@ public class TestAdmin1 {
}
}
@Test (timeout=120000)
@Test
public void testTableExist() throws IOException {
final TableName table = TableName.valueOf(name.getMethodName());
boolean exist;
@ -906,7 +906,7 @@ public class TestAdmin1 {
* @throws Exception
* @throws IOException
*/
@Test (timeout=400000)
@Test
public void testForceSplit() throws Exception {
byte[][] familyNames = new byte[][] { Bytes.toBytes("cf") };
int[] rowCounts = new int[] { 6000 };
@ -925,7 +925,7 @@ public class TestAdmin1 {
*
* @throws IOException
*/
@Test (timeout=300000)
@Test
public void testEnableTableRetainAssignment() throws IOException {
final TableName tableName = TableName.valueOf(name.getMethodName());
byte[][] splitKeys = { new byte[] { 1, 1, 1 }, new byte[] { 2, 2, 2 },
@ -961,7 +961,7 @@ public class TestAdmin1 {
* @throws Exception
* @throws IOException
*/
@Test (timeout=800000)
@Test
public void testForceSplitMultiFamily() throws Exception {
int numVersions = HColumnDescriptor.DEFAULT_VERSIONS;
@ -1234,12 +1234,12 @@ public class TestAdmin1 {
assertTrue(gotException);
}
@Test (expected=IllegalArgumentException.class, timeout=300000)
@Test (expected=IllegalArgumentException.class)
public void testInvalidHColumnDescriptor() throws IOException {
new HColumnDescriptor("/cfamily/name");
}
@Test (timeout=300000)
@Test
public void testEnableDisableAddColumnDeleteColumn() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close();
@ -1264,7 +1264,7 @@ public class TestAdmin1 {
this.admin.deleteTable(tableName);
}
@Test (timeout=300000)
@Test
public void testDeleteLastColumnFamily() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close();
@ -1297,7 +1297,7 @@ public class TestAdmin1 {
* Test DFS replication for column families, where one CF has default replication(3) and the other
* is set to 1.
*/
@Test(timeout = 300000)
@Test
public void testHFileReplication() throws Exception {
final TableName tableName = TableName.valueOf(this.name.getMethodName());
String fn1 = "rep1";
@ -1348,7 +1348,7 @@ public class TestAdmin1 {
}
}
@Test (timeout=300000)
@Test
public void testMergeRegions() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
HColumnDescriptor cd = new HColumnDescriptor("d");

View File

@ -118,7 +118,7 @@ public class TestAdmin2 {
}
}
@Test (timeout=300000)
@Test
public void testCreateBadTables() throws IOException {
String msg = null;
try {
@ -175,7 +175,7 @@ public class TestAdmin2 {
* Test for hadoop-1581 'HBASE: Unopenable tablename bug'.
* @throws Exception
*/
@Test (timeout=300000)
@Test
public void testTableNameClash() throws Exception {
final String name = this.name.getMethodName();
HTableDescriptor htd1 = new HTableDescriptor(TableName.valueOf(name + "SOMEUPPERCASE"));
@ -195,7 +195,7 @@ public class TestAdmin2 {
* issue anymore
* @throws Exception
*/
@Test (timeout=300000)
@Test
public void testCreateTableRPCTimeOut() throws Exception {
final String name = this.name.getMethodName();
int oldTimeout = TEST_UTIL.getConfiguration().
@ -219,7 +219,7 @@ public class TestAdmin2 {
* Test read only tables
* @throws Exception
*/
@Test (timeout=300000)
@Test
public void testReadOnlyTable() throws Exception {
final TableName name = TableName.valueOf(this.name.getMethodName());
Table table = TEST_UTIL.createTable(name, HConstants.CATALOG_FAMILY);
@ -236,7 +236,7 @@ public class TestAdmin2 {
* start with same. HBASE-771
* @throws IOException
*/
@Test (timeout=300000)
@Test
public void testTableNames() throws IOException {
byte[][] illegalNames = new byte[][] {
Bytes.toBytes("-bad"),
@ -265,7 +265,7 @@ public class TestAdmin2 {
* For HADOOP-2579
* @throws IOException
*/
@Test (expected=TableExistsException.class, timeout=300000)
@Test (expected=TableExistsException.class)
public void testTableExistsExceptionWithATable() throws IOException {
final TableName name = TableName.valueOf(this.name.getMethodName());
TEST_UTIL.createTable(name, HConstants.CATALOG_FAMILY).close();
@ -276,7 +276,7 @@ public class TestAdmin2 {
* Can't disable a table if the table isn't in enabled state
* @throws IOException
*/
@Test (expected=TableNotEnabledException.class, timeout=300000)
@Test (expected=TableNotEnabledException.class)
public void testTableNotEnabledExceptionWithATable() throws IOException {
final TableName name = TableName.valueOf(this.name.getMethodName());
TEST_UTIL.createTable(name, HConstants.CATALOG_FAMILY).close();
@ -288,7 +288,7 @@ public class TestAdmin2 {
* Can't enable a table if the table isn't in disabled state
* @throws IOException
*/
@Test (expected=TableNotDisabledException.class, timeout=300000)
@Test (expected=TableNotDisabledException.class)
public void testTableNotDisabledExceptionWithATable() throws IOException {
final TableName name = TableName.valueOf(this.name.getMethodName());
Table t = TEST_UTIL.createTable(name, HConstants.CATALOG_FAMILY);
@ -303,7 +303,7 @@ public class TestAdmin2 {
* For HADOOP-2579
* @throws IOException
*/
@Test (expected=TableNotFoundException.class, timeout=300000)
@Test (expected=TableNotFoundException.class)
public void testTableNotFoundExceptionWithoutAnyTables() throws IOException {
TableName tableName = TableName
.valueOf("testTableNotFoundExceptionWithoutAnyTables");
@ -311,7 +311,7 @@ public class TestAdmin2 {
ht.get(new Get(Bytes.toBytes("e")));
}
@Test (timeout=300000)
@Test
public void testShouldUnassignTheRegion() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
createTableWithDefaultConf(tableName);
@ -338,7 +338,7 @@ public class TestAdmin2 {
isInList);
}
@Test (timeout=300000)
@Test
public void testCloseRegionIfInvalidRegionNameIsPassed() throws Exception {
final String name = this.name.getMethodName();
byte[] tableName = Bytes.toBytes(name);
@ -364,7 +364,7 @@ public class TestAdmin2 {
onlineRegions.contains(info));
}
@Test (timeout=300000)
@Test
public void testCloseRegionThatFetchesTheHRIFromMeta() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
createTableWithDefaultConf(tableName);
@ -421,7 +421,7 @@ public class TestAdmin2 {
* For HBASE-2556
* @throws IOException
*/
@Test (timeout=300000)
@Test
public void testGetTableRegions() throws IOException {
final TableName tableName = TableName.valueOf(name.getMethodName());
@ -443,7 +443,7 @@ public class TestAdmin2 {
expectedRegions, RegionInfos.size());
}
@Test (timeout=300000)
@Test
public void testMoveToPreviouslyAssignedRS() throws IOException, InterruptedException {
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
HMaster master = cluster.getMaster();
@ -458,7 +458,7 @@ public class TestAdmin2 {
am.getRegionStates().getRegionServerOfRegion(hri));
}
@Test (timeout=300000)
@Test
public void testWALRollWriting() throws Exception {
setUpforLogRolling();
String className = this.getClass().getName();
@ -557,7 +557,7 @@ public class TestAdmin2 {
/**
* Check that we have an exception if the cluster is not there.
*/
@Test (timeout=300000)
@Test
public void testCheckHBaseAvailableWithoutCluster() {
Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
@ -578,7 +578,7 @@ public class TestAdmin2 {
" HBase was not available");
}
@Test (timeout=300000)
@Test
public void testDisableCatalogTable() throws Exception {
try {
this.admin.disableTable(TableName.META_TABLE_NAME);
@ -594,7 +594,7 @@ public class TestAdmin2 {
TEST_UTIL.getHBaseAdmin().createTable(htd);
}
@Test (timeout=300000)
@Test
public void testIsEnabledOrDisabledOnUnknownTable() throws Exception {
try {
admin.isTableEnabled(TableName.valueOf(name.getMethodName()));
@ -609,7 +609,7 @@ public class TestAdmin2 {
}
}
@Test (timeout=300000)
@Test
public void testGetRegion() throws Exception {
// We use actual HBaseAdmin instance instead of going via Admin interface in
// here because makes use of an internal HBA method (TODO: Fix.).
@ -630,7 +630,7 @@ public class TestAdmin2 {
}
}
@Test(timeout = 30000)
@Test
public void testBalancer() throws Exception {
boolean initialState = admin.isBalancerEnabled();
@ -652,7 +652,7 @@ public class TestAdmin2 {
assertEquals(initialState, admin.isBalancerEnabled());
}
@Test(timeout = 30000)
@Test
public void testRegionNormalizer() throws Exception {
boolean initialState = admin.isNormalizerEnabled();
@ -674,7 +674,7 @@ public class TestAdmin2 {
assertEquals(initialState, admin.isNormalizerEnabled());
}
@Test(timeout = 30000)
@Test
public void testAbortProcedureFail() throws Exception {
Random randomGenerator = new Random();
long procId = randomGenerator.nextLong();
@ -683,19 +683,19 @@ public class TestAdmin2 {
assertFalse(abortResult);
}
@Test(timeout = 300000)
@Test
public void testGetProcedures() throws Exception {
String procList = admin.getProcedures();
assertTrue(procList.startsWith("["));
}
@Test(timeout = 300000)
@Test
public void testGetLocks() throws Exception {
String lockList = admin.getLocks();
assertTrue(lockList.startsWith("["));
}
@Test(timeout = 30000)
@Test
public void testDecommissionRegionServers() throws Exception {
List<ServerName> decommissionedRegionServers = admin.listDecommissionedRegionServers();
assertTrue(decommissionedRegionServers.isEmpty());

View File

@ -44,7 +44,7 @@ public class TestAsyncDecommissionAdminApi extends TestAsyncAdminBase {
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestAsyncDecommissionAdminApi.class);
@Test(timeout = 30000)
@Test
public void testAsyncDecommissionRegionServers() throws Exception {
List<ServerName> decommissionedRegionServers = admin.listDecommissionedRegionServers().get();
assertTrue(decommissionedRegionServers.isEmpty());

View File

@ -75,7 +75,7 @@ public class TestAsyncNamespaceAdminApi extends TestAsyncAdminBase {
LOG.info("Done initializing cluster");
}
@Test(timeout = 60000)
@Test
public void testCreateAndDelete() throws Exception {
String testName = "testCreateAndDelete";
String nsName = prefix + "_" + testName;
@ -97,7 +97,7 @@ public class TestAsyncNamespaceAdminApi extends TestAsyncAdminBase {
assertNull(zkNamespaceManager.get(nsName));
}
@Test(timeout = 60000)
@Test
public void testDeleteReservedNS() throws Exception {
boolean exceptionCaught = false;
try {
@ -119,7 +119,7 @@ public class TestAsyncNamespaceAdminApi extends TestAsyncAdminBase {
}
}
@Test(timeout = 60000)
@Test
public void testNamespaceOperations() throws Exception {
admin.createNamespace(NamespaceDescriptor.create(prefix + "ns1").build()).join();
admin.createNamespace(NamespaceDescriptor.create(prefix + "ns2").build()).join();

View File

@ -92,7 +92,7 @@ public class TestClientPushback {
UTIL.shutdownMiniCluster();
}
@Test(timeout=60000)
@Test
public void testClientTracksServerPushback() throws Exception{
Configuration conf = UTIL.getConfiguration();

View File

@ -85,7 +85,7 @@ public class TestEnableTable {
TEST_UTIL.shutdownMiniCluster();
}
@Test(timeout = 300000)
@Test
public void testEnableTableWithNoRegionServers() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
final MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
@ -149,7 +149,7 @@ public class TestEnableTable {
* @throws IOException
* @throws InterruptedException
*/
@Test(timeout=60000)
@Test
public void testDeleteForSureClearsAllTableRowsFromMeta()
throws IOException, InterruptedException {
final TableName tableName = TableName.valueOf(name.getMethodName());

View File

@ -269,7 +269,7 @@ public class TestFromClientSide3 {
}
// override the config settings at the CF level and ensure priority
@Test(timeout = 60000)
@Test
public void testAdvancedConfigOverride() throws Exception {
/*
* Overall idea: (1) create 3 store files and issue a compaction. config's
@ -681,7 +681,7 @@ public class TestFromClientSide3 {
assertTrue(con.hasCellBlockSupport());
}
@Test(timeout = 60000)
@Test
public void testPutWithPreBatchMutate() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
testPreBatchMutate(tableName, () -> {
@ -696,7 +696,7 @@ public class TestFromClientSide3 {
});
}
@Test(timeout = 60000)
@Test
public void testRowMutationsWithPreBatchMutate() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
testPreBatchMutate(tableName, () -> {
@ -743,7 +743,7 @@ public class TestFromClientSide3 {
TEST_UTIL.deleteTable(tableName);
}
@Test(timeout = 30000)
@Test
public void testLockLeakWithDelta() throws Exception, Throwable {
final TableName tableName = TableName.valueOf(name.getMethodName());
HTableDescriptor desc = new HTableDescriptor(tableName);
@ -796,7 +796,7 @@ public class TestFromClientSide3 {
assertEquals(0, readLockCount);
}
@Test(timeout = 30000)
@Test
public void testMultiRowMutations() throws Exception, Throwable {
final TableName tableName = TableName.valueOf(name.getMethodName());
HTableDescriptor desc = new HTableDescriptor(tableName);

View File

@ -178,7 +178,7 @@ public class TestMultiParallel {
* @throws NoSuchFieldException
* @throws SecurityException
*/
@Test(timeout=300000)
@Test
public void testActiveThreadsCount() throws Exception {
UTIL.getConfiguration().setLong("hbase.htable.threads.coresize", slaves + 1);
try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration())) {
@ -202,7 +202,7 @@ public class TestMultiParallel {
}
}
@Test(timeout=300000)
@Test
public void testBatchWithGet() throws Exception {
LOG.info("test=testBatchWithGet");
Table table = UTIL.getConnection().getTable(TEST_TABLE);
@ -271,7 +271,7 @@ public class TestMultiParallel {
table.close();
}
@Test (timeout=300000)
@Test
public void testFlushCommitsNoAbort() throws Exception {
LOG.info("test=testFlushCommitsNoAbort");
doTestFlushCommits(false);
@ -283,7 +283,7 @@ public class TestMultiParallel {
*
* @throws Exception
*/
@Test (timeout=360000)
@Test
public void testFlushCommitsWithAbort() throws Exception {
LOG.info("test=testFlushCommitsWithAbort");
doTestFlushCommits(true);
@ -354,7 +354,7 @@ public class TestMultiParallel {
LOG.info("done");
}
@Test (timeout=300000)
@Test
public void testBatchWithPut() throws Exception {
LOG.info("test=testBatchWithPut");
Table table = CONNECTION.getTable(TEST_TABLE);
@ -387,7 +387,7 @@ public class TestMultiParallel {
table.close();
}
@Test(timeout=300000)
@Test
public void testBatchWithDelete() throws Exception {
LOG.info("test=testBatchWithDelete");
Table table = UTIL.getConnection().getTable(TEST_TABLE);
@ -418,7 +418,7 @@ public class TestMultiParallel {
table.close();
}
@Test(timeout=300000)
@Test
public void testHTableDeleteWithList() throws Exception {
LOG.info("test=testHTableDeleteWithList");
Table table = UTIL.getConnection().getTable(TEST_TABLE);
@ -448,7 +448,7 @@ public class TestMultiParallel {
table.close();
}
@Test(timeout=300000)
@Test
public void testBatchWithManyColsInOneRowGetAndPut() throws Exception {
LOG.info("test=testBatchWithManyColsInOneRowGetAndPut");
Table table = UTIL.getConnection().getTable(TEST_TABLE);
@ -487,7 +487,7 @@ public class TestMultiParallel {
table.close();
}
@Test(timeout=300000)
@Test
public void testBatchWithIncrementAndAppend() throws Exception {
LOG.info("test=testBatchWithIncrementAndAppend");
final byte[] QUAL1 = Bytes.toBytes("qual1");
@ -522,7 +522,7 @@ public class TestMultiParallel {
table.close();
}
@Test(timeout=300000)
@Test
public void testNonceCollision() throws Exception {
LOG.info("test=testNonceCollision");
final Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
@ -622,7 +622,7 @@ public class TestMultiParallel {
}
}
@Test(timeout=300000)
@Test
public void testBatchWithMixedActions() throws Exception {
LOG.info("test=testBatchWithMixedActions");
Table table = UTIL.getConnection().getTable(TEST_TABLE);

View File

@ -280,7 +280,7 @@ public class TestReplicaWithCluster {
HTU.shutdownMiniCluster();
}
@Test (timeout=30000)
@Test
public void testCreateDeleteTable() throws IOException {
// Create table then get the single region for our new table.
HTableDescriptor hdt = HTU.createTableDescriptor("testCreateDeleteTable");
@ -313,7 +313,7 @@ public class TestReplicaWithCluster {
HTU.deleteTable(hdt.getTableName());
}
@Test (timeout=120000)
@Test
public void testChangeTable() throws Exception {
TableDescriptor td = TableDescriptorBuilder.newBuilder(TableName.valueOf("testChangeTable"))
.setRegionReplication(NB_SERVERS)
@ -373,7 +373,7 @@ public class TestReplicaWithCluster {
}
@SuppressWarnings("deprecation")
@Test (timeout=300000)
@Test
public void testReplicaAndReplication() throws Exception {
HTableDescriptor hdt = HTU.createTableDescriptor("testReplicaAndReplication");
hdt.setRegionReplication(NB_SERVERS);
@ -457,7 +457,7 @@ public class TestReplicaWithCluster {
// the minicluster has negative impact of deleting all HConnections in JVM.
}
@Test (timeout=30000)
@Test
public void testBulkLoad() throws IOException {
// Create table then get the single region for our new table.
LOG.debug("Creating test table");

View File

@ -102,7 +102,7 @@ public class TestScannerTimeout {
* from failed. Before 2772, it reused the same scanner id.
* @throws Exception
*/
@Test(timeout=300000)
@Test
public void test2772() throws Exception {
LOG.info("START************ test2772");
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TABLE_NAME);
@ -133,7 +133,7 @@ public class TestScannerTimeout {
* from failed. Before 3686, it would skip rows in the scan.
* @throws Exception
*/
@Test(timeout=300000)
@Test
public void test3686a() throws Exception {
LOG.info("START ************ TEST3686A---1");
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TABLE_NAME);
@ -179,7 +179,7 @@ public class TestScannerTimeout {
* client.
* @throws Exception
*/
@Test(timeout=300000)
@Test
public void test3686b() throws Exception {
LOG.info("START ************ test3686b");
HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(TABLE_NAME);

View File

@ -148,7 +148,7 @@ public class TestSnapshotFromClient {
* Test snapshotting not allowed hbase:meta and -ROOT-
* @throws Exception
*/
@Test (timeout=300000)
@Test
public void testMetaTablesSnapshot() throws Exception {
Admin admin = UTIL.getAdmin();
byte[] snapshotName = Bytes.toBytes("metaSnapshot");
@ -166,7 +166,7 @@ public class TestSnapshotFromClient {
*
* @throws Exception
*/
@Test (timeout=300000)
@Test
public void testSnapshotDeletionWithRegex() throws Exception {
Admin admin = UTIL.getAdmin();
// make sure we don't fail on listing snapshots
@ -202,7 +202,7 @@ public class TestSnapshotFromClient {
* Test snapshotting a table that is offline
* @throws Exception
*/
@Test (timeout=300000)
@Test
public void testOfflineTableSnapshot() throws Exception {
Admin admin = UTIL.getAdmin();
// make sure we don't fail on listing snapshots
@ -250,7 +250,7 @@ public class TestSnapshotFromClient {
SnapshotTestingUtils.assertNoSnapshots(admin);
}
@Test (timeout=300000)
@Test
public void testSnapshotFailsOnNonExistantTable() throws Exception {
Admin admin = UTIL.getAdmin();
// make sure we don't fail on listing snapshots
@ -279,7 +279,7 @@ public class TestSnapshotFromClient {
}
}
@Test (timeout=300000)
@Test
public void testOfflineTableSnapshotWithEmptyRegions() throws Exception {
// test with an empty table with one region
@ -323,7 +323,7 @@ public class TestSnapshotFromClient {
SnapshotTestingUtils.assertNoSnapshots(admin);
}
@Test(timeout = 300000)
@Test
public void testListTableSnapshots() throws Exception {
Admin admin = null;
final TableName tableName = TableName.valueOf(name.getMethodName());
@ -369,7 +369,7 @@ public class TestSnapshotFromClient {
}
}
@Test(timeout = 300000)
@Test
public void testListTableSnapshotsWithRegex() throws Exception {
Admin admin = null;
try {
@ -408,7 +408,7 @@ public class TestSnapshotFromClient {
}
}
@Test(timeout = 300000)
@Test
public void testDeleteTableSnapshots() throws Exception {
Admin admin = null;
final TableName tableName = TableName.valueOf(name.getMethodName());
@ -443,7 +443,7 @@ public class TestSnapshotFromClient {
}
}
@Test(timeout = 300000)
@Test
public void testDeleteTableSnapshotsWithRegex() throws Exception {
Admin admin = null;
Pattern tableNamePattern = Pattern.compile("test.*");

View File

@ -188,7 +188,7 @@ public class TestSnapshotMetadata {
/**
* Verify that the describe for a cloned table matches the describe from the original.
*/
@Test (timeout=300000)
@Test
public void testDescribeMatchesAfterClone() throws Exception {
// Clone the original table
final String clonedTableNameAsString = "clone" + originalTableName;
@ -229,7 +229,7 @@ public class TestSnapshotMetadata {
/**
* Verify that the describe for a restored table matches the describe for one the original.
*/
@Test (timeout=300000)
@Test
public void testDescribeMatchesAfterRestore() throws Exception {
runRestoreWithAdditionalMetadata(false);
}
@ -238,7 +238,7 @@ public class TestSnapshotMetadata {
* Verify that if metadata changed after a snapshot was taken, that the old metadata replaces the
* new metadata during a restore
*/
@Test (timeout=300000)
@Test
public void testDescribeMatchesAfterMetadataChangeAndRestore() throws Exception {
runRestoreWithAdditionalMetadata(true);
}
@ -248,7 +248,7 @@ public class TestSnapshotMetadata {
* the restored table's original metadata
* @throws Exception
*/
@Test (timeout=300000)
@Test
public void testDescribeOnEmptyTableMatchesAfterMetadataChangeAndRestore() throws Exception {
runRestoreWithAdditionalMetadata(true, false);
}

View File

@ -87,7 +87,7 @@ public class TestReplicationAdminWithClusters extends TestReplicationBase {
TestReplicationBase.tearDownAfterClass();
}
@Test(timeout = 300000)
@Test
public void disableNotFullReplication() throws Exception {
HTableDescriptor table = new HTableDescriptor(admin2.getTableDescriptor(tableName));
HColumnDescriptor f = new HColumnDescriptor("notReplicatedFamily");
@ -104,7 +104,7 @@ public class TestReplicationAdminWithClusters extends TestReplicationBase {
}
}
@Test(timeout = 300000)
@Test
public void testEnableReplicationWhenSlaveClusterDoesntHaveTable() throws Exception {
admin1.disableTableReplication(tableName);
admin2.disableTable(tableName);
@ -114,7 +114,7 @@ public class TestReplicationAdminWithClusters extends TestReplicationBase {
assertTrue(admin2.tableExists(tableName));
}
@Test(timeout = 300000)
@Test
public void testEnableReplicationWhenReplicationNotEnabled() throws Exception {
HTableDescriptor table = new HTableDescriptor(admin1.getTableDescriptor(tableName));
for (HColumnDescriptor fam : table.getColumnFamilies()) {
@ -135,7 +135,7 @@ public class TestReplicationAdminWithClusters extends TestReplicationBase {
}
}
@Test(timeout = 300000)
@Test
public void testEnableReplicationWhenTableDescriptorIsNotSameInClusters() throws Exception {
HTableDescriptor table = new HTableDescriptor(admin2.getTableDescriptor(tableName));
HColumnDescriptor f = new HColumnDescriptor("newFamily");
@ -160,7 +160,7 @@ public class TestReplicationAdminWithClusters extends TestReplicationBase {
}
}
@Test(timeout = 300000)
@Test
public void testDisableAndEnableReplication() throws Exception {
admin1.disableTableReplication(tableName);
HTableDescriptor table = admin1.getTableDescriptor(tableName);
@ -174,22 +174,22 @@ public class TestReplicationAdminWithClusters extends TestReplicationBase {
}
}
@Test(timeout = 300000, expected = TableNotFoundException.class)
@Test(expected = TableNotFoundException.class)
public void testDisableReplicationForNonExistingTable() throws Exception {
admin1.disableTableReplication(TableName.valueOf(name.getMethodName()));
}
@Test(timeout = 300000, expected = TableNotFoundException.class)
@Test(expected = TableNotFoundException.class)
public void testEnableReplicationForNonExistingTable() throws Exception {
admin1.enableTableReplication(TableName.valueOf(name.getMethodName()));
}
@Test(timeout = 300000, expected = IllegalArgumentException.class)
@Test(expected = IllegalArgumentException.class)
public void testDisableReplicationWhenTableNameAsNull() throws Exception {
admin1.disableTableReplication(null);
}
@Test(timeout = 300000, expected = IllegalArgumentException.class)
@Test(expected = IllegalArgumentException.class)
public void testEnableReplicationWhenTableNameAsNull() throws Exception {
admin1.enableTableReplication(null);
}
@ -198,7 +198,7 @@ public class TestReplicationAdminWithClusters extends TestReplicationBase {
* Test enable table replication should create table only in user explicit specified table-cfs.
* HBASE-14717
*/
@Test(timeout = 300000)
@Test
public void testEnableReplicationForExplicitSetTableCfs() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
String peerId = "2";
@ -239,7 +239,7 @@ public class TestReplicationAdminWithClusters extends TestReplicationBase {
}
}
@Test(timeout=300000)
@Test
public void testReplicationPeerConfigUpdateCallback() throws Exception {
String peerId = "1";
ReplicationPeerConfig rpc = new ReplicationPeerConfig();

View File

@ -106,7 +106,7 @@ public class TestConstraint {
* @throws Exception
*/
@SuppressWarnings("unchecked")
@Test(timeout = 60000)
@Test
public void testConstraintFails() throws Exception {
// create the table

View File

@ -165,7 +165,7 @@ public class TestMasterCoprocessorExceptionWithAbort {
UTIL.shutdownMiniCluster();
}
@Test(timeout=30000)
@Test
public void testExceptionFromCoprocessorWhenCreatingTable()
throws IOException {
MiniHBaseCluster cluster = UTIL.getHBaseCluster();

View File

@ -146,7 +146,7 @@ public class TestMasterCoprocessorExceptionWithRemove {
UTIL.shutdownMiniCluster();
}
@Test(timeout=30000)
@Test
public void testExceptionFromCoprocessorWhenCreatingTable()
throws IOException {
MiniHBaseCluster cluster = UTIL.getHBaseCluster();

View File

@ -1261,7 +1261,7 @@ public class TestMasterObserver {
UTIL.shutdownMiniCluster();
}
@Test (timeout=180000)
@Test
public void testStarted() throws Exception {
MiniHBaseCluster cluster = UTIL.getHBaseCluster();
@ -1280,7 +1280,7 @@ public class TestMasterObserver {
cp.wasStartMasterCalled());
}
@Test (timeout=180000)
@Test
public void testTableOperations() throws Exception {
MiniHBaseCluster cluster = UTIL.getHBaseCluster();
final TableName tableName = TableName.valueOf(name.getMethodName());
@ -1409,7 +1409,7 @@ public class TestMasterObserver {
}
}
@Test (timeout=180000)
@Test
public void testSnapshotOperations() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
MiniHBaseCluster cluster = UTIL.getHBaseCluster();
@ -1470,7 +1470,7 @@ public class TestMasterObserver {
}
}
@Test (timeout=180000)
@Test
public void testNamespaceOperations() throws Exception {
MiniHBaseCluster cluster = UTIL.getHBaseCluster();
String testNamespace = "observed_ns";
@ -1503,7 +1503,7 @@ public class TestMasterObserver {
}
}
@Test (timeout=180000)
@Test
public void testRegionTransitionOperations() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
MiniHBaseCluster cluster = UTIL.getHBaseCluster();
@ -1592,7 +1592,7 @@ public class TestMasterObserver {
}
}
@Test (timeout=180000)
@Test
public void testTableDescriptorsEnumeration() throws Exception {
MiniHBaseCluster cluster = UTIL.getHBaseCluster();
@ -1609,7 +1609,7 @@ public class TestMasterObserver {
cp.wasGetTableDescriptorsCalled());
}
@Test (timeout=180000)
@Test
public void testTableNamesEnumeration() throws Exception {
MiniHBaseCluster cluster = UTIL.getHBaseCluster();
@ -1624,7 +1624,7 @@ public class TestMasterObserver {
cp.wasGetTableNamesCalled());
}
@Test (timeout=180000)
@Test
public void testAbortProcedureOperation() throws Exception {
MiniHBaseCluster cluster = UTIL.getHBaseCluster();
@ -1639,7 +1639,7 @@ public class TestMasterObserver {
cp.wasAbortProcedureCalled());
}
@Test (timeout=180000)
@Test
public void testGetProceduresOperation() throws Exception {
MiniHBaseCluster cluster = UTIL.getHBaseCluster();
@ -1654,7 +1654,7 @@ public class TestMasterObserver {
cp.wasGetProceduresCalled());
}
@Test (timeout=180000)
@Test
public void testGetLocksOperation() throws Exception {
MiniHBaseCluster cluster = UTIL.getHBaseCluster();

View File

@ -378,7 +378,7 @@ public class TestRegionObserverInterface {
}
@Test(timeout = 300000)
@Test
// HBase-3758
public void testHBase3758() throws IOException {
final TableName tableName = TableName.valueOf(name.getMethodName());

View File

@ -61,7 +61,7 @@ public class TestRegionServerCoprocessorExceptionWithAbort {
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private static final TableName TABLE_NAME = TableName.valueOf("observed_table");
@Test(timeout=60000)
@Test
public void testExceptionDuringInitialization() throws Exception {
Configuration conf = TEST_UTIL.getConfiguration();
conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2); // Let's fail fast.
@ -88,7 +88,7 @@ public class TestRegionServerCoprocessorExceptionWithAbort {
}
}
@Test(timeout=60000)
@Test
public void testExceptionFromCoprocessorDuringPut() throws Exception {
// set configure to indicate which cp should be loaded
Configuration conf = TEST_UTIL.getConfiguration();

View File

@ -87,7 +87,7 @@ public class TestRegionServerCoprocessorExceptionWithRemove {
TEST_UTIL.shutdownMiniCluster();
}
@Test(timeout=60000)
@Test
public void testExceptionFromCoprocessorDuringPut()
throws IOException, InterruptedException {
// Set watches on the zookeeper nodes for all of the regionservers in the

View File

@ -44,7 +44,7 @@ public class TestTimeoutExceptionInjector {
/**
* Test that a manually triggered timer fires an exception.
*/
@Test(timeout = 60000)
@Test
public void testTimerTrigger() {
final long time = 10000000; // pick a value that is very far in the future
ForeignExceptionListener listener = Mockito.mock(ForeignExceptionListener.class);
@ -71,7 +71,7 @@ public class TestTimeoutExceptionInjector {
* Demonstrate TimeoutExceptionInjector semantics -- completion means no more exceptions passed to
* error listener.
*/
@Test(timeout = 60000)
@Test
public void testStartAfterComplete() throws InterruptedException {
final long time = 10;
ForeignExceptionListener listener = Mockito.mock(ForeignExceptionListener.class);
@ -91,7 +91,7 @@ public class TestTimeoutExceptionInjector {
* Demonstrate TimeoutExceptionInjector semantics -- triggering fires exception and completes
* the timer.
*/
@Test(timeout = 60000)
@Test
public void testStartAfterTrigger() throws InterruptedException {
final long time = 10;
ForeignExceptionListener listener = Mockito.mock(ForeignExceptionListener.class);

View File

@ -198,7 +198,7 @@ public class TestChangingEncoding {
TEST_UTIL.waitUntilNoRegionsInTransition(TIMEOUT_MS);
}
@Test(timeout=TIMEOUT_MS)
@Test
public void testChangingEncoding() throws Exception {
prepareTest("ChangingEncoding");
for (boolean onlineChange : new boolean[]{false, true}) {
@ -210,7 +210,7 @@ public class TestChangingEncoding {
}
}
@Test(timeout=TIMEOUT_MS)
@Test
public void testChangingEncodingWithCompaction() throws Exception {
prepareTest("ChangingEncodingWithCompaction");
for (boolean onlineChange : new boolean[]{false, true}) {

View File

@ -73,7 +73,7 @@ public class TestLoadAndSwitchEncodeOnDisk extends
}
@Override
@Test(timeout=TIMEOUT_MS)
@Test
public void loadTest() throws Exception {
Admin admin = TEST_UTIL.getAdmin();

View File

@ -326,7 +326,7 @@ public class TestCacheConfig {
* Assert that when BUCKET_CACHE_COMBINED_KEY is false, the non-default, that we deploy
* LruBlockCache as L1 with a BucketCache for L2.
*/
@Test (timeout=10000)
@Test
public void testBucketCacheConfigL1L2Setup() {
this.conf.set(HConstants.BUCKET_CACHE_IOENGINE_KEY, "offheap");
// Make lru size is smaller than bcSize for sure. Need this to be true so when eviction

View File

@ -744,12 +744,12 @@ public class TestHFileBlockIndex {
valueRead);
}
@Test(timeout=10000)
@Test
public void testIntermediateLevelIndicesWithLargeKeys() throws IOException {
testIntermediateLevelIndicesWithLargeKeys(16);
}
@Test(timeout=10000)
@Test
public void testIntermediateLevelIndicesWithLargeKeysWithMinNumEntries() throws IOException {
// because of the large rowKeys, we will end up with a 50-level block index without sanity check
testIntermediateLevelIndicesWithLargeKeys(2);

View File

@ -128,7 +128,7 @@ public class TestHFileEncryption {
return b.getOnDiskSizeWithHeader();
}
@Test(timeout=20000)
@Test
public void testDataBlockEncryption() throws IOException {
final int blocks = 10;
final int[] blockSizes = new int[blocks];
@ -164,7 +164,7 @@ public class TestHFileEncryption {
}
}
@Test(timeout=20000)
@Test
public void testHFileEncryptionMetadata() throws Exception {
Configuration conf = TEST_UTIL.getConfiguration();
CacheConfig cacheConf = new CacheConfig(conf);
@ -202,7 +202,7 @@ public class TestHFileEncryption {
}
}
@Test(timeout=6000000)
@Test
public void testHFileEncryption() throws Exception {
// Create 1000 random test KVs
RedundantKVGenerator generator = new RedundantKVGenerator();

View File

@ -77,7 +77,7 @@ public class TestPrefetch {
assertTrue(cc.shouldPrefetchOnOpen());
}
@Test(timeout=60000)
@Test
public void testPrefetch() throws Exception {
Path storeFile = writeStoreFile();
readStoreFile(storeFile);

View File

@ -111,7 +111,7 @@ public class TestBucketWriterThread {
* @throws IOException
* @throws InterruptedException
*/
@Test (timeout=30000)
@Test
public void testNonErrorCase() throws IOException, InterruptedException {
bc.cacheBlock(this.plainKey, this.plainCacheable);
doDrainOfOneEntry(this.bc, this.wt, this.q);
@ -137,7 +137,7 @@ public class TestBucketWriterThread {
* @throws InterruptedException
*/
@SuppressWarnings("unchecked")
@Test (timeout=30000)
@Test
public void testIOE() throws IOException, InterruptedException {
this.bc.cacheBlock(this.plainKey, plainCacheable);
RAMQueueEntry rqe = q.remove();
@ -156,7 +156,7 @@ public class TestBucketWriterThread {
* @throws IOException
* @throws InterruptedException
*/
@Test (timeout=30000)
@Test
public void testCacheFullException()
throws IOException, InterruptedException {
this.bc.cacheBlock(this.plainKey, plainCacheable);

View File

@ -77,7 +77,7 @@ public class TestNettyRpcServer {
TEST_UTIL.shutdownMiniCluster();
}
@Test(timeout = 180000)
@Test
public void testNettyRpcServer() throws Exception {
final Table table = TEST_UTIL.createTable(TABLE, FAMILY);
try {

View File

@ -171,7 +171,7 @@ public abstract class AbstractTestDLS {
ZKUtil.deleteNodeRecursively(TEST_UTIL.getZooKeeperWatcher(), "/hbase");
}
@Test(timeout = 300000)
@Test
public void testRecoveredEdits() throws Exception {
conf.setLong("hbase.regionserver.hlog.blocksize", 30 * 1024); // create more than one wal
startCluster(NUM_RS);
@ -250,7 +250,7 @@ public abstract class AbstractTestDLS {
}
}
@Test(timeout = 300000)
@Test
public void testMasterStartsUpWithLogSplittingWork() throws Exception {
conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, NUM_RS - 1);
startCluster(NUM_RS);
@ -310,7 +310,7 @@ public abstract class AbstractTestDLS {
* @throws Exception
*/
// Was marked flaky before Distributed Log Replay cleanup.
@Test(timeout = 300000)
@Test
public void testWorkerAbort() throws Exception {
LOG.info("testWorkerAbort");
startCluster(3);
@ -367,7 +367,7 @@ public abstract class AbstractTestDLS {
}
}
@Test(timeout = 300000)
@Test
public void testThreeRSAbort() throws Exception {
LOG.info("testThreeRSAbort");
int numRegionsToCreate = 40;
@ -402,7 +402,7 @@ public abstract class AbstractTestDLS {
}
}
@Test(timeout = 30000)
@Test
public void testDelayedDeleteOnFailure() throws Exception {
LOG.info("testDelayedDeleteOnFailure");
startCluster(1);
@ -470,7 +470,7 @@ public abstract class AbstractTestDLS {
}
}
@Test(timeout = 300000)
@Test
public void testReadWriteSeqIdFiles() throws Exception {
LOG.info("testReadWriteSeqIdFiles");
startCluster(2);

View File

@ -174,7 +174,7 @@ public class TestAssignmentListener {
TEST_UTIL.shutdownMiniCluster();
}
@Test(timeout=60000)
@Test
public void testServerListener() throws IOException, InterruptedException {
ServerManager serverManager = TEST_UTIL.getHBaseCluster().getMaster().getServerManager();
@ -216,7 +216,7 @@ public class TestAssignmentListener {
}
}
@Test(timeout=60000)
@Test
public void testAssignmentListener() throws IOException, InterruptedException {
AssignmentManager am = TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager();
Admin admin = TEST_UTIL.getAdmin();

View File

@ -90,7 +90,7 @@ public class TestCatalogJanitorInMemoryStates {
/**
* Test clearing a split parent from memory.
*/
@Test(timeout = 180000)
@Test
public void testInMemoryParentCleanup() throws IOException, InterruptedException {
final AssignmentManager am = TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager();
final ServerManager sm = TEST_UTIL.getHBaseCluster().getMaster().getServerManager();

View File

@ -103,7 +103,7 @@ public class TestDeadServer {
assertFalse(ds.cleanPreviousInstance(deadServerHostComingAlive));
}
@Test(timeout = 15000)
@Test
public void testCrashProcedureReplay() {
HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
final ProcedureExecutor<MasterProcedureEnv> pExecutor = master.getMasterProcedureExecutor();

View File

@ -64,7 +64,7 @@ public class TestMasterBalanceThrottling {
TEST_UTIL.shutdownMiniCluster();
}
@Test(timeout = 60000)
@Test
public void testThrottlingByBalanceInterval() throws Exception {
// Use default config and start a cluster of two regionservers.
TEST_UTIL.startMiniCluster(2);
@ -88,7 +88,7 @@ public class TestMasterBalanceThrottling {
TEST_UTIL.deleteTable(tableName);
}
@Test(timeout = 60000)
@Test
public void testThrottlingByMaxRitPercent() throws Exception {
// Set max balancing time to 500 ms and max percent of regions in transition to 0.05
TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_BALANCER_MAX_BALANCING, 500);

View File

@ -47,7 +47,7 @@ public class TestMasterFailoverBalancerPersistence {
*
* @throws Exception
*/
@Test(timeout = 240000)
@Test
public void testMasterFailoverBalancerPersistence() throws Exception {
final int NUM_MASTERS = 3;
final int NUM_RS = 1;

View File

@ -85,7 +85,7 @@ public class TestMasterMetrics {
}
}
@Test(timeout = 300000)
@Test
public void testClusterRequests() throws Exception {
// sending fake request to master to see how metric value has changed

View File

@ -58,7 +58,7 @@ public class TestMasterMetricsWrapper {
TEST_UTIL.shutdownMiniCluster();
}
@Test (timeout = 30000)
@Test
public void testInfo() {
HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
MetricsMasterWrapperImpl info = new MetricsMasterWrapperImpl(master);

View File

@ -57,7 +57,7 @@ public class TestMasterShutdown {
* Verifies that all masters are properly shutdown.
* @throws Exception
*/
@Test (timeout=120000)
@Test
public void testMasterShutdown() throws Exception {
final int NUM_MASTERS = 3;
final int NUM_RS = 3;
@ -103,7 +103,7 @@ public class TestMasterShutdown {
htu.shutdownMiniCluster();
}
@Test(timeout = 60000)
@Test
public void testMasterShutdownBeforeStartingAnyRegionServer() throws Exception {
final int NUM_MASTERS = 1;
final int NUM_RS = 0;

View File

@ -198,7 +198,8 @@ public class TestMasterTransitions {
* in.
* @see <a href="https://issues.apache.org/jira/browse/HBASE-2428">HBASE-2428</a>
*/
@Ignore @Test (timeout=300000) public void testRegionCloseWhenNoMetaHBase2428()
@Ignore @Test
public void testRegionCloseWhenNoMetaHBase2428()
throws Exception {
/*
LOG.info("Running testRegionCloseWhenNoMetaHBase2428");
@ -253,7 +254,8 @@ public class TestMasterTransitions {
* If confusion between old and new, purportedly meta never comes back. Test
* that meta gets redeployed.
*/
@Ignore @Test (timeout=300000) public void testAddingServerBeforeOldIsDead2413()
@Ignore @Test
public void testAddingServerBeforeOldIsDead2413()
throws IOException {
/*
LOG.info("Running testAddingServerBeforeOldIsDead2413");
@ -380,7 +382,8 @@ public class TestMasterTransitions {
* done.
* @see <a href="https://issues.apache.org/jira/browse/HBASE-2482">HBASE-2482</a>
*/
@Ignore @Test (timeout=300000) public void testKillRSWithOpeningRegion2482()
@Ignore @Test
public void testKillRSWithOpeningRegion2482()
throws Exception {
/*
LOG.info("Running testKillRSWithOpeningRegion2482");

View File

@ -78,7 +78,7 @@ public class TestMetaShutdownHandler {
* the region server so that it is still up during the master SSH.
* We will check that the master SSH is still successfully done.
*/
@Test (timeout=180000)
@Test
public void testExpireMetaRegionServer() throws Exception {
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
HMaster master = cluster.getMaster();

View File

@ -66,7 +66,7 @@ public class TestRestartCluster {
UTIL.shutdownMiniCluster();
}
@Test (timeout=300000)
@Test
public void testClusterRestart() throws Exception {
UTIL.startMiniCluster(3);
while (!UTIL.getMiniHBaseCluster().getMaster().isInitialized()) {
@ -112,7 +112,7 @@ public class TestRestartCluster {
/**
* This tests retaining assignments on a cluster restart
*/
@Test (timeout=300000)
@Test
public void testRetainAssignmentOnRestart() throws Exception {
UTIL.startMiniCluster(2);
while (!UTIL.getMiniHBaseCluster().getMaster().isInitialized()) {

View File

@ -64,7 +64,7 @@ public class TestRollingRestart {
@Rule
public TestName name = new TestName();
@Test (timeout=500000)
@Test
public void testBasicRollingRestart() throws Exception {
// Start a cluster with 2 masters and 4 regionservers

View File

@ -215,7 +215,7 @@ public class TestSplitLogManager {
* Test whether the splitlog correctly creates a task in zookeeper
* @throws Exception
*/
@Test (timeout=180000)
@Test
public void testTaskCreation() throws Exception {
LOG.info("TestTaskCreation - test the creation of a task in zk");
@ -230,7 +230,7 @@ public class TestSplitLogManager {
assertTrue(slt.isUnassigned(master.getServerName()));
}
@Test (timeout=180000)
@Test
public void testOrphanTaskAcquisition() throws Exception {
LOG.info("TestOrphanTaskAcquisition");
@ -254,7 +254,7 @@ public class TestSplitLogManager {
waitForCounter(tot_mgr_rescan, 0, 1, to + to/2);
}
@Test (timeout=180000)
@Test
public void testUnassignedOrphan() throws Exception {
LOG.info("TestUnassignedOrphan - an unassigned task is resubmitted at" +
" startup");
@ -283,7 +283,7 @@ public class TestSplitLogManager {
assertTrue(ZKUtil.checkExists(zkw, tasknode) > version);
}
@Test (timeout=180000)
@Test
public void testMultipleResubmits() throws Exception {
LOG.info("TestMultipleResbmits - no indefinite resubmissions");
conf.setInt("hbase.splitlog.max.resubmit", 2);
@ -315,7 +315,7 @@ public class TestSplitLogManager {
assertEquals(2L, tot_mgr_resubmit.sum() - tot_mgr_resubmit_force.sum());
}
@Test (timeout=180000)
@Test
public void testRescanCleanup() throws Exception {
LOG.info("TestRescanCleanup - ensure RESCAN nodes are cleaned up");
@ -344,7 +344,7 @@ public class TestSplitLogManager {
waitForCounter(tot_mgr_rescan_deleted, 0, 1, to/2);
}
@Test (timeout=180000)
@Test
public void testTaskDone() throws Exception {
LOG.info("TestTaskDone - cleanup task node once in DONE state");
@ -363,7 +363,7 @@ public class TestSplitLogManager {
assertTrue(ZKUtil.checkExists(zkw, tasknode) == -1);
}
@Test (timeout=180000)
@Test
public void testTaskErr() throws Exception {
LOG.info("TestTaskErr - cleanup task node once in ERR state");
@ -386,7 +386,7 @@ public class TestSplitLogManager {
conf.setInt("hbase.splitlog.max.resubmit", ZKSplitLogManagerCoordination.DEFAULT_MAX_RESUBMIT);
}
@Test (timeout=180000)
@Test
public void testTaskResigned() throws Exception {
LOG.info("TestTaskResigned - resubmit task node once in RESIGNED state");
assertEquals(0, tot_mgr_resubmit.sum());
@ -412,7 +412,7 @@ public class TestSplitLogManager {
assertTrue(slt.isUnassigned(master.getServerName()));
}
@Test (timeout=180000)
@Test
public void testUnassignedTimeout() throws Exception {
LOG.info("TestUnassignedTimeout - iff all tasks are unassigned then" +
" resubmit");
@ -448,7 +448,7 @@ public class TestSplitLogManager {
waitForCounter(tot_mgr_resubmit_unassigned, 0, 1, 2 * to + to/2);
}
@Test (timeout=180000)
@Test
public void testDeadWorker() throws Exception {
LOG.info("testDeadWorker");
@ -476,7 +476,7 @@ public class TestSplitLogManager {
return;
}
@Test (timeout=180000)
@Test
public void testWorkerCrash() throws Exception {
slm = new SplitLogManager(master, conf);
TaskBatch batch = new TaskBatch();
@ -500,7 +500,7 @@ public class TestSplitLogManager {
Assert.assertEquals(1, tot_mgr_resubmit.sum());
}
@Test (timeout=180000)
@Test
public void testEmptyLogDir() throws Exception {
LOG.info("testEmptyLogDir");
slm = new SplitLogManager(master, conf);
@ -512,7 +512,7 @@ public class TestSplitLogManager {
assertFalse(fs.exists(emptyLogDirPath));
}
@Test (timeout = 60000)
@Test
public void testLogFilesAreArchived() throws Exception {
LOG.info("testLogFilesAreArchived");
slm = new SplitLogManager(master, conf);

View File

@ -59,7 +59,7 @@ public class TestTableStateManager {
TEST_UTIL.shutdownMiniCluster();
}
@Test(timeout = 60000)
@Test
public void testUpgradeFromZk() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
TEST_UTIL.startMiniCluster(2, 1);

View File

@ -82,22 +82,22 @@ public class TestAssignmentOnRSCrash {
UTIL.shutdownMiniCluster();
}
@Test(timeout=30000)
@Test
public void testKillRsWithUserRegionWithData() throws Exception {
testCrashRsWithUserRegion(true, true);
}
@Test(timeout=30000)
@Test
public void testKillRsWithUserRegionWithoutData() throws Exception {
testCrashRsWithUserRegion(true, false);
}
@Test(timeout=30000)
@Test
public void testStopRsWithUserRegionWithData() throws Exception {
testCrashRsWithUserRegion(false, true);
}
@Test(timeout=30000)
@Test
public void testStopRsWithUserRegionWithoutData() throws Exception {
testCrashRsWithUserRegion(false, false);
}
@ -133,12 +133,12 @@ public class TestAssignmentOnRSCrash {
assertTrue("expected RSs to be killed", nkilled > 0);
}
@Test(timeout=60000)
@Test
public void testKillRsWithMetaRegion() throws Exception {
testCrashRsWithMetaRegion(true);
}
@Test(timeout=60000)
@Test
public void testStopRsWithMetaRegion() throws Exception {
testCrashRsWithMetaRegion(false);
}

View File

@ -137,7 +137,7 @@ public class TestRogueRSAssignment {
admin.setBalancerRunning(true, false);
}
@Test(timeout = 120000)
@Test
public void testReportRSWithWrongRegion() throws Exception {
final TableName tableName = TableName.valueOf(this.name.getMethodName());

View File

@ -137,7 +137,7 @@ public class TestSplitTableRegionProcedure {
}
}
@Test(timeout=60000)
@Test
public void testSplitTableRegion() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();

View File

@ -149,7 +149,7 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
*
* @throws Exception
*/
@Test (timeout=180000)
@Test
public void testBulkAssignment() throws Exception {
List<ServerName> tmp = getListOfServerNames(randomServers(5, 0));
List<RegionInfo> hris = randomRegions(20);
@ -190,7 +190,7 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
* assignment info.
* @throws Exception
*/
@Test (timeout=180000)
@Test
public void testRetainAssignment() throws Exception {
// Test simple case where all same servers are there
List<ServerAndLoad> servers = randomServers(10, 10);
@ -226,7 +226,7 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
assertRetainedAssignment(existing, listOfServerNames, assignment);
}
@Test (timeout=30000)
@Test
public void testRandomAssignment() throws Exception {
for (int i = 1; i != 5; ++i) {
LOG.info("run testRandomAssignment() with idle servers:" + i);
@ -272,7 +272,7 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
}
}
@Test (timeout=180000)
@Test
public void testRegionAvailability() throws Exception {
// Create a cluster with a few servers, assign them to specific racks
// then assign some regions. The tests should check whether moving a
@ -351,7 +351,7 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
assertTrue(!cluster.wouldLowerAvailability(hri1, servers[6]));
}
@Test (timeout=180000)
@Test
public void testRegionAvailabilityWithRegionMoves() throws Exception {
List<RegionInfo> list0 = new ArrayList<>();
List<RegionInfo> list1 = new ArrayList<>();
@ -466,7 +466,7 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
}
}
@Test (timeout=180000)
@Test
public void testClusterServersWithSameHostPort() {
// tests whether the BaseLoadBalancer.Cluster can be constructed with servers
// sharing same host and port
@ -506,7 +506,7 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
}
}
@Test (timeout=180000)
@Test
public void testClusterRegionLocations() {
// tests whether region locations are handled correctly in Cluster
List<ServerName> servers = getListOfServerNames(randomServers(10, 10));

View File

@ -131,7 +131,7 @@ public class TestDefaultLoadBalancer extends BalancerTestBase {
*
* @throws Exception
*/
@Test (timeout=60000)
@Test
public void testBalanceClusterOverall() throws Exception {
Map<TableName, Map<ServerName, List<RegionInfo>>> clusterLoad = new TreeMap<>();
for (int[] mockCluster : clusterStateMocks) {
@ -169,7 +169,7 @@ public class TestDefaultLoadBalancer extends BalancerTestBase {
* level balance while the bytable strategy cannot
* @throws Exception
*/
@Test (timeout=60000)
@Test
public void testImpactOfBalanceClusterOverall() throws Exception {
Map<TableName, Map<ServerName, List<RegionInfo>>> clusterLoad = new TreeMap<>();
Map<ServerName, List<RegionInfo>> clusterServers = mockUniformClusterServers(mockUniformCluster);

View File

@ -103,7 +103,7 @@ public class TestHFileCleaner {
+ status.getAccessTime();
}
@Test(timeout = 60 *1000)
@Test
public void testHFileCleaning() throws Exception {
final EnvironmentEdge originalEdge = EnvironmentEdgeManager.getDelegate();
String prefix = "someHFileThatWouldBeAUUID";
@ -340,7 +340,7 @@ public class TestHFileCleaner {
Assert.assertEquals(SMALL_FILE_NUM, cleaner.getNumOfDeletedSmallFiles());
}
@Test(timeout = 60 * 1000)
@Test
public void testOnConfigurationChange() throws Exception {
// constants
final int ORIGINAL_THROTTLE_POINT = 512 * 1024;

View File

@ -201,7 +201,7 @@ public class TestLogsCleaner {
}
}
@Test(timeout=5000)
@Test
public void testZnodeCversionChange() throws Exception {
Configuration conf = TEST_UTIL.getConfiguration();
ReplicationLogCleaner cleaner = new ReplicationLogCleaner();

View File

@ -204,7 +204,7 @@ public class TestReplicationHFileCleaner {
* Test for HBASE-14621. This test will not assert directly anything. Without the fix the test
* will end up in a infinite loop, so it will timeout.
*/
@Test(timeout = 15000)
@Test
public void testForDifferntHFileRefsZnodeVersion() throws Exception {
// 1. Create a file
Path file = new Path(root, "testForDifferntHFileRefsZnodeVersion");

View File

@ -168,7 +168,7 @@ public class TestSnapshotFromMaster {
* <li>If asking about a snapshot has hasn't occurred, you should get an error.</li>
* </ol>
*/
@Test(timeout = 300000)
@Test
public void testIsDoneContract() throws Exception {
IsSnapshotDoneRequest.Builder builder = IsSnapshotDoneRequest.newBuilder();
@ -222,7 +222,7 @@ public class TestSnapshotFromMaster {
assertTrue("Completed, on-disk snapshot not found", response.getDone());
}
@Test(timeout = 300000)
@Test
public void testGetCompletedSnapshots() throws Exception {
// first check when there are no snapshots
GetCompletedSnapshotsRequest request = GetCompletedSnapshotsRequest.newBuilder().build();
@ -253,7 +253,7 @@ public class TestSnapshotFromMaster {
assertEquals("Returned snapshots don't match created snapshots", expected, snapshots);
}
@Test(timeout = 300000)
@Test
public void testDeleteSnapshot() throws Exception {
String snapshotName = "completed";
@ -280,7 +280,7 @@ public class TestSnapshotFromMaster {
* should be retained, while those that are not in a snapshot should be deleted.
* @throws Exception on failure
*/
@Test(timeout = 300000)
@Test
public void testSnapshotHFileArchiving() throws Exception {
Admin admin = UTIL.getAdmin();
// make sure we don't fail on listing snapshots

View File

@ -408,25 +408,25 @@ public class TestLockProcedure {
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
}
@Test(timeout = 20000)
@Test
public void testRemoteTableLockRecovery() throws Exception {
LockRequest lock = getTableExclusiveLock(tableName1, testMethodName);
testRemoteLockRecovery(lock);
}
@Test(timeout = 20000)
@Test
public void testRemoteNamespaceLockRecovery() throws Exception {
LockRequest lock = getNamespaceLock(namespace, testMethodName);
testRemoteLockRecovery(lock);
}
@Test(timeout = 20000)
@Test
public void testRemoteRegionLockRecovery() throws Exception {
LockRequest lock = getRegionLock(tableRegions1, testMethodName);
testRemoteLockRecovery(lock);
}
@Test (timeout = 20000)
@Test
public void testLocalMasterLockRecovery() throws Exception {
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
CountDownLatch latch = new CountDownLatch(1);

View File

@ -92,7 +92,7 @@ public class TestSimpleRegionNormalizerOnCluster {
TEST_UTIL.shutdownMiniCluster();
}
@Test(timeout = 90000)
@Test
@SuppressWarnings("deprecation")
public void testRegionNormalizationSplitOnCluster() throws Exception {
testRegionNormalizationSplitOnCluster(false);
@ -179,7 +179,7 @@ public class TestSimpleRegionNormalizerOnCluster {
admin.deleteTable(TABLENAME);
}
@Test(timeout = 60000)
@Test
@SuppressWarnings("deprecation")
public void testRegionNormalizationMergeOnCluster() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());

View File

@ -100,7 +100,7 @@ public class TestCloneSnapshotProcedure extends TestTableDDLProcedureBase {
return htd;
}
@Test(timeout=60000)
@Test
public void testCloneSnapshot() throws Exception {
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
final TableName clonedTableName = TableName.valueOf("testCloneSnapshot2");
@ -117,7 +117,7 @@ public class TestCloneSnapshotProcedure extends TestTableDDLProcedureBase {
clonedTableName);
}
@Test(timeout=60000)
@Test
public void testCloneSnapshotToSameTable() throws Exception {
// take the snapshot
SnapshotProtos.SnapshotDescription snapshotDesc = getSnapshot();
@ -135,7 +135,7 @@ public class TestCloneSnapshotProcedure extends TestTableDDLProcedureBase {
ProcedureTestingUtility.getExceptionCause(result) instanceof TableExistsException);
}
@Test(timeout=60000)
@Test
public void testRecoveryAndDoubleExecution() throws Exception {
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
final TableName clonedTableName = TableName.valueOf("testRecoveryAndDoubleExecution");
@ -158,7 +158,7 @@ public class TestCloneSnapshotProcedure extends TestTableDDLProcedureBase {
clonedTableName);
}
@Test(timeout = 60000)
@Test
public void testRollbackAndDoubleExecution() throws Exception {
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
final TableName clonedTableName = TableName.valueOf("testRollbackAndDoubleExecution");

View File

@ -84,7 +84,7 @@ public class TestCreateNamespaceProcedure {
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
}
@Test(timeout = 60000)
@Test
public void testCreateNamespace() throws Exception {
final NamespaceDescriptor nsd = NamespaceDescriptor.create("testCreateNamespace").build();
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
@ -98,7 +98,7 @@ public class TestCreateNamespaceProcedure {
validateNamespaceCreated(nsd);
}
@Test(timeout=60000)
@Test
public void testCreateSameNamespaceTwice() throws Exception {
final NamespaceDescriptor nsd =
NamespaceDescriptor.create("testCreateSameNamespaceTwice").build();
@ -124,7 +124,7 @@ public class TestCreateNamespaceProcedure {
ProcedureTestingUtility.getExceptionCause(result) instanceof NamespaceExistException);
}
@Test(timeout=60000)
@Test
public void testCreateSystemNamespace() throws Exception {
final NamespaceDescriptor nsd =
UTIL.getAdmin().getNamespaceDescriptor(NamespaceDescriptor.SYSTEM_NAMESPACE.getName());
@ -141,7 +141,7 @@ public class TestCreateNamespaceProcedure {
ProcedureTestingUtility.getExceptionCause(result) instanceof NamespaceExistException);
}
@Test(timeout=60000)
@Test
public void testCreateNamespaceWithInvalidRegionCount() throws Exception {
final NamespaceDescriptor nsd =
NamespaceDescriptor.create("testCreateNamespaceWithInvalidRegionCount").build();
@ -161,7 +161,7 @@ public class TestCreateNamespaceProcedure {
assertTrue(ProcedureTestingUtility.getExceptionCause(result) instanceof ConstraintException);
}
@Test(timeout=60000)
@Test
public void testCreateNamespaceWithInvalidTableCount() throws Exception {
final NamespaceDescriptor nsd =
NamespaceDescriptor.create("testCreateNamespaceWithInvalidTableCount").build();
@ -181,7 +181,7 @@ public class TestCreateNamespaceProcedure {
assertTrue(ProcedureTestingUtility.getExceptionCause(result) instanceof ConstraintException);
}
@Test(timeout = 60000)
@Test
public void testRecoveryAndDoubleExecution() throws Exception {
final NamespaceDescriptor nsd =
NamespaceDescriptor.create("testRecoveryAndDoubleExecution").build();
@ -202,7 +202,7 @@ public class TestCreateNamespaceProcedure {
validateNamespaceCreated(nsd);
}
@Test(timeout = 60000)
@Test
public void testRollbackAndDoubleExecution() throws Exception {
final NamespaceDescriptor nsd =
NamespaceDescriptor.create("testRollbackAndDoubleExecution").build();

View File

@ -94,7 +94,7 @@ public class TestDeleteNamespaceProcedure {
}
}
@Test(timeout = 60000)
@Test
public void testDeleteNamespace() throws Exception {
final String namespaceName = "testDeleteNamespace";
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
@ -110,7 +110,7 @@ public class TestDeleteNamespaceProcedure {
validateNamespaceNotExist(namespaceName);
}
@Test(timeout=60000)
@Test
public void testDeleteNonExistNamespace() throws Exception {
final String namespaceName = "testDeleteNonExistNamespace";
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
@ -129,7 +129,7 @@ public class TestDeleteNamespaceProcedure {
ProcedureTestingUtility.getExceptionCause(result) instanceof NamespaceNotFoundException);
}
@Test(timeout=60000)
@Test
public void testDeleteSystemNamespace() throws Exception {
final String namespaceName = NamespaceDescriptor.SYSTEM_NAMESPACE.getName();
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
@ -144,7 +144,7 @@ public class TestDeleteNamespaceProcedure {
assertTrue(ProcedureTestingUtility.getExceptionCause(result) instanceof ConstraintException);
}
@Test(timeout=60000)
@Test
public void testDeleteNonEmptyNamespace() throws Exception {
final String namespaceName = "testDeleteNonExistNamespace";
final TableName tableName = TableName.valueOf("testDeleteNonExistNamespace:" + name.getMethodName());
@ -164,7 +164,7 @@ public class TestDeleteNamespaceProcedure {
assertTrue(ProcedureTestingUtility.getExceptionCause(result) instanceof ConstraintException);
}
@Test(timeout = 60000)
@Test
public void testRecoveryAndDoubleExecution() throws Exception {
final String namespaceName = "testRecoveryAndDoubleExecution";
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
@ -186,7 +186,7 @@ public class TestDeleteNamespaceProcedure {
validateNamespaceNotExist(namespaceName);
}
@Test(timeout = 60000)
@Test
public void testRollbackAndDoubleExecution() throws Exception {
final String namespaceName = "testRollbackAndDoubleExecution";
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();

View File

@ -48,7 +48,7 @@ public class TestDeleteTableProcedure extends TestTableDDLProcedureBase {
private static final Logger LOG = LoggerFactory.getLogger(TestDeleteTableProcedure.class);
@Rule public TestName name = new TestName();
@Test(timeout=60000, expected=TableNotFoundException.class)
@Test(expected=TableNotFoundException.class)
public void testDeleteNotExistentTable() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
@ -59,7 +59,7 @@ public class TestDeleteTableProcedure extends TestTableDDLProcedureBase {
latch.await();
}
@Test(timeout=60000, expected=TableNotDisabledException.class)
@Test(expected=TableNotDisabledException.class)
public void testDeleteNotDisabledTable() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
@ -72,7 +72,7 @@ public class TestDeleteTableProcedure extends TestTableDDLProcedureBase {
latch.await();
}
@Test(timeout=60000)
@Test
public void testDeleteDeletedTable() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
@ -103,14 +103,14 @@ public class TestDeleteTableProcedure extends TestTableDDLProcedureBase {
assertTrue(ProcedureTestingUtility.getExceptionCause(result) instanceof TableNotFoundException);
}
@Test(timeout=60000)
@Test
public void testSimpleDelete() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
final byte[][] splitKeys = null;
testSimpleDelete(tableName, splitKeys);
}
@Test(timeout=60000)
@Test
public void testSimpleDeleteWithSplits() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
final byte[][] splitKeys = new byte[][] {
@ -132,7 +132,7 @@ public class TestDeleteTableProcedure extends TestTableDDLProcedureBase {
MasterProcedureTestingUtility.validateTableDeletion(getMaster(), tableName);
}
@Test(timeout=60000)
@Test
public void testRecoveryAndDoubleExecution() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());

View File

@ -48,7 +48,7 @@ public class TestDisableTableProcedure extends TestTableDDLProcedureBase {
@Rule public TestName name = new TestName();
@Test(timeout = 60000)
@Test
public void testDisableTable() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
@ -64,7 +64,7 @@ public class TestDisableTableProcedure extends TestTableDDLProcedureBase {
MasterProcedureTestingUtility.validateTableIsDisabled(getMaster(), tableName);
}
@Test(timeout = 60000)
@Test
public void testDisableTableMultipleTimes() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
@ -112,7 +112,7 @@ public class TestDisableTableProcedure extends TestTableDDLProcedureBase {
MasterProcedureTestingUtility.validateTableIsDisabled(getMaster(), tableName);
}
@Test(timeout=60000)
@Test
public void testRecoveryAndDoubleExecution() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();

View File

@ -47,7 +47,7 @@ public class TestEnableTableProcedure extends TestTableDDLProcedureBase {
private static final Logger LOG = LoggerFactory.getLogger(TestEnableTableProcedure.class);
@Rule public TestName name = new TestName();
@Test(timeout = 60000)
@Test
public void testEnableTable() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
@ -64,7 +64,7 @@ public class TestEnableTableProcedure extends TestTableDDLProcedureBase {
MasterProcedureTestingUtility.validateTableIsEnabled(getMaster(), tableName);
}
@Test(timeout=60000, expected=TableNotDisabledException.class)
@Test(expected=TableNotDisabledException.class)
public void testEnableNonDisabledTable() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
@ -97,7 +97,7 @@ public class TestEnableTableProcedure extends TestTableDDLProcedureBase {
Assert.fail("Enable should throw exception through latch.");
}
@Test(timeout = 60000)
@Test
public void testRecoveryAndDoubleExecution() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
@ -120,7 +120,7 @@ public class TestEnableTableProcedure extends TestTableDDLProcedureBase {
MasterProcedureTestingUtility.validateTableIsEnabled(getMaster(), tableName);
}
@Test(timeout = 60000)
@Test
public void testRollbackAndDoubleExecution() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();

View File

@ -36,7 +36,7 @@ public class TestFastFailOnProcedureNotRegistered extends TestTableDDLProcedureB
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestFastFailOnProcedureNotRegistered.class);
@Test(expected=DoNotRetryIOException.class, timeout = 3000)
@Test(expected=DoNotRetryIOException.class)
public void testFastFailOnProcedureNotRegistered() throws IOException {
Admin admin = UTIL.getAdmin();
Map<String, String> props = new HashMap<String, String>();

View File

@ -90,7 +90,7 @@ public class TestMasterProcedureEvents {
}
}
@Test(timeout = 30000)
@Test
public void testMasterInitializedEvent() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
HMaster master = UTIL.getMiniHBaseCluster().getMaster();
@ -108,7 +108,7 @@ public class TestMasterProcedureEvents {
new CreateTableProcedure(procExec.getEnvironment(), htd, new HRegionInfo[] { hri }));
}
@Test(timeout = 30000)
@Test
public void testServerCrashProcedureEvent() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
HMaster master = UTIL.getMiniHBaseCluster().getMaster();

View File

@ -71,7 +71,7 @@ public class TestMasterProcedureSchedulerConcurrency {
* Verify that "write" operations for a single table are serialized,
* but different tables can be executed in parallel.
*/
@Test(timeout=60000)
@Test
public void testConcurrentWriteOps() throws Exception {
final TestTableProcSet procSet = new TestTableProcSet(queue);
@ -157,7 +157,7 @@ public class TestMasterProcedureSchedulerConcurrency {
}
}
@Test(timeout=60000)
@Test
public void testMasterProcedureSchedulerPerformanceEvaluation() throws Exception {
// Make sure the tool does not get stuck
MasterProcedureSchedulerPerformanceEvaluation.main(new String[] {

View File

@ -88,7 +88,7 @@ public class TestModifyNamespaceProcedure {
}
@Test(timeout = 60000)
@Test
public void testModifyNamespace() throws Exception {
final NamespaceDescriptor nsd = NamespaceDescriptor.create("testModifyNamespace").build();
final String nsKey1 = "hbase.namespace.quota.maxregions";
@ -124,7 +124,7 @@ public class TestModifyNamespaceProcedure {
assertEquals(nsValue2, currentNsDescriptor.getConfigurationValue(nsKey2));
}
@Test(timeout=60000)
@Test
public void testModifyNonExistNamespace() throws Exception {
final String namespaceName = "testModifyNonExistNamespace";
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
@ -152,7 +152,7 @@ public class TestModifyNamespaceProcedure {
ProcedureTestingUtility.getExceptionCause(result) instanceof NamespaceNotFoundException);
}
@Test(timeout=60000)
@Test
public void testModifyNamespaceWithInvalidRegionCount() throws Exception {
final NamespaceDescriptor nsd =
NamespaceDescriptor.create("testModifyNamespaceWithInvalidRegionCount").build();
@ -175,7 +175,7 @@ public class TestModifyNamespaceProcedure {
assertTrue(ProcedureTestingUtility.getExceptionCause(result) instanceof ConstraintException);
}
@Test(timeout=60000)
@Test
public void testModifyNamespaceWithInvalidTableCount() throws Exception {
final NamespaceDescriptor nsd =
NamespaceDescriptor.create("testModifyNamespaceWithInvalidTableCount").build();
@ -198,7 +198,7 @@ public class TestModifyNamespaceProcedure {
assertTrue(ProcedureTestingUtility.getExceptionCause(result) instanceof ConstraintException);
}
@Test(timeout = 60000)
@Test
public void testRecoveryAndDoubleExecution() throws Exception {
final NamespaceDescriptor nsd =
NamespaceDescriptor.create("testRecoveryAndDoubleExecution").build();
@ -227,7 +227,7 @@ public class TestModifyNamespaceProcedure {
assertEquals(nsValue, currentNsDescriptor.getConfigurationValue(nsKey));
}
@Test(timeout = 60000)
@Test
public void testRollbackAndDoubleExecution() throws Exception {
final NamespaceDescriptor nsd =
NamespaceDescriptor.create("testRollbackAndDoubleExecution").build();

View File

@ -48,7 +48,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
@Rule public TestName name = new TestName();
@Test(timeout=60000)
@Test
public void testModifyTable() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
@ -86,7 +86,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
assertEquals(newMemStoreFlushSize, currentHtd.getMemStoreFlushSize());
}
@Test(timeout = 60000)
@Test
public void testModifyTableAddCF() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
@ -126,7 +126,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
assertEquals(3, currentHtd.getFamiliesKeys().size());
}
@Test(timeout = 60000)
@Test
public void testModifyTableDeleteCF() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
final String cf1 = "cf1";
@ -185,7 +185,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
assertTrue(currentHtd.hasFamily(Bytes.toBytes(cf1)));
}
@Test(timeout=60000)
@Test
public void testRecoveryAndDoubleExecutionOffline() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
final String cf2 = "cf2";
@ -225,7 +225,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
tableName, regions, false, "cf1", cf2);
}
@Test(timeout = 60000)
@Test
public void testRecoveryAndDoubleExecutionOnline() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
final String cf2 = "cf2";
@ -264,7 +264,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
tableName, regions, "cf1", cf2);
}
@Test(timeout = 60000)
@Test
public void testRollbackAndDoubleExecutionOnline() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
final String familyName = "cf2";
@ -293,7 +293,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
tableName, regions, "cf1");
}
@Test(timeout = 60000)
@Test
public void testRollbackAndDoubleExecutionOffline() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
final String familyName = "cf2";

View File

@ -94,7 +94,7 @@ public class TestProcedureAdmin {
}
}
@Test(timeout=60000)
@Test
public void testAbortProcedureSuccess() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
@ -119,7 +119,7 @@ public class TestProcedureAdmin {
tableName);
}
@Test(timeout=60000)
@Test
public void testAbortProcedureFailure() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
@ -151,7 +151,7 @@ public class TestProcedureAdmin {
UTIL.getHBaseCluster().getMaster(), tableName);
}
@Test(timeout=60000)
@Test
public void testAbortProcedureInterruptedNotAllowed() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
@ -179,7 +179,7 @@ public class TestProcedureAdmin {
UTIL.getHBaseCluster().getMaster(), tableName);
}
@Test(timeout=60000)
@Test
public void testAbortNonExistProcedure() throws Exception {
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
Random randomGenerator = new Random();
@ -193,7 +193,7 @@ public class TestProcedureAdmin {
assertFalse(abortResult);
}
@Test(timeout=60000)
@Test
public void testGetProcedure() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();

View File

@ -148,7 +148,7 @@ public class TestRestoreSnapshotProcedure extends TestTableDDLProcedureBase {
return htd;
}
@Test(timeout=600000)
@Test
public void testRestoreSnapshot() throws Exception {
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
@ -160,7 +160,7 @@ public class TestRestoreSnapshotProcedure extends TestTableDDLProcedureBase {
validateSnapshotRestore();
}
@Test(timeout=60000)
@Test
public void testRestoreSnapshotToDifferentTable() throws Exception {
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
final TableName restoredTableName = TableName.valueOf(name.getMethodName());
@ -175,7 +175,7 @@ public class TestRestoreSnapshotProcedure extends TestTableDDLProcedureBase {
ProcedureTestingUtility.getExceptionCause(result) instanceof TableNotFoundException);
}
@Test(timeout=60000)
@Test
public void testRestoreSnapshotToEnabledTable() throws Exception {
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
@ -195,7 +195,7 @@ public class TestRestoreSnapshotProcedure extends TestTableDDLProcedureBase {
}
}
@Test(timeout=60000)
@Test
public void testRecoveryAndDoubleExecution() throws Exception {
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();

View File

@ -94,7 +94,7 @@ public class TestSafemodeBringsDownMaster {
public void tearDown() throws Exception {
}
@Test(timeout=60000)
@Test
public void testSafemodeBringsDownMaster() throws Exception {
final TableName tableName = TableName.valueOf("testSafemodeBringsDownMaster");
final byte[][] splitKeys = new byte[][] {

View File

@ -87,17 +87,17 @@ public class TestServerCrashProcedure {
}
@Test(timeout=60000)
@Test
public void testCrashTargetRs() throws Exception {
testRecoveryAndDoubleExecution(false, false);
}
@Test(timeout=60000)
@Test
public void testRecoveryAndDoubleExecutionOnRsWithMeta() throws Exception {
testRecoveryAndDoubleExecution(true, true);
}
@Test(timeout=60000)
@Test
public void testRecoveryAndDoubleExecutionOnRsWithoutMeta() throws Exception {
testRecoveryAndDoubleExecution(false, true);
}

View File

@ -104,7 +104,7 @@ public class TestWALProcedureStoreOnHDFS {
}
}
@Test(timeout=60000, expected=RuntimeException.class)
@Test(expected=RuntimeException.class)
public void testWalAbortOnLowReplication() throws Exception {
setupDFS();
@ -123,7 +123,7 @@ public class TestWALProcedureStoreOnHDFS {
assertFalse(store.isRunning());
}
@Test(timeout=60000)
@Test
public void testWalAbortOnLowReplicationWithQueuedWriters() throws Exception {
setupDFS();
@ -167,7 +167,7 @@ public class TestWALProcedureStoreOnHDFS {
reCount.get() < thread.length);
}
@Test(timeout=60000)
@Test
public void testWalRollOnLowReplication() throws Exception {
UTIL.getConfiguration().setInt("dfs.namenode.replication.min", 1);
setupDFS();

View File

@ -92,7 +92,7 @@ public class TestSnapshotFileCache {
fs.delete(snapshotDir, true);
}
@Test(timeout = 10000000)
@Test
public void testLoadAndDelete() throws IOException {
// don't refresh the cache unless we tell it to
long period = Long.MAX_VALUE;

Some files were not shown because too many files have changed in this diff Show More