HADOOP-2017 TestRegionServerAbort failure in patch build #903 and
nightly #266 git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@583309 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
5233c28aaa
commit
c9f105acd5
|
@ -72,6 +72,8 @@ Trunk (unreleased changes)
|
|||
HADOOP-2004 webapp hql formatting bugs
|
||||
HADOOP_2011 Make hbase daemon scripts take args in same order as hadoop
|
||||
daemon scripts
|
||||
HADOOP-2017 TestRegionServerAbort failure in patch build #903 and
|
||||
nightly #266
|
||||
|
||||
IMPROVEMENTS
|
||||
HADOOP-1737 Make HColumnDescriptor data publically members settable
|
||||
|
|
|
@ -24,7 +24,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.util.Sleeper;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
|
||||
/**
|
||||
* Chore is a task performed on a period in hbase. The chore is run in its own
|
||||
|
|
|
@ -205,7 +205,7 @@ HMasterRegionInterface {
|
|||
HRegionInterface regionServer = null;
|
||||
long scannerId = -1L;
|
||||
LOG.info(Thread.currentThread().getName() + " scanning meta region " +
|
||||
region.getRegionName() + " on " + region.getServer().toString());
|
||||
region.toString());
|
||||
|
||||
// Array to hold list of split parents found. Scan adds to list. After
|
||||
// scan we go check if parents can be removed.
|
||||
|
@ -247,7 +247,7 @@ HMasterRegionInterface {
|
|||
}
|
||||
numberOfRegionsFound += 1;
|
||||
}
|
||||
if (rootRegion) {
|
||||
if (this.rootRegion) {
|
||||
numberOfMetaRegions.set(numberOfRegionsFound);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
|
@ -282,7 +282,7 @@ HMasterRegionInterface {
|
|||
}
|
||||
}
|
||||
LOG.info(Thread.currentThread().getName() + " scan of meta region " +
|
||||
region.getRegionName() + " complete");
|
||||
region.toString() + " complete");
|
||||
}
|
||||
|
||||
/*
|
||||
|
@ -405,8 +405,8 @@ HMasterRegionInterface {
|
|||
}
|
||||
|
||||
protected void checkAssigned(final HRegionInfo info,
|
||||
final String serverName, final long startCode) throws IOException {
|
||||
|
||||
final String serverName, final long startCode)
|
||||
throws IOException {
|
||||
// Skip region - if ...
|
||||
if(info.offLine // offline
|
||||
|| killedRegions.contains(info.regionName) // queued for offline
|
||||
|
@ -415,7 +415,6 @@ HMasterRegionInterface {
|
|||
assignAttempts.remove(info.regionName);
|
||||
return;
|
||||
}
|
||||
|
||||
HServerInfo storedInfo = null;
|
||||
if (serverName.length() != 0) {
|
||||
Map<Text, HRegionInfo> regionsToKill = killList.get(serverName);
|
||||
|
@ -432,14 +431,17 @@ HMasterRegionInterface {
|
|||
storedInfo = serversToServerInfo.get(serverName);
|
||||
}
|
||||
}
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Checking " + info.regionName + " is assigned");
|
||||
}
|
||||
if (!(unassignedRegions.containsKey(info.regionName) ||
|
||||
pendingRegions.contains(info.regionName))
|
||||
pendingRegions.contains(info.regionName))
|
||||
&& (storedInfo == null || storedInfo.getStartCode() != startCode)) {
|
||||
|
||||
// The current assignment is no good
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Current assignment of " + info.regionName + " is no good");
|
||||
}
|
||||
// Recover the region server's log if there is one.
|
||||
|
||||
if (serverName.length() != 0) {
|
||||
StringBuilder dirName = new StringBuilder("log_");
|
||||
dirName.append(serverName.replace(":", "_"));
|
||||
|
@ -449,20 +451,19 @@ HMasterRegionInterface {
|
|||
splitLogLock.lock();
|
||||
try {
|
||||
HLog.splitLog(dir, logDir, fs, conf);
|
||||
|
||||
} finally {
|
||||
splitLogLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Split " + logDir.toString());
|
||||
}
|
||||
} catch (IOException e) {
|
||||
LOG.warn("unable to split region server log because: ", e);
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
// Now get the region assigned
|
||||
|
||||
unassignedRegions.put(info.regionName, info);
|
||||
assignAttempts.put(info.regionName, Long.valueOf(0L));
|
||||
}
|
||||
|
@ -479,6 +480,7 @@ HMasterRegionInterface {
|
|||
}
|
||||
|
||||
private void scanRoot() {
|
||||
boolean succeeded = false;
|
||||
int tries = 0;
|
||||
while (!closed.get() && tries < numRetries) {
|
||||
synchronized (rootRegionLocation) {
|
||||
|
@ -503,6 +505,7 @@ HMasterRegionInterface {
|
|||
scanRegion(new MetaRegion(rootRegionLocation.get(),
|
||||
HGlobals.rootRegionInfo.regionName, null));
|
||||
}
|
||||
succeeded = true;
|
||||
break;
|
||||
} catch (IOException e) {
|
||||
e = RemoteExceptionHandler.checkIOException(e);
|
||||
|
@ -511,8 +514,9 @@ HMasterRegionInterface {
|
|||
LOG.warn("Scan ROOT region", e);
|
||||
} else {
|
||||
LOG.error("Scan ROOT region", e);
|
||||
if (tries == numRetries - 1) {
|
||||
// We ran out of tries. Make sure the file system is still available
|
||||
if (tries == numRetries - 1) {
|
||||
// We ran out of tries. Make sure the file system is still
|
||||
// available
|
||||
if (!checkFileSystem()) {
|
||||
continue; // Avoid sleeping.
|
||||
}
|
||||
|
@ -524,7 +528,13 @@ HMasterRegionInterface {
|
|||
LOG.error("Unexpected exception", e);
|
||||
}
|
||||
sleeper.sleep();
|
||||
}
|
||||
}
|
||||
if (!succeeded) {
|
||||
// We tried numretries to reach root and failed. Is it gone.
|
||||
// Currently we just flounder. Should we reallocate root?
|
||||
// This would be catastrophic?
|
||||
// unassignRootRegion();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -564,6 +574,12 @@ HMasterRegionInterface {
|
|||
this.startKey.set(startKey);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "regionname: " + this.regionName.toString() + ", startKey: <" +
|
||||
this.startKey.toString() + ">, server: " + this.server.toString() + "}";
|
||||
}
|
||||
|
||||
/** @return the regionName */
|
||||
public Text getRegionName() {
|
||||
|
@ -599,23 +615,28 @@ HMasterRegionInterface {
|
|||
/** {@inheritDoc} */
|
||||
public int compareTo(Object o) {
|
||||
MetaRegion other = (MetaRegion)o;
|
||||
|
||||
int result = this.regionName.compareTo(other.getRegionName());
|
||||
if(result == 0) {
|
||||
result = this.startKey.compareTo(other.getStartKey());
|
||||
if (result == 0) {
|
||||
// Might be on different host?
|
||||
result = this.server.compareTo(other.server);
|
||||
}
|
||||
}
|
||||
return result;
|
||||
}
|
||||
}
|
||||
|
||||
/** Set by root scanner to indicate the number of meta regions */
|
||||
AtomicInteger numberOfMetaRegions;
|
||||
final AtomicInteger numberOfMetaRegions = new AtomicInteger();
|
||||
|
||||
/** Work for the meta scanner is queued up here */
|
||||
BlockingQueue<MetaRegion> metaRegionsToScan;
|
||||
final BlockingQueue<MetaRegion> metaRegionsToScan =
|
||||
new LinkedBlockingQueue<MetaRegion>();
|
||||
|
||||
/** These are the online meta regions */
|
||||
SortedMap<Text, MetaRegion> onlineMetaRegions;
|
||||
final SortedMap<Text, MetaRegion> onlineMetaRegions =
|
||||
Collections.synchronizedSortedMap(new TreeMap<Text, MetaRegion>());
|
||||
|
||||
/** Set by meta scanner after initial scan */
|
||||
volatile boolean initialMetaScanComplete;
|
||||
|
@ -657,15 +678,24 @@ HMasterRegionInterface {
|
|||
e = RemoteExceptionHandler.checkIOException(e);
|
||||
tries += 1;
|
||||
if (tries == 1) {
|
||||
LOG.warn("Scan one META region", e);
|
||||
LOG.warn("Scan one META region: " + region.toString(), e);
|
||||
} else {
|
||||
LOG.error("Scan one META region", e);
|
||||
if (tries == numRetries - 1) {
|
||||
// We ran out of tries. Make sure the file system is still
|
||||
// available
|
||||
if (!checkFileSystem()) {
|
||||
continue; // avoid sleeping
|
||||
}
|
||||
LOG.error("Scan one META region: " + region.toString(), e);
|
||||
}
|
||||
// The region may have moved (TestRegionServerAbort, etc.). If
|
||||
// so, either it won't be in the onlineMetaRegions list or its host
|
||||
// address has changed and the containsValue will fail. If not
|
||||
// found, best thing to do here is probably break.
|
||||
if (!onlineMetaRegions.containsValue(region)) {
|
||||
LOG.debug("Scanned region is no longer in map of online " +
|
||||
"regions or its value has changed");
|
||||
break;
|
||||
}
|
||||
if (tries == numRetries - 1) {
|
||||
// We ran out of tries. Make sure the file system is still
|
||||
// available
|
||||
if (!checkFileSystem()) {
|
||||
continue; // avoid sleeping
|
||||
}
|
||||
}
|
||||
} catch (Exception e) {
|
||||
|
@ -673,6 +703,7 @@ HMasterRegionInterface {
|
|||
// at least log it rather than go out silently.
|
||||
LOG.error("Unexpected exception", e);
|
||||
}
|
||||
// Sleep before going around again.
|
||||
sleeper.sleep();
|
||||
}
|
||||
}
|
||||
|
@ -710,9 +741,8 @@ HMasterRegionInterface {
|
|||
* regions. This wakes up any threads that were waiting for this to happen.
|
||||
*/
|
||||
private synchronized boolean metaRegionsScanned() {
|
||||
if (!rootScanned ||
|
||||
if (!rootScanned ||
|
||||
numberOfMetaRegions.get() != onlineMetaRegions.size()) {
|
||||
|
||||
return false;
|
||||
}
|
||||
LOG.info("all meta regions scanned");
|
||||
|
@ -754,14 +784,16 @@ HMasterRegionInterface {
|
|||
* <p>Items are removed from this list when a region server reports in that
|
||||
* the region has been deployed.
|
||||
*/
|
||||
SortedMap<Text, HRegionInfo> unassignedRegions;
|
||||
final SortedMap<Text, HRegionInfo> unassignedRegions =
|
||||
Collections.synchronizedSortedMap(new TreeMap<Text, HRegionInfo>());
|
||||
|
||||
/**
|
||||
* The 'assignAttempts' table maps from regions to a timestamp that indicates
|
||||
* the last time we *tried* to assign the region to a RegionServer. If the
|
||||
* timestamp is out of date, then we can try to reassign it.
|
||||
*/
|
||||
Map<Text, Long> assignAttempts;
|
||||
final Map<Text, Long> assignAttempts =
|
||||
Collections.synchronizedMap(new HashMap<Text, Long>());
|
||||
|
||||
/**
|
||||
* Regions that have been assigned, and the server has reported that it has
|
||||
|
@ -897,28 +929,11 @@ HMasterRegionInterface {
|
|||
this.rootScannerThread = new RootScanner();
|
||||
|
||||
// Scans the meta table
|
||||
|
||||
this.numberOfMetaRegions = new AtomicInteger();
|
||||
this.metaRegionsToScan = new LinkedBlockingQueue<MetaRegion>();
|
||||
|
||||
this.onlineMetaRegions =
|
||||
Collections.synchronizedSortedMap(new TreeMap<Text, MetaRegion>());
|
||||
|
||||
this.initialMetaScanComplete = false;
|
||||
|
||||
this.metaScannerThread = new MetaScanner();
|
||||
|
||||
this.unassignedRegions =
|
||||
Collections.synchronizedSortedMap(new TreeMap<Text, HRegionInfo>());
|
||||
|
||||
this.unassignedRegions.put(HGlobals.rootRegionInfo.regionName,
|
||||
HGlobals.rootRegionInfo);
|
||||
|
||||
this.assignAttempts =
|
||||
Collections.synchronizedMap(new HashMap<Text, Long>());
|
||||
|
||||
this.assignAttempts.put(HGlobals.rootRegionInfo.regionName,
|
||||
Long.valueOf(0L));
|
||||
|
||||
unassignRootRegion();
|
||||
|
||||
this.pendingRegions =
|
||||
Collections.synchronizedSet(new HashSet<Text>());
|
||||
|
@ -943,6 +958,22 @@ HMasterRegionInterface {
|
|||
this.closed.set(false);
|
||||
LOG.info("HMaster initialized on " + this.address.toString());
|
||||
}
|
||||
|
||||
/*
|
||||
* Unassign the root region.
|
||||
* This method would be used in case where root region server had died
|
||||
* without reporting in. Currently, we just flounder and never recover. We
|
||||
* could 'notice' dead region server in root scanner -- if we failed access
|
||||
* multiple times -- but reassigning root is catastrophic.
|
||||
*/
|
||||
void unassignRootRegion() {
|
||||
this.rootRegionLocation.set(null);
|
||||
this.unassignedRegions.put(HGlobals.rootRegionInfo.regionName,
|
||||
HGlobals.rootRegionInfo);
|
||||
this.assignAttempts.put(HGlobals.rootRegionInfo.regionName,
|
||||
Long.valueOf(0L));
|
||||
// TODO: If the old root region server had a log, it needs splitting.
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks to see if the file system is still accessible.
|
||||
|
@ -1271,17 +1302,13 @@ HMasterRegionInterface {
|
|||
|
||||
LOG.info("Region server " + serverName +
|
||||
": MSG_REPORT_EXITING -- lease cancelled");
|
||||
|
||||
// Get all the regions the server was serving reassigned
|
||||
// (if we are not shutting down).
|
||||
|
||||
if (!closed.get()) {
|
||||
for (int i = 1; i < msgs.length; i++) {
|
||||
HRegionInfo info = msgs[i].getRegionInfo();
|
||||
|
||||
if (info.tableDesc.getName().equals(ROOT_TABLE_NAME)) {
|
||||
rootRegionLocation.set(null);
|
||||
|
||||
} else if (info.tableDesc.getName().equals(META_TABLE_NAME)) {
|
||||
onlineMetaRegions.remove(info.getStartKey());
|
||||
}
|
||||
|
@ -1448,14 +1475,11 @@ HMasterRegionInterface {
|
|||
} else {
|
||||
LOG.info(info.getServerAddress().toString() + " serving " +
|
||||
region.regionName);
|
||||
|
||||
// Remove from unassigned list so we don't assign it to someone else
|
||||
this.unassignedRegions.remove(region.regionName);
|
||||
this.assignAttempts.remove(region.regionName);
|
||||
|
||||
if (region.regionName.compareTo(
|
||||
HGlobals.rootRegionInfo.regionName) == 0) {
|
||||
|
||||
// Store the Root Region location (in memory)
|
||||
synchronized (rootRegionLocation) {
|
||||
this.rootRegionLocation.
|
||||
|
@ -2226,30 +2250,22 @@ HMasterRegionInterface {
|
|||
* root region which is handled specially.
|
||||
*/
|
||||
private class PendingOpenReport extends PendingOperation {
|
||||
private boolean rootRegion;
|
||||
private HRegionInfo region;
|
||||
private HServerAddress serverAddress;
|
||||
private byte [] startCode;
|
||||
private final boolean rootRegion;
|
||||
private final HRegionInfo region;
|
||||
private final HServerAddress serverAddress;
|
||||
private final byte [] startCode;
|
||||
|
||||
PendingOpenReport(HServerInfo info, HRegionInfo region)
|
||||
throws IOException {
|
||||
if (region.tableDesc.getName().equals(META_TABLE_NAME)) {
|
||||
// The region which just came on-line is a META region.
|
||||
// We need to look in the ROOT region for its information.
|
||||
|
||||
this.rootRegion = true;
|
||||
|
||||
} else {
|
||||
// Just an ordinary region. Look for it in the META table.
|
||||
|
||||
this.rootRegion = false;
|
||||
}
|
||||
// If true, the region which just came on-line is a META region.
|
||||
// We need to look in the ROOT region for its information. Otherwise,
|
||||
// its just an ordinary region. Look for it in the META table.
|
||||
this.rootRegion = region.tableDesc.getName().equals(META_TABLE_NAME);
|
||||
this.region = region;
|
||||
this.serverAddress = info.getServerAddress();
|
||||
this.startCode = Writables.longToBytes(info.getStartCode());
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
@Override
|
||||
public String toString() {
|
||||
return "PendingOpenOperation from " + serverAddress.toString();
|
||||
|
@ -2261,15 +2277,15 @@ HMasterRegionInterface {
|
|||
if (closed.get()) {
|
||||
return true;
|
||||
}
|
||||
LOG.info(region.getRegionName() + " open on " +
|
||||
LOG.info(region.toString() + " open on " +
|
||||
this.serverAddress.toString());
|
||||
|
||||
// Register the newly-available Region's location.
|
||||
Text metaRegionName;
|
||||
HRegionInterface server;
|
||||
if (rootRegion) {
|
||||
if (this.rootRegion) {
|
||||
if (rootRegionLocation.get() == null || !rootScanned) {
|
||||
// We can't proceed until the root region is online and has been scanned
|
||||
// We can't proceed until root region is online and scanned
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("root region: " +
|
||||
((rootRegionLocation.get() != null)?
|
||||
|
@ -2283,12 +2299,10 @@ HMasterRegionInterface {
|
|||
} else {
|
||||
if (!rootScanned ||
|
||||
numberOfMetaRegions.get() != onlineMetaRegions.size()) {
|
||||
|
||||
// We can't proceed because not all of the meta regions are online.
|
||||
// We can't block either because that would prevent the meta region
|
||||
// online message from being processed. So return false to have this
|
||||
// operation requeued.
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Requeuing open because rootScanned: " +
|
||||
rootScanned + ", numberOfMetaRegions: " +
|
||||
|
@ -2298,21 +2312,18 @@ HMasterRegionInterface {
|
|||
return false;
|
||||
}
|
||||
|
||||
MetaRegion r = null;
|
||||
if (onlineMetaRegions.containsKey(region.getRegionName())) {
|
||||
r = onlineMetaRegions.get(region.getRegionName());
|
||||
} else {
|
||||
r = onlineMetaRegions.get(onlineMetaRegions.headMap(
|
||||
region.getRegionName()).lastKey());
|
||||
}
|
||||
MetaRegion r = onlineMetaRegions.containsKey(region.getRegionName())?
|
||||
onlineMetaRegions.get(region.getRegionName()):
|
||||
onlineMetaRegions.get(onlineMetaRegions.
|
||||
headMap(region.getRegionName()).lastKey());
|
||||
metaRegionName = r.getRegionName();
|
||||
server = connection.getHRegionConnection(r.getServer());
|
||||
}
|
||||
|
||||
LOG.info("updating row " + region.getRegionName() + " in table " +
|
||||
metaRegionName + " with startcode " +
|
||||
Writables.bytesToLong(this.startCode) + " and server "+
|
||||
serverAddress.toString());
|
||||
|
||||
try {
|
||||
BatchUpdate b = new BatchUpdate(rand.nextLong());
|
||||
long lockid = b.startUpdate(region.getRegionName());
|
||||
|
@ -2320,34 +2331,28 @@ HMasterRegionInterface {
|
|||
Writables.stringToBytes(serverAddress.toString()));
|
||||
b.put(lockid, COL_STARTCODE, startCode);
|
||||
server.batchUpdate(metaRegionName, System.currentTimeMillis(), b);
|
||||
|
||||
if (region.tableDesc.getName().equals(META_TABLE_NAME)) {
|
||||
// It's a meta region.
|
||||
|
||||
MetaRegion m =
|
||||
new MetaRegion(serverAddress, region.regionName, region.startKey);
|
||||
|
||||
MetaRegion m = new MetaRegion(this.serverAddress,
|
||||
this.region.regionName, this.region.startKey);
|
||||
if (!initialMetaScanComplete) {
|
||||
// Put it on the queue to be scanned for the first time.
|
||||
|
||||
try {
|
||||
LOG.debug("Adding " + m.toString() + " to regions to scan");
|
||||
metaRegionsToScan.put(m);
|
||||
|
||||
} catch (InterruptedException e) {
|
||||
throw new RuntimeException(
|
||||
"Putting into metaRegionsToScan was interrupted.", e);
|
||||
}
|
||||
} else {
|
||||
// Add it to the online meta regions
|
||||
|
||||
onlineMetaRegions.put(region.startKey, m);
|
||||
LOG.debug("Adding to onlineMetaRegions: " + m.toString());
|
||||
onlineMetaRegions.put(this.region.startKey, m);
|
||||
}
|
||||
}
|
||||
// If updated successfully, remove from pending list.
|
||||
|
||||
pendingRegions.remove(region.getRegionName());
|
||||
break;
|
||||
|
||||
} catch (IOException e) {
|
||||
if (tries == numRetries - 1) {
|
||||
throw RemoteExceptionHandler.checkIOException(e);
|
||||
|
|
|
@ -68,41 +68,12 @@ public class DFSAbort extends HBaseClusterTestCase {
|
|||
cluster.getDFSCluster().shutdown();
|
||||
// Now wait for Mini HBase Cluster to shut down
|
||||
// cluster.join();
|
||||
join();
|
||||
threadDumpingJoin();
|
||||
} catch (Exception e) {
|
||||
e.printStackTrace();
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
private void join() {
|
||||
if (this.cluster.regionThreads != null) {
|
||||
synchronized(this.cluster.regionThreads) {
|
||||
for(Thread t: this.cluster.regionThreads) {
|
||||
join(t);
|
||||
}
|
||||
}
|
||||
}
|
||||
join(this.cluster.getMasterThread());
|
||||
}
|
||||
|
||||
private void join(final Thread t) {
|
||||
if (t == null) {
|
||||
return;
|
||||
}
|
||||
for (int i = 0; t.isAlive(); i++) {
|
||||
try {
|
||||
Thread.sleep(1000);
|
||||
} catch (InterruptedException e) {
|
||||
LOG.info("Continuing...", e);
|
||||
}
|
||||
if (i != 0 && i % 30 == 0) {
|
||||
ReflectionUtils.printThreadInfo(new PrintWriter(System.out),
|
||||
"Automatic Stack Trace every 30 seconds waiting on " +
|
||||
t.getName());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @param args unused
|
||||
|
|
|
@ -19,11 +19,19 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase;
|
||||
|
||||
import java.io.PrintWriter;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.util.ReflectionUtils;
|
||||
|
||||
/**
|
||||
* Abstract base class for HBase cluster junit tests. Spins up cluster on
|
||||
* {@link #setUp()} and takes it down again in {@link #tearDown()}.
|
||||
*/
|
||||
public abstract class HBaseClusterTestCase extends HBaseTestCase {
|
||||
private static final Log LOG =
|
||||
LogFactory.getLog(HBaseClusterTestCase.class.getName());
|
||||
protected MiniHBaseCluster cluster;
|
||||
final boolean miniHdfs;
|
||||
int regionServers;
|
||||
|
@ -69,7 +77,6 @@ public abstract class HBaseClusterTestCase extends HBaseTestCase {
|
|||
this.regionServers = 1;
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
@Override
|
||||
protected void setUp() throws Exception {
|
||||
super.setUp();
|
||||
|
@ -77,7 +84,6 @@ public abstract class HBaseClusterTestCase extends HBaseTestCase {
|
|||
new MiniHBaseCluster(this.conf, this.regionServers, this.miniHdfs);
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
@Override
|
||||
protected void tearDown() throws Exception {
|
||||
super.tearDown();
|
||||
|
@ -86,4 +92,41 @@ public abstract class HBaseClusterTestCase extends HBaseTestCase {
|
|||
}
|
||||
HConnectionManager.deleteConnection(conf);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Use this utility method debugging why cluster won't go down. On a
|
||||
* period it throws a thread dump. Method ends when all cluster
|
||||
* regionservers and master threads are no long alive.
|
||||
*/
|
||||
public void threadDumpingJoin() {
|
||||
if (this.cluster.regionThreads != null) {
|
||||
synchronized(this.cluster.regionThreads) {
|
||||
for(Thread t: this.cluster.regionThreads) {
|
||||
threadDumpingJoin(t);
|
||||
}
|
||||
}
|
||||
}
|
||||
threadDumpingJoin(this.cluster.getMasterThread());
|
||||
}
|
||||
|
||||
public void threadDumpingJoin(final Thread t) {
|
||||
if (t == null) {
|
||||
return;
|
||||
}
|
||||
long startTime = System.currentTimeMillis();
|
||||
while (t.isAlive()) {
|
||||
try {
|
||||
Thread.sleep(1000);
|
||||
} catch (InterruptedException e) {
|
||||
LOG.info("Continuing...", e);
|
||||
}
|
||||
if (System.currentTimeMillis() - startTime > 60000) {
|
||||
startTime = System.currentTimeMillis();
|
||||
ReflectionUtils.printThreadInfo(new PrintWriter(System.out),
|
||||
"Automatic Stack Trace every 60 seconds waiting on " +
|
||||
t.getName());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -21,11 +21,8 @@ package org.apache.hadoop.hbase;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import junit.framework.TestCase;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.dfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
@ -38,22 +35,17 @@ import org.apache.hadoop.io.WritableComparable;
|
|||
/**
|
||||
* Test HStoreFile
|
||||
*/
|
||||
public class TestHStoreFile extends TestCase {
|
||||
public class TestHStoreFile extends HBaseTestCase {
|
||||
static final Log LOG = LogFactory.getLog(TestHStoreFile.class);
|
||||
private static String DIR = "/";
|
||||
private static final char FIRST_CHAR = 'a';
|
||||
private static final char LAST_CHAR = 'z';
|
||||
private MiniDFSCluster cluster;
|
||||
private FileSystem fs;
|
||||
private Configuration conf;
|
||||
private Path dir = null;
|
||||
|
||||
/** {@inheritDoc} */
|
||||
@Override
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
this.conf = new HBaseConfiguration();
|
||||
this.cluster = null;
|
||||
this.cluster = new MiniDFSCluster(this.conf, 2, true, (String[])null);
|
||||
this.fs = cluster.getFileSystem();
|
||||
this.dir = new Path(DIR, getName());
|
||||
|
|
|
@ -30,15 +30,15 @@ import org.apache.hadoop.io.Text;
|
|||
* Tests region server failover when a region server exits.
|
||||
*/
|
||||
public class TestRegionServerAbort extends HBaseClusterTestCase {
|
||||
private final Log LOG = LogFactory.getLog(this.getClass().getName());
|
||||
private HTable table;
|
||||
final Log LOG = LogFactory.getLog(this.getClass().getName());
|
||||
HTable table;
|
||||
|
||||
/** constructor */
|
||||
public TestRegionServerAbort() {
|
||||
super(2);
|
||||
conf.setInt("ipc.client.timeout", 5000); // reduce client timeout
|
||||
conf.setInt("ipc.client.connect.max.retries", 5); // and number of retries
|
||||
conf.setInt("hbase.client.retries.number", 5); // reduce HBase retries
|
||||
conf.setInt("ipc.client.timeout", 10000); // reduce client timeout
|
||||
conf.setInt("ipc.client.connect.max.retries", 5); // and number of retries
|
||||
conf.setInt("hbase.client.retries.number", 5); // reduce HBase retries
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -50,14 +50,14 @@ public class TestRegionServerAbort extends HBaseClusterTestCase {
|
|||
@SuppressWarnings("unused")
|
||||
HTable meta = new HTable(conf, HConstants.META_TABLE_NAME);
|
||||
// Put something into the meta table.
|
||||
String tableName = getName();
|
||||
final String tableName = getName();
|
||||
HTableDescriptor desc = new HTableDescriptor(tableName);
|
||||
desc.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY.toString()));
|
||||
HBaseAdmin admin = new HBaseAdmin(conf);
|
||||
admin.createTable(desc);
|
||||
// put some values in the table
|
||||
this.table = new HTable(conf, new Text(tableName));
|
||||
Text row = new Text("row1");
|
||||
final Text row = new Text("row1");
|
||||
long lockid = table.startUpdate(row);
|
||||
table.put(lockid, HConstants.COLUMN_FAMILY,
|
||||
tableName.getBytes(HConstants.UTF8_ENCODING));
|
||||
|
@ -68,29 +68,45 @@ public class TestRegionServerAbort extends HBaseClusterTestCase {
|
|||
// Now shutdown the region server and wait for it to go down.
|
||||
this.cluster.abortRegionServer(0);
|
||||
LOG.info(this.cluster.waitOnRegionServer(0) + " has been shutdown");
|
||||
HScannerInterface scanner = null;
|
||||
try {
|
||||
// Verify that the client can find the data after the region has moved
|
||||
// to a different server
|
||||
scanner =
|
||||
table.obtainScanner(HConstants.COLUMN_FAMILY_ARRAY, new Text());
|
||||
LOG.info("Obtained scanner " + scanner);
|
||||
HStoreKey key = new HStoreKey();
|
||||
TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
|
||||
while (scanner.next(key, results)) {
|
||||
assertTrue(key.getRow().equals(row));
|
||||
assertEquals(1, results.size());
|
||||
byte[] bytes = results.get(HConstants.COLUMN_FAMILY);
|
||||
assertNotNull(bytes);
|
||||
assertTrue(tableName.equals(new String(bytes,
|
||||
HConstants.UTF8_ENCODING)));
|
||||
// Run verification in a thread so I can concurrently run a thread-dumper
|
||||
// while we're waiting (because in this test sometimes the meta scanner
|
||||
// looks to be be stuck).
|
||||
Runnable runnable = new Runnable() {
|
||||
public void run() {
|
||||
HScannerInterface scanner = null;
|
||||
try {
|
||||
// Verify that the client can find the data after the region has moved
|
||||
// to a different server
|
||||
scanner =
|
||||
table.obtainScanner(HConstants.COLUMN_FAMILY_ARRAY, new Text());
|
||||
LOG.info("Obtained scanner " + scanner);
|
||||
HStoreKey key = new HStoreKey();
|
||||
TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
|
||||
while (scanner.next(key, results)) {
|
||||
assertTrue(key.getRow().equals(row));
|
||||
assertEquals(1, results.size());
|
||||
byte[] bytes = results.get(HConstants.COLUMN_FAMILY);
|
||||
assertNotNull(bytes);
|
||||
assertTrue(tableName.equals(new String(bytes,
|
||||
HConstants.UTF8_ENCODING)));
|
||||
}
|
||||
LOG.info("Success!");
|
||||
} catch (IOException e) {
|
||||
e.printStackTrace();
|
||||
} finally {
|
||||
if (scanner != null) {
|
||||
LOG.info("Closing scanner " + scanner);
|
||||
try {
|
||||
scanner.close();
|
||||
} catch (IOException e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
LOG.info("Success!");
|
||||
} finally {
|
||||
if (scanner != null) {
|
||||
LOG.info("Closing scanner " + scanner);
|
||||
scanner.close();
|
||||
}
|
||||
}
|
||||
};
|
||||
Thread t = new Thread(runnable);
|
||||
t.start();
|
||||
threadDumpingJoin(t);
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue