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 21:12:53 UTC

svn commit: r1182030 - in /hbase/branches/0.89/src: main/java/org/apache/hadoop/hbase/regionserver/ main/java/org/apache/hadoop/hbase/regionserver/compactions/ test/java/org/apache/hadoop/hbase/regionserver/

Author: nspiegelberg
Date: Tue Oct 11 19:12:52 2011
New Revision: 1182030

URL: http://svn.apache.org/viewvc?rev=1182030&view=rev
Log:
Part 2 - Run more aggressive compactions during off peak hours

Summary: If there was a bulk imported file, a removeAll method was being called
on the abstract list. This diff changes CompactSelection to no longer implement
a list.

Test Plan: Running unit tests, made sure TestCompactSelection passes.

Reviewers: kannan, mbautin, nspiegelberg

Reviewed By: kannan

CC: hbase-eng@lists, kannan

Differential Revision: 334585

Revert Plan: OK

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
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactSelection.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionRequest.java
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSelection.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=1182030&r1=1182029&r2=1182030&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 19:12:52 2011
@@ -1065,7 +1065,7 @@ public class HRegion implements HeapSize
           }
         }
         LOG.info("Starting compaction on " + cr.getStore() + " in region "
-            + this + (cr.getFiles().isOffPeakCompaction()?" as an off-peak compaction":""));
+            + this + (cr.getCompactSelection().isOffPeakCompaction()?" as an off-peak compaction":""));
         doRegionCompactionPrep();
         boolean completed = false;
         try {

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=1182030&r1=1182029&r2=1182030&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 19:12:52 2011
@@ -880,21 +880,21 @@ public class Store implements HeapSize {
         CompactSelection filesToCompact = compactSelection(candidates);
 
         // no files to compact
-        if (filesToCompact.isEmpty()) {
+        if (filesToCompact.getFilesToCompact().isEmpty()) {
           return null;
         }
 
         // basic sanity check: do not try to compact the same StoreFile twice.
-        if (!Collections.disjoint(filesCompacting, filesToCompact)) {
+        if (!Collections.disjoint(filesCompacting, filesToCompact.getFilesToCompact())) {
           // TODO: change this from an IAE to LOG.error after sufficient testing
           Preconditions.checkArgument(false, "%s overlaps with %s",
               filesToCompact, filesCompacting);
         }
-        filesCompacting.addAll(filesToCompact);
+        filesCompacting.addAll(filesToCompact.getFilesToCompact());
         Collections.sort(filesCompacting, StoreFile.Comparators.FLUSH_TIME);
 
         // major compaction iff all StoreFiles are included
-        boolean isMajor = (filesToCompact.size() == this.storefiles.size());
+        boolean isMajor = (filesToCompact.getFilesToCompact().size() == this.storefiles.size());
         if (isMajor) {
           // since we're enqueuing a major, update the compaction wait interval
           this.forceMajor = false;
@@ -955,37 +955,41 @@ public class Store implements HeapSize {
      *    | |  | |  | |  | | | | | |
      *    | |  | |  | |  | | | | | |
      */
-    CompactSelection filesToCompact = new CompactSelection(conf, candidates);
+    CompactSelection compactSelection = new CompactSelection(conf, candidates);
 
     boolean forcemajor = this.forceMajor && filesCompacting.isEmpty();
     if (!forcemajor) {
       // do not compact old files above a configurable threshold
       // save all references. we MUST compact them
       int pos = 0;
-      while (pos < filesToCompact.size() &&
-             filesToCompact.get(pos).getReader().length() > maxCompactSize &&
-             !filesToCompact.get(pos).isReference()) ++pos;
-      filesToCompact.clearSubList(0, pos);
+      while (pos < compactSelection.getFilesToCompact().size() &&
+             compactSelection.getFilesToCompact().get(pos).getReader().length()
+               > maxCompactSize &&
+             !compactSelection.getFilesToCompact().get(pos).isReference()) ++pos;
+      compactSelection.clearSubList(0, pos);
     }
 
-    if (filesToCompact.isEmpty()) {
+    if (compactSelection.getFilesToCompact().isEmpty()) {
       LOG.debug(this.storeNameStr + ": no store files to compact");
-      filesToCompact.emptyFileList();
-      return filesToCompact;
+      compactSelection.emptyFileList();
+      return compactSelection;
     }
 
     // major compact on user action or age (caveat: we have too many files)
-    boolean majorcompaction = (forcemajor || isMajorCompaction(filesToCompact))
-      && filesToCompact.size() < this.maxFilesToCompact;
+    boolean majorcompaction =
+      (forcemajor || isMajorCompaction(compactSelection.getFilesToCompact()))
+      && compactSelection.getFilesToCompact().size() < this.maxFilesToCompact;
 
-    if (!majorcompaction && !hasReferences(filesToCompact)) {
+    if (!majorcompaction &&
+        !hasReferences(compactSelection.getFilesToCompact())) {
       // we're doing a minor compaction, let's see what files are applicable
       int start = 0;
-      double r = filesToCompact.getCompactSelectionRatio();
+      double r = compactSelection.getCompactSelectionRatio();
 
       // exclude bulk import files from minor compactions, if configured
       if (conf.getBoolean("hbase.hstore.compaction.exclude.bulk", false)) {
-        filesToCompact.removeAll(Collections2.filter(filesToCompact,
+        compactSelection.getFilesToCompact().removeAll(Collections2.filter(
+            compactSelection.getFilesToCompact(),
             new Predicate<StoreFile>() {
               @Override
               public boolean apply(StoreFile input) {
@@ -995,9 +999,9 @@ public class Store implements HeapSize {
       }
 
       // skip selection algorithm if we don't have enough files
-      if (filesToCompact.size() < this.minFilesToCompact) {
-        filesToCompact.emptyFileList();
-        return filesToCompact;
+      if (compactSelection.getFilesToCompact().size() < this.minFilesToCompact) {
+        compactSelection.emptyFileList();
+        return compactSelection;
       }
 
       /* TODO: add sorting + unit test back in when HBASE-2856 is fixed
@@ -1006,11 +1010,11 @@ public class Store implements HeapSize {
        */
 
       // get store file sizes for incremental compacting selection.
-      int countOfFiles = filesToCompact.size();
+      int countOfFiles = compactSelection.getFilesToCompact().size();
       long [] fileSizes = new long[countOfFiles];
       long [] sumSize = new long[countOfFiles];
       for (int i = countOfFiles-1; i >= 0; --i) {
-        StoreFile file = filesToCompact.get(i);
+        StoreFile file = compactSelection.getFilesToCompact().get(i);
         fileSizes[i] = file.getReader().length();
         // calculate the sum of fileSizes[i,i+maxFilesToCompact-1) for algo
         int tooFar = i + this.maxFilesToCompact - 1;
@@ -1040,27 +1044,28 @@ public class Store implements HeapSize {
       int end = Math.min(countOfFiles, start + this.maxFilesToCompact);
       long totalSize = fileSizes[start]
                      + ((start+1 < countOfFiles) ? sumSize[start+1] : 0);
-      filesToCompact = filesToCompact.getSubList(start, end);
+      compactSelection = compactSelection.getSubList(start, end);
 
       // if we don't have enough files to compact, just wait
-      if (filesToCompact.size() < this.minFilesToCompact) {
+      if (compactSelection.getFilesToCompact().size() < this.minFilesToCompact) {
         if (LOG.isDebugEnabled()) {
           LOG.debug("Skipped compaction of " + this.storeNameStr
             + ".  Only " + (end - start) + " file(s) of size "
             + StringUtils.humanReadableInt(totalSize)
             + " have met compaction criteria.");
         }
-        filesToCompact.emptyFileList();
-        return filesToCompact;
+        compactSelection.emptyFileList();
+        return compactSelection;
       }
     } else {
       // all files included in this compaction, up to max
-      if (filesToCompact.size() > this.maxFilesToCompact) {
-        int pastMax = filesToCompact.size() - this.maxFilesToCompact;
-        filesToCompact.clearSubList(0, pastMax);
+      if (compactSelection.getFilesToCompact().size() > this.maxFilesToCompact) {
+        int pastMax =
+          compactSelection.getFilesToCompact().size() - this.maxFilesToCompact;
+        compactSelection.clearSubList(0, pastMax);
       }
     }
-    return filesToCompact;
+    return compactSelection;
   }
 
   /**

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactSelection.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactSelection.java?rev=1182030&r1=1182029&r2=1182030&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactSelection.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactSelection.java Tue Oct 11 19:12:52 2011
@@ -30,7 +30,7 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
 
-public class CompactSelection extends AbstractList<StoreFile> {
+public class CompactSelection {
   private static final long serialVersionUID = 1L;
   static final Log LOG = LogFactory.getLog(CompactSelection.class);
   // the actual list - this is needed to handle methods like "sublist"
@@ -113,6 +113,10 @@ public class CompactSelection extends Ab
     }
   }
 
+  public List<StoreFile> getFilesToCompact() {
+    return filesToCompact;
+  }
+
   /**
    * Removes all files from the current compaction list, and resets off peak
    * compactions is set.
@@ -162,14 +166,4 @@ public class CompactSelection extends Ab
   private boolean isValidHour(int hour) {
     return (hour >= 0 && hour <= 23);
   }
-
-  @Override
-  public StoreFile get(int index) {
-    return filesToCompact.get(index);
-  }
-
-  @Override
-  public int size() {
-    return filesToCompact.size();
-  }
 }

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionRequest.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionRequest.java?rev=1182030&r1=1182029&r2=1182030&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionRequest.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionRequest.java Tue Oct 11 19:12:52 2011
@@ -49,7 +49,7 @@ public class CompactionRequest implement
     static final Log LOG = LogFactory.getLog(CompactionRequest.class);
     private final HRegion r;
     private final Store s;
-    private final CompactSelection files;
+    private final CompactSelection compactSelection;
     private final long totalSize;
     private final boolean isMajor;
     private int p;
@@ -63,9 +63,9 @@ public class CompactionRequest implement
 
       this.r = r;
       this.s = s;
-      this.files = files;
+      this.compactSelection = files;
       long sz = 0;
-      for (StoreFile sf : files) {
+      for (StoreFile sf : files.getFilesToCompact()) {
         sz += sf.getReader().length();
       }
       this.totalSize = sz;
@@ -75,7 +75,7 @@ public class CompactionRequest implement
     }
 
     public void finishRequest() {
-      this.files.finishRequest();
+      this.compactSelection.finishRequest();
     }
 
     /**
@@ -121,9 +121,14 @@ public class CompactionRequest implement
       return s;
     }
 
+    /** Gets the compact selection object for the request */
+    public CompactSelection getCompactSelection() {
+      return compactSelection;
+    }
+
     /** Gets the StoreFiles for the request */
-    public CompactSelection getFiles() {
-      return files;
+    public List<StoreFile> getFiles() {
+      return compactSelection.getFilesToCompact();
     }
 
     /** Gets the total size of all StoreFiles in compaction */
@@ -152,7 +157,7 @@ public class CompactionRequest implement
   @Override
     public String toString() {
       String fsList = Joiner.on(", ").join(
-        Collections2.transform(Collections2.filter(files,
+        Collections2.transform(Collections2.filter(compactSelection.getFilesToCompact(),
             new Predicate<StoreFile>() {
               public boolean apply(StoreFile sf) {
                 return sf.getReader() != null;
@@ -167,7 +172,7 @@ public class CompactionRequest implement
 
       return "regionName=" + r.getRegionNameAsString() +
         ", storeName=" + new String(s.getFamily().getName()) +
-        ", fileCount=" + files.size() +
+        ", fileCount=" + compactSelection.getFilesToCompact().size() +
         ", fileSize=" + StringUtils.humanReadableInt(totalSize) +
           ((fsList.isEmpty()) ? "" : " (" + fsList + ")") +
         ", priority=" + p + ", date=" + date;

Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSelection.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSelection.java?rev=1182030&r1=1182029&r2=1182030&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSelection.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSelection.java Tue Oct 11 19:12:52 2011
@@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.HColumnDe
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactSelection;
 import org.apache.hadoop.hbase.regionserver.wal.HLog;
 import org.apache.hadoop.hbase.util.Bytes;
 
@@ -159,7 +160,7 @@ public class TestCompactSelection extend
       long ... expected)
   throws IOException {
     store.forceMajor = forcemajor;
-    List<StoreFile> actual = store.compactSelection(candidates);
+    List<StoreFile> actual = store.compactSelection(candidates).getFilesToCompact();
     store.forceMajor = false;
     assertEquals(Arrays.toString(expected), Arrays.toString(getSizes(actual)));
   }
@@ -189,7 +190,7 @@ public class TestCompactSelection extend
      */
     // don't exceed max file compact threshold
     assertEquals(maxFiles,
-        store.compactSelection(sfCreate(7,6,5,4,3,2,1)).size());
+        store.compactSelection(sfCreate(7,6,5,4,3,2,1)).getFilesToCompact().size());
 
     /* MAJOR COMPACTION */
     // if a major compaction has been forced, then compact everything
@@ -201,7 +202,7 @@ public class TestCompactSelection extend
     // don't exceed max file compact threshold, even with major compaction
     store.forceMajor = true;
     assertEquals(maxFiles,
-        store.compactSelection(sfCreate(7,6,5,4,3,2,1)).size());
+        store.compactSelection(sfCreate(7,6,5,4,3,2,1)).getFilesToCompact().size());
     store.forceMajor = false;
     // if we exceed maxCompactSize, downgrade to minor
     // if not, it creates a 'snowball effect' when files >> maxCompactSize:
@@ -221,7 +222,7 @@ public class TestCompactSelection extend
     compactEquals(sfCreate(true, tooBig, 12,12), tooBig, 12, 12);
     // reference files should obey max file compact to avoid OOM
     assertEquals(maxFiles,
-        store.compactSelection(sfCreate(true, 7,6,5,4,3,2,1)).size());
+        store.compactSelection(sfCreate(true, 7,6,5,4,3,2,1)).getFilesToCompact().size());
 
     // empty case
     compactEquals(new ArrayList<StoreFile>() /* empty */);