Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -296,7 +296,7 @@ protected HStore(final HRegion region, final ColumnFamilyDescriptor family,
List<HStoreFile> hStoreFiles = loadStoreFiles(warmup);
// Move the storeSize calculation out of loadStoreFiles() method, because the secondary read
// replica's refreshStoreFiles() will also use loadStoreFiles() to refresh its store files and
// update the storeSize in the completeCompaction(..) finally (just like compaction) , so
// update the storeSize in the refreshStoreSizeAndTotalBytes() finally (just like compaction) , so
// no need calculate the storeSize twice.
this.storeSize.addAndGet(getStorefilesSize(hStoreFiles, sf -> true));
this.totalUncompressedBytes.addAndGet(getTotalUncompressedBytes(hStoreFiles));
Expand Down Expand Up @@ -713,7 +713,7 @@ private void refreshStoreFilesInternal(Collection<StoreFileInfo> newFiles) throw
region.getMVCC().advanceTo(this.getMaxSequenceId().getAsLong());
}

completeCompaction(toBeRemovedStoreFiles);
refreshStoreSizeAndTotalBytes();
}

protected HStoreFile createStoreFileAndReader(final Path p) throws IOException {
Expand Down Expand Up @@ -1543,7 +1543,7 @@ protected List<HStoreFile> doCompaction(CompactionRequestImpl cr,
long outputBytes = getTotalSize(sfs);

// At this point the store will use new files for all new scanners.
completeCompaction(filesToCompact); // update store size.
refreshStoreSizeAndTotalBytes(); // update store size.

long now = EnvironmentEdgeManager.currentTime();
if (region.getRegionServerServices() != null
Expand Down Expand Up @@ -1769,7 +1769,7 @@ public void replayCompactionMarker(CompactionDescriptor compaction, boolean pick
LOG.info("Replaying compaction marker, replacing input files: " +
inputStoreFiles + " with output files : " + outputStoreFiles);
this.replaceStoreFiles(inputStoreFiles, outputStoreFiles);
this.completeCompaction(inputStoreFiles);
this.refreshStoreSizeAndTotalBytes();
}
}

Expand Down Expand Up @@ -1822,7 +1822,7 @@ public void compactRecentForTestingAssumingDefaultPolicy(int N) throws IOExcepti
this.getCoprocessorHost().postCompact(this, sf, null, null, null);
}
replaceStoreFiles(filesToCompact, Collections.singletonList(sf));
completeCompaction(filesToCompact);
refreshStoreSizeAndTotalBytes();
}
} finally {
synchronized (filesCompacting) {
Expand Down Expand Up @@ -2010,7 +2010,7 @@ private void removeUnneededFiles() throws IOException {
Collection<HStoreFile> newFiles = Collections.emptyList(); // No new files.
writeCompactionWalRecord(delSfs, newFiles);
replaceStoreFiles(delSfs, newFiles);
completeCompaction(delSfs);
refreshStoreSizeAndTotalBytes();
LOG.info("Completed removal of " + delSfs.size() + " unnecessary (expired) file(s) in "
+ this + "; total size is "
+ TraditionalBinaryPrefix.long2String(storeSize.get(), "", 1));
Expand Down Expand Up @@ -2052,10 +2052,8 @@ private void validateStoreFile(Path path) throws IOException {

/**
* Update counts.
* @param compactedFiles list of files that were compacted
*/
protected void completeCompaction(Collection<HStoreFile> compactedFiles)
// Rename this method! TODO.
protected void refreshStoreSizeAndTotalBytes()
throws IOException {
this.storeSize.set(0L);
this.totalUncompressedBytes.set(0L);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@
import static org.junit.Assert.assertTrue;

import java.io.IOException;
import java.util.Collection;
import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicInteger;
Expand All @@ -39,7 +38,6 @@
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
Expand Down Expand Up @@ -211,14 +209,14 @@ protected BlockCompactionsInCompletionHStore(HRegion region, ColumnFamilyDescrip
}

@Override
protected void completeCompaction(Collection<HStoreFile> compactedFiles) throws IOException {
protected void refreshStoreSizeAndTotalBytes() throws IOException {
try {
r.compactionsWaiting.countDown();
r.compactionsBlocked.await();
} catch (InterruptedException ex) {
throw new IOException(ex);
}
super.completeCompaction(compactedFiles);
super.refreshStoreSizeAndTotalBytes();
}
}

Expand Down