HBASE-501 Empty region server address in info:server entry and a startcode of -1 in .META.

M conf/hbase-default.xml
Add hbase.hbasemaster.maxregionopen property.
M src/java/org/apache/hadoop/hbase/HStore.java
Change way we log. Do way less. Just emit sums of edits applied
and skipped rather than individual edits.
M src/java/org/apache/hadoop/hbase/HRegionServer.java
Make sleeper instance a local rather than data member.
(reportForDuty): Take a sleeper instance.
(run): Removed redundant wrap of a 'for' by a 'while'.
(constructor): If IOE, do not offline the region. Seen to be
an overreaction.
M src/java/org/apache/hadoop/hbase/HLog.java
Don't output map of all files being cleaned everytime a new
entry is added; instead just log new entry. Remove emission
of every 10k edits.
M src/java/org/apache/hadoop/hbase/HMaster.java
Up default for maxregionopen. Was seeing that playing edits
could take a long time (mostly because we used log every
edit) but no harm in this being longer. On REPORT_CLOSE,
emit region info, not just region so can see the properties
(W/o, made it hard to figure who was responsible for offlining).
Add logging of attempt # in shutdown processing.
Add logging of state flags passed to the close region. Helps
debugging. Also in close offline ONLY if we are NOT reassigning
the region (jimk find).
M src/java/org/apache/hadoop/hbase/util/Sleeper.java
Add logging of extraordinary sleeps or calculated periods
(suspicion is that we're sleeping way longer on loaded machies
and the regionserver appears hung).


git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@636849 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2008-03-13 19:37:21 +00:00
parent 8c815c72f4
commit c1f974bbee
10 changed files with 179 additions and 165 deletions

View File

@ -38,6 +38,8 @@ Hbase Change Log
HBASE-433 HBASE-251 Region server should delete restore log after successful
restore, Stuck replaying the edits of crashed machine.
HBASE-27 hregioninfo cell empty in meta table
HBASE-501 Empty region server address in info:server entry and a
startcode of -1 in .META.
IMPROVEMENTS
HBASE-415 Rewrite leases to use DelayedBlockingQueue instead of polling

View File

@ -107,6 +107,13 @@
they are considered dead. On loaded cluster, may need to up this
period.</description>
</property>
<property>
<name>hbase.hbasemaster.maxregionopen</name>
<value>60000</value>
<description>Period to wait for a region open. If regionserver
takes longer than this interval, assign to a new regionserver.
</description>
</property>
<property>
<name>hbase.regionserver.lease.period</name>
<value>30000</value>

View File

