HADOOP-2392 HADOOP-2324 HADOOP-2396 HADOOP-2397

HADOOP-2392, HADOOP-2324:

Chore
- initialChore() now returns boolean

HMaster
- rather than retry in root and meta scanners, return if a scan fails. It will get retried on the next scan. This has two effects: 1) scanners exit more quickly during shutdown and 2) they don't keep retrying to connect to a dead server, allowing them to recover from a server going down more quickly.
- initialScan in root and meta scanners return boolean and do not progress to maintenanceScan until the initial scan completes successfully.

HRegionServer
- speed up region server exit by reordering join's so that we join with threads in the order that we told them to stop

TestTableMapReduce
- remove overrides of heartbeat and thread wake intervals

HADOOP-2396:

HMaster
- move check for null HRegionInfo before first attempt to dereference it.

HADOOP-2397:
- HMaster$BaseScanner.checkAssigned: don't try to split dead server's log if initial startup has completed.

HADOOP-2353:

HMsg
- change toString() to only output the region name rather than calling HRegionInfo.toString()

StaticTestEnvironment
- make logging a bit less verbose

TestHLog
- was writing to local file system and failing on Windows


git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@603428 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Jim Kellerman 2007-12-12 00:40:02 +00:00
parent c70f6c5122
commit c499da8b7b
8 changed files with 179 additions and 178 deletions

View File

@ -72,6 +72,11 @@ Trunk (unreleased changes)
different column families do not have entries for some rows different column families do not have entries for some rows
HADOOP-2283 AlreadyBeingCreatedException (Was: Stuck replay of failed HADOOP-2283 AlreadyBeingCreatedException (Was: Stuck replay of failed
regionserver edits) regionserver edits)
HADOOP-2392 TestRegionServerExit has new failure mode since HADOOP-2338
HADOOP-2324 Fix assertion failures in TestTableMapReduce
HADOOP-2396 NPE in HMaster.cancelLease
HADOOP-2397 The only time that a meta scanner should try to recover a log is
when the master is starting
IMPROVEMENTS IMPROVEMENTS
HADOOP-2401 Add convenience put method that takes writable HADOOP-2401 Add convenience put method that takes writable

View File

