Index: src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java =================================================================== --- src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java (revision 611707) +++ src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java (working copy) @@ -232,6 +232,7 @@ private HTable root = null; private HTable meta = null; + private long startTime; /** constructor */ public Splitter() { @@ -240,6 +241,7 @@ /** {@inheritDoc} */ public void closing(final Text regionName) { + startTime = System.currentTimeMillis(); lock.writeLock().lock(); try { // Remove region from regions Map and add it to the Map of retiring @@ -367,10 +369,11 @@ } reportSplit(oldRegionInfo, newRegions[0].getRegionInfo(), newRegions[1].getRegionInfo()); - LOG.info("region split, META update, and report to master all" + + LOG.info("region split, META updated, and report to master all" + " successful. Old region=" + oldRegionInfo.getRegionName() + ", new regions: " + newRegions[0].getRegionName() + ", " + - newRegions[1].getRegionName()); + newRegions[1].getRegionName() + ". Split took " + + StringUtils.formatTimeDiff(System.currentTimeMillis(), startTime)); // Do not serve the new regions. Let the Master assign them. } Index: src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java =================================================================== --- src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java (revision 611707) +++ src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java (working copy) @@ -373,31 +373,45 @@ */ List close(boolean abort, final RegionUnavailableListener listener) throws IOException { + Text regionName = this.regionInfo.getRegionName(); if (isClosed()) { - LOG.info("region " + this.regionInfo.getRegionName() + " already closed"); + LOG.info("region " + regionName + " already closed"); return null; } synchronized (splitLock) { + synchronized (writestate) { + while (writestate.compacting || writestate.flushing) { + LOG.debug("waiting for" + + (writestate.compacting ? " compaction" : "") + + (writestate.flushing ? + (writestate.compacting ? "," : "") + " cache flush" : + "" + ) + " to complete for region " + regionName + ); + try { + writestate.wait(); + } catch (InterruptedException iex) { + // continue + } + } + // Disable compacting and flushing by background threads for this + // region. + writestate.writesEnabled = false; + LOG.debug("compactions and cache flushes disabled for region " + + regionName); + } lock.writeLock().lock(); + LOG.debug("new updates and scanners for region " + regionName + + " disabled"); + try { - synchronized (writestate) { - while (writestate.compacting || writestate.flushing) { - try { - writestate.wait(); - } catch (InterruptedException iex) { - // continue - } - } - // Disable compacting and flushing by background threads for this - // region. - writestate.writesEnabled = false; - } - // Wait for active scanners to finish. The write lock we hold will prevent // new scanners from being created. synchronized (activeScannerCount) { while (activeScannerCount.get() != 0) { + LOG.debug("waiting for " + activeScannerCount.get() + + " scanners to finish"); try { activeScannerCount.wait(); @@ -406,12 +420,14 @@ } } } + LOG.debug("no more active scanners for region " + regionName); // Write lock means no more row locks can be given out. Wait on // outstanding row locks to come in before we close so we do not drop // outstanding updates. waitOnRowLocks(); - + LOG.debug("no more write locks outstanding on region " + regionName); + if (listener != null) { // If there is a listener, let them know that we have now // acquired all the necessary locks and are starting to @@ -551,7 +567,6 @@ if (closed.get() || !needsSplit(midKey)) { return null; } - long startTime = System.currentTimeMillis(); Path splits = new Path(this.regiondir, SPLITDIR); if(!this.fs.exists(splits)) { this.fs.mkdirs(splits); @@ -617,10 +632,6 @@ LOG.debug("Cleaned up " + splits.toString() + " " + deleted); } HRegion regions[] = new HRegion [] {regionA, regionB}; - LOG.info("Region split of " + this.regionInfo.getRegionName() + - " complete; " + "new regions: " + regions[0].getRegionName() + ", " + - regions[1].getRegionName() + ". Split took " + - StringUtils.formatTimeDiff(System.currentTimeMillis(), startTime)); return regions; } } @@ -770,39 +781,39 @@ * because a Snapshot was not properly persisted. */ boolean flushcache() throws IOException { - lock.readLock().lock(); // Prevent splits and closes - try { - if (this.closed.get()) { - return false; - } - synchronized (writestate) { - if ((!writestate.flushing) && writestate.writesEnabled) { - writestate.flushing = true; + if (this.closed.get()) { + return false; + } + synchronized (writestate) { + if ((!writestate.flushing) && writestate.writesEnabled) { + writestate.flushing = true; - } else { - if(LOG.isDebugEnabled()) { - LOG.debug("NOT flushing memcache for region " + - this.regionInfo.getRegionName() + ", flushing=" + - writestate.flushing + ", writesEnabled=" + - writestate.writesEnabled); - } - return false; + } else { + if(LOG.isDebugEnabled()) { + LOG.debug("NOT flushing memcache for region " + + this.regionInfo.getRegionName() + ", flushing=" + + writestate.flushing + ", writesEnabled=" + + writestate.writesEnabled); } + return false; } - long startTime = -1; - synchronized (updateLock) {// Stop updates while we snapshot the memcaches - startTime = snapshotMemcaches(); - } + } + try { + lock.readLock().lock(); // Prevent splits and closes try { + long startTime = -1; + synchronized (updateLock) {// Stop updates while we snapshot the memcaches + startTime = snapshotMemcaches(); + } return internalFlushcache(startTime); } finally { - synchronized (writestate) { - writestate.flushing = false; - writestate.notifyAll(); - } + lock.readLock().unlock(); } } finally { - lock.readLock().unlock(); + synchronized (writestate) { + writestate.flushing = false; + writestate.notifyAll(); + } } } @@ -1481,6 +1492,7 @@ private void waitOnRowLocks() { synchronized (rowsToLocks) { while (this.rowsToLocks.size() > 0) { + LOG.debug("waiting for " + this.rowsToLocks.size() + " row locks"); try { this.rowsToLocks.wait(); } catch (InterruptedException e) {