HBASE-790 During import, single region blocks requests for >10 minutes, thread dumps, throws out pending requests, and continues

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@682874 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2008-08-05 19:05:48 +00:00
parent 13d0f9c527
commit 942c54cf56
3 changed files with 21 additions and 15 deletions

View File

@ -224,6 +224,9 @@ Release 0.2.0
HBASE-751 dfs exception and regionserver stuck during heavy write load HBASE-751 dfs exception and regionserver stuck during heavy write load
HBASE-793 HTable.getStartKeys() ignores table names when matching columns HBASE-793 HTable.getStartKeys() ignores table names when matching columns
(Andrew Purtell and Dru Jensen via Stack) (Andrew Purtell and Dru Jensen via Stack)
HBASE-790 During import, single region blocks requests for >10 minutes,
thread dumps, throws out pending requests, and continues
(Jonathan Gray via Stack)
IMPROVEMENTS IMPROVEMENTS
HBASE-559 MR example job to count table rows HBASE-559 MR example job to count table rows

View File

@ -237,7 +237,6 @@ class Flusher extends Thread implements FlushRequester {
if (!regionsInQueue.contains(r)) { if (!regionsInQueue.contains(r)) {
regionsInQueue.add(r); regionsInQueue.add(r);
flushQueue.add(r); flushQueue.add(r);
r.setLastFlushTime(now);
} }
} }
} }

View File

@ -317,7 +317,6 @@ public class HRegion implements HConstants {
new ConcurrentHashMap<Integer, byte []>(); new ConcurrentHashMap<Integer, byte []>();
private final Map<Integer, TreeMap<HStoreKey, byte []>> targetColumns = private final Map<Integer, TreeMap<HStoreKey, byte []>> targetColumns =
new ConcurrentHashMap<Integer, TreeMap<HStoreKey, byte []>>(); new ConcurrentHashMap<Integer, TreeMap<HStoreKey, byte []>>();
private volatile boolean flushRequested = false;
// Default access because read by tests. // Default access because read by tests.
final Map<Integer, HStore> stores = new ConcurrentHashMap<Integer, HStore>(); final Map<Integer, HStore> stores = new ConcurrentHashMap<Integer, HStore>();
final AtomicLong memcacheSize = new AtomicLong(0); final AtomicLong memcacheSize = new AtomicLong(0);
@ -337,6 +336,8 @@ public class HRegion implements HConstants {
static class WriteState { static class WriteState {
// Set while a memcache flush is happening. // Set while a memcache flush is happening.
volatile boolean flushing = false; volatile boolean flushing = false;
// Set when a flush has been requested.
volatile boolean flushRequested = false;
// Set while a compaction is running. // Set while a compaction is running.
volatile boolean compacting = false; volatile boolean compacting = false;
// Gets set in close. If set, cannot compact or flush again. // Gets set in close. If set, cannot compact or flush again.
@ -355,6 +356,10 @@ public class HRegion implements HConstants {
boolean isReadOnly() { boolean isReadOnly() {
return this.readOnly; return this.readOnly;
} }
boolean isFlushRequested() {
return this.flushRequested;
}
} }
private volatile WriteState writestate = new WriteState(); private volatile WriteState writestate = new WriteState();
@ -689,11 +694,6 @@ public class HRegion implements HConstants {
return this.lastFlushTime; return this.lastFlushTime;
} }
/** @param t the lastFlushTime */
void setLastFlushTime(long t) {
this.lastFlushTime = t;
}
////////////////////////////////////////////////////////////////////////////// //////////////////////////////////////////////////////////////////////////////
// HRegion maintenance. // HRegion maintenance.
// //
@ -946,7 +946,7 @@ public class HRegion implements HConstants {
} }
synchronized (writestate) { synchronized (writestate) {
if (!writestate.flushing && writestate.writesEnabled) { if (!writestate.flushing && writestate.writesEnabled) {
writestate.flushing = true; this.writestate.flushing = true;
} else { } else {
if(LOG.isDebugEnabled()) { if(LOG.isDebugEnabled()) {
LOG.debug("NOT flushing memcache for region " + this + LOG.debug("NOT flushing memcache for region " + this +
@ -968,6 +968,7 @@ public class HRegion implements HConstants {
} finally { } finally {
synchronized (writestate) { synchronized (writestate) {
writestate.flushing = false; writestate.flushing = false;
this.writestate.flushRequested = false;
writestate.notifyAll(); writestate.notifyAll();
} }
} }
@ -1008,7 +1009,6 @@ public class HRegion implements HConstants {
private boolean internalFlushcache() throws IOException { private boolean internalFlushcache() throws IOException {
final long startTime = System.currentTimeMillis(); final long startTime = System.currentTimeMillis();
// Clear flush flag. // Clear flush flag.
this.flushRequested = false;
// Record latest flush time // Record latest flush time
this.lastFlushTime = startTime; this.lastFlushTime = startTime;
// If nothing to flush, return and avoid logging start/stop flush. // If nothing to flush, return and avoid logging start/stop flush.
@ -1411,7 +1411,7 @@ public class HRegion implements HConstants {
releaseRowLock(lid); releaseRowLock(lid);
} }
} }
/* /*
* Check if resources to support an update. * Check if resources to support an update.
* *
@ -1605,8 +1605,7 @@ public class HRegion implements HConstants {
size = this.memcacheSize.addAndGet( size = this.memcacheSize.addAndGet(
getStore(key.getColumn()).add(key, e.getValue())); getStore(key.getColumn()).add(key, e.getValue()));
} }
flush = this.flushListener != null && !this.flushRequested && flush = isFlushSize(size);
isFlushSize(size);
} finally { } finally {
this.updatesLock.readLock().unlock(); this.updatesLock.readLock().unlock();
} }
@ -1617,11 +1616,16 @@ public class HRegion implements HConstants {
} }
private void requestFlush() { private void requestFlush() {
if (this.flushListener == null || this.flushRequested) { if (this.flushListener == null) {
return; return;
} }
this.flushListener.request(this); synchronized (writestate) {
this.flushRequested = true; if (this.writestate.isFlushRequested()) {
return;
}
writestate.flushRequested = true;
this.flushListener.request(this);
}
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("Flush requested on " + this); LOG.debug("Flush requested on " + this);
} }