@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.util.Sleeper;
public abstract class Chore extends Thread { public abstract class Chore extends Thread {
private final Log LOG = LogFactory.getLog(this.getClass()); private final Log LOG = LogFactory.getLog(this.getClass());
private final Sleeper sleeper; private final Sleeper sleeper;
protected final AtomicBoolean stop; protected volatile AtomicBoolean stop;
/** /**
* @param p Period at which we should run. Will be adjusted appropriately * @param p Period at which we should run. Will be adjusted appropriately
@ -49,9 +49,13 @@ public abstract class Chore extends Thread {
this.stop = s; this.stop = s;
} }
/** {@inheritDoc} */
@Override
public void run() { public void run() {
try { try {
initialChore(); while (!initialChore()) {
this.sleeper.sleep();
}
this.sleeper.sleep(); this.sleeper.sleep();
while(!this.stop.get()) { while(!this.stop.get()) {
long startTime = System.currentTimeMillis(); long startTime = System.currentTimeMillis();
@ -65,9 +69,11 @@ public abstract class Chore extends Thread {
/** /**
* Override to run a task before we start looping. * Override to run a task before we start looping.
* @return true if initial chore was successful
*/ */
protected void initialChore() { protected boolean initialChore() {
// Default does nothing. // Default does nothing.
return true;
} }
/** /**

View File

@ -92,38 +92,38 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
// started here in HMaster rather than have them have to know about the // started here in HMaster rather than have them have to know about the
// hosting class // hosting class
volatile AtomicBoolean closed = new AtomicBoolean(true); volatile AtomicBoolean closed = new AtomicBoolean(true);
volatile AtomicBoolean shutdownRequested = new AtomicBoolean(false); volatile boolean shutdownRequested = false;
volatile AtomicInteger quiescedMetaServers = new AtomicInteger(0); volatile AtomicInteger quiescedMetaServers = new AtomicInteger(0);
volatile boolean fsOk; volatile boolean fsOk = true;
Path dir; final Path dir;
HBaseConfiguration conf; final HBaseConfiguration conf;
FileSystem fs; final FileSystem fs;
Random rand; final Random rand;
int threadWakeFrequency; final int threadWakeFrequency;
int numRetries; final int numRetries;
long maxRegionOpenTime; final long maxRegionOpenTime;
DelayQueue<RegionServerOperation> delayedToDoQueue = volatile DelayQueue<RegionServerOperation> delayedToDoQueue =
new DelayQueue<RegionServerOperation>(); new DelayQueue<RegionServerOperation>();
BlockingQueue<RegionServerOperation> toDoQueue = volatile BlockingQueue<RegionServerOperation> toDoQueue =
new LinkedBlockingQueue<RegionServerOperation>(); new LinkedBlockingQueue<RegionServerOperation>();
int leaseTimeout; final int leaseTimeout;
private Leases serverLeases; private final Leases serverLeases;
private Server server; private final Server server;
private HServerAddress address; private final HServerAddress address;
HConnection connection; final HConnection connection;
int metaRescanInterval; final int metaRescanInterval;
final AtomicReference<HServerAddress> rootRegionLocation = volatile AtomicReference<HServerAddress> rootRegionLocation =
new AtomicReference<HServerAddress>(null); new AtomicReference<HServerAddress>(null);
Lock splitLogLock = new ReentrantLock(); final Lock splitLogLock = new ReentrantLock();
// A Sleeper that sleeps for threadWakeFrequency // A Sleeper that sleeps for threadWakeFrequency
protected Sleeper sleeper; protected final Sleeper sleeper;
// Default access so accesible from unit tests. MASTER is name of the webapp // Default access so accesible from unit tests. MASTER is name of the webapp
// and the attribute name used stuffing this instance into web context. // and the attribute name used stuffing this instance into web context.
@ -183,7 +183,7 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
protected boolean rootRegion; protected boolean rootRegion;
protected final Text tableName; protected final Text tableName;
protected abstract void initialScan(); protected abstract boolean initialScan();
protected abstract void maintenanceScan(); protected abstract void maintenanceScan();
BaseScanner(final Text tableName, final int period, BaseScanner(final Text tableName, final int period,
@ -194,8 +194,8 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
} }
@Override @Override
protected void initialChore() { protected boolean initialChore() {
initialScan(); return initialScan();
} }
@Override @Override
@ -473,7 +473,7 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
// This is only done from here if we are restarting and there is stale // This is only done from here if we are restarting and there is stale
// data in the meta region. Once we are on-line, dead server log // data in the meta region. Once we are on-line, dead server log
// recovery is handled by lease expiration and ProcessServerShutdown // recovery is handled by lease expiration and ProcessServerShutdown
if (serverName.length() != 0) { if (!initialMetaScanComplete && serverName.length() != 0) {
StringBuilder dirName = new StringBuilder("log_"); StringBuilder dirName = new StringBuilder("log_");
dirName.append(serverName.replace(":", "_")); dirName.append(serverName.replace(":", "_"));
Path logDir = new Path(dir, dirName.toString()); Path logDir = new Path(dir, dirName.toString());
@ -500,7 +500,7 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
} }
} }
volatile boolean rootScanned; volatile boolean rootScanned = false;
/** Scanner for the <code>ROOT</code> HRegion. */ /** Scanner for the <code>ROOT</code> HRegion. */
class RootScanner extends BaseScanner { class RootScanner extends BaseScanner {
@ -509,60 +509,51 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
super(HConstants.ROOT_TABLE_NAME, metaRescanInterval, closed); super(HConstants.ROOT_TABLE_NAME, metaRescanInterval, closed);
} }
private void scanRoot() { private boolean scanRoot() {
int tries = 0; // Don't retry if we get an error while scanning. Errors are most often
while (!closed.get() && tries < numRetries) { // caused by the server going away. Wait until next rescan interval when
synchronized (rootRegionLocation) { // things should be back to normal
while(!closed.get() && rootRegionLocation.get() == null) { boolean scanSuccessful = false;
// rootRegionLocation will be filled in when we get an 'open region' synchronized (rootRegionLocation) {
// regionServerReport message from the HRegionServer that has been while(!closed.get() && rootRegionLocation.get() == null) {
// allocated the ROOT region below. // rootRegionLocation will be filled in when we get an 'open region'
try { // regionServerReport message from the HRegionServer that has been
rootRegionLocation.wait(); // allocated the ROOT region below.
} catch (InterruptedException e) { try {
// continue rootRegionLocation.wait();
} } catch (InterruptedException e) {
// continue
} }
} }
if (closed.get()) {
continue;
}
try {
// Don't interrupt us while we're working
synchronized(rootScannerLock) {
scanRegion(new MetaRegion(rootRegionLocation.get(),
HRegionInfo.rootRegionInfo.getRegionName(), null));
}
break;
} catch (IOException e) {
e = RemoteExceptionHandler.checkIOException(e);
tries += 1;
if (tries == 1) {
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 (!checkFileSystem()) {
continue; // Avoid sleeping.
}
}
}
} catch (Exception e) {
// If for some reason we get some other kind of exception,
// at least log it rather than go out silently.
LOG.error("Unexpected exception", e);
}
sleeper.sleep();
} }
if (closed.get()) {
return scanSuccessful;
}
try {
// Don't interrupt us while we're working
synchronized(rootScannerLock) {
scanRegion(new MetaRegion(rootRegionLocation.get(),
HRegionInfo.rootRegionInfo.getRegionName(), null));
}
scanSuccessful = true;
} catch (IOException e) {
e = RemoteExceptionHandler.checkIOException(e);
LOG.warn("Scan ROOT region", e);
// Make sure the file system is still available
checkFileSystem();
} catch (Exception e) {
// If for some reason we get some other kind of exception,
// at least log it rather than go out silently.
LOG.error("Unexpected exception", e);
}
return scanSuccessful;
} }
@Override @Override
protected void initialScan() { protected boolean initialScan() {
scanRoot(); rootScanned = scanRoot();
rootScanned = true; return rootScanned;
} }
@Override @Override
@ -571,8 +562,8 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
} }
} }
private RootScanner rootScannerThread; private final RootScanner rootScannerThread;
Integer rootScannerLock = new Integer(0); final Integer rootScannerLock = new Integer(0);
/** Describes a meta region and its server */ /** Describes a meta region and its server */
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
@ -652,18 +643,18 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
} }
/** Set by root scanner to indicate the number of meta regions */ /** Set by root scanner to indicate the number of meta regions */
final AtomicInteger numberOfMetaRegions = new AtomicInteger(); volatile AtomicInteger numberOfMetaRegions = new AtomicInteger();
/** Work for the meta scanner is queued up here */ /** Work for the meta scanner is queued up here */
final BlockingQueue<MetaRegion> metaRegionsToScan = volatile BlockingQueue<MetaRegion> metaRegionsToScan =
new LinkedBlockingQueue<MetaRegion>(); new LinkedBlockingQueue<MetaRegion>();
/** These are the online meta regions */ /** These are the online meta regions */
final SortedMap<Text, MetaRegion> onlineMetaRegions = volatile SortedMap<Text, MetaRegion> onlineMetaRegions =
Collections.synchronizedSortedMap(new TreeMap<Text, MetaRegion>()); Collections.synchronizedSortedMap(new TreeMap<Text, MetaRegion>());
/** Set by meta scanner after initial scan */ /** Set by meta scanner after initial scan */
volatile boolean initialMetaScanComplete; volatile boolean initialMetaScanComplete = false;
/** /**
* MetaScanner <code>META</code> table. * MetaScanner <code>META</code> table.
@ -675,65 +666,58 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
* action would prevent other work from getting done. * action would prevent other work from getting done.
*/ */
class MetaScanner extends BaseScanner { class MetaScanner extends BaseScanner {
private final List<MetaRegion> metaRegionsToRescan =
new ArrayList<MetaRegion>();
/** Constructor */ /** Constructor */
public MetaScanner() { public MetaScanner() {
super(HConstants.META_TABLE_NAME, metaRescanInterval, closed); super(HConstants.META_TABLE_NAME, metaRescanInterval, closed);
} }
private void scanOneMetaRegion(MetaRegion region) { private boolean scanOneMetaRegion(MetaRegion region) {
int tries = 0; // Don't retry if we get an error while scanning. Errors are most often
while (!closed.get() && tries < numRetries) { // caused by the server going away. Wait until next rescan interval when
while (!closed.get() && !rootScanned && // things should be back to normal
rootRegionLocation.get() == null) { boolean scanSuccessful = false;
sleeper.sleep(); while (!closed.get() && !rootScanned &&
} rootRegionLocation.get() == null) {
if (closed.get()) {
continue;
}
try {
// Don't interrupt us while we're working
synchronized (metaScannerLock) {
scanRegion(region);
onlineMetaRegions.put(region.getStartKey(), region);
}
break;
} catch (IOException e) {
e = RemoteExceptionHandler.checkIOException(e);
tries += 1;
if (tries == 1) {
LOG.warn("Scan one META region: " + region.toString(), e);
} else {
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) {
// If for some reason we get some other kind of exception,
// at least log it rather than go out silently.
LOG.error("Unexpected exception", e);
}
// Sleep before going around again.
sleeper.sleep(); sleeper.sleep();
} }
if (closed.get()) {
return scanSuccessful;
}
try {
// Don't interrupt us while we're working
synchronized (metaScannerLock) {
scanRegion(region);
onlineMetaRegions.put(region.getStartKey(), region);
}
scanSuccessful = true;
} catch (IOException e) {
e = RemoteExceptionHandler.checkIOException(e);
LOG.warn("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 return.
if (!onlineMetaRegions.containsValue(region.getStartKey())) {
LOG.debug("Scanned region is no longer in map of online " +
"regions or its value has changed");
return scanSuccessful;
}
// Make sure the file system is still available
checkFileSystem();
} catch (Exception e) {
// If for some reason we get some other kind of exception,
// at least log it rather than go out silently.
LOG.error("Unexpected exception", e);
}
return scanSuccessful;
} }
@Override @Override
protected void initialScan() { protected boolean initialScan() {
MetaRegion region = null; MetaRegion region = null;
while (!closed.get() && region == null && !metaRegionsScanned()) { while (!closed.get() && region == null && !metaRegionsScanned()) {
try { try {
@ -742,12 +726,17 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
} catch (InterruptedException e) { } catch (InterruptedException e) {
// continue // continue
} }
if (region == null && metaRegionsToRescan.size() != 0) {
region = metaRegionsToRescan.remove(0);
}
if (region != null) { if (region != null) {
scanOneMetaRegion(region); if (!scanOneMetaRegion(region)) {
metaRegionsToRescan.add(region);
}
} }
} }
initialMetaScanComplete = true; initialMetaScanComplete = true;
return true;
} }
@Override @Override
@ -797,23 +786,23 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
} }
} }
MetaScanner metaScannerThread; final MetaScanner metaScannerThread;
Integer metaScannerLock = new Integer(0); final Integer metaScannerLock = new Integer(0);
/** The map of known server names to server info */ /** The map of known server names to server info */
final Map<String, HServerInfo> serversToServerInfo = volatile Map<String, HServerInfo> serversToServerInfo =
new ConcurrentHashMap<String, HServerInfo>(); new ConcurrentHashMap<String, HServerInfo>();
/** Set of known dead servers */ /** Set of known dead servers */
final Set<String> deadServers = volatile Set<String> deadServers =
Collections.synchronizedSet(new HashSet<String>()); Collections.synchronizedSet(new HashSet<String>());
/** SortedMap server load -> Set of server names */ /** SortedMap server load -> Set of server names */
final SortedMap<HServerLoad, Set<String>> loadToServers = volatile SortedMap<HServerLoad, Set<String>> loadToServers =
Collections.synchronizedSortedMap(new TreeMap<HServerLoad, Set<String>>()); Collections.synchronizedSortedMap(new TreeMap<HServerLoad, Set<String>>());
/** Map of server names -> server load */ /** Map of server names -> server load */
final Map<String, HServerLoad> serversToLoad = volatile Map<String, HServerLoad> serversToLoad =
new ConcurrentHashMap<String, HServerLoad>(); new ConcurrentHashMap<String, HServerLoad>();
/** /**
@ -827,36 +816,36 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
* <p>Items are removed from this list when a region server reports in that * <p>Items are removed from this list when a region server reports in that
* the region has been deployed. * the region has been deployed.
*/ */
final SortedMap<HRegionInfo, Long> unassignedRegions = volatile SortedMap<HRegionInfo, Long> unassignedRegions =
Collections.synchronizedSortedMap(new TreeMap<HRegionInfo, Long>()); Collections.synchronizedSortedMap(new TreeMap<HRegionInfo, Long>());
/** /**
* Regions that have been assigned, and the server has reported that it has * Regions that have been assigned, and the server has reported that it has
* started serving it, but that we have not yet recorded in the meta table. * started serving it, but that we have not yet recorded in the meta table.
*/ */
final Set<Text> pendingRegions = volatile Set<Text> pendingRegions =
Collections.synchronizedSet(new HashSet<Text>()); Collections.synchronizedSet(new HashSet<Text>());
/** /**
* The 'killList' is a list of regions that are going to be closed, but not * The 'killList' is a list of regions that are going to be closed, but not
* reopened. * reopened.
*/ */
final Map<String, HashMap<Text, HRegionInfo>> killList = volatile Map<String, HashMap<Text, HRegionInfo>> killList =
new ConcurrentHashMap<String, HashMap<Text, HRegionInfo>>(); new ConcurrentHashMap<String, HashMap<Text, HRegionInfo>>();
/** 'killedRegions' contains regions that are in the process of being closed */ /** 'killedRegions' contains regions that are in the process of being closed */
final Set<Text> killedRegions = volatile Set<Text> killedRegions =
Collections.synchronizedSet(new HashSet<Text>()); Collections.synchronizedSet(new HashSet<Text>());
/** /**
* 'regionsToDelete' contains regions that need to be deleted, but cannot be * 'regionsToDelete' contains regions that need to be deleted, but cannot be
* until the region server closes it * until the region server closes it
*/ */
final Set<Text> regionsToDelete = volatile Set<Text> regionsToDelete =
Collections.synchronizedSet(new HashSet<Text>()); Collections.synchronizedSet(new HashSet<Text>());
/** Set of tables currently in creation. */ /** Set of tables currently in creation. */
private Set<Text> tableInCreation = private volatile Set<Text> tableInCreation =
Collections.synchronizedSet(new HashSet<Text>()); Collections.synchronizedSet(new HashSet<Text>());
/** Build the HMaster out of a raw configuration item. /** Build the HMaster out of a raw configuration item.
@ -881,7 +870,6 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
public HMaster(Path dir, HServerAddress address, HBaseConfiguration conf) public HMaster(Path dir, HServerAddress address, HBaseConfiguration conf)
throws IOException { throws IOException {
this.fsOk = true;
this.dir = dir; this.dir = dir;
this.conf = conf; this.conf = conf;
this.fs = FileSystem.get(conf); this.fs = FileSystem.get(conf);
@ -945,11 +933,9 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
conf.getInt("hbase.master.meta.thread.rescanfrequency", 60 * 1000); conf.getInt("hbase.master.meta.thread.rescanfrequency", 60 * 1000);
// The root region // The root region
this.rootScanned = false;
this.rootScannerThread = new RootScanner(); this.rootScannerThread = new RootScanner();
// Scans the meta table // Scans the meta table
this.initialMetaScanComplete = false;
this.metaScannerThread = new MetaScanner(); this.metaScannerThread = new MetaScanner();
unassignRootRegion(); unassignRootRegion();
@ -1021,7 +1007,7 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
*/ */
public HServerAddress getRootRegionLocation() { public HServerAddress getRootRegionLocation() {
HServerAddress rootServer = null; HServerAddress rootServer = null;
if (!shutdownRequested.get() && !closed.get()) { if (!shutdownRequested && !closed.get()) {
rootServer = this.rootRegionLocation.get(); rootServer = this.rootRegionLocation.get();
} }
return rootServer; return rootServer;
@ -1313,9 +1299,6 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
throws IOException { throws IOException {
String serverName = serverInfo.getServerAddress().toString().trim(); String serverName = serverInfo.getServerAddress().toString().trim();
long serverLabel = getServerLabel(serverName); long serverLabel = getServerLabel(serverName);
// if (LOG.isDebugEnabled()) {
// LOG.debug("received heartbeat from " + serverName);
// }
if (msgs.length > 0) { if (msgs.length > 0) {
if (msgs[0].getMsg() == HMsg.MSG_REPORT_EXITING) { if (msgs[0].getMsg() == HMsg.MSG_REPORT_EXITING) {
synchronized (serversToServerInfo) { synchronized (serversToServerInfo) {
@ -1373,7 +1356,7 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
} }
} }
if (shutdownRequested.get() && !closed.get()) { if (shutdownRequested && !closed.get()) {
// Tell the server to stop serving any user regions // Tell the server to stop serving any user regions
return new HMsg[]{new HMsg(HMsg.MSG_REGIONSERVER_QUIESCE)}; return new HMsg[]{new HMsg(HMsg.MSG_REGIONSERVER_QUIESCE)};
} }
@ -1411,7 +1394,10 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
LOG.debug("region server race condition detected: " + serverName); LOG.debug("region server race condition detected: " + serverName);
} }
cancelLease(serverName, serverLabel); synchronized (serversToServerInfo) {
cancelLease(serverName, serverLabel);
serversToServerInfo.notifyAll();
}
return new HMsg[]{new HMsg(HMsg.MSG_REGIONSERVER_STOP)}; return new HMsg[]{new HMsg(HMsg.MSG_REGIONSERVER_STOP)};
} else { } else {
@ -1460,13 +1446,13 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
private boolean cancelLease(final String serverName, final long serverLabel) { private boolean cancelLease(final String serverName, final long serverLabel) {
boolean leaseCancelled = false; boolean leaseCancelled = false;
HServerInfo info = serversToServerInfo.remove(serverName); HServerInfo info = serversToServerInfo.remove(serverName);
if (rootRegionLocation.get() != null &&
info.getServerAddress().equals(rootRegionLocation.get())) {
unassignRootRegion();
}
if (info != null) { if (info != null) {
// Only cancel lease and update load information once. // Only cancel lease and update load information once.
// This method can be called a couple of times during shutdown. // This method can be called a couple of times during shutdown.
if (rootRegionLocation.get() != null &&
info.getServerAddress().equals(rootRegionLocation.get())) {
unassignRootRegion();
}
LOG.info("Cancelling lease for " + serverName); LOG.info("Cancelling lease for " + serverName);
serverLeases.cancelLease(serverLabel, serverLabel); serverLeases.cancelLease(serverLabel, serverLabel);
leaseCancelled = true; leaseCancelled = true;
@ -1502,7 +1488,7 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
for (int i = 0; i < incomingMsgs.length; i++) { for (int i = 0; i < incomingMsgs.length; i++) {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("Received " + incomingMsgs[i].toString() + "from " + LOG.debug("Received " + incomingMsgs[i].toString() + " from " +
serverName); serverName);
} }
HRegionInfo region = incomingMsgs[i].getRegionInfo(); HRegionInfo region = incomingMsgs[i].getRegionInfo();
@ -2513,7 +2499,7 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
/** {@inheritDoc} */ /** {@inheritDoc} */
public void shutdown() { public void shutdown() {
LOG.info("Cluster shutdown requested. Starting to quiesce servers"); LOG.info("Cluster shutdown requested. Starting to quiesce servers");
this.shutdownRequested.set(true); this.shutdownRequested = true;
} }
/** {@inheritDoc} */ /** {@inheritDoc} */

View File

@ -188,7 +188,7 @@ public class HMsg implements Writable {
message.append(") : "); message.append(") : ");
break; break;
} }
message.append(info == null ? "null" : info.toString()); message.append(info == null ? "null" : info.getRegionName());
return message.toString(); return message.toString();
} }

View File

@ -1005,11 +1005,11 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
* Presumption is that all closes and stops have already been called. * Presumption is that all closes and stops have already been called.
*/ */
void join() { void join() {
join(this.logRoller); join(this.workerThread);
join(this.cacheFlusher); join(this.cacheFlusher);
join(this.compactor); join(this.compactor);
join(this.splitter); join(this.splitter);
join(this.workerThread); join(this.logRoller);
} }
private void join(final Thread t) { private void join(final Thread t) {
@ -1523,9 +1523,9 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
region.deleteAll(row, timestamp); region.deleteAll(row, timestamp);
} }
/** {@inheritDoc} */
public void deleteFamily(Text regionName, Text row, Text family, public void deleteFamily(Text regionName, Text row, Text family,
long timestamp) long timestamp) throws IOException{
throws IOException{
getRegion(regionName).deleteFamily(row, family, timestamp); getRegion(regionName).deleteFamily(row, family, timestamp);
} }

View File

@ -108,7 +108,7 @@ public class StaticTestEnvironment {
Layout layout = consoleAppender.getLayout(); Layout layout = consoleAppender.getLayout();
if(layout instanceof PatternLayout) { if(layout instanceof PatternLayout) {
PatternLayout consoleLayout = (PatternLayout)layout; PatternLayout consoleLayout = (PatternLayout)layout;
consoleLayout.setConversionPattern("%d %-5p [%t] %l: %m%n"); consoleLayout.setConversionPattern("%d %-5p [%t] %C{2}(%L): %m%n");
} }
} }
LOG.setLevel(logLevel); LOG.setLevel(logLevel);

View File

@ -22,6 +22,7 @@ package org.apache.hadoop.hbase;
import java.io.IOException; import java.io.IOException;
import java.util.TreeMap; import java.util.TreeMap;
import org.apache.hadoop.dfs.MiniDFSCluster;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.SequenceFile;
@ -30,24 +31,33 @@ import org.apache.hadoop.io.SequenceFile.Reader;
/** JUnit test case for HLog */ /** JUnit test case for HLog */
public class TestHLog extends HBaseTestCase implements HConstants { public class TestHLog extends HBaseTestCase implements HConstants {
private Path dir; private final Path dir = new Path("/hbase");
private FileSystem fs; private FileSystem fs;
private MiniDFSCluster cluster;
/** constructor */
public TestHLog() {
this.cluster = null;
}
/** {@inheritDoc} */
@Override @Override
protected void setUp() throws Exception { public void setUp() throws Exception {
super.setUp(); super.setUp();
this.dir = getUnitTestdir(getName()); cluster = new MiniDFSCluster(conf, 2, true, (String[])null);
this.fs = FileSystem.get(this.conf); this.fs = cluster.getFileSystem();
if (fs.exists(dir)) { if (fs.exists(dir)) {
fs.delete(dir); fs.delete(dir);
} }
} }
/** {@inheritDoc} */
@Override @Override
protected void tearDown() throws Exception { public void tearDown() throws Exception {
if (this.fs.exists(this.dir)) { if (this.fs.exists(this.dir)) {
this.fs.delete(this.dir); this.fs.delete(this.dir);
} }
StaticTestEnvironment.shutdownDfs(cluster);
super.tearDown(); super.tearDown();
} }
@ -76,6 +86,7 @@ public class TestHLog extends HBaseTestCase implements HConstants {
log.rollWriter(); log.rollWriter();
} }
HLog.splitLog(this.testDir, this.dir, this.fs, this.conf); HLog.splitLog(this.testDir, this.dir, this.fs, this.conf);
log = null;
} finally { } finally {
if (log != null) { if (log != null) {
log.closeAndDelete(); log.closeAndDelete();

View File

@ -87,13 +87,6 @@ public class TestTableMapReduce extends MultiRegionTable {
public TestTableMapReduce() { public TestTableMapReduce() {
super(); super();
// The region server doesn't have to talk to the master quite so often
conf.setInt("hbase.regionserver.msginterval", 2000);
// Make the thread wake frequency a little slower so other threads
// can run
conf.setInt("hbase.server.thread.wakefrequency", 2000);
// Make sure the cache gets flushed so we trigger a compaction(s) and // Make sure the cache gets flushed so we trigger a compaction(s) and
// hence splits. // hence splits.
conf.setInt("hbase.hregion.memcache.flush.size", 1024 * 1024); conf.setInt("hbase.hregion.memcache.flush.size", 1024 * 1024);