HBASE-5015 Remove some leaks in tests due to lack of HTable.close()

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1213823 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Zhihong Yu 2011-12-13 18:21:33 +00:00
parent 34dc2a4b66
commit 1b1725188e
37 changed files with 128 additions and 19 deletions

View File

@ -795,6 +795,7 @@ public class PerformanceEvaluation {
if (flushCommits) {
this.table.flushCommits();
}
table.close();
}
/*

View File

@ -26,6 +26,7 @@ import org.apache.commons.logging.LogFactory;
import java.lang.management.ManagementFactory;
import java.lang.management.OperatingSystemMXBean;
import java.util.*;
/**
@ -161,4 +162,18 @@ public class ResourceChecker {
return isOk;
}
/**
* Helper function: print the threads
*/
public static void printThreads(){
Set<Thread> threads = Thread.getAllStackTraces().keySet();
System.out.println("name; state; isDameon; isAlive; isInterrupted");
for (Thread t: threads){
System.out.println(
t.getName()+";"+t.getState()+";"+t.isDaemon()+";"+t.isAlive()+
";"+t.isInterrupted()
);
}
}
}

View File

@ -121,6 +121,8 @@ public class TestFullLogReconstruction {
newCount++;
}
assertEquals(count, newCount);
results.close();
table.close();
}
@org.junit.Rule

View File

@ -127,6 +127,7 @@ public class TestGlobalMemStoreSize {
assertEquals("Server=" + server.getServerName() + ", i=" + i++, 0, size);
}
ht.close();
TEST_UTIL.shutdownMiniCluster();
}

View File

