From a6af6f6443134f0915692b9bd6a82a701ce61591 Mon Sep 17 00:00:00 2001 From: Andrew Purtell Date: Wed, 12 Nov 2014 12:56:42 -0800 Subject: [PATCH] HBASE-12454 Setting didPerformCompaction early in HRegion#compact --- .../apache/hadoop/hbase/regionserver/HRegion.java | 8 ++-- .../apache/hadoop/hbase/regionserver/HStore.java | 43 ++++++++++++---------- 2 files changed, 29 insertions(+), 22 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index fbf151a..e65dbe5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -1528,7 +1528,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // return false; } MonitoredTask status = null; - boolean didPerformCompaction = false; + boolean requestNeedsCancellation = true; // block waiting for the lock for compaction lock.readLock().lock(); try { @@ -1565,7 +1565,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // doRegionCompactionPrep(); try { status.setStatus("Compacting store " + store); - didPerformCompaction = true; + // We no longer need to cancel the request on the way out of this + // method because Store#compact will clean up unconditionally + requestNeedsCancellation = false; store.compact(compaction); } catch (InterruptedIOException iioe) { String msg = "compaction interrupted"; @@ -1587,7 +1589,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { // return true; } finally { try { - if (!didPerformCompaction) store.cancelRequestedCompaction(compaction); + if (requestNeedsCancellation) store.cancelRequestedCompaction(compaction); if (status != null) status.cleanup(); } finally { lock.readLock().unlock(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java index 8b41401..02c5b8f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java @@ -1132,25 +1132,29 @@ public class HStore implements Store { */ @Override public List compact(CompactionContext compaction) throws IOException { - assert compaction != null && compaction.hasSelection(); - CompactionRequest cr = compaction.getRequest(); - Collection filesToCompact = cr.getFiles(); - assert !filesToCompact.isEmpty(); - synchronized (filesCompacting) { - // sanity check: we're compacting files that this store knows about - // TODO: change this to LOG.error() after more debugging - Preconditions.checkArgument(filesCompacting.containsAll(filesToCompact)); - } - - // Ready to go. Have list of files to compact. - LOG.info("Starting compaction of " + filesToCompact.size() + " file(s) in " - + this + " of " + this.getRegionInfo().getRegionNameAsString() - + " into tmpdir=" + fs.getTempDir() + ", totalSize=" - + StringUtils.humanReadableInt(cr.getSize())); - - long compactionStartTime = EnvironmentEdgeManager.currentTime(); + assert compaction != null; List sfs = null; + CompactionRequest cr = compaction.getRequest();; try { + // Do all sanity checking in here if we have a valid CompactionRequest + // because we need to clean up after it on the way out in a finally + // block below + long compactionStartTime = EnvironmentEdgeManager.currentTime(); + assert compaction.hasSelection(); + Collection filesToCompact = cr.getFiles(); + assert !filesToCompact.isEmpty(); + synchronized (filesCompacting) { + // sanity check: we're compacting files that this store knows about + // TODO: change this to LOG.error() after more debugging + Preconditions.checkArgument(filesCompacting.containsAll(filesToCompact)); + } + + // Ready to go. Have list of files to compact. + LOG.info("Starting compaction of " + filesToCompact.size() + " file(s) in " + + this + " of " + this.getRegionInfo().getRegionNameAsString() + + " into tmpdir=" + fs.getTempDir() + ", totalSize=" + + StringUtils.humanReadableInt(cr.getSize())); + // Commence the compaction. List newFiles = compaction.compact(); @@ -1179,11 +1183,12 @@ public class HStore implements Store { } // At this point the store will use new files for all new scanners. completeCompaction(filesToCompact, true); // Archive old files & update store size. + + logCompactionEndMessage(cr, sfs, compactionStartTime); + return sfs; } finally { finishCompactionRequest(cr); } - logCompactionEndMessage(cr, sfs, compactionStartTime); - return sfs; } private List moveCompatedFilesIntoPlace( -- 1.7.12.4 (Apple Git-37)