You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2008/08/05 21:05:49 UTC
svn commit: r682874 - in /hadoop/hbase/trunk: CHANGES.txt
src/java/org/apache/hadoop/hbase/regionserver/Flusher.java
src/java/org/apache/hadoop/hbase/regionserver/HRegion.java
Author: stack
Date: Tue Aug 5 12:05:48 2008
New Revision: 682874
URL: http://svn.apache.org/viewvc?rev=682874&view=rev
Log:
HBASE-790 During import, single region blocks requests for >10 minutes, thread dumps, throws out pending requests, and continues
Modified:
hadoop/hbase/trunk/CHANGES.txt
hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/Flusher.java
hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java
Modified: hadoop/hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/CHANGES.txt?rev=682874&r1=682873&r2=682874&view=diff
==============================================================================
--- hadoop/hbase/trunk/CHANGES.txt (original)
+++ hadoop/hbase/trunk/CHANGES.txt Tue Aug 5 12:05:48 2008
@@ -224,6 +224,9 @@
HBASE-751 dfs exception and regionserver stuck during heavy write load
HBASE-793 HTable.getStartKeys() ignores table names when matching columns
(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
HBASE-559 MR example job to count table rows
Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/Flusher.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/Flusher.java?rev=682874&r1=682873&r2=682874&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/Flusher.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/Flusher.java Tue Aug 5 12:05:48 2008
@@ -237,7 +237,6 @@
if (!regionsInQueue.contains(r)) {
regionsInQueue.add(r);
flushQueue.add(r);
- r.setLastFlushTime(now);
}
}
}
Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=682874&r1=682873&r2=682874&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java Tue Aug 5 12:05:48 2008
@@ -317,7 +317,6 @@
new ConcurrentHashMap<Integer, byte []>();
private final Map<Integer, TreeMap<HStoreKey, byte []>> targetColumns =
new ConcurrentHashMap<Integer, TreeMap<HStoreKey, byte []>>();
- private volatile boolean flushRequested = false;
// Default access because read by tests.
final Map<Integer, HStore> stores = new ConcurrentHashMap<Integer, HStore>();
final AtomicLong memcacheSize = new AtomicLong(0);
@@ -337,6 +336,8 @@
static class WriteState {
// Set while a memcache flush is happening.
volatile boolean flushing = false;
+ // Set when a flush has been requested.
+ volatile boolean flushRequested = false;
// Set while a compaction is running.
volatile boolean compacting = false;
// Gets set in close. If set, cannot compact or flush again.
@@ -355,6 +356,10 @@
boolean isReadOnly() {
return this.readOnly;
}
+
+ boolean isFlushRequested() {
+ return this.flushRequested;
+ }
}
private volatile WriteState writestate = new WriteState();
@@ -689,11 +694,6 @@
return this.lastFlushTime;
}
- /** @param t the lastFlushTime */
- void setLastFlushTime(long t) {
- this.lastFlushTime = t;
- }
-
//////////////////////////////////////////////////////////////////////////////
// HRegion maintenance.
//
@@ -946,7 +946,7 @@
}
synchronized (writestate) {
if (!writestate.flushing && writestate.writesEnabled) {
- writestate.flushing = true;
+ this.writestate.flushing = true;
} else {
if(LOG.isDebugEnabled()) {
LOG.debug("NOT flushing memcache for region " + this +
@@ -968,6 +968,7 @@
} finally {
synchronized (writestate) {
writestate.flushing = false;
+ this.writestate.flushRequested = false;
writestate.notifyAll();
}
}
@@ -1008,7 +1009,6 @@
private boolean internalFlushcache() throws IOException {
final long startTime = System.currentTimeMillis();
// Clear flush flag.
- this.flushRequested = false;
// Record latest flush time
this.lastFlushTime = startTime;
// If nothing to flush, return and avoid logging start/stop flush.
@@ -1411,7 +1411,7 @@
releaseRowLock(lid);
}
}
-
+
/*
* Check if resources to support an update.
*
@@ -1605,8 +1605,7 @@
size = this.memcacheSize.addAndGet(
getStore(key.getColumn()).add(key, e.getValue()));
}
- flush = this.flushListener != null && !this.flushRequested &&
- isFlushSize(size);
+ flush = isFlushSize(size);
} finally {
this.updatesLock.readLock().unlock();
}
@@ -1617,11 +1616,16 @@
}
private void requestFlush() {
- if (this.flushListener == null || this.flushRequested) {
+ if (this.flushListener == null) {
return;
}
- this.flushListener.request(this);
- this.flushRequested = true;
+ synchronized (writestate) {
+ if (this.writestate.isFlushRequested()) {
+ return;
+ }
+ writestate.flushRequested = true;
+ this.flushListener.request(this);
+ }
if (LOG.isDebugEnabled()) {
LOG.debug("Flush requested on " + this);
}