diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java index 3e139a5..7b998a6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java @@ -220,6 +220,17 @@ public class StoreFile { } /** + * Clone + */ + public StoreFile(final StoreFile other) { + this.fs = other.fs; + this.fileInfo = other.fileInfo; + this.cacheConf = other.cacheConf; + this.cfBloomType = other.cfBloomType; + this.modificationTimeStamp = other.modificationTimeStamp; + } + + /** * @return the StoreFile object associated to this StoreFile. * null if the StoreFile is not a reference. */ diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java index aae3968..2940d93 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java @@ -50,7 +50,14 @@ public class DefaultCompactor extends Compactor { // Find the smallest read point across all the Scanners. long smallestReadPoint = getSmallestReadPoint(); - List scanners = createFileScanners(request.getFiles(), smallestReadPoint); + + // clone all StoreFiles, so we'll do the compaction on a independent copy of StoreFiles, + // HFileFiles, and readers + Collection cloned = new ArrayList(request.getFiles().size()); + for (StoreFile f : request.getFiles()) { + cloned.add(new StoreFile(f)); + } + List scanners = createFileScanners(cloned, smallestReadPoint); StoreFile.Writer writer = null; List newFiles = new ArrayList(); @@ -108,6 +115,9 @@ public class DefaultCompactor extends Compactor { newFiles.add(writer.getPath()); } } + for (StoreFile f : cloned) { + f.closeReader(true); + } } return newFiles; }