@ -110,6 +110,9 @@ public class TestHBaseTestingUtility {
res = table2.get(get);
assertEquals(0, res.size());
table1.close();
table2.close();
} finally {
htu3.shutdownMiniCluster();
htu2.shutdownMiniCluster();

View File

@ -62,7 +62,7 @@ public class TestInfoServers {
@Test
public void testInfoServersRedirect() throws Exception {
// give the cluster time to start up
new HTable(UTIL.getConfiguration(), ".META.");
new HTable(UTIL.getConfiguration(), ".META.").close();
int port = UTIL.getHBaseCluster().getMaster().getInfoServer().getPort();
assertHasExpectedContent(new URL("http://localhost:" + port +
"/index.html"), "master-status");
@ -82,7 +82,7 @@ public class TestInfoServers {
@Test
public void testInfoServersStatusPages() throws Exception {
// give the cluster time to start up
new HTable(UTIL.getConfiguration(), ".META.");
new HTable(UTIL.getConfiguration(), ".META.").close();
int port = UTIL.getHBaseCluster().getMaster().getInfoServer().getPort();
assertHasExpectedContent(new URL("http://localhost:" + port +
"/master-status"), "META");

View File

@ -110,6 +110,8 @@ public class TestMultiVersions {
UTIL.getMiniHBaseCluster().flushcache();
}
});
table.close();
}
/**
@ -135,6 +137,7 @@ public class TestMultiVersions {
HTable table = new HTable(UTIL.getConfiguration(), tableName);
table.put(put);
// Shut down and restart the HBase cluster
table.close();
UTIL.shutdownMiniHBaseCluster();
LOG.debug("HBase cluster shut down -- restarting");
UTIL.startMiniHBaseCluster(1, 1);
@ -171,6 +174,7 @@ public class TestMultiVersions {
assertTrue(versionMap.size() == 2);
assertTrue(Bytes.equals(value1, versionMap.get(timestamp1)));
assertTrue(Bytes.equals(value2, versionMap.get(timestamp2)));
table.close();
}
/**
@ -231,6 +235,7 @@ public class TestMultiVersions {
}
assertTrue(cellCount == 1);
}
table.close();
}
// Case 1: scan with LATEST_TIMESTAMP. Should get two rows

View File

@ -126,6 +126,7 @@ public class TestRegionRebalancing {
}
UTIL.getHBaseCluster().getMaster().balance();
assertRegionsAreBalanced();
table.close();
}
/** figure out how many regions are currently being served. */

View File

@ -96,7 +96,7 @@ public class TestZooKeeper {
throws IOException, InterruptedException {
LOG.info("testClientSessionExpired");
Configuration c = new Configuration(TEST_UTIL.getConfiguration());
new HTable(c, HConstants.META_TABLE_NAME);
new HTable(c, HConstants.META_TABLE_NAME).close();
String quorumServers = ZKConfig.getZKQuorumServersString(c);
int sessionTimeout = 5 * 1000; // 5 seconds
HConnection connection = HConnectionManager.getConnection(c);
@ -165,7 +165,7 @@ public class TestZooKeeper {
Bytes.toBytes("col"), Bytes.toBytes("testdata"));
LOG.info("Putting table " + tableName);
table.put(put);
table.close();
}
@Test
@ -187,6 +187,8 @@ public class TestZooKeeper {
assertFalse(HConnectionManager.getConnection(localMeta.getConfiguration())
.getZooKeeperWatcher().getQuorum().equals(HConnectionManager
.getConnection(otherConf).getZooKeeperWatcher().getQuorum()));
localMeta.close();
ipMeta.close();
} catch (Exception e) {
e.printStackTrace();
fail();

View File

@ -141,6 +141,7 @@ public class TestMetaReaderEditor {
writer.stop = true;
reader.join();
writer.join();
t.close();
}
/**

View File

@ -140,6 +140,7 @@ public class TestHCM {
conn.deleteCachedLocation(TABLE_NAME, ROW);
HRegionLocation rl = conn.getCachedLocation(TABLE_NAME, ROW);
assertNull("What is this location?? " + rl, rl);
table.close();
}
/**

View File

@ -89,6 +89,7 @@ public class TestHTableUtil {
}
LOG.info("bucket put count=" + count);
assertEquals(count, puts.size());
ht.close();
}
private Put createPut(String row) {
@ -123,6 +124,7 @@ public class TestHTableUtil {
}
LOG.info("bucket batch count=" + count);
assertEquals(count, rows.size());
ht.close();
}

View File

@ -71,7 +71,7 @@ public class TestInstantSchemaChange extends InstantSchemaChangeTestBase {
int result = Bytes.compareTo(value, tvalue);
assertEquals(result, 0);
LOG.info("END testInstantSchemaChangeForModifyTable()");
ht.close();
}
@Test
@ -100,7 +100,7 @@ public class TestInstantSchemaChange extends InstantSchemaChangeTestBase {
int result = Bytes.compareTo(value, tvalue);
assertEquals(result, 0);
LOG.info("End testInstantSchemaChangeForAddColumn() ");
ht.close();
}
@Test
@ -158,6 +158,7 @@ public class TestInstantSchemaChange extends InstantSchemaChangeTestBase {
HColumnDescriptor hcd = modifiedHtd.getFamily(Bytes.toBytes("C"));
assertTrue(hcd == null);
LOG.info("End testInstantSchemaChangeForDeleteColumn() ");
ht.close();
}
@Test
@ -175,6 +176,7 @@ public class TestInstantSchemaChange extends InstantSchemaChangeTestBase {
MasterSchemaChangeTracker msct =
TEST_UTIL.getHBaseCluster().getMaster().getSchemaChangeTracker();
assertTrue(msct.doesSchemaChangeNodeExists(tableName) == false);
ht.close();
}
/**
@ -246,6 +248,7 @@ public class TestInstantSchemaChange extends InstantSchemaChangeTestBase {
int result2 = Bytes.compareTo(value, tvalue2);
assertEquals(result2, 0);
LOG.info("END testConcurrentInstantSchemaChangeForModifyTable()");
ht.close();
}
/**
@ -307,6 +310,7 @@ public class TestInstantSchemaChange extends InstantSchemaChangeTestBase {
assertEquals(result, 0);
LOG.info("End testInstantSchemaChangeWithLoadBalancerRunning() ");
ht.close();
}

View File

@ -161,6 +161,7 @@ public class TestInstantSchemaChangeFailover {
assertTrue(ZKUtil.checkExists(zkw, nodePath) == -1);
LOG.info("result2 = " + result2);
LOG.info("end testInstantSchemaChangeWhileRSCrash()");
ht.close();
}
/**
@ -205,6 +206,7 @@ public class TestInstantSchemaChangeFailover {
assertEquals(result2, 0);
LOG.info("result2 = " + result2);
LOG.info("end testInstantSchemaChangeWhileRandomRSCrashAndStart()");
ht.close();
}
/**
@ -246,6 +248,7 @@ public class TestInstantSchemaChangeFailover {
assertEquals(result2, 0);
LOG.info("result2 = " + result2);
LOG.info("end testInstantSchemaChangeWhileMasterFailover()");
ht.close();
}
/**

View File

@ -89,7 +89,7 @@ public class TestInstantSchemaChangeSplit extends InstantSchemaChangeTestBase {
assertEquals(tableHcd.getMaxVersions(), 99);
}
LOG.info("End testInstantSchemaChangeExclusions() ");
ht.close();
}
/**
@ -145,6 +145,7 @@ public class TestInstantSchemaChangeSplit extends InstantSchemaChangeTestBase {
MasterSchemaChangeTracker.MasterAlterStatus.AlterState.FAILURE);
assertTrue(mas.getErrorCause() != null);
LOG.info("End testInstantSchemaChangeWhileRSOpenRegionFailure() ");
ht.close();
}
@Test
@ -209,6 +210,7 @@ public class TestInstantSchemaChangeSplit extends InstantSchemaChangeTestBase {
int result = Bytes.compareTo(value, tvalue);
assertEquals(result, 0);
LOG.info("End testConcurrentInstantSchemaChangeAndSplit() ");
ht.close();
}

View File

@ -94,6 +94,8 @@ public class TestMetaMigrationRemovingHTD {
// Assert that we find all 100 rows that are in the data we loaded. If
// so then we must have migrated it from 0.90 to 0.92.
Assert.assertEquals(ROWCOUNT, count);
scanner.close();
t.close();
}
private static File untar(final File testdir) throws IOException {
@ -305,6 +307,7 @@ public class TestMetaMigrationRemovingHTD {
newRegions.add(hri);
count++;
}
meta.close();
return count;
}
@ -349,6 +352,7 @@ public class TestMetaMigrationRemovingHTD {
newRegions.add(hri);
count++;
}
meta.close();
return count;
}

View File

@ -92,7 +92,7 @@ public class TestMetaScanner {
doReturn(true).when(visitor).processRow((Result)anyObject());
MetaScanner.metaScan(conf, visitor, TABLENAME, Bytes.toBytes("region_ac"), 1);
verify(visitor, times(1)).processRow((Result)anyObject());
table.close();
}
@org.junit.Rule

View File

@ -57,6 +57,7 @@ public class TestMultiParallel {
UTIL.startMiniCluster(slaves);
HTable t = UTIL.createTable(Bytes.toBytes(TEST_TABLE), Bytes.toBytes(FAMILY));
UTIL.createMultiRegions(t, Bytes.toBytes(FAMILY));
t.close();
}
@AfterClass public static void afterClass() throws Exception {
@ -126,6 +127,7 @@ public class TestMultiParallel {
poolField.setAccessible(true);
ThreadPoolExecutor tExecutor = (ThreadPoolExecutor) poolField.get(table);
assertEquals(slaves, tExecutor.getLargestPoolSize());
table.close();
}
@Test(timeout=300000)
@ -165,6 +167,7 @@ public class TestMultiParallel {
.getValue()));
}
}
table.close();
}
@Test
@ -194,6 +197,7 @@ public class TestMultiParallel {
assertEquals(2, r.length);
assertTrue(r[0] instanceof Throwable);
assertTrue(r[1] instanceof Result);
table.close();
}
/**
@ -259,6 +263,7 @@ public class TestMultiParallel {
int regions = t.getRegionServer().getOnlineRegions().size();
Assert.assertTrue("Count of regions=" + regions, regions > 10);
}
table.close();
LOG.info("done");
}
@ -282,6 +287,7 @@ public class TestMultiParallel {
}
validateLoadedData(table);
table.close();
}
@Test(timeout=300000)
@ -310,7 +316,7 @@ public class TestMultiParallel {
get.addColumn(BYTES_FAMILY, QUALIFIER);
Assert.assertFalse(table.exists(get));
}
table.close();
}
@Test(timeout=300000)
@ -339,7 +345,7 @@ public class TestMultiParallel {
get.addColumn(BYTES_FAMILY, QUALIFIER);
Assert.assertFalse(table.exists(get));
}
table.close();
}
@Test(timeout=300000)
@ -376,7 +382,7 @@ public class TestMultiParallel {
validateResult(r, qual, VALUE);
idx++;
}
table.close();
}
@Test(timeout=300000)
@ -445,6 +451,8 @@ public class TestMultiParallel {
get.addColumn(BYTES_FAMILY, qual2);
Result r = table.get(get);
validateResult(r, qual2, val2);
table.close();
}
// // Helper methods ////

View File

@ -115,6 +115,8 @@ public class TestMultipleTimestamps {
assertEquals(2, kvs.length);
checkOneCell(kvs[0], FAMILY, 5, 3, 4);
checkOneCell(kvs[1], FAMILY, 5, 3, 3);
ht.close();
}
@Test
@ -152,6 +154,8 @@ public class TestMultipleTimestamps {
kvs = scanner.next().raw();
assertEquals(1, kvs.length);
checkOneCell(kvs[0], FAMILY, 5, 3, 3);
ht.close();
}
@Test
@ -197,6 +201,8 @@ public class TestMultipleTimestamps {
checkOneCell(kvs[1], FAMILY, 7, 3, 2);
checkOneCell(kvs[2], FAMILY, 7, 5, 3);
checkOneCell(kvs[3], FAMILY, 7, 5, 2);
ht.close();
}
@Test
@ -254,6 +260,8 @@ public class TestMultipleTimestamps {
kvs = scanner.next().raw();
assertEquals(1, kvs.length);
checkOneCell(kvs[0], FAMILY, 7, 3, 4);
ht.close();
}
@Test
@ -297,6 +305,8 @@ public class TestMultipleTimestamps {
checkOneCell(kvs[0], FAMILY, 0, 0, 5);
checkOneCell(kvs[1], FAMILY, 0, 0, 3);
checkOneCell(kvs[2], FAMILY, 0, 0, 2);
ht.close();
}
@Test
@ -322,6 +332,8 @@ public class TestMultipleTimestamps {
// only get back entries for the versions that exist.
KeyValue kvs[] = getNVersions(ht, FAMILY, 0, 0, Arrays.asList(2L, 3L));
assertEquals(0, kvs.length);
ht.close();
}
@Test
@ -345,6 +357,8 @@ public class TestMultipleTimestamps {
// only get back entries for the versions that exist.
KeyValue kvs[] = getNVersions(ht, FAMILY, 0, 0, Arrays.asList(2L, 3L));
assertEquals(0, kvs.length);
ht.close();
}
@Test
@ -368,6 +382,8 @@ public class TestMultipleTimestamps {
// only get back entries for the versions that exist.
KeyValue kvs[] = getNVersions(ht, FAMILY, 0, 0, Arrays.asList(2L, 3L));
assertEquals(0, kvs.length);
ht.close();
}
/**

View File

@ -68,6 +68,7 @@ public class TestScannerTimeout {
put.add(SOME_BYTES, SOME_BYTES, SOME_BYTES);
table.put(put);
}
table.close();
}
/**
@ -111,6 +112,8 @@ public class TestScannerTimeout {
} catch (ScannerTimeoutException e) {
LOG.info("Got the timeout " + e.getMessage(), e);
return;
} finally {
table.close();
}
fail("We should be timing out");
LOG.info("END ************ test2481");
@ -140,6 +143,7 @@ public class TestScannerTimeout {
Result[] results = r.next(NB_ROWS);
assertEquals(NB_ROWS, results.length);
r.close();
higherScanTimeoutTable.close();
LOG.info("END ************ test2772");
}
@ -176,6 +180,7 @@ public class TestScannerTimeout {
}
assertEquals(NB_ROWS, count);
r.close();
table.close();
LOG.info("************ END TEST3686A");
}
@ -209,6 +214,7 @@ public class TestScannerTimeout {
}
assertEquals(NB_ROWS, count);
r.close();
higherScanTimeoutTable.close();
LOG.info("END ************ END test3686b");
}

View File

@ -162,6 +162,7 @@ public class TestTimestampsFilter {
checkOneCell(kvs[offset + 2], FAMILY, rowIdx, colIdx, 6);
}
}
ht.close();
}
@Test
@ -208,6 +209,8 @@ public class TestTimestampsFilter {
"value2-3");
assertEquals(Bytes.toString(result.list().get(1).getValue()),
"value4-3");
ht.close();
}
/**
@ -251,6 +254,8 @@ public class TestTimestampsFilter {
checkOneCell(kvs[0], FAMILY, 0, 0, 5);
checkOneCell(kvs[1], FAMILY, 0, 0, 3);
checkOneCell(kvs[2], FAMILY, 0, 0, 2);
ht.close();
}
private void verifyInsertedValues(HTable ht, byte[] cf) throws IOException {

View File

@ -97,6 +97,7 @@ public class TestAggregateProtocol {
.toBytes(l * 10));
table.put(p2);
}
table.close();
}
/**

View File

@ -94,6 +94,7 @@ public class TestCoprocessorEndpoint {
(table.getRegionsInfo().size() != 2)) {
Thread.sleep(250);
}
table.close();
}
@AfterClass
@ -128,12 +129,12 @@ public class TestCoprocessorEndpoint {
assertEquals(6f, workResult10, 0.01);
Text workResult11 = protocol.doWork(new Text("foo"));
assertEquals(new Text("foo"), workResult11);
table.close();
}
@Test
public void testAggregation() throws Throwable {
HTable table = new HTable(util.getConfiguration(), TEST_TABLE);
Scan scan;
Map<byte[], Long> results;
// scan: for all regions
@ -177,6 +178,7 @@ public class TestCoprocessorEndpoint {
expectedResult += i;
}
assertEquals("Invalid result", sumResult, expectedResult);
table.close();
}
@Test

View File

@ -706,6 +706,7 @@ public class TestMasterObserver {
boolean balanceRun = master.balance();
assertTrue("Coprocessor should be called on region rebalancing",
cp.wasBalanceCalled());
table.close();
}
@org.junit.Rule

View File

@ -81,6 +81,7 @@ public class TestRegionObserverBypass {
// before HBASE-4331, this would throw an exception
t.put(p);
checkRowAndDelete(t,row1,0);
t.close();
}
/**
@ -169,6 +170,7 @@ public class TestRegionObserverBypass {
checkRowAndDelete(t,row1,0);
checkRowAndDelete(t,row2,1);
checkRowAndDelete(t,row3,0);
t.close();
}
private void checkRowAndDelete(HTable t, byte[] row, int count) throws IOException {

View File

@ -128,6 +128,7 @@ public class TestRegionObserverInterface {
new Boolean[] {true, true, true, true, true}
);
util.deleteTable(tableName);
table.close();
}
@Test
@ -152,6 +153,7 @@ public class TestRegionObserverInterface {
new Boolean[] {true, true}
);
util.deleteTable(tableName);
table.close();
}
@Test
@ -201,6 +203,7 @@ public class TestRegionObserverInterface {
new Boolean[] {true, true}
);
util.deleteTable(tableName);
table.close();
}
@Test
@ -245,6 +248,7 @@ public class TestRegionObserverInterface {
new Boolean[] {true}
);
util.deleteTable(tableName);
table.close();
}
/* Overrides compaction to only output rows with keys that are even numbers */
@ -374,6 +378,7 @@ public class TestRegionObserverInterface {
} finally {
scanner.close();
}
table.close();
}
// check each region whether the coprocessor upcalls are called or not.

View File

@ -154,6 +154,7 @@ public class TestRegionServerCoprocessorExceptionWithAbort {
assertTrue("Main thread caught interruption.",caughtInterruption);
assertTrue("RegionServer aborted on coprocessor exception, as expected.",
rsTracker.regionZKNodeWasDeleted);
table.close();
}
public static class BuggyRegionObserver extends SimpleRegionObserver {

View File

@ -134,6 +134,7 @@ public class TestRegionServerCoprocessorExceptionWithRemove {
"zk node to be deleted.");
}
}
table.close();
}
@org.junit.Rule

View File

@ -239,6 +239,7 @@ public class TestColumnRangeFilter {
assertEquals(rangeMap.get(s).size(), results.size());
}
ht.close();
}
List<String> generateRandomWords(int numberOfWords, int maxLengthOfWords) {

View File

@ -140,6 +140,7 @@ public class TestDistributedLogSplitting {
assertEquals(NUM_REGIONS_TO_CREATE * NUM_ROWS_PER_REGION,
TEST_UTIL.countRows(ht));
ht.close();
}
@Test(expected=OrphanHLogAfterSplitException.class, timeout=300000)

View File

@ -69,9 +69,9 @@ public class TestMaster {
HMaster m = cluster.getMaster();
HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
TEST_UTIL.createTable(TABLENAME, FAMILYNAME);
TEST_UTIL.loadTable(new HTable(TEST_UTIL.getConfiguration(), TABLENAME),
FAMILYNAME);
HTable ht = TEST_UTIL.createTable(TABLENAME, FAMILYNAME);
TEST_UTIL.loadTable(ht, FAMILYNAME);
ht.close();
List<Pair<HRegionInfo, ServerName>> tableRegions =
MetaReader.getTableRegionsAndLocations(m.getCatalogTracker(),
@ -113,6 +113,7 @@ public class TestMaster {
} finally {
proceed.countDown();
}
admin.close();
}
static class RegionSplitListener implements EventHandlerListener {

View File

@ -104,6 +104,7 @@ public class TestMasterRestartAfterDisablingTable {
assertEquals(
"The assigned regions were not onlined after master switch except for the catalog tables.",
6, regions.size());
ht.close();
TEST_UTIL.shutdownMiniCluster();
}

View File

@ -64,6 +64,7 @@ public class TestMasterTransitions {
int countOfRegions = TEST_UTIL.createMultiRegions(t, getTestFamily());
TEST_UTIL.waitUntilAllRegionsAssigned(countOfRegions);
addToEachStartKey(countOfRegions);
t.close();
}
@AfterClass public static void afterAllTests() throws Exception {
@ -496,6 +497,8 @@ public class TestMasterTransitions {
}
s.close();
Assert.assertEquals(expected, rows);
t.close();
meta.close();
return rows;
}
@ -513,6 +516,7 @@ public class TestMasterTransitions {
}
s.close();
LOG.info("Counted=" + rows);
t.close();
return rows;
}

View File

@ -295,7 +295,7 @@ public class TestRollingRestart {
// TODO: Bring random 3 of 4 RS down at the same time
ht.close();
// Stop the cluster
TEST_UTIL.shutdownMiniCluster();
}

View File

@ -73,6 +73,7 @@ public class TestZKBasedOpenCloseRegion {
countOfRegions = TEST_UTIL.createMultiRegions(t, getTestFamily());
waitUntilAllRegionsAssigned();
addToEachStartKey(countOfRegions);
t.close();
}
@AfterClass public static void afterAllTests() throws Exception {
@ -332,6 +333,7 @@ public class TestZKBasedOpenCloseRegion {
LOG.info("Found=" + rows);
Threads.sleep(1000);
}
meta.close();
}
/*
@ -366,6 +368,8 @@ public class TestZKBasedOpenCloseRegion {
}
s.close();
Assert.assertEquals(expected, rows);
t.close();
meta.close();
return rows;
}

View File

@ -56,9 +56,9 @@ public class TestEndToEndSplitTransaction {
public void testMasterOpsWhileSplitting() throws Exception {
byte[] tableName = Bytes.toBytes("TestSplit");
byte[] familyName = Bytes.toBytes("fam");
TEST_UTIL.createTable(tableName, familyName);
TEST_UTIL.loadTable(new HTable(TEST_UTIL.getConfiguration(), tableName),
familyName);
HTable ht = TEST_UTIL.createTable(tableName, familyName);
TEST_UTIL.loadTable(ht, familyName);
ht.close();
HRegionServer server = TEST_UTIL.getHBaseCluster().getRegionServer(0);
byte []firstRow = Bytes.toBytes("aaa");
byte []splitRow = Bytes.toBytes("lll");

View File

@ -3178,6 +3178,8 @@ public class TestHRegion extends HBaseTestCase {
blocksDistribution2.getUniqueBlocksTotalWeight();
assertTrue(uniqueBlocksWeight1 == uniqueBlocksWeight2);
ht.close();
} finally {
htu.shutdownMiniCluster();
}