You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2016/02/28 04:32:54 UTC

[2/2] hbase git commit: HBASE-15181 A simple implementation of date based tiered compaction (Clara Xiong)

HBASE-15181 A simple implementation of date based tiered compaction (Clara Xiong)


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/72169b4a
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/72169b4a
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/72169b4a

Branch: refs/heads/0.98
Commit: 72169b4a8a88c2375f668cfd681aec905c063ba3
Parents: bc370c9
Author: tedyu <yu...@gmail.com>
Authored: Sat Feb 27 19:32:47 2016 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Sat Feb 27 19:32:47 2016 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/StoreFile.java    |  11 +-
 .../compactions/CompactionConfiguration.java    |  76 +++++++-
 .../compactions/RatioBasedCompactionPolicy.java |  21 ++-
 .../hbase/regionserver/MockStoreFile.java       |  12 ++
 .../TestDefaultCompactSelection.java            | 177 +------------------
 .../regionserver/TestDefaultStoreEngine.java    |   1 +
 6 files changed, 113 insertions(+), 185 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/72169b4a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
index 6913efb..eb23570 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
@@ -682,10 +682,17 @@ public class StoreFile {
 
   public Long getMinimumTimestamp() {
     return (getReader().timeRangeTracker == null) ?
-        null :
-        getReader().timeRangeTracker.getMinimumTimestamp();
+      null :
+      getReader().timeRangeTracker.getMinimumTimestamp();
   }
 
+  public Long getMaximumTimestamp() {
+    return (getReader().timeRangeTracker == null) ?
+      null :
+      getReader().timeRangeTracker.getMaximumTimestamp();
+  }
+
+
   /**
    * Gets the approximate mid-point of this file that is optimal for use in splitting it.
    * @param comparator Comparator used to compare KVs.

http://git-wip-us.apache.org/repos/asf/hbase/blob/72169b4a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionConfiguration.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionConfiguration.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionConfiguration.java
index a08dc17..85516df 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionConfiguration.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionConfiguration.java
@@ -21,9 +21,9 @@ package org.apache.hadoop.hbase.regionserver.compactions;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
 
 /**
@@ -53,6 +53,21 @@ public class CompactionConfiguration {
   public static final String MIN_KEY = CONFIG_PREFIX + "min";
   public static final String MAX_KEY = CONFIG_PREFIX + "max";
 
+  /*
+   * The epoch time length for the windows we no longer compact
+   */
+  public static final String MAX_AGE_KEY =CONFIG_PREFIX + "date.tiered.max.storefile.age.millis";
+  public static final String BASE_WINDOW_MILLIS_KEY =
+    CONFIG_PREFIX + "date.tiered.base.window.millis";
+  public static final String WINDOWS_PER_TIER_KEY = CONFIG_PREFIX + "date.tiered.windows.per.tier";
+  public static final String INCOMING_WINDOW_MIN_KEY =
+    CONFIG_PREFIX + "date.tiered.incoming.window.min";
+  public static final String COMPACTION_POLICY_CLASS_FOR_TIERED_WINDOWS_KEY =
+      CONFIG_PREFIX + "date.tiered.window.policy.class";
+
+  private static final Class<? extends RatioBasedCompactionPolicy>
+    DEFAULT_TIER_COMPACTION_POLICY_CLASS = ExploringCompactionPolicy.class;
+
   Configuration conf;
   StoreConfigInformation storeConfigInfo;
 
@@ -66,7 +81,11 @@ public class CompactionConfiguration {
   long majorCompactionPeriod;
   float majorCompactionJitter;
   final float minLocalityToForceCompact;
-
+  private final long maxStoreFileAgeMillis;
+  private final long baseWindowMillis;
+  private final int windowsPerTier;
+  private final int incomingWindowMin;
+  private final String compactionPolicyForTieredWindow;
 
   CompactionConfiguration(Configuration conf, StoreConfigInformation storeConfigInfo) {
     this.conf = conf;
@@ -88,6 +107,12 @@ public class CompactionConfiguration {
     majorCompactionJitter = conf.getFloat("hbase.hregion.majorcompaction.jitter", 0.50F);
     minLocalityToForceCompact = conf.getFloat(HBASE_HSTORE_MIN_LOCALITY_TO_SKIP_MAJOR_COMPACT, 0f);
 
+    maxStoreFileAgeMillis = conf.getLong(MAX_AGE_KEY, Long.MAX_VALUE);
+    baseWindowMillis = conf.getLong(BASE_WINDOW_MILLIS_KEY, 3600000 * 6);
+    windowsPerTier = conf.getInt(WINDOWS_PER_TIER_KEY, 4);
+    incomingWindowMin = conf.getInt(INCOMING_WINDOW_MIN_KEY, 6);
+    compactionPolicyForTieredWindow = conf.get(COMPACTION_POLICY_CLASS_FOR_TIERED_WINDOWS_KEY,
+        DEFAULT_TIER_COMPACTION_POLICY_CLASS.getName());
     LOG.info(this);
   }
 
@@ -95,7 +120,9 @@ public class CompactionConfiguration {
   public String toString() {
     return String.format(
       "size [%d, %d); files [%d, %d); ratio %f; off-peak ratio %f; throttle point %d;"
-      + " major period %d, major jitter %f, min locality to compact %f\"",
+      + " major period %d, major jitter %f, min locality to compact %f;"
+      + " tiered compaction: max_age %d, base window in milliseconds %d, windows per tier %d, "
+      + "incoming window threshold %d",
       minCompactSize,
       maxCompactSize,
       minFilesToCompact,
@@ -105,8 +132,11 @@ public class CompactionConfiguration {
       throttlePoint,
       majorCompactionPeriod,
       majorCompactionJitter,
-      minLocalityToForceCompact
-    );
+      minLocalityToForceCompact,
+      maxStoreFileAgeMillis,
+      baseWindowMillis,
+      windowsPerTier,
+      incomingWindowMin);
   }
 
   /**
@@ -131,6 +161,14 @@ public class CompactionConfiguration {
   }
 
   /**
+   * Set upper bound on number of files to be included in minor compactions
+   * @param threshold
+   */
+  public void setMinFilesToCompact(int threshold) {
+    minFilesToCompact = threshold;
+  }
+
+  /**
    * @return upper bound on number of files to be included in minor compactions
    */
   int getMaxFilesToCompact() {
@@ -160,7 +198,7 @@ public class CompactionConfiguration {
 
   /**
    * @return Major compaction period from compaction.
-   * Major compactions are selected periodically according to this parameter plus jitter
+   *   Major compactions are selected periodically according to this parameter plus jitter
    */
   long getMajorCompactionPeriod() {
     return majorCompactionPeriod;
@@ -168,7 +206,7 @@ public class CompactionConfiguration {
 
   /**
    * @return Major the jitter fraction, the fraction within which the major compaction
-   *  period is randomly chosen from the majorCompactionPeriod in each store.
+   *    period is randomly chosen from the majorCompactionPeriod in each store.
    */
   float getMajorCompactionJitter() {
     return majorCompactionJitter;
@@ -176,10 +214,30 @@ public class CompactionConfiguration {
 
   /**
    * @return Block locality ratio, the ratio at which we will include old regions with a single
-   * store file for major compaction.  Used to improve block locality for regions that
-   * haven't had writes in a while but are still being read.
+   *   store file for major compaction.  Used to improve block locality for regions that
+   *   haven't had writes in a while but are still being read.
    */
   float getMinLocalityToForceCompact() {
     return minLocalityToForceCompact;
   }
+
+  public long getMaxStoreFileAgeMillis() {
+    return maxStoreFileAgeMillis;
+  }
+
+  public long getBaseWindowMillis() {
+    return baseWindowMillis;
+  }
+
+  public int getWindowsPerTier() {
+    return windowsPerTier;
+  }
+
+  public int getIncomingWindowMin() {
+    return incomingWindowMin;
+  }
+
+  public String getCompactionPolicyForTieredWindow() {
+    return compactionPolicyForTieredWindow;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/72169b4a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/RatioBasedCompactionPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/RatioBasedCompactionPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/RatioBasedCompactionPolicy.java
index f1c04d4..e632a79 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/RatioBasedCompactionPolicy.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/RatioBasedCompactionPolicy.java
@@ -34,8 +34,6 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreUtils;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-
 import com.google.common.base.Preconditions;
 import com.google.common.base.Predicate;
 import com.google.common.collect.Collections2;
@@ -74,7 +72,9 @@ public class RatioBasedCompactionPolicy extends CompactionPolicy {
   }
 
   /**
-   * @param candidateFiles candidate files, ordered from oldest to newest
+   * @param candidateFiles candidate files, ordered from oldest to newest by seqId. We rely on
+   *   DefaultStoreFileManager to sort the files by seqId to guarantee contiguous compaction based
+   *   on seqId for data consistency.
    * @return subset copy of candidate list that meets compaction criteria
    * @throws java.io.IOException
    */
@@ -127,7 +127,7 @@ public class RatioBasedCompactionPolicy extends CompactionPolicy {
    * exclude all files above maxCompactSize
    * Also save all references. We MUST compact them
    */
-  private ArrayList<StoreFile> skipLargeFiles(ArrayList<StoreFile> candidates) {
+  protected ArrayList<StoreFile> skipLargeFiles(ArrayList<StoreFile> candidates) {
     int pos = 0;
     while (pos < candidates.size() && !candidates.get(pos).isReference()
       && (candidates.get(pos).getReader().length() > comConf.getMaxCompactSize())) {
@@ -146,7 +146,7 @@ public class RatioBasedCompactionPolicy extends CompactionPolicy {
    * @return filtered subset
    * exclude all bulk load files if configured
    */
-  private ArrayList<StoreFile> filterBulk(ArrayList<StoreFile> candidates) {
+  protected ArrayList<StoreFile> filterBulk(ArrayList<StoreFile> candidates) {
     candidates.removeAll(Collections2.filter(candidates,
         new Predicate<StoreFile>() {
           @Override
@@ -182,7 +182,7 @@ public class RatioBasedCompactionPolicy extends CompactionPolicy {
    * @return filtered subset
    * forget the compactionSelection if we don't have enough files
    */
-  private ArrayList<StoreFile> checkMinFilesCriteria(ArrayList<StoreFile> candidates) {
+  protected ArrayList<StoreFile> checkMinFilesCriteria(ArrayList<StoreFile> candidates) {
     int minFiles = comConf.getMinFilesToCompact();
     if (candidates.size() < minFiles) {
       if(LOG.isDebugEnabled()) {
@@ -368,4 +368,13 @@ public class RatioBasedCompactionPolicy extends CompactionPolicy {
     int numCandidates = storeFiles.size() - filesCompacting.size();
     return numCandidates >= comConf.getMinFilesToCompact();
   }
+
+  /**
+   * Overwrite min threshold for compaction
+   * @param minThreshold
+   */
+  public void setMinThreshold(int minThreshold)
+  {
+    comConf.setMinFilesToCompact(minThreshold);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/72169b4a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java
index 3a12674..db25edb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java
@@ -95,6 +95,18 @@ public class MockStoreFile extends StoreFile {
     this.entryCount = entryCount;
   }
 
+  public Long getMinimumTimestamp() {
+    return (timeRangeTracker == null) ?
+      null :
+      timeRangeTracker.getMinimumTimestamp();
+  }
+
+  public Long getMaximumTimestamp() {
+    return (timeRangeTracker == null) ?
+      null :
+      timeRangeTracker.getMaximumTimestamp();
+  }
+
   @Override
   public StoreFile.Reader getReader() {
     final long len = this.length;

http://git-wip-us.apache.org/repos/asf/hbase/blob/72169b4a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java
index e813f41..11a63b4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java
@@ -19,182 +19,20 @@ package org.apache.hadoop.hbase.regionserver;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.List;
 
-import junit.framework.TestCase;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
-import org.apache.hadoop.hbase.regionserver.wal.HLog;
-import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import com.google.common.collect.Lists;
-
 @Category(SmallTests.class)
-public class TestDefaultCompactSelection extends TestCase {
-  private final static Log LOG = LogFactory.getLog(TestDefaultCompactSelection.class);
-  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-
-  protected Configuration conf;
-  protected HStore store;
-  private static final String DIR=
-    TEST_UTIL.getDataTestDir(TestDefaultCompactSelection.class.getSimpleName()).toString();
-  private static Path TEST_FILE;
-
-  protected static final int minFiles = 3;
-  protected static final int maxFiles = 5;
-
-  protected static final long minSize = 10;
-  protected static final long maxSize = 2100;
-
-  private HLog hlog;
-  private HRegion region;
-
-  @Override
-  public void setUp() throws Exception {
-    // setup config values necessary for store
-    this.conf = TEST_UTIL.getConfiguration();
-    this.conf.setLong(HConstants.MAJOR_COMPACTION_PERIOD, 0);
-    this.conf.setInt("hbase.hstore.compaction.min", minFiles);
-    this.conf.setInt("hbase.hstore.compaction.max", maxFiles);
-    this.conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, minSize);
-    this.conf.setLong("hbase.hstore.compaction.max.size", maxSize);
-    this.conf.setFloat("hbase.hstore.compaction.ratio", 1.0F);
-
-    //Setting up a Store
-    Path basedir = new Path(DIR);
-    String logName = "logs";
-    Path logdir = new Path(DIR, logName);
-    HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toBytes("family"));
-    FileSystem fs = FileSystem.get(conf);
-
-    fs.delete(logdir, true);
-
-    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(Bytes.toBytes("table")));
-    htd.addFamily(hcd);
-    HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
-
-    hlog = HLogFactory.createHLog(fs, basedir, logName, conf);
-    region = HRegion.createHRegion(info, basedir, conf, htd);
-    HRegion.closeHRegion(region);
-    Path tableDir = FSUtils.getTableDir(basedir, htd.getTableName());
-    region = new HRegion(tableDir, hlog, fs, conf, info, htd, null);
-
-    store = new HStore(region, hcd, conf);
-
-    TEST_FILE = region.getRegionFileSystem().createTempName();
-    fs.createNewFile(TEST_FILE);
-  }
-
-  @After
-  public void tearDown() throws IOException {
-    IOException ex = null;
-    try {
-      region.close();
-    } catch (IOException e) {
-      LOG.warn("Caught Exception", e);
-      ex = e;
-    }
-    try {
-      hlog.closeAndDelete();
-    } catch (IOException e) {
-      LOG.warn("Caught Exception", e);
-      ex = e;
-    }
-    if (ex != null) {
-      throw ex;
-    }
-  }
-
-  ArrayList<Long> toArrayList(long... numbers) {
-    ArrayList<Long> result = new ArrayList<Long>();
-    for (long i : numbers) {
-      result.add(i);
-    }
-    return result;
-  }
-
-  List<StoreFile> sfCreate(long... sizes) throws IOException {
-    ArrayList<Long> ageInDisk = new ArrayList<Long>();
-    for (int i = 0; i < sizes.length; i++) {
-      ageInDisk.add(0L);
-    }
-    return sfCreate(toArrayList(sizes), ageInDisk);
-  }
-
-  List<StoreFile> sfCreate(ArrayList<Long> sizes, ArrayList<Long> ageInDisk)
-    throws IOException {
-    return sfCreate(false, sizes, ageInDisk);
-  }
-
-  List<StoreFile> sfCreate(boolean isReference, long... sizes) throws IOException {
-    ArrayList<Long> ageInDisk = new ArrayList<Long>(sizes.length);
-    for (int i = 0; i < sizes.length; i++) {
-      ageInDisk.add(0L);
-    }
-    return sfCreate(isReference, toArrayList(sizes), ageInDisk);
-  }
-
-  List<StoreFile> sfCreate(boolean isReference, ArrayList<Long> sizes, ArrayList<Long> ageInDisk)
-      throws IOException {
-    List<StoreFile> ret = Lists.newArrayList();
-    for (int i = 0; i < sizes.size(); i++) {
-      ret.add(new MockStoreFile(TEST_UTIL, TEST_FILE,
-          sizes.get(i), ageInDisk.get(i), isReference, i));
-    }
-    return ret;
-  }
-
-  long[] getSizes(List<StoreFile> sfList) {
-    long[] aNums = new long[sfList.size()];
-    for (int i = 0; i < sfList.size(); ++i) {
-      aNums[i] = sfList.get(i).getReader().length();
-    }
-    return aNums;
-  }
-
-  void compactEquals(List<StoreFile> candidates, long... expected)
-    throws IOException {
-    compactEquals(candidates, false, false, expected);
-  }
-
-  void compactEquals(List<StoreFile> candidates, boolean forcemajor, long... expected)
-    throws IOException {
-    compactEquals(candidates, forcemajor, false, expected);
-  }
-
-  void compactEquals(List<StoreFile> candidates, boolean forcemajor, boolean isOffPeak,
-      long ... expected)
-  throws IOException {
-    store.forceMajor = forcemajor;
-    //Test Default compactions
-    CompactionRequest result = ((RatioBasedCompactionPolicy)store.storeEngine.getCompactionPolicy())
-        .selectCompaction(candidates, new ArrayList<StoreFile>(), false, isOffPeak, forcemajor);
-    List<StoreFile> actual = new ArrayList<StoreFile>(result.getFiles());
-    if (isOffPeak && !forcemajor) {
-      assertTrue(result.isOffPeak());
-    }
-    assertEquals(Arrays.toString(expected), Arrays.toString(getSizes(actual)));
-    store.forceMajor = false;
-  }
+public class TestDefaultCompactSelection extends TestCompactionPolicy {
 
+  @Test
   public void testCompactionRatio() throws IOException {
     /**
      * NOTE: these tests are specific to describe the implementation of the
@@ -272,9 +110,10 @@ public class TestDefaultCompactSelection extends TestCase {
     // empty case
     compactEquals(new ArrayList<StoreFile>() /* empty */);
     // empty case (because all files are too big)
-   compactEquals(sfCreate(tooBig, tooBig) /* empty */);
+    compactEquals(sfCreate(tooBig, tooBig) /* empty */);
   }
 
+  @Test
   public void testOffPeakCompactionRatio() throws IOException {
     /*
      * NOTE: these tests are specific to describe the implementation of the
@@ -289,6 +128,7 @@ public class TestDefaultCompactSelection extends TestCase {
     compactEquals(sfCreate(999, 50, 12, 12, 1), 12, 12, 1);
   }
 
+  @Test
   public void testStuckStoreCompaction() throws IOException {
     // Select the smallest compaction if the store is stuck.
     compactEquals(sfCreate(99,99,99,99,99,99, 30,30,30,30), 30, 30, 30);
@@ -303,6 +143,7 @@ public class TestDefaultCompactSelection extends TestCase {
     compactEquals(sfCreate(99,99,99,99, 27,27,27,20,20,20), 20, 20, 20);
   }
 
+  @Test
   public void testCompactionEmptyHFile() throws IOException {
     // Set TTL
     ScanInfo oldScanInfo = store.getScanInfo();
@@ -324,7 +165,7 @@ public class TestDefaultCompactSelection extends TestCase {
     CompactionRequest result = ((RatioBasedCompactionPolicy) store.storeEngine
         .getCompactionPolicy()).selectCompaction(candidates,
         new ArrayList<StoreFile>(), false, false, false);
-    assertTrue(result.getFiles().size() == 0);
+    Assert.assertTrue(result.getFiles().size() == 0);
     store.setScanInfo(oldScanInfo);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/72169b4a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java
index b76b20f..5ed494e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.regionserver;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.KeyValue.KVComparator;
 import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
 import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;