@ -229,7 +229,7 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
this.threadWakeFrequency = conf.getInt(THREAD_WAKE_FREQUENCY, 10 * 1000);
this.numRetries = conf.getInt("hbase.client.retries.number", 2);
this.maxRegionOpenTime =
conf.getLong("hbase.hbasemaster.maxregionopen", 30 * 1000);
conf.getLong("hbase.hbasemaster.maxregionopen", 60 * 1000);
this.leaseTimeout = conf.getInt("hbase.master.lease.period", 30 * 1000);
this.server = HbaseRPC.getServer(this, address.getBindAddress(),
@ -589,6 +589,7 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
private void createTable(final HRegionInfo newRegion) throws IOException {
Text tableName = newRegion.getTableDesc().getName();
// TODO: Not thread safe check.
if (tableInCreation.contains(tableName)) {
throw new TableExistsException("Table " + tableName + " in process "
+ "of being created");

View File

@ -49,7 +49,8 @@ class ProcessRegionClose extends ProcessRegionStatusChange {
/** {@inheritDoc} */
@Override
public String toString() {
return "ProcessRegionClose of " + this.regionInfo.getRegionName();
return "ProcessRegionClose of " + this.regionInfo.getRegionName() +
", " + this.reassignRegion + ", " + this.deleteRegion;
}
@Override
@ -74,7 +75,7 @@ class ProcessRegionClose extends ProcessRegionStatusChange {
if (deleteRegion) {
HRegion.removeRegionFromMETA(getMetaServer(), metaRegionName,
regionInfo.getRegionName());
} else {
} else if (!this.reassignRegion) {
HRegion.offlineRegionInMETA(getMetaServer(), metaRegionName,
regionInfo);
}

View File

@ -295,7 +295,7 @@ class ProcessServerShutdown extends RegionServerOperation {
if (LOG.isDebugEnabled()) {
LOG.debug("process server shutdown scanning " +
r.getRegionName() + " on " + r.getServer() + " " +
Thread.currentThread().getName());
Thread.currentThread().getName() + " attempt " + tries);
}
server = master.connection.getHRegionConnection(r.getServer());

View File

@ -315,7 +315,7 @@ class ServerManager implements HConstants {
case HMsg.MSG_REPORT_CLOSE:
LOG.info(serverInfo.getServerAddress().toString() + " no longer serving " +
region.getRegionName());
region);
if (region.isRootRegion()) {
// Root region

View File

@ -546,7 +546,6 @@ public class HLog implements HConstants {
),
HREGION_OLDLOGFILE_NAME
);
Path oldlogfile = null;
SequenceFile.Reader old = null;
if (fs.exists(logfile)) {
@ -556,16 +555,15 @@ public class HLog implements HConstants {
fs.rename(logfile, oldlogfile);
old = new SequenceFile.Reader(fs, oldlogfile, conf);
}
if (LOG.isDebugEnabled()) {
LOG.debug("Creating new log file writer for path " + logfile +
"; map content " + logWriters.toString());
}
w = SequenceFile.createWriter(fs, conf, logfile, HLogKey.class,
HLogEdit.class, getCompressionType(conf));
// Use copy of regionName; regionName object is reused inside in
// HStoreKey.getRegionName so its content changes as we iterate.
logWriters.put(new Text(regionName), w);
if (LOG.isDebugEnabled()) {
LOG.debug("Creating new log file writer for path " + logfile +
" and region " + regionName);
}
if (old != null) {
// Copy from existing log file
@ -581,9 +579,6 @@ public class HLog implements HConstants {
fs.delete(oldlogfile);
}
}
if (LOG.isDebugEnabled() && count > 0 && count % 10000 == 0) {
LOG.debug("Applied " + count + " edits");
}
w.append(key, val);
}
if (LOG.isDebugEnabled()) {

View File

@ -121,7 +121,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
new ConcurrentHashMap<Text, HRegion>();
protected final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
private volatile List<HMsg> outboundMsgs =
private final List<HMsg> outboundMsgs =
Collections.synchronizedList(new ArrayList<HMsg>());
final int numRetries;
@ -142,9 +142,6 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
// Request counter
private volatile AtomicInteger requestCount = new AtomicInteger();
// A sleeper that sleeps for msgInterval.
private final Sleeper sleeper;
// Info server. Default access so can be used by unit tests. REGIONSERVER
// is name of the webapp and the attribute name used stuffing this instance
// into web context.
@ -234,7 +231,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
// Task thread to process requests from Master
this.worker = new Worker();
this.workerThread = new Thread(worker);
this.sleeper = new Sleeper(this.msgInterval, this.stopRequested);
// Server to handle client requests
this.server = HbaseRPC.getServer(this, address.getBindAddress(),
address.getPort(), conf.getInt("hbase.regionserver.handler.count", 10),
@ -259,10 +256,12 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
*/
public void run() {
boolean quiesceRequested = false;
// A sleeper that sleeps for msgInterval.
Sleeper sleeper =
new Sleeper(this.msgInterval, this.stopRequested);
try {
init(reportForDuty());
init(reportForDuty(sleeper));
long lastMsg = 0;
while(!stopRequested.get()) {
// Now ask master what it wants us to do and tell it what we have done
for (int tries = 0; !stopRequested.get();) {
long now = System.currentTimeMillis();
@ -276,11 +275,11 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
}
if ((now - lastMsg) >= msgInterval) {
HMsg outboundArray[] = null;
synchronized(outboundMsgs) {
synchronized(this.outboundMsgs) {
outboundArray =
this.outboundMsgs.toArray(new HMsg[outboundMsgs.size()]);
}
this.outboundMsgs.clear();
}
try {
this.serverInfo.setLoad(new HServerLoad(requestCount.get(),
@ -331,7 +330,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
break;
}
}
reportForDuty();
reportForDuty(sleeper);
restart = true;
} else {
LOG.fatal("file system available check failed. " +
@ -366,7 +365,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
"interrupted.", e);
}
if (msgs[i].getMsg() == HMsg.MSG_REGION_OPEN) {
outboundMsgs.add(new HMsg(HMsg.MSG_REPORT_PROCESS_OPEN,
this.outboundMsgs.add(new HMsg(HMsg.MSG_REPORT_PROCESS_OPEN,
msgs[i].getRegionInfo()));
}
}
@ -395,9 +394,8 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
}
}
}
this.sleeper.sleep(lastMsg);
sleeper.sleep(lastMsg);
} // for
} // while (!stopRequested.get())
} catch (Throwable t) {
LOG.fatal("Unhandled exception. Aborting...", t);
abort();
@ -627,7 +625,8 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
* Let the master know we're here
* Run initialization using parameters passed us by the master.
*/
private HbaseMapWritable reportForDuty() throws IOException {
private HbaseMapWritable reportForDuty(final Sleeper sleeper)
throws IOException {
if (LOG.isDebugEnabled()) {
LOG.debug("Telling master at " +
conf.get(MASTER_ADDRESS) + " that we are up");
@ -651,7 +650,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
break;
} catch(IOException e) {
LOG.warn("error telling master we are up", e);
this.sleeper.sleep(lastMsg);
sleeper.sleep(lastMsg);
continue;
}
}
@ -794,12 +793,9 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
} catch (IOException e) {
LOG.error("error opening region " + regionInfo.getRegionName(), e);
// Mark the region offline.
// TODO: add an extra field in HRegionInfo to indicate that there is
// an error. We can't do that now because that would be an incompatible
// change that would require a migration
regionInfo.setOffline(true);
reportClose(regionInfo);
return;
}

View File

@ -299,7 +299,8 @@ public class HStore implements HConstants {
* reflected in the MapFiles.)
*/
private void doReconstructionLog(final Path reconstructionLog,
final long maxSeqID) throws UnsupportedEncodingException, IOException {
final long maxSeqID)
throws UnsupportedEncodingException, IOException {
if (reconstructionLog == null || !fs.exists(reconstructionLog)) {
// Nothing to do.
@ -316,16 +317,13 @@ public class HStore implements HConstants {
HLogKey key = new HLogKey();
HLogEdit val = new HLogEdit();
long skippedEdits = 0;
long editsCount = 0;
while (logReader.next(key, val)) {
maxSeqIdInLog = Math.max(maxSeqIdInLog, key.getLogSeqNum());
if (key.getLogSeqNum() <= maxSeqID) {
skippedEdits++;
continue;
}
if (skippedEdits > 0 && LOG.isDebugEnabled()) {
LOG.debug("Skipped " + skippedEdits +
" edits because sequence id <= " + maxSeqID);
}
// Check this edit is for me. Also, guard against writing
// METACOLUMN info such as HBASE::CACHEFLUSH entries
Text column = val.getColumn();
@ -335,11 +333,12 @@ public class HStore implements HConstants {
continue;
}
HStoreKey k = new HStoreKey(key.getRow(), column, val.getTimestamp());
if (LOG.isDebugEnabled()) {
LOG.debug("Applying edit <" + k.toString() + "=" + val.toString() +
">");
}
reconstructedCache.put(k, val.getVal());
editsCount++;
}
if (LOG.isDebugEnabled()) {
LOG.debug("Applied " + editsCount + ", skipped " + skippedEdits +
" because sequence id <= " + maxSeqID);
}
} finally {
logReader.close();

View File

@ -21,6 +21,9 @@ package org.apache.hadoop.hbase.util;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
/**
* Sleeper for current thread.
* Sleeps for passed period. Also checks passed boolean and if interrupted,
@ -28,6 +31,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
* sleep time is up).
*/
public class Sleeper {
private final Log LOG = LogFactory.getLog(this.getClass().getName());
private final int period;
private AtomicBoolean stop;
@ -56,10 +60,19 @@ public class Sleeper {
if (this.stop.get()) {
return;
}
long waitTime = this.period - (System.currentTimeMillis() - startTime);
long now = System.currentTimeMillis();
long waitTime = this.period - (now - startTime);
if (waitTime > this.period) {
LOG.warn("Calculated wait time > " + this.period +
"; setting to this.period: " + System.currentTimeMillis() + ", " +
startTime);
}
if (waitTime > 0) {
try {
Thread.sleep(waitTime);
if ((System.currentTimeMillis() - now) > (10 * this.period)) {
LOG.warn("We slept ten times longer than scheduled: " + this.period);
}
} catch(InterruptedException iex) {
// We we interrupted because we're meant to stop? If not, just
// continue ignoring the interruption