You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ni...@apache.org on 2021/03/06 10:49:16 UTC

[hbase] branch branch-2 updated: HBASE-25637 Rename method completeCompaction to refreshStoreSizeAndTotalBytes (#3023)

This is an automated email from the ASF dual-hosted git repository.

niuyulin pushed a commit to branch branch-2
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2 by this push:
     new 88b9e98  HBASE-25637 Rename method completeCompaction to refreshStoreSizeAndTotalBytes (#3023)
88b9e98 is described below

commit 88b9e98a7df3149743b8814a27ffa1cfc8581e33
Author: niuyulin <yu...@gmail.com>
AuthorDate: Sat Mar 6 04:41:07 2021 -0600

    HBASE-25637 Rename method completeCompaction to refreshStoreSizeAndTotalBytes (#3023)
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
---
 .../org/apache/hadoop/hbase/regionserver/HStore.java     | 16 +++++++---------
 .../test/java/org/apache/hadoop/hbase/TestIOFencing.java |  6 ++----
 2 files changed, 9 insertions(+), 13 deletions(-)

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 3e6a8a0..15f13bf 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
@@ -303,7 +303,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));
@@ -727,7 +727,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 {
@@ -1557,7 +1557,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
@@ -1783,7 +1783,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();
     }
   }
 
@@ -1836,7 +1836,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) {
@@ -2024,7 +2024,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));
@@ -2066,10 +2066,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);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
index 3880516..cfff0c4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
@@ -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;
@@ -38,7 +37,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;
@@ -210,14 +208,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();
     }
   }