HBASE-25637 Rename method completeCompaction to refreshStoreSizeAndTotalBytes (#3023)

Signed-off-by: Duo Zhang <zhangduo@apache.org>
This commit is contained in:
niuyulin 2021-03-06 04:41:07 -06:00 committed by GitHub
parent 830d2895b2
commit dd4a11e421
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
2 changed files with 9 additions and 13 deletions

View File

@ -296,7 +296,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
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));
@ -713,7 +713,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
region.getMVCC().advanceTo(this.getMaxSequenceId().getAsLong());
}
completeCompaction(toBeRemovedStoreFiles);
refreshStoreSizeAndTotalBytes();
}
protected HStoreFile createStoreFileAndReader(final Path p) throws IOException {
@ -1543,7 +1543,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
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
@ -1769,7 +1769,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
LOG.info("Replaying compaction marker, replacing input files: " +
inputStoreFiles + " with output files : " + outputStoreFiles);
this.replaceStoreFiles(inputStoreFiles, outputStoreFiles);
this.completeCompaction(inputStoreFiles);
this.refreshStoreSizeAndTotalBytes();
}
}
@ -1822,7 +1822,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
this.getCoprocessorHost().postCompact(this, sf, null, null, null);
}
replaceStoreFiles(filesToCompact, Collections.singletonList(sf));
completeCompaction(filesToCompact);
refreshStoreSizeAndTotalBytes();
}
} finally {
synchronized (filesCompacting) {
@ -2010,7 +2010,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
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));
@ -2052,10 +2052,8 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
/**
* 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);

View File

@ -21,7 +21,6 @@ import static org.junit.Assert.assertEquals;
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;
@ -39,7 +38,6 @@ import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
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;
@ -211,14 +209,14 @@ public class TestIOFencing {
}
@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();
}
}