HBASE-5833 0.92 build has been failing pretty consistently on TestMasterFailover

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1329400 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2012-04-23 19:40:13 +00:00
parent f8c7f1b0fb
commit fabaf478ed
20 changed files with 2837 additions and 2383 deletions

View File

@ -50,6 +50,7 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.HasThread;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.util.StringUtils;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
@ -579,16 +580,17 @@ public class LruBlockCache implements BlockCache, HeapSize {
*/
private static class EvictionThread extends HasThread {
private WeakReference<LruBlockCache> cache;
private boolean go = true;
public EvictionThread(LruBlockCache cache) {
super("LruBlockCache.EvictionThread");
super(Thread.currentThread().getName() + ".LruBlockCache.EvictionThread");
setDaemon(true);
this.cache = new WeakReference<LruBlockCache>(cache);
}
@Override
public void run() {
while(true) {
while (this.go) {
synchronized(this) {
try {
this.wait();
@ -599,11 +601,17 @@ public class LruBlockCache implements BlockCache, HeapSize {
cache.evict();
}
}
public void evict() {
synchronized(this) {
this.notify(); // FindBugs NN_NAKED_NOTIFY
}
}
void shutdown() {
this.go = false;
interrupt();
}
}
/*
@ -727,6 +735,14 @@ public class LruBlockCache implements BlockCache, HeapSize {
public void shutdown() {
this.scheduleThreadPool.shutdown();
for (int i = 0; i < 10; i++) {
if (!this.scheduleThreadPool.isShutdown()) Threads.sleep(10);
}
if (!this.scheduleThreadPool.isShutdown()) {
List<Runnable> runnables = this.scheduleThreadPool.shutdownNow();
LOG.debug("Still running " + runnables);
}
this.evictionThread.shutdown();
}
/** Clears the cache. Used in tests. */

View File

@ -44,7 +44,7 @@ public class ExponentiallyDecayingSample implements Sample {
private static final ScheduledExecutorService TICK_SERVICE =
Executors.newScheduledThreadPool(1,
getNamedDaemonThreadFactory("decayingSampleTick"));
getNamedDaemonThreadFactory(Thread.currentThread().getName() + ".decayingSampleTick."));
private static volatile long CURRENT_TICK =
TimeUnit.MILLISECONDS.toSeconds(System.currentTimeMillis());

View File

@ -3525,7 +3525,11 @@ public class HRegion implements HeapSize { // , Writable{
* bootstrap code in the HMaster constructor.
* Note, this method creates an {@link HLog} for the created region. It
* needs to be closed explicitly. Use {@link HRegion#getLog()} to get
* access.
* access. <b>When done with a region created using this method, you will
* need to explicitly close the {@link HLog} it created too; it will not be
* done for you. Not closing the log will leave at least a daemon thread
* running.</b> Call {@link #closeHRegion(HRegion)} and it will do
* necessary cleanup for you.
* @param info Info for region to create.
* @param rootDir Root directory for HBase instance
* @param conf
@ -3540,6 +3544,23 @@ public class HRegion implements HeapSize { // , Writable{
return createHRegion(info, rootDir, conf, hTableDescriptor, null);
}
/**
* This will do the necessary cleanup a call to {@link #createHRegion(HRegionInfo, Path, Configuration, HTableDescriptor)}
* requires. This method will close the region and then close its
* associated {@link HLog} file. You use it if you call the other createHRegion,
* the one that takes an {@link HLog} instance but don't be surprised by the
* call to the {@link HLog#closeAndDelete()} on the {@link HLog} the
* HRegion was carrying.
* @param r
* @throws IOException
*/
public static void closeHRegion(final HRegion r) throws IOException {
if (r == null) return;
r.close();
if (r.getLog() == null) return;
r.getLog().closeAndDelete();
}
/**
* Convenience method creating new HRegions. Used by createTable.
* The {@link HLog} for the created region needs to be closed explicitly.

View File

@ -223,13 +223,17 @@ public class JVMClusterUtil {
final List<RegionServerThread> regionservers) {
LOG.debug("Shutting down HBase Cluster");
if (masters != null) {
// Do backups first.
JVMClusterUtil.MasterThread activeMaster = null;
for (JVMClusterUtil.MasterThread t : masters) {
if (t.master.isActiveMaster()) {
t.master.shutdown();
} else {
if (!t.master.isActiveMaster()) {
t.master.stopMaster();
} else {
activeMaster = t;
}
}
// Do active after.
if (activeMaster != null) activeMaster.master.shutdown();
}
if (regionservers != null) {
for (RegionServerThread t : regionservers) {

View File

@ -151,6 +151,16 @@ public abstract class HBaseTestCase extends TestCase {
return testUtil.getDataTestDir(testName);
}
/**
* You must call close on the returned region and then close on the log file
* it created. Do {@link HRegion#close()} followed by {@link HRegion#getLog()}
* and on it call close.
* @param desc
* @param startKey
* @param endKey
* @return An {@link HRegion}
* @throws IOException
*/
protected HRegion createNewHRegion(HTableDescriptor desc, byte [] startKey,
byte [] endKey)
throws IOException {
@ -636,6 +646,11 @@ public abstract class HBaseTestCase extends TestCase {
}
}
/**
* You must call {@link #closeRootAndMeta()} when done after calling this
* method. It does cleanup.
* @throws IOException
*/
protected void createRootAndMetaRegions() throws IOException {
root = HRegion.createHRegion(HRegionInfo.ROOT_REGIONINFO, testDir,
conf, HTableDescriptor.ROOT_TABLEDESC);

View File

@ -50,7 +50,7 @@ public class TestColumnPrefixFilter {
HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
HRegion region = HRegion.createHRegion(info, TEST_UTIL.
getDataTestDir(), TEST_UTIL.getConfiguration(), htd);
try {
List<String> rows = generateRandomWords(100, "row");
List<String> columns = generateRandomWords(10000, "column");
long maxTimestamp = 2;
@ -97,6 +97,10 @@ public class TestColumnPrefixFilter {
while(scanner.next(results));
assertEquals(prefixMap.get(s).size(), results.size());
}
} finally {
region.close();
region.getLog().closeAndDelete();
}
region.close();
region.getLog().closeAndDelete();
@ -110,7 +114,7 @@ public class TestColumnPrefixFilter {
HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
HRegion region = HRegion.createHRegion(info, TEST_UTIL.
getDataTestDir(), TEST_UTIL.getConfiguration(), htd);
try {
List<String> rows = generateRandomWords(100, "row");
List<String> columns = generateRandomWords(10000, "column");
long maxTimestamp = 2;
@ -160,6 +164,10 @@ public class TestColumnPrefixFilter {
while(scanner.next(results));
assertEquals(prefixMap.get(s).size(), results.size());
}
} finally {
region.close();
region.getLog().closeAndDelete();
}
region.close();
region.getLog().closeAndDelete();

View File

@ -24,6 +24,7 @@ import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Set;
@ -56,6 +57,55 @@ import org.junit.experimental.categories.Category;
public class TestMasterFailover {
private static final Log LOG = LogFactory.getLog(TestMasterFailover.class);
@Test (timeout=180000)
public void testShouldCheckMasterFailOverWhenMETAIsInOpenedState()
throws Exception {
LOG.info("Starting testShouldCheckMasterFailOverWhenMETAIsInOpenedState");
final int NUM_MASTERS = 1;
final int NUM_RS = 2;
Configuration conf = HBaseConfiguration.create();
conf.setInt("hbase.master.assignment.timeoutmonitor.period", 2000);
conf.setInt("hbase.master.assignment.timeoutmonitor.timeout", 8000);
// Start the cluster
HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(conf);
TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
// Find regionserver carrying meta.
List<RegionServerThread> regionServerThreads =
cluster.getRegionServerThreads();
int count = -1;
HRegion metaRegion = null;
for (RegionServerThread regionServerThread : regionServerThreads) {
HRegionServer regionServer = regionServerThread.getRegionServer();
metaRegion = regionServer.getOnlineRegion(HRegionInfo.FIRST_META_REGIONINFO.getRegionName());
count++;
regionServer.abort("");
if (null != metaRegion) break;
}
HRegionServer regionServer = cluster.getRegionServer(count);
TEST_UTIL.shutdownMiniHBaseCluster();
// Create a ZKW to use in the test
ZooKeeperWatcher zkw =
HBaseTestingUtility.createAndForceNodeToOpenedState(TEST_UTIL,
metaRegion, regionServer.getServerName());
LOG.info("Staring cluster for second time");
TEST_UTIL.startMiniHBaseCluster(NUM_MASTERS, NUM_RS);
// Failover should be completed, now wait for no RIT
log("Waiting for no more RIT");
ZKAssign.blockUntilNoRIT(zkw);
zkw.close();
// Stop the cluster
TEST_UTIL.shutdownMiniCluster();
}
/**
* Simple test of master failover.
* <p>
@ -101,6 +151,7 @@ public class TestMasterFailover {
}
assertEquals(1, numActive);
assertEquals(NUM_MASTERS, masterThreads.size());
LOG.info("Active master " + activeName);
// Check that ClusterStatus reports the correct active and backup masters
assertNotNull(active);
@ -110,16 +161,16 @@ public class TestMasterFailover {
assertEquals(2, status.getBackupMasters().size());
// attempt to stop one of the inactive masters
LOG.debug("\n\nStopping a backup master\n");
int backupIndex = (activeIndex == 0 ? 1 : activeIndex - 1);
HMaster master = cluster.getMaster(backupIndex);
LOG.debug("\n\nStopping a backup master: " + master.getServerName() + "\n");
cluster.stopMaster(backupIndex, false);
cluster.waitOnMaster(backupIndex);
// verify still one active master and it's the same
// Verify still one active master and it's the same
for (int i = 0; i < masterThreads.size(); i++) {
if (masterThreads.get(i).getMaster().isActiveMaster()) {
assertTrue(activeName.equals(
masterThreads.get(i).getMaster().getServerName()));
assertTrue(activeName.equals(masterThreads.get(i).getMaster().getServerName()));
activeIndex = i;
active = masterThreads.get(activeIndex).getMaster();
}
@ -127,7 +178,7 @@ public class TestMasterFailover {
assertEquals(1, numActive);
assertEquals(2, masterThreads.size());
int rsCount = masterThreads.get(activeIndex).getMaster().getClusterStatus().getServersSize();
LOG.info("Active master managing " + rsCount + " regions servers");
LOG.info("Active master " + active.getServerName() + " managing " + rsCount + " regions servers");
assertEquals(3, rsCount);
// Check that ClusterStatus reports the correct active and backup masters
@ -138,7 +189,7 @@ public class TestMasterFailover {
assertEquals(1, status.getBackupMasters().size());
// kill the active master
LOG.debug("\n\nStopping the active master\n");
LOG.debug("\n\nStopping the active master " + active.getServerName() + "\n");
cluster.stopMaster(activeIndex, false);
cluster.waitOnMaster(activeIndex);
@ -159,7 +210,7 @@ public class TestMasterFailover {
assertEquals(0, status.getBackupMastersSize());
assertEquals(0, status.getBackupMasters().size());
int rss = status.getServersSize();
LOG.info("Active master " + mastername.getHostname() + " managing " +
LOG.info("Active master " + mastername.getServerName() + " managing " +
rss + " region servers");
assertEquals(3, rss);
@ -167,83 +218,6 @@ public class TestMasterFailover {
TEST_UTIL.shutdownMiniCluster();
}
@Test
public void testShouldCheckMasterFailOverWhenMETAIsInOpenedState()
throws Exception {
final int NUM_MASTERS = 1;
final int NUM_RS = 2;
Configuration conf = HBaseConfiguration.create();
conf.setInt("hbase.master.assignment.timeoutmonitor.period", 2000);
conf.setInt("hbase.master.assignment.timeoutmonitor.timeout", 8000);
// Start the cluster
HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(conf);
TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
// get all the master threads
List<MasterThread> masterThreads = cluster.getMasterThreads();
// wait for each to come online
for (MasterThread mt : masterThreads) {
assertTrue(mt.isAlive());
}
// verify only one is the active master and we have right number
int numActive = 0;
ServerName activeName = null;
for (int i = 0; i < masterThreads.size(); i++) {
if (masterThreads.get(i).getMaster().isActiveMaster()) {
numActive++;
activeName = masterThreads.get(i).getMaster().getServerName();
}
}
assertEquals(1, numActive);
assertEquals(NUM_MASTERS, masterThreads.size());
// verify still one active master and it's the same
for (int i = 0; i < masterThreads.size(); i++) {
if (masterThreads.get(i).getMaster().isActiveMaster()) {
assertTrue(activeName.equals(masterThreads.get(i).getMaster()
.getServerName()));
}
}
assertEquals(1, numActive);
assertEquals(1, masterThreads.size());
List<RegionServerThread> regionServerThreads = cluster
.getRegionServerThreads();
int count = -1;
HRegion metaRegion = null;
for (RegionServerThread regionServerThread : regionServerThreads) {
HRegionServer regionServer = regionServerThread.getRegionServer();
metaRegion = regionServer
.getOnlineRegion(HRegionInfo.FIRST_META_REGIONINFO.getRegionName());
count++;
regionServer.abort("");
if (null != metaRegion) {
break;
}
}
HRegionServer regionServer = cluster.getRegionServer(count);
cluster.shutdown();
// Create a ZKW to use in the test
ZooKeeperWatcher zkw =
HBaseTestingUtility.createAndForceNodeToOpenedState(TEST_UTIL,
metaRegion, regionServer.getServerName());
TEST_UTIL.startMiniHBaseCluster(1, 1);
// Failover should be completed, now wait for no RIT
log("Waiting for no more RIT");
ZKAssign.blockUntilNoRIT(zkw);
// Stop the cluster
TEST_UTIL.shutdownMiniCluster();
}
/**
* Complex test of master failover that tests as many permutations of the
* different possible states that regions in transition could be in within ZK.
@ -379,7 +353,7 @@ public class TestMasterFailover {
FSTableDescriptors.createTableDescriptor(filesystem, rootdir, htdEnabled);
HRegionInfo hriEnabled = new HRegionInfo(htdEnabled.getName(), null, null);
HRegion.createHRegion(hriEnabled, rootdir, conf, htdEnabled);
createRegion(hriEnabled, rootdir, conf, htdEnabled);
List<HRegionInfo> enabledRegions = TEST_UTIL.createMultiRegionsInMeta(
TEST_UTIL.getConfiguration(), htdEnabled, SPLIT_KEYS);
@ -390,7 +364,7 @@ public class TestMasterFailover {
// Write the .tableinfo
FSTableDescriptors.createTableDescriptor(filesystem, rootdir, htdDisabled);
HRegionInfo hriDisabled = new HRegionInfo(htdDisabled.getName(), null, null);
HRegion.createHRegion(hriDisabled, rootdir, conf, htdDisabled);
createRegion(hriDisabled, rootdir, conf, htdDisabled);
List<HRegionInfo> disabledRegions = TEST_UTIL.createMultiRegionsInMeta(
TEST_UTIL.getConfiguration(), htdDisabled, SPLIT_KEYS);
@ -692,7 +666,7 @@ public class TestMasterFailover {
FSTableDescriptors.createTableDescriptor(filesystem, rootdir, htdEnabled);
HRegionInfo hriEnabled = new HRegionInfo(htdEnabled.getName(),
null, null);
HRegion.createHRegion(hriEnabled, rootdir, conf, htdEnabled);
createRegion(hriEnabled, rootdir, conf, htdEnabled);
List<HRegionInfo> enabledRegions = TEST_UTIL.createMultiRegionsInMeta(
TEST_UTIL.getConfiguration(), htdEnabled, SPLIT_KEYS);
@ -703,7 +677,7 @@ public class TestMasterFailover {
// Write the .tableinfo
FSTableDescriptors.createTableDescriptor(filesystem, rootdir, htdDisabled);
HRegionInfo hriDisabled = new HRegionInfo(htdDisabled.getName(), null, null);
HRegion.createHRegion(hriDisabled, rootdir, conf, htdDisabled);
createRegion(hriDisabled, rootdir, conf, htdDisabled);
List<HRegionInfo> disabledRegions = TEST_UTIL.createMultiRegionsInMeta(
TEST_UTIL.getConfiguration(), htdDisabled, SPLIT_KEYS);
@ -1024,6 +998,19 @@ public class TestMasterFailover {
TEST_UTIL.shutdownMiniCluster();
}
HRegion createRegion(final HRegionInfo hri, final Path rootdir, final Configuration c,
final HTableDescriptor htd)
throws IOException {
HRegion r = HRegion.createHRegion(hri, rootdir, c, htd);
// The above call to create a region will create an hlog file. Each
// log file create will also create a running thread to do syncing. We need
// to close out this log else we will have a running thread trying to sync
// the file system continuously which is ugly when dfs is taken away at the
// end of the test.
HRegion.closeHRegion(r);
return r;
}
// TODO: Next test to add is with testing permutations of the RIT or the RS
// killed are hosting ROOT and META regions.

View File

@ -112,6 +112,7 @@ public class TestOpenedRegionHandler {
@Test
public void testShouldNotCompeleteOpenedRegionSuccessfullyIfVersionMismatches()
throws Exception {
HRegion region = null;
try {
int testIndex = 0;
TEST_UTIL.startMiniZKCluster();
@ -120,8 +121,7 @@ public class TestOpenedRegionHandler {
"testShouldNotCompeleteOpenedRegionSuccessfullyIfVersionMismatches");
HRegionInfo hri = new HRegionInfo(htd.getName(),
Bytes.toBytes(testIndex), Bytes.toBytes(testIndex + 1));
HRegion region = HRegion.createHRegion(hri, TEST_UTIL
.getDataTestDir(), TEST_UTIL.getConfiguration(), htd);
region = HRegion.createHRegion(hri, TEST_UTIL.getDataTestDir(), TEST_UTIL.getConfiguration(), htd);
assertNotNull(region);
AssignmentManager am = Mockito.mock(AssignmentManager.class);
when(am.isRegionInTransition(hri)).thenReturn(
@ -160,6 +160,8 @@ public class TestOpenedRegionHandler {
assertEquals("The region should not be opened successfully.", regionName,
region.getRegionInfo().getEncodedName());
} finally {
region.close();
region.getLog().closeAndDelete();
TEST_UTIL.shutdownMiniZKCluster();
}
}

View File

@ -85,7 +85,16 @@ public class TestBlocksRead extends HBaseTestCase {
EnvironmentEdgeManagerTestHelper.reset();
}
private void initHRegion(byte[] tableName, String callingMethod,
/**
* Callers must afterward call {@link HRegion#closeHRegion(HRegion)}
* @param tableName
* @param callingMethod
* @param conf
* @param families
* @throws IOException
* @return created and initialized region.
*/
private HRegion initHRegion(byte[] tableName, String callingMethod,
HBaseConfiguration conf, String family) throws IOException {
HTableDescriptor htd = new HTableDescriptor(tableName);
HColumnDescriptor familyDesc;
@ -99,8 +108,9 @@ public class TestBlocksRead extends HBaseTestCase {
HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
Path path = new Path(DIR + callingMethod);
region = HRegion.createHRegion(info, path, conf, htd);
HRegion r = HRegion.createHRegion(info, path, conf, htd);
blockCache = new CacheConfig(conf).getBlockCache();
return r;
}
private void putData(String family, String row, String col, long version)
@ -212,8 +222,9 @@ public class TestBlocksRead extends HBaseTestCase {
String FAMILY = "cf1";
KeyValue kvs[];
HBaseConfiguration conf = getConf();
initHRegion(TABLE, getName(), conf, FAMILY);
this.region = initHRegion(TABLE, getName(), conf, FAMILY);
try {
putData(FAMILY, "row", "col1", 1);
putData(FAMILY, "row", "col2", 2);
putData(FAMILY, "row", "col3", 3);
@ -251,6 +262,10 @@ public class TestBlocksRead extends HBaseTestCase {
kvs = getData(FAMILY, "row", Arrays.asList("col5"), 2);
assertEquals(1, kvs.length);
verifyData(kvs[0], "row", "col5", 5);
} finally {
HRegion.closeHRegion(this.region);
this.region = null;
}
}
/**
@ -264,8 +279,9 @@ public class TestBlocksRead extends HBaseTestCase {
String FAMILY = "cf1";
KeyValue kvs[];
HBaseConfiguration conf = getConf();
initHRegion(TABLE, getName(), conf, FAMILY);
this.region = initHRegion(TABLE, getName(), conf, FAMILY);
try {
// File 1
putData(FAMILY, "row", "col1", 1);
putData(FAMILY, "row", "col2", 2);
@ -355,6 +371,10 @@ public class TestBlocksRead extends HBaseTestCase {
verifyData(kvs[0], "row", "col1", 11);
verifyData(kvs[1], "row", "col2", 12);
verifyData(kvs[2], "row", "col3", 13);
} finally {
HRegion.closeHRegion(this.region);
this.region = null;
}
}
/**
@ -367,8 +387,9 @@ public class TestBlocksRead extends HBaseTestCase {
String FAMILY = "cf1";
HBaseConfiguration conf = getConf();
initHRegion(TABLE, getName(), conf, FAMILY);
this.region = initHRegion(TABLE, getName(), conf, FAMILY);
try {
putData(FAMILY, "row", "col1", 1);
putData(FAMILY, "row", "col2", 2);
region.flushcache();
@ -399,6 +420,10 @@ public class TestBlocksRead extends HBaseTestCase {
blocksEnd = getBlkCount();
assertEquals(2 * BLOOM_TYPE.length, blocksEnd - blocksStart);
} finally {
HRegion.closeHRegion(this.region);
this.region = null;
}
}
@Test
@ -407,8 +432,8 @@ public class TestBlocksRead extends HBaseTestCase {
String FAMILY = "cf1";
KeyValue kvs[];
HBaseConfiguration conf = getConf();
initHRegion(TABLE, getName(), conf, FAMILY);
this.region = initHRegion(TABLE, getName(), conf, FAMILY);
try {
deleteFamily(FAMILY, "row", 200);
for (int i = 0; i < 100; i++) {
putData(FAMILY, "row", "col" + i, i);
@ -422,6 +447,10 @@ public class TestBlocksRead extends HBaseTestCase {
kvs = getData(FAMILY, "row", Arrays.asList("col99"), 2);
assertEquals(1, kvs.length);
verifyData(kvs[0], "row", "col99", 201);
} finally {
HRegion.closeHRegion(this.region);
this.region = null;
}
}
@org.junit.Rule

View File

@ -62,7 +62,7 @@ public class TestColumnSeeking {
HRegion region =
HRegion.createHRegion(info, TEST_UTIL.getDataTestDir(), TEST_UTIL
.getConfiguration(), htd);
try {
List<String> rows = generateRandomWords(10, "row");
List<String> allColumns = generateRandomWords(10, "column");
List<String> values = generateRandomWords(100, "value");
@ -152,6 +152,9 @@ public class TestColumnSeeking {
assertEquals(kvSet.size(), results.size());
assertTrue(results.containsAll(kvSet));
}
} finally {
HRegion.closeHRegion(region);
}
region.close();
region.getLog().closeAndDelete();

View File

@ -86,9 +86,10 @@ public class TestCompactSelection extends TestCase {
HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
HLog hlog = new HLog(fs, logdir, oldLogDir, conf);
HRegion.createHRegion(info, basedir, conf, htd);
HRegion region = HRegion.createHRegion(info, basedir, conf, htd);
HRegion.closeHRegion(region);
Path tableDir = new Path(basedir, Bytes.toString(htd.getName()));
HRegion region = new HRegion(tableDir, hlog, fs, conf, info, htd, null);
region = new HRegion(tableDir, hlog, fs, conf, info, htd, null);
store = new Store(basedir, region, hcd, fs, conf);
TEST_FILE = StoreFile.getRandomFilename(fs, store.getHomedir());

View File

@ -0,0 +1,145 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver;
import static org.junit.Assert.*;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.MediumTests;
import org.junit.Test;
/**
* Tests that need to spin up a cluster testing an {@link HRegion}. Use
* {@link TestHRegion} if you don't need a cluster, if you can test w/ a
* standalone {@link HRegion}.
*/
@Category(MediumTests.class)
public class TestHRegionOnCluster {
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
@Test (timeout=180000)
public void testDataCorrectnessReplayingRecoveredEdits() throws Exception {
final int NUM_MASTERS = 1;
final int NUM_RS = 3;
TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
try {
final byte[] TABLENAME = Bytes
.toBytes("testDataCorrectnessReplayingRecoveredEdits");
final byte[] FAMILY = Bytes.toBytes("family");
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
HMaster master = cluster.getMaster();
// Create table
HTableDescriptor desc = new HTableDescriptor(TABLENAME);
desc.addFamily(new HColumnDescriptor(FAMILY));
HBaseAdmin hbaseAdmin = TEST_UTIL.getHBaseAdmin();
hbaseAdmin.createTable(desc);
assertTrue(hbaseAdmin.isTableAvailable(TABLENAME));
// Put data: r1->v1
HTable table = new HTable(TEST_UTIL.getConfiguration(), TABLENAME);
putDataAndVerify(table, "r1", FAMILY, "v1", 1);
// Move region to target server
HRegionInfo regionInfo = table.getRegionLocation("r1").getRegionInfo();
int originServerNum = cluster.getServerWith(regionInfo.getRegionName());
HRegionServer originServer = cluster.getRegionServer(originServerNum);
int targetServerNum = (originServerNum + 1) % NUM_RS;
HRegionServer targetServer = cluster.getRegionServer(targetServerNum);
assertFalse(originServer.equals(targetServer));
hbaseAdmin.move(regionInfo.getEncodedNameAsBytes(),
Bytes.toBytes(targetServer.getServerName().getServerName()));
do {
Thread.sleep(1);
} while (cluster.getServerWith(regionInfo.getRegionName()) == originServerNum);
// Put data: r2->v2
putDataAndVerify(table, "r2", FAMILY, "v2", 2);
// Move region to origin server
hbaseAdmin.move(regionInfo.getEncodedNameAsBytes(),
Bytes.toBytes(originServer.getServerName().getServerName()));
do {
Thread.sleep(1);
} while (cluster.getServerWith(regionInfo.getRegionName()) == targetServerNum);
// Put data: r3->v3
putDataAndVerify(table, "r3", FAMILY, "v3", 3);
// Kill target server
targetServer.kill();
cluster.getRegionServerThreads().get(targetServerNum).join();
// Wait until finish processing of shutdown
while (master.getServerManager().areDeadServersInProgress()) {
Thread.sleep(5);
}
// Kill origin server
originServer.kill();
cluster.getRegionServerThreads().get(originServerNum).join();
// Put data: r4->v4
putDataAndVerify(table, "r4", FAMILY, "v4", 4);
} finally {
TEST_UTIL.shutdownMiniCluster();
}
}
private void putDataAndVerify(HTable table, String row, byte[] family,
String value, int verifyNum) throws IOException {
System.out.println("=========Putting data :" + row);
Put put = new Put(Bytes.toBytes(row));
put.add(family, Bytes.toBytes("q1"), Bytes.toBytes(value));
table.put(put);
ResultScanner resultScanner = table.getScanner(new Scan());
List<Result> results = new ArrayList<Result>();
while (true) {
Result r = resultScanner.next();
if (r == null)
break;
results.add(r);
}
resultScanner.close();
if (results.size() != verifyNum) {
System.out.println(results);
}
assertEquals(verifyNum, results.size());
}
@org.junit.Rule
public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
}

View File

@ -52,6 +52,7 @@ public class TestMinVersions extends HBaseTestCase {
public void testGetClosestBefore() throws Exception {
HTableDescriptor htd = createTableDescriptor(getName(), 1, 1000, 1, false);
HRegion region = createNewHRegion(htd, null, null);
try {
// 2s in the past
long ts = EnvironmentEdgeManager.currentTimeMillis() - 2000;
@ -86,6 +87,10 @@ public class TestMinVersions extends HBaseTestCase {
r = region.getClosestRowBefore(T2, c0);
checkResult(r, c0, T4);
} finally {
region.close();
region.getLog().closeAndDelete();
}
}
/**
@ -96,10 +101,10 @@ public class TestMinVersions extends HBaseTestCase {
// keep 3 versions minimum
HTableDescriptor htd = createTableDescriptor(getName(), 3, 1000, 1, false);
HRegion region = createNewHRegion(htd, null, null);
// 2s in the past
long ts = EnvironmentEdgeManager.currentTimeMillis() - 2000;
try {
Put p = new Put(T1, ts-1);
p.add(c0, c0, T2);
region.put(p);
@ -138,6 +143,10 @@ public class TestMinVersions extends HBaseTestCase {
g.addColumn(c0, c0);
r = region.get(g, null); // this'll use ExplicitColumnTracker
checkResult(r, c0, T3,T2,T1);
} finally {
region.close();
region.getLog().closeAndDelete();
}
}
/**
@ -150,6 +159,7 @@ public class TestMinVersions extends HBaseTestCase {
// 2s in the past
long ts = EnvironmentEdgeManager.currentTimeMillis() - 2000;
try {
Put p = new Put(T1, ts-2);
p.add(c0, c0, T1);
region.put(p);
@ -191,6 +201,10 @@ public class TestMinVersions extends HBaseTestCase {
g.addColumn(c0, c0);
r = region.get(g, null); // this'll use ExplicitColumnTracker
checkResult(r, c0, T3);
} finally {
region.close();
region.getLog().closeAndDelete();
}
}
/**
@ -203,6 +217,7 @@ public class TestMinVersions extends HBaseTestCase {
// 2s in the past
long ts = EnvironmentEdgeManager.currentTimeMillis() - 2000;
try {
// 2nd version
Put p = new Put(T1, ts-2);
p.add(c0, c0, T2);
@ -260,6 +275,10 @@ public class TestMinVersions extends HBaseTestCase {
g.addColumn(c0, c0);
r = region.get(g, null); // this'll use ExplicitColumnTracker
checkResult(r, c0, T5,T4);
} finally {
region.close();
region.getLog().closeAndDelete();
}
}
/**
@ -269,6 +288,7 @@ public class TestMinVersions extends HBaseTestCase {
// 1 version minimum, 1000 versions maximum, ttl = 1s
HTableDescriptor htd = createTableDescriptor(getName(), 2, 1000, 1, false);
HRegion region = createNewHRegion(htd, null, null);
try {
// 2s in the past
long ts = EnvironmentEdgeManager.currentTimeMillis() - 2000;
@ -346,6 +366,10 @@ public class TestMinVersions extends HBaseTestCase {
g.setTimeRange(0L, ts-1);
r = region.get(g, null);
assertTrue(r.isEmpty());
} finally {
region.close();
region.getLog().closeAndDelete();
}
}
/**
@ -359,6 +383,7 @@ public class TestMinVersions extends HBaseTestCase {
// 2s in the past
long ts = EnvironmentEdgeManager.currentTimeMillis() - 2000;
try {
Put p = new Put(T1, ts-3);
p.add(c0, c0, T0);
@ -415,6 +440,10 @@ public class TestMinVersions extends HBaseTestCase {
g.setMaxVersions();
r = region.get(g, null);
checkResult(r, c0, T2);
} finally {
region.close();
region.getLog().closeAndDelete();
}
}
private void checkResult(Result r, byte[] col, byte[] ... vals) {

View File

@ -68,7 +68,7 @@ public class TestResettingCounters {
}
}
HRegion region = HRegion.createHRegion(hri, path, conf, htd);
try {
Increment odd = new Increment(rows[0]);
Increment even = new Increment(rows[0]);
Increment all = new Increment(rows[0]);
@ -94,6 +94,9 @@ public class TestResettingCounters {
assertTrue(Bytes.equals(kvs[i].getQualifier(), qualifiers[i]));
assertEquals(6, Bytes.toLong(kvs[i].getValue()));
}
} finally {
HRegion.closeHRegion(region);
}
region.close();
region.getLog().closeAndDelete();
}

View File

@ -319,7 +319,9 @@ public class TestSplitTransaction {
HColumnDescriptor hcd = new HColumnDescriptor(CF);
htd.addFamily(hcd);
HRegionInfo hri = new HRegionInfo(htd.getName(), STARTROW, ENDROW);
HRegion.createHRegion(hri, testdir, TEST_UTIL.getConfiguration(), htd);
HRegion r = HRegion.createHRegion(hri, testdir, TEST_UTIL.getConfiguration(), htd);
r.close();
r.getLog().closeAndDelete();
return HRegion.openHRegion(testdir, hri, htd, wal,
TEST_UTIL.getConfiguration());
}

View File

@ -99,6 +99,7 @@ public class TestCloseRegionHandler {
HRegion region =
HRegion.createHRegion(hri, HTU.getDataTestDir(),
HTU.getConfiguration(), htd);
try {
assertNotNull(region);
// Spy on the region so can throw exception when close is called.
HRegion spy = Mockito.spy(region);
@ -122,6 +123,9 @@ public class TestCloseRegionHandler {
// Abort calls stop so stopped flag should be set.
assertTrue(server.isStopped());
}
} finally {
HRegion.closeHRegion(region);
}
}
/**

View File

@ -98,6 +98,7 @@ public class TestOpenRegionHandler {
HRegion.createHRegion(hri, HTU.getDataTestDir(), HTU
.getConfiguration(), htd);
assertNotNull(region);
try {
OpenRegionHandler handler = new OpenRegionHandler(server, rss, hri, htd) {
HRegion openRegion() {
// Open region first, then remove znode as though it'd been hijacked.
@ -122,6 +123,9 @@ public class TestOpenRegionHandler {
// Call process again but this time yank the zk znode out from under it
// post OPENING; again will expect it to come back w/o NPE or exception.
handler.process();
} finally {
HRegion.closeHRegion(region);
}
}
@Test

View File

@ -133,7 +133,8 @@ public class TestWALReplay {
HTableDescriptor htd = createBasic3FamilyHTD(tableNameStr);
HRegion region2 = HRegion.createHRegion(hri,
hbaseRootDir, this.conf, htd);
region2.close();
region2.getLog().closeAndDelete();
final byte [] tableName = Bytes.toBytes(tableNameStr);
final byte [] rowName = tableName;
@ -193,6 +194,8 @@ public class TestWALReplay {
final HTableDescriptor htd = createBasic3FamilyHTD(tableNameStr);
HRegion region2 = HRegion.createHRegion(hri,
hbaseRootDir, this.conf, htd);
region2.close();
region2.getLog().closeAndDelete();
HLog wal = createWAL(this.conf);
HRegion region = HRegion.openHRegion(hri, htd, wal, this.conf);
Path f = new Path(basedir, "hfile");
@ -252,7 +255,8 @@ public class TestWALReplay {
final HTableDescriptor htd = createBasic3FamilyHTD(tableNameStr);
HRegion region3 = HRegion.createHRegion(hri,
hbaseRootDir, this.conf, htd);
region3.close();
region3.getLog().closeAndDelete();
// Write countPerFamily edits into the three families. Do a flush on one
// of the families during the load of edits so its seqid is not same as
// others to test we do right thing when different seqids.
@ -369,7 +373,8 @@ public class TestWALReplay {
final HTableDescriptor htd = createBasic3FamilyHTD(tableNameStr);
HRegion region3 = HRegion.createHRegion(hri,
hbaseRootDir, this.conf, htd);
region3.close();
region3.getLog().closeAndDelete();
// Write countPerFamily edits into the three families. Do a flush on one
// of the families during the load of edits so its seqid is not same as
// others to test we do right thing when different seqids.
@ -435,7 +440,8 @@ public class TestWALReplay {
final HTableDescriptor htd = createBasic3FamilyHTD(tableNameStr);
HRegion region2 = HRegion.createHRegion(hri,
hbaseRootDir, this.conf, htd);
region2.close();
region2.getLog().closeAndDelete();
final HLog wal = createWAL(this.conf);
final byte[] tableName = Bytes.toBytes(tableNameStr);
final byte[] rowName = tableName;

View File

@ -169,6 +169,13 @@ public class TestMergeTool extends HBaseTestCase {
@Override
public void tearDown() throws Exception {
super.tearDown();
for (int i = 0; i < sourceRegions.length; i++) {
HRegion r = regions[i];
if (r != null) {
r.close();
r.getLog().closeAndDelete();
}
}
TEST_UTIL.shutdownMiniCluster();
}