You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ns...@apache.org on 2011/10/11 04:10:22 UTC

svn commit: r1181459 - in /hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver: HRegion.java Store.java

Author: nspiegelberg
Date: Tue Oct 11 02:10:21 2011
New Revision: 1181459

URL: http://svn.apache.org/viewvc?rev=1181459&view=rev
Log:
Fixed miscellaneous bugs that would ignore a user-requested major compaction

Modified:
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=1181459&r1=1181458&r2=1181459&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java Tue Oct 11 02:10:21 2011
@@ -172,11 +172,6 @@ public class HRegion implements HeapSize
   final Path regiondir;
   KeyValue.KVComparator comparator;
 
-  /*
-   * Set this when scheduling compaction if want the next compaction to be a
-   * major compaction.  Cleared each time through compaction code.
-   */
-  private volatile boolean forceMajorCompaction = false;
   private Pair<Long,Long> lastCompactInfo = null;
 
   /*
@@ -781,11 +776,9 @@ public class HRegion implements HeapSize
   }
 
   void setForceMajorCompaction(final boolean b) {
-    this.forceMajorCompaction = b;
-  }
-
-  boolean getForceMajorCompaction() {
-    return this.forceMajorCompaction;
+    for (Store h: stores.values()) {
+      h.setForceMajorCompaction(b);
+    }
   }
 
   /**
@@ -799,13 +792,14 @@ public class HRegion implements HeapSize
    * conflicts with a region split, and that cannot happen because the region
    * server does them sequentially and not in parallel.
    *
-   * @return mid key if split is needed
+   * @param majorCompaction True to force a major compaction regardless of thresholds
+   * @return split row if split is needed
    * @throws IOException e
    */
-  public byte [] compactStores() throws IOException {
-    boolean majorCompaction = this.forceMajorCompaction;
-    this.forceMajorCompaction = false;
-    return compactStores(majorCompaction);
+  byte [] compactStores(final boolean majorCompaction)
+  throws IOException {
+    this.setForceMajorCompaction(majorCompaction);
+    return compactStores();
   }
 
   /*
@@ -823,7 +817,7 @@ public class HRegion implements HeapSize
    * @return split row if split is needed
    * @throws IOException e
    */
-  byte [] compactStores(final boolean majorCompaction)
+  byte [] compactStores()
   throws IOException {
     if (this.closing.get() || this.closed.get()) {
       LOG.debug("Skipping compaction on " + this + " because closing/closed");
@@ -847,8 +841,7 @@ public class HRegion implements HeapSize
               return splitRow;
           }
         }
-        LOG.info("Starting" + (majorCompaction? " major " : " ") +
-            "compaction on region " + this);
+        LOG.info("Starting compaction on region " + this);
         long startTime = EnvironmentEdgeManager.currentTimeMillis();
         doRegionCompactionPrep();
         long lastCompactSize = 0;
@@ -856,7 +849,7 @@ public class HRegion implements HeapSize
         boolean completed = false;
         try {
           for (Store store: stores.values()) {
-            final Store.StoreSize ss = store.compact(majorCompaction);
+            final Store.StoreSize ss = store.compact();
             lastCompactSize += store.getLastCompactSize();
             if (ss != null && ss.getSize() > maxSize) {
               maxSize = ss.getSize();
@@ -3144,7 +3137,7 @@ public class HRegion implements HeapSize
   }
 
   public static final long FIXED_OVERHEAD = ClassSize.align(
-      (4 * Bytes.SIZEOF_LONG) + Bytes.SIZEOF_BOOLEAN + ClassSize.ARRAY +
+      (4 * Bytes.SIZEOF_LONG) + ClassSize.ARRAY +
       (23 * ClassSize.REFERENCE) + ClassSize.OBJECT + Bytes.SIZEOF_INT);
 
   public static final long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD +

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java?rev=1181459&r1=1181458&r2=1181459&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java Tue Oct 11 02:10:21 2011
@@ -103,6 +103,7 @@ public class Store implements HeapSize {
   // With float, java will downcast your long to float for comparisons (bad)
   private double compactRatio;
   private long lastCompactSize = 0;
+  private volatile boolean forceMajor = false;
   /* how many bytes to write between status checks */
   static int closeCheckInterval = 0;
   private final long desiredMaxFileSize;
@@ -589,11 +590,10 @@ public class Store implements HeapSize {
    * <p>We don't want to hold the structureLock for the whole time, as a compact()
    * can be lengthy and we want to allow cache-flushes during this period.
    *
-   * @param forceMajor True to force a major compaction regardless of thresholds
    * @return row to split around if a split is needed, null otherwise
    * @throws IOException
    */
-  StoreSize compact(final boolean forceMajor) throws IOException {
+  StoreSize compact() throws IOException {
     boolean forceSplit = this.region.shouldSplit(false);
     synchronized (compactLock) {
       this.lastCompactSize = 0; // reset first in case compaction is aborted
@@ -609,12 +609,12 @@ public class Store implements HeapSize {
 
       // if the user wants to force a split, skip compaction unless necessary
       boolean references = hasReferences(this.storefiles);
-      if (forceSplit && !forceMajor && !references) {
+      if (forceSplit && !this.forceMajor && !references) {
         return checkSplit(forceSplit);
       }
 
       Collection<StoreFile> filesToCompact
-        = compactSelection(this.storefiles, forceMajor);
+        = compactSelection(this.storefiles, this.forceMajor);
 
       // empty == do not compact
       if (filesToCompact.isEmpty()) {
@@ -632,6 +632,9 @@ public class Store implements HeapSize {
       // major compaction iff all StoreFiles are included
       boolean majorcompaction
         = (filesToCompact.size() == this.storefiles.size());
+      if (majorcompaction) {
+        this.forceMajor = false;
+      }
 
       // Max-sequenceID is the last key in the files we're compacting
       long maxId = StoreFile.getMaxSequenceIdInList(filesToCompact);
@@ -1359,6 +1362,14 @@ public class Store implements HeapSize {
     return storeSize;
   }
 
+  void setForceMajorCompaction(final boolean b) {
+    this.forceMajor = b;
+  }
+
+  boolean getForceMajorCompaction() {
+    return this.forceMajor;
+  }
+
   //////////////////////////////////////////////////////////////////////////////
   // File administration
   //////////////////////////////////////////////////////////////////////////////
@@ -1540,7 +1551,7 @@ public class Store implements HeapSize {
 
   public static final long FIXED_OVERHEAD = ClassSize.align(
       ClassSize.OBJECT + (14 * ClassSize.REFERENCE) + (1 * Bytes.SIZEOF_DOUBLE) +
-      (7 * Bytes.SIZEOF_LONG) + (4 * Bytes.SIZEOF_INT) + (Bytes.SIZEOF_BOOLEAN * 2));
+      (7 * Bytes.SIZEOF_LONG) + (4 * Bytes.SIZEOF_INT) + (3 * Bytes.SIZEOF_BOOLEAN));
 
   public static final long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD +
       ClassSize.OBJECT + ClassSize.REENTRANT_LOCK +