You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by li...@apache.org on 2012/12/21 20:19:28 UTC

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

Author: liyin
Date: Fri Dec 21 19:19:27 2012
New Revision: 1425101

URL: http://svn.apache.org/viewvc?rev=1425101&view=rev
Log:
[HBASE-7416] augments tier compaction with fixed boundary option

Author: liyintang

Summary: Currently, in tier compaction the aged-based algorithm considers HFile's age in disk relative to the current time, thus the tiers are actually shifting along the time. In order to best use our prior information about how applications consume the data, it needs another feature to perceive the tiers relative to a fixed time point. For more details, please refer to https://our.intern.facebook.com/intern/wiki/index.php/HBase/Boundary_Based_Tier_Compaction

Test Plan: Modify the TestTierCompaction.java to accommodate the new feature.

Reviewers: aaiyer, kannan, cjin

Reviewed By: kannan

CC: hbase-eng@

Differential Revision: https://phabricator.fb.com/D619488

Modified:
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/TierCompactionConfiguration.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/TierCompactionManager.java
    hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestTierCompactSelection.java

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/TierCompactionConfiguration.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/TierCompactionConfiguration.java?rev=1425101&r1=1425100&r2=1425101&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/TierCompactionConfiguration.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/TierCompactionConfiguration.java Fri Dec 21 19:19:27 2012
@@ -20,11 +20,14 @@
 
 package org.apache.hadoop.hbase.regionserver;
 
+import java.text.DecimalFormat;
+import java.util.Calendar;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.StringUtils;
 
-import java.text.DecimalFormat;
-
 /**
  * Control knobs for default compaction algorithm
  */
@@ -32,7 +35,7 @@ public class TierCompactionConfiguration
 
   private CompactionTier[] compactionTier;
   private boolean recentFirstOrder;
-
+  private boolean tierBoundaryFixed = false;
   TierCompactionConfiguration(Configuration conf, Store store) {
     super(conf, store);
 
@@ -101,7 +104,11 @@ public class TierCompactionConfiguration
               majorCompactionJitter
           )
       );
-
+    
+    // This allows us to compact files in hourly, daily boundary and etc
+    strAttribute = "IsTierBoundaryFixed";
+    tierBoundaryFixed = conf.getBoolean(strPrefix + strSchema + strAttribute,
+        conf.getBoolean(strPrefix + strDefault + strAttribute, false));
     for (int i = 0; i < compactionTier.length; i++) {
       compactionTier[i] = new CompactionTier(i);
     }
@@ -128,6 +135,12 @@ public class TierCompactionConfiguration
   }
 
   /**
+   * @return Whether the tiers will be compacted based on fixed boundary
+   */
+  boolean isTierBoundaryFixed() {
+    return tierBoundaryFixed;
+  }
+  /**
    * Parameters for each tier
    */
   class CompactionTier {
@@ -138,7 +151,8 @@ public class TierCompactionConfiguration
     private int tierMinFilesToCompact;
     private int tierMaxFilesToCompact;
     private int endingIndexForTier;
-
+    private CronTierBoundary cronTierBoundary;
+    
     CompactionTier(int tier) {
       String strPrefix = "hbase.hstore.compaction.";
       String strSchema = "tbl." + store.getHRegion().getTableDesc().getNameAsString()
@@ -203,7 +217,33 @@ public class TierCompactionConfiguration
         LOG.error("EndingIndexForTier improperly set. Using default value.");
         endingIndexForTier = tier;
       }
+      
+      strAttribute = "Boundary";
+        String cronExpression = conf.get(strPrefix + strSchema + strTier + strAttribute);
+        if (cronExpression == null) {
+          cronExpression = conf.get(strPrefix + strSchema + strDefTier + strAttribute);
+          if (cronExpression == null) {
+            cronExpression = conf.get(strPrefix + strDefault + strTier + strAttribute);
+            if (cronExpression == null) {
+              cronExpression = conf.get(strPrefix + strDefault + strDefTier + strAttribute);
+          }
+        }
+      }
+      if (cronExpression != null) {
+        LOG.info("Tier " + tier + " has cron expression " + cronExpression);
+        cronTierBoundary = new CronTierBoundary(cronExpression);
+      } else {
+        LOG.info("Tier " + tier + " has an empty cron expression");
+        cronTierBoundary = new CronTierBoundary();
+      }
+    }
 
+    /**
+     * 
+     * @return tierBoundary
+     */
+    Calendar getTierBoundary(Calendar prevTierBoundary) {
+      return cronTierBoundary.getTierBoundary(prevTierBoundary);
     }
 
     /**
@@ -269,4 +309,453 @@ public class TierCompactionConfiguration
 
   }
 
-}
\ No newline at end of file
+  /*
+   * * * * * * [num]                        Additional chars are allowed
+   * _ _ _ _ _ _ 
+   * | | | | | |___year (empty, 1970-2099)  empty, * or an explicit value
+   * | | | | |_____day_of_week (0-6)        , - * / L # 
+   * | | | |_______month (1-12)             , - * / ?
+   * | | |_________day_of_month (1-31)      , - * / ? L 
+   * | |___________hour_of_day (0-23)       , - * /
+   * |_____________min (0-59)               , - * /
+   *
+   * Examples: 
+   * "0 0 * * 5#3" :     3rd Friday of every month midnight
+   * "0 0 1 * *" :       first day of every month midnight
+   * "0 0 * * 5L" :      last Friday per month midnight
+   * "0 0 1-7 * *":      every first 7 days per month
+   * "0 0 * * 3,4":      every Wednesday, Thursday 
+   * "0 0 8 7 * 2012" :  July 8th, 2012, midnight
+   * Currently we only support only one "L" tagged value at day_of_month and
+   * month fields since we can always make separate expressions to specify last
+   * two or more days. 
+   */
+
+  public class CronTierBoundary {
+    private static final int NUMBER_OF_FIELDS = 5;
+    private static final int MINUTE = 0;
+    private static final int HOUR_OF_DAY = 1;
+    private static final int DAY_OF_MONTH = 2;
+    private static final int MONTH = 3;
+    private static final int DAY_OF_WEEK = 4;
+    private static final int YEAR = 5;
+    private int numFields = 0;
+    private final String[] fieldNames = { "MINUTE", "HOUR_OF_DAY",
+        "DAY_OF_MONTH", "MONTH", "DAY_OF_WEEK" };
+    private final int[] calIdx = { Calendar.MINUTE, Calendar.HOUR_OF_DAY,
+        Calendar.DAY_OF_MONTH, Calendar.MONTH, Calendar.DAY_OF_WEEK,
+        Calendar.YEAR };
+
+    private TimeField[] fields;
+
+    public CronTierBoundary() {
+    }
+
+    public CronTierBoundary(String inputString) {
+      if (inputString.equals("")) {
+        return;
+      }
+      String[] cronPhrases = inputString.split(" ");
+      if (cronPhrases.length < NUMBER_OF_FIELDS
+          || cronPhrases.length > (NUMBER_OF_FIELDS + 1)) {
+        throw new IllegalArgumentException("Wrong CronExpression "
+            + inputString);
+      } else {
+        numFields = cronPhrases.length;
+        fields = new TimeField[numFields];
+        for (int i = 0; i < numFields; i++) {
+          fields[i] = new TimeField(cronPhrases[i], i);
+        }
+      }
+    }
+
+    /**
+     * 
+     * the main API used to find the next legitimate boundary
+     * 
+     * @param from when we find the available boundary
+     * @return the next legitimate boundary
+     */
+    public Calendar getTierBoundary(Calendar currCal) {
+
+      if (numFields == 0) {
+        // default case: the boundary is 0, in this case 1970/1/1
+        // clone() is much more efficient than getInstance()
+        Calendar cal = (Calendar) currCal.clone();
+        cal.setTimeInMillis(0);
+        return cal;
+      }
+      return getNextAvail(currCal);
+    }
+
+    /**
+     * 
+     * the main function to find the next legitimate boundary
+     * @param from when we find the available boundary
+     * @return the next legitimate boundary
+     */
+    private Calendar getNextAvail(Calendar present) {
+      // clone() is much more efficient than getInstance()
+      Calendar nextCal = (Calendar) present.clone();
+      
+      int val, nextVal;
+      // This will guarantee the time go backward at least at minute level
+      boolean carry = true;
+      // minute and hour are simple cases, only need to consider borrowing
+      // from day or higher unit
+      for (int i = 0; i < DAY_OF_MONTH; i++) {
+        val = nextCal.get(calIdx[i]);
+        nextVal = fields[i].getNext(val, nextCal, carry);
+        nextCal.set(calIdx[i], nextVal);
+        carry = (nextVal == val) ? carry : (val < nextVal);
+      }
+
+      // check whether the year is a single value
+      // we don't deal with the multiple values for year although it could be
+      // implemented. For simplicity, we just consider the three cases:
+      // empty, or an explicit value
+      if (numFields > NUMBER_OF_FIELDS) {
+        if (fields[YEAR].getSize() != 1) {
+          throw new IllegalArgumentException("Cron Expression inValid: "
+              + " need either * or a particular year can be specified");
+        }
+        nextCal.set(Calendar.YEAR, fields[YEAR].values.first());
+      }
+
+      // special character "#" at DAY_OF_WEEK
+      if (fields[DAY_OF_WEEK].hashId > 0) {
+        int weekID = nextCal.get(Calendar.WEEK_OF_MONTH);
+        int weekday = nextCal.get(Calendar.DAY_OF_WEEK);
+        int weekDayDesired = fields[DAY_OF_WEEK].getNext(
+            nextCal.get(Calendar.DAY_OF_WEEK), nextCal, false);
+        // if in the current month the desired weekday is ahead
+        // please roll back in month
+        if (weekID < fields[DAY_OF_WEEK].hashId
+            || (weekID == fields[DAY_OF_WEEK].hashId && weekday < weekDayDesired)) {
+          carry = true;
+        }
+        // update month and year
+        updateMonthYear(fields, nextCal, carry);
+        // set the desired week and weekday
+        nextCal.set(Calendar.DAY_OF_WEEK, weekDayDesired);
+        val = nextCal.get(Calendar.MONTH);
+        nextCal.set(Calendar.WEEK_OF_MONTH, fields[DAY_OF_WEEK].hashId);
+        // when the first day_of_week in month greater than the desired one
+        // it falls back to the previous month, to compensate it, we add it
+        // back to the current month 
+        if (nextCal.get(Calendar.MONTH) < val) {
+          nextCal.add(Calendar.DAY_OF_MONTH, 7);
+        }
+      } else if (fields[DAY_OF_WEEK].isLast) {
+        // special character "L" at DAY_OF_WEEK
+        int dayOfMonth = nextCal.get(Calendar.DAY_OF_MONTH);
+        int daysInMonth = nextCal.getActualMaximum(Calendar.DAY_OF_MONTH);
+        nextCal.set(Calendar.DAY_OF_MONTH, daysInMonth);
+        int dayOfWeekLast = nextCal.get(Calendar.DAY_OF_WEEK);
+        int dayOfWeekDesired = fields[DAY_OF_WEEK].getLastValue();
+        // If the desired day_of_week is the future compared with day_of_week
+        // for the end of the current month, we need roll back in month
+        if ((dayOfWeekLast - dayOfWeekDesired + 7) < (daysInMonth - dayOfMonth)
+            || (carry && (dayOfWeekLast - dayOfWeekDesired + 7)
+                         == (daysInMonth - dayOfMonth))) {
+          updateMonthYear(fields, nextCal, true);
+          nextCal.set(Calendar.DAY_OF_MONTH,
+              nextCal.getActualMaximum(Calendar.DAY_OF_MONTH));
+          dayOfWeekLast = nextCal.get(Calendar.DAY_OF_WEEK);
+        }
+        // move to the desired day_of_week
+        nextCal.add(Calendar.DAY_OF_MONTH,
+            -((dayOfWeekLast - dayOfWeekDesired + 7) % 7));
+
+      } else if (fields[DAY_OF_MONTH].isOffset) {
+        // special character "?" with DAY_OF_MONTH
+        nextCal.add(calIdx[DAY_OF_MONTH], -fields[DAY_OF_MONTH].step);
+        updateMonthYear(fields, nextCal, false);
+
+      } else if (fields[DAY_OF_MONTH].isLast) {
+        // special character "L" with DAY_OF_MONTH
+        val = fields[DAY_OF_MONTH].getLastValue();
+        int days = nextCal.getActualMaximum(Calendar.DAY_OF_MONTH);
+        if (days - val + 1 > nextCal.get(DAY_OF_MONTH)) {
+          carry = true;
+        } 
+        updateMonthYear(fields, nextCal, carry);
+        days = nextCal.getActualMaximum(Calendar.DAY_OF_MONTH);
+        nextCal.set(Calendar.DAY_OF_MONTH, days - val + 1);
+      } else {
+        // This process all the rest cases, such as comma, *, hyphen
+        do {
+          nextVal = 0;
+          int oldNextVal = 0;
+          for (int i = DAY_OF_MONTH; i <= MONTH; i++) {
+            val = nextCal.get(calIdx[i]);
+            oldNextVal = nextVal;
+            nextVal = fields[i].getNext(val, nextCal, carry);
+            carry = (nextVal == val) ? carry : (nextVal > val);
+          }
+          nextCal.set(calIdx[MONTH], nextVal);
+          while (oldNextVal > nextCal.getActualMaximum(calIdx[DAY_OF_MONTH])) {
+            oldNextVal = fields[DAY_OF_MONTH]
+                .getNext(oldNextVal, nextCal, true);
+          }
+          nextCal.set(calIdx[DAY_OF_MONTH], oldNextVal);
+          // If day_of_week is satisfied, then we've found what we want
+          if (fields[DAY_OF_WEEK].values.contains(nextCal
+              .get(Calendar.DAY_OF_WEEK))) {
+            break;
+          }
+          carry = true;
+
+        } while (true);
+        // when we have roll over in month
+        if (carry && numFields == NUMBER_OF_FIELDS) {
+          nextCal.set(Calendar.YEAR, nextCal.get(Calendar.YEAR) - 1);
+          carry = false;
+        }
+      }
+      return nextCal;
+    }
+
+    // Helper function to update month/year
+    private void updateMonthYear(TimeField[] fields, Calendar nextCal,
+        boolean carry) {
+      int val = nextCal.get(Calendar.MONTH);
+      int nextVal = fields[MONTH].getNext(val, nextCal, carry);
+      nextCal.add(Calendar.MONTH, nextVal - val);
+    }
+  
+    // this is used for debugging 
+    public String displayCronConfigure() {
+      StringBuilder str = new StringBuilder();
+      for (int i = 0; i < numFields - 1; i++) {
+        str.append(fields[i].toString() + "\n");
+      }
+      str.append(fields[numFields - 1].toString());
+      return str.toString();
+    }
+
+    class TimeField {
+      private int field = 0;
+      private int step = 1;
+      private int start = 0;
+      private int end = 0;
+      private int hashId = -1;
+      private int lastValue;
+      SortedSet<Integer> values;
+      
+      private boolean isOffset = false;   // the indicator for ?/stepSize
+      private boolean isLast;             // the indicator whether it starts from the 
+                                          // end of the month or the end of the week 
+
+      /*
+       * convert the cron expression to java Calendar:
+       * 
+       * Java calendar defines the following: 
+       *   month: Jan-Dec (0 - 11)
+       *   day_of_week: Sun-Sat (1-7)
+       * 
+       * This is different from what is defined our cron expression, which has
+       *   month: Jan-Dec (1 - 12) 
+       *   day_of_week: Sun-Sat (0 - 6) 
+       * therefore, we are going to convert our value range to java 
+       * calendar accordingly
+       */
+      TimeField(String phrase, int idx) {
+        field = idx;
+        switch (idx) {
+        case MINUTE:
+          start = 0;
+          end = 59;
+          break;
+        case HOUR_OF_DAY:
+          start = 0;
+          end = 23;
+          break;
+        case DAY_OF_MONTH:
+          start = 1;
+          end = 31;
+          break;
+        case MONTH:
+          start = 1;
+          end = 12;
+          break;
+        case DAY_OF_WEEK:
+          start = 0;
+          end = 6;
+          break;
+        case YEAR:
+          start = 1969;
+          end = Integer.MAX_VALUE;
+          break;
+        default:
+          throw new IllegalArgumentException("Illegal Field Index number["
+              + (DAY_OF_WEEK + 1) + "]: " + idx);
+        }
+        values = new TreeSet<Integer>();
+        if (phrase != null) {
+          parseField(phrase);
+        }
+      }
+
+      // parse each field
+      public void parseField(String phrase) {
+        for (String word : phrase.split(",")) {
+          String[] cronStep = word.split("/");
+          if (cronStep.length > 2) {
+            throw new IllegalArgumentException("multiple division \"/\": "
+                + phrase);
+          } else if (cronStep.length == 2) {
+            try {
+              step = Integer.valueOf(cronStep[1]);
+            } catch (Exception e) {
+              throw new IllegalArgumentException("Illegal denominator: "
+                  + cronStep[1]);
+            }
+          } // end of step
+
+          /* range */
+          String startStr = null, endStr = null;
+          if (!cronStep[0].equals("*") && !cronStep[0].equals("?")) {
+            String[] cronRange = cronStep[0].split("-");
+            if (cronRange.length > 2) {
+              throw new IllegalArgumentException("multiple range \"-\": "
+                  + cronStep[0]);
+            } else if (cronRange.length == 2) {
+              startStr = cronRange[0];
+              endStr = cronRange[1];
+            } else {
+              try {
+                startStr = cronRange[0];
+                endStr = startStr;
+              } catch (Exception e) {
+                throw new IllegalArgumentException("Illegal range value");
+              }
+            }
+
+            if (field == DAY_OF_WEEK) {
+              String[] cronHash = cronStep[0].split("#");
+              if (cronHash.length > 2) {
+                throw new IllegalArgumentException("multiple hash \"#\": "
+                    + cronStep[0]);
+              } else if (cronHash.length == 2) {
+                hashId = Integer.valueOf(cronHash[1]);
+                startStr = cronHash[0];
+                endStr = startStr;
+              }
+            }
+
+            if ((field == DAY_OF_WEEK || field == DAY_OF_MONTH)
+                && startStr != null && startStr.endsWith("L")) {
+              isLast = true;
+              if (endStr != null && !endStr.endsWith("L")) {
+                throw new IllegalArgumentException("Illegal arguments: "
+                    + startStr + ", " + endStr);
+              }
+              start = Integer.valueOf(startStr.split("L")[0]);
+              end = Integer.valueOf(endStr.split("L")[0]);
+            } else {
+              if (startStr != null) {
+                start = Integer.valueOf(startStr);
+                end = Integer.valueOf(endStr);
+              }
+            }
+          } // end of range
+          if (cronStep[0].equals("?")) {
+            isOffset = true;
+          }
+          // In Java calendar, Jan-Dec map to 0-11
+          if (field == MONTH) {
+            start--;
+            end--;
+          }
+          // In java calendar, Sun-Sat map to 1-7
+          if (field == DAY_OF_WEEK) {
+            start++;
+            end++;
+          }
+          if (isLast) {
+            if (start != end) {
+              throw new IllegalArgumentException(
+                  "only one value can be specified here");
+            }
+            lastValue = start;
+          } else {
+            for (int i = start; i <= end; i += step) {
+              values.add(i);
+            }
+          }
+        } // end of phrase loop
+      }
+
+      /**
+       * 
+       * @return the value tagged by "L"
+       */
+      public int getLastValue() {
+        return lastValue;
+      }
+      /**
+       * 
+       * @return the size of valid values at a given field
+       */
+      public int getSize() {
+        return values != null ? values.size() : 1;
+      }
+
+      /**
+       * 
+       * To seek the next valid value before the present one
+       * 
+       * @param present the present value at field i
+       * @param cal the valid Calendar so far
+       * @param carry the carry bit
+       * @return the next valid value before the present one
+       */
+      public int getNext(int present, Calendar cal, boolean carry) {
+        SortedSet<Integer> fvalues = values;
+        // reconstruct the value's treeset in order to 
+        // accommodate the case: ?/step
+        if (field == MONTH && isOffset) {
+          fvalues = new TreeSet<Integer>();
+          int val = cal.get(Calendar.MONTH);
+          carry = true;
+          for (int i = 0; i < 12 / step; i++) {
+            fvalues.add((val - i * step + 12) % 12);
+          }
+        }
+        if (!carry && fvalues.contains(present)) {
+          return present;
+        } else {
+          // backward
+          if (fvalues.size() == 1) {
+            return fvalues.first();
+          }
+          if (fvalues.headSet(present).size() > 0)
+            return fvalues.headSet(present).last();
+          else
+            return fvalues.last();
+        }
+      }
+
+      public String toString() {
+        StringBuilder sb = new StringBuilder(fieldNames[field] + ":\t");
+        for (Integer val : values) {
+          if (values.last() == val) {
+            sb.append(val);
+          } else {
+            sb.append(val + ", ");
+          }
+        }
+        
+        if (isOffset) {
+          sb.append("from now go back :" + step);
+        }
+        
+        if (isLast && (field == DAY_OF_WEEK || field == DAY_OF_MONTH)) {
+          sb.append(lastValue + "L");
+        }
+        return sb.toString();
+      }
+    }
+  }
+}

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/TierCompactionManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/TierCompactionManager.java?rev=1425101&r1=1425100&r2=1425101&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/TierCompactionManager.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/TierCompactionManager.java Fri Dec 21 19:19:27 2012
@@ -20,13 +20,14 @@
 
 package org.apache.hadoop.hbase.regionserver;
 
+import java.io.IOException;
+import java.util.Calendar;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.util.StringUtils;
-
-import java.io.IOException;
 public class TierCompactionManager extends CompactionManager {
 
   private static final Log LOG = LogFactory.getLog(TierCompactionManager.class);
@@ -121,13 +122,19 @@ public class TierCompactionManager exten
 
     int i;
     int j = countOfFiles;
-
+    //
+    Calendar tierBoundary = tierConf.isTierBoundaryFixed() ? Calendar
+        .getInstance() : null;
     for (i = 0; i < numTiers; i++) {
       tier = tierConf.getCompactionTier(i);
+      // assign the current tier as the start point of the next tier
+      if (tierConf.isTierBoundaryFixed()) {
+        tierBoundary = tier.getTierBoundary(tierBoundary);
+      }
       endInTier[i] = j;
       while (j > 0) {
         file = candidates.getFilesToCompact().get(j - 1);
-        if (!isInTier(file, tier)) {
+        if (!isInTier(file, tier, tierBoundary)) {
           break;
         }
         j--;
@@ -188,11 +195,18 @@ public class TierCompactionManager exten
     return candidates;
   }
 
-  private boolean isInTier(StoreFile file, TierCompactionConfiguration.CompactionTier tier) {
-    return file.getReader().length() <= tier.getMaxSize() &&
-      EnvironmentEdgeManager.currentTimeMillis()-file.getMinFlushTime() <= tier.getMaxAgeInDisk();
+  // Either we use age-based tier compaction or boundary-based tier compaction
+  private boolean isInTier(StoreFile file, TierCompactionConfiguration.CompactionTier tier, 
+      Calendar tierBoundary) {
+    if (tierConf.isTierBoundaryFixed()) {
+      return file.getMinFlushTime() >= tierBoundary.getTimeInMillis();
+    } else {
+      return file.getReader().length() <= tier.getMaxSize()
+          && EnvironmentEdgeManager.currentTimeMillis()
+          - file.getMinFlushTime() <= tier.getMaxAgeInDisk();
+    }
   }
-
+  
   /**
    * This function iterates over the start values in order.
    * Whenever an admissible compaction is found, we return the selection.

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestTierCompactSelection.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestTierCompactSelection.java?rev=1425101&r1=1425100&r2=1425101&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestTierCompactSelection.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestTierCompactSelection.java Fri Dec 21 19:19:27 2012
@@ -19,20 +19,23 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Calendar;
+import java.util.List;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HConstants;
 
-import java.io.IOException;
-import java.util.*;
-
 public class TestTierCompactSelection extends TestDefaultCompactSelection {
   private final static Log LOG = LogFactory.getLog(TestTierCompactSelection.class);
 
-  private static final int numTiers = 4;
+  private static final int numTiers = 5;
 
   private String strPrefix, strSchema, strTier;
-
+  private Calendar currCal;
+  private Calendar[] expectedCals;
 
   @Override
   public void setUp() throws Exception {
@@ -84,11 +87,16 @@ public class TestTierCompactSelection ex
     conf.setFloat(strPrefix + strSchema + strTier + "CompactionRatio", 1.0F);
     // Also include files in tier 1 here
     conf.setInt(strPrefix + strSchema + strTier + "EndingIndexForTier", 1);
+    conf.setBoolean(strPrefix + strSchema + "IsTierBoundaryFixed", false);
 
     // Last tier - least aggressive compaction
     // has default tier settings only
     // Max Time elapsed is Infinity by default
-
+    currCal = Calendar.getInstance();
+    expectedCals = new Calendar[numTiers + 1];
+    for (int i = 0; i < numTiers + 1; i++) {
+      expectedCals[i] = Calendar.getInstance();
+    }
   }
 
   @Override
@@ -301,6 +309,521 @@ public class TestTierCompactSelection ex
 
   }
 
+  /**
+   * 
+   * @throws IOException
+   */
+  public void testTierCompactionBoundary() throws IOException {
+    conf.setBoolean(strPrefix + strSchema + "IsTierBoundaryFixed", true);
+    String strTierPrefix = strPrefix + strSchema;
+    conf.setInt(strPrefix + strSchema + "NumCompactionTiers", 3);
+
+    // -------------------------------------
+    // two Tiers
+    // -------------------------------------
+    // NOW: 2012 Nov 10, 12:11
+    // wild card
+    // -------------------------------------
+    this.conf.setStrings(strTierPrefix + "Tier.0.Boundary", "1 0 * * *");
+    boundaryEquals(new int[] {2012, 11, 10, 12, 11}, // pseudo now
+                   new int[] {2012, 11, 10,  0, 1});  // expected tier boundary
+
+    // -------------------------------------
+    // ?/2 at DAY_OF_MONTH
+    // -------------------------------------
+    conf.setStrings(strTierPrefix + "Tier.0.Boundary", "0 0 ?/2 * *");
+    boundaryEquals(new int[] {2012, 11, 10, 12, 11},
+                   new int[] {2012, 11, 8, 0, 0});
+    // -------------------------------------
+    // different step size at DAY_OF_MONTH
+    // -------------------------------------
+    conf.setStrings(strTierPrefix + "Tier.0.Boundary", "0 0 ?/4 * *");
+    boundaryEquals(new int[] {2012, 11, 10, 12, 11},
+                   new int[] {2012, 11,  6,  0,  0});
+
+    // -------------------------------------
+    // */4 at DAY_OF_MONTH
+    // [1, 5, 9... ]
+    // -------------------------------------
+    conf.setStrings(strTierPrefix + "Tier.0.Boundary", "0 0 */4 * *");
+    boundaryEquals(new int[] {2012, 11, 10, 12, 11},
+                   new int[] {2012, 11,  9,  0,  0});
+
+    conf.setStrings(strTierPrefix + "Tier.0.Boundary", "0 0 */4 * *");
+    boundaryEquals(new int[] { 2012, 11, 8, 12, 11 },
+                   new int[] {2012, 11, 5, 0, 0});
+
+    // -------------------------------------
+    // an explicit value at DAY_OF_MONTH
+    // -------------------------------------
+    conf.setStrings(strTierPrefix + "Tier.0.Boundary", "0 0 7 * *");
+    boundaryEquals(new int[] {2012, 11, 10, 12, 11},
+                   new int[] {2012, 11, 7, 0, 0});
+
+    conf.setStrings(strTierPrefix + "Tier.0.Boundary", "0 0 7 8 * 2011");
+    boundaryEquals(new int[] {2012, 11, 10, 12, 11}, 
+                   new int[] { 2011, 8, 7, 0, 0 });
+
+    // -------------------------------------
+    // comma at DAY_OF_MONTH
+    // -------------------------------------
+    conf.setStrings(strTierPrefix + "Tier.0.Boundary", "0 0 6,7 * *");
+    boundaryEquals(new int[] {2012, 11, 10, 12, 11},
+                   new int[] {2012, 11, 7, 0, 0});
+    
+    conf.setStrings(strTierPrefix + "Tier.0.Boundary", "0 0 6,7 * *");
+    boundaryEquals(new int[] {2012, 11, 7, 0, 1},
+                   new int[] {2012, 11, 7, 0, 0});
+
+    conf.setStrings(strTierPrefix + "Tier.0.Boundary", "0 0 6,7 * *");
+    boundaryEquals(new int[] { 2012, 11, 7, 0, 0 },
+                   new int[] {2012, 11, 6, 0, 0});
+
+    // -------------------------------------
+    // hyphen at DAY_OF_MONTH
+    // -------------------------------------
+    conf.setStrings(strTierPrefix + "Tier.0.Boundary", "0 0 6-7 * *");
+    boundaryEquals(new int[] {2012, 11, 10, 12, 11},
+                   new int[] {2012, 11, 7, 0, 0});
+
+    conf.setStrings(strTierPrefix + "Tier.0.Boundary", "0 0 6-7 * *");
+    boundaryEquals(new int[] { 2012, 11, 7, 0, 0 },
+                   new int[] {2012, 11, 6, 0, 0});
+
+    // -------------------------------------
+    // an explicit value at DAY_OF_MONTH
+    // -------------------------------------
+    conf.setStrings(strTierPrefix + "Tier.0.Boundary", "0 0 1L * *");
+    boundaryEquals(new int[] {2012, 11, 10, 12, 11},
+                   new int[] {2012, 10, 31, 0, 0});
+
+    // -------------------------------------
+    // an explicit value at DAY_OF_MONTH
+    // -------------------------------------
+    conf.setStrings(strTierPrefix + "Tier.0.Boundary", "0 0 1L ?/2 *");
+    boundaryEquals(new int[] {2012, 11, 10, 12, 11},
+                   new int[] {2012, 9, 30, 0, 0});
+
+    // -------------------------------------
+    // an explicit value at DAY_OF_MONTH
+    // [1, 6, 11]
+    // -------------------------------------
+    conf.setStrings(strTierPrefix + "Tier.0.Boundary", "0 0 1L */5 *");
+    boundaryEquals(new int[] {2012, 11, 10, 12, 11},
+                   new int[] {2012, 6, 30, 0, 0});
+
+    // -------------------------------------
+    // comma at DAY_OF_WEEK 
+    // -------------------------------------
+    this.conf.setStrings(strTierPrefix + "Tier.0.Boundary", "1 0 * * 1,3,5");
+    boundaryEquals(new int[] {2012, 11, 10, 12, 11},
+                   new int[] {2012, 11, 9, 0, 1});
+
+    this.conf.setStrings(strTierPrefix + "Tier.0.Boundary", "1 0 * * 1,3,5");
+    boundaryEquals(new int[] {2012, 11, 10, 12, 11},
+                   new int[] {2012, 11, 9, 0, 1});
+
+    this.conf.setStrings(strTierPrefix + "Tier.0.Boundary", "1 0 * * 1,3,5");
+    boundaryEquals(new int[] {2012, 11, 9, 0, 1},
+                   new int[] {2012, 11, 7, 0, 1});
+
+    // -------------------------------------
+    // hyphen at DAY_OF_WEEK 
+    // -------------------------------------
+    this.conf.setStrings(strTierPrefix + "Tier.0.Boundary", "1 0 * * 1-5");
+    boundaryEquals(new int[] {2012, 11, 10, 12, 11},
+                   new int[] {2012, 11, 9, 0, 1});
+
+    this.conf.setStrings(strTierPrefix + "Tier.0.Boundary", "1 0 * * 1-4");
+    boundaryEquals(new int[] {2012, 11, 10, 12, 11},
+                   new int[] {2012, 11, 8, 0, 1});
+
+    // -------------------------------------
+    // DAY_OF_WEEK with "#"
+    // -------------------------------------
+    conf.setStrings(strTierPrefix + "Tier.0.Boundary", "0 12 * * 5#1");
+    boundaryEquals(new int[] {2012, 11, 10, 12, 11},
+                   new int[] {2012, 11, 2, 12, 0});
+
+    // -------------------------------------
+    // week_of_day with "#" with roll over in month
+    // -------------------------------------
+    conf.setStrings(strTierPrefix + "Tier.0.Boundary", "0 12 * * 5#1");
+    boundaryEquals(new int[] {2012, 11, 2, 12, 0},
+                   new int[] {2012, 10, 5, 12, 0});
+
+    // -------------------------------------
+    // week_of_day with "#" with explicit month
+    // -------------------------------------
+    conf.setStrings(strTierPrefix + "Tier.0.Boundary", "0 12 * 8 5#1");
+    boundaryEquals(new int[] {2012, 11, 10, 12, 11},
+                   new int[] {2012, 8, 3, 12, 0});
+    
+    // -------------------------------------
+    // week_of_day with "#" with ?/2 month
+    // -------------------------------------
+    conf.setStrings(strTierPrefix + "Tier.0.Boundary", "0 12 * ?/2 5#1");
+    boundaryEquals(new int[] {2012, 11, 10, 12, 11},
+                   new int[] {2012, 9, 7, 12, 0});
+    
+    // -------------------------------------
+    // week_of_day with "#" with */ month
+    // -------------------------------------
+    conf.setStrings(strTierPrefix + "Tier.0.Boundary", "0 12 * */2 5#1");
+    boundaryEquals(new int[] {2012, 11, 10, 12, 11},
+                   new int[] {2012, 11, 2, 12, 0});
+    
+    // -------------------------------------
+    // week_of_day with "L"
+    // -------------------------------------
+    conf.setStrings(strTierPrefix + "Tier.0.Boundary", "0 12 * * 5L");
+    boundaryEquals(new int[] {2012, 11, 10, 12, 11},
+                   new int[] {2012, 10, 26, 12, 0});
+    
+    // -------------------------------------
+    // week_of_day with "L" and roll over in month
+    // -------------------------------------
+    conf.setStrings(strTierPrefix + "Tier.0.Boundary", "0 12 * * 5L");
+    boundaryEquals(new int[] { 2012, 10, 22, 12, 11 },
+                   new int[] {2012, 9, 28, 12, 0});
+    
+    // -------------------------------------
+    // week_of_day with "L" in a given month
+    // -------------------------------------
+    conf.setStrings(strTierPrefix + "Tier.0.Boundary", "0 12 * 8 5L");
+    boundaryEquals(new int[] { 2012, 10, 22, 12, 11 },
+                   new int[] {2012, 8, 31, 12, 0});
+    
+    conf.setStrings(strTierPrefix + "Tier.0.Boundary", "0 12 * 8 5L 2011");
+    boundaryEquals(new int[] { 2012, 10, 22, 12, 11 },
+                   new int[] {2011, 8, 26, 12, 0});
+    
+    // -------------------------------------
+    // week_of_day with "L" for dynamic month
+    // [1, 3, 5, ...]
+    // -------------------------------------
+    conf.setStrings(strTierPrefix + "Tier.0.Boundary", "0 12 * */2 5L");
+    boundaryEquals(new int[] { 2012, 10, 22, 12, 11 },
+                   new int[] {2012, 9, 28, 12, 0});
+
+    // -------------------------------------
+    // week_of_day with "L" with ?
+    // -------------------------------------
+    conf.setStrings(strTierPrefix + "Tier.0.Boundary", "0 12 * ?/2 5L");
+    boundaryEquals(new int[] { 2012, 10, 22, 12, 11 },
+                   new int[] {2012, 8, 31, 12, 0});
+
+    // -------------------------------------
+    // week_of_day with "L" with ? and different stepsize
+    // -------------------------------------
+    conf.setStrings(strTierPrefix + "Tier.0.Boundary", "0 12 * ?/5 5L");
+    boundaryEquals(new int[] { 2012, 10, 22, 12, 11 },
+                   new int[] {2012, 5, 25, 12, 0});
+
+    // -------------------------------------
+    // week_of_day with "L" with ? and different stepsize
+    // [1, 6, 11]
+    // -------------------------------------
+    conf.setStrings(strTierPrefix + "Tier.0.Boundary", "0 12 * */5 5L");
+    boundaryEquals(new int[] { 2012, 10, 22, 12, 11 },
+                   new int[] {2012, 6, 29, 12, 0});
+
+    // -------------------------------------
+    // Three Tiers
+    // -------------------------------------
+    // test wild card
+    // -------------------------------------
+    conf.setStrings(strTierPrefix + "Tier.0.Boundary", "0 0 * * *");
+    conf.setStrings(strTierPrefix + "Tier.1.Boundary", "0 0 * * *");
+    boundaryEquals(new int[] {2012, 11, 10, 12, 11},
+                   new int[] {2012, 11, 10, 0, 0},
+                   new int[] {2012, 11, 9, 0, 0});
+
+    // -------------------------------------
+    // test combination of wild card & ?/2
+    // -------------------------------------
+    conf.setStrings(strTierPrefix + "Tier.0.Boundary", "0 0 * * *");
+    conf.setStrings(strTierPrefix + "Tier.1.Boundary", "0 0 ?/2 * *");
+    boundaryEquals(new int[] {2012, 11, 10, 12, 11},
+                   new int[] {2012, 11, 10, 0, 0}, 
+                   new int[] {2012, 11, 8, 0, 0});
+
+    // -------------------------------------
+    // test stepsize
+    // -------------------------------------
+    conf.setStrings(strTierPrefix + "Tier.0.Boundary", "0 0 * * *");
+    conf.setStrings(strTierPrefix + "Tier.1.Boundary", "0 0 ?/3 * *");
+    boundaryEquals(new int[] {2012, 11, 10, 12, 11},
+                   new int[] {2012, 11, 10, 0, 0}, 
+                   new int[] {2012, 11, 7, 0, 0});
+
+    // -------------------------------------
+    // test combination of wildcard and "L"
+    // -------------------------------------
+    conf.setStrings(strTierPrefix + "Tier.0.Boundary", "0 0 * * *");
+    conf.setStrings(strTierPrefix + "Tier.1.Boundary", "59 23 1L * *");
+    boundaryEquals(new int[] {2012, 11, 10, 12, 11},
+                   new int[] {2012, 11, 10, 0, 0}, 
+                   new int[] {2012, 10, 31, 23, 59});
+
+    // -------------------------------------
+    // test with a given year/month/day
+    // -------------------------------------
+    conf.setStrings(strTierPrefix + "Tier.0.Boundary", "0 4 1 1 * 2012");
+    conf.setStrings(strTierPrefix + "Tier.1.Boundary", "0 4 1 1 * 2011");
+    boundaryEquals(new int[] {2012, 11, 10, 12, 11},
+                   new int[] {2012, 1, 1, 4, 0}, 
+                   new int[] {2011, 1, 1, 4, 0});
+
+    // -------------------------------------
+    // five Tiers
+    // -------------------------------------
+    // T0 :0 0 * * *
+    // T1: 0 0 * * *
+    // T2: 0 0 * * 0
+    // T3: 0 0 1 * *
+    // 
+    // NOW: 2012 Nov 10, 12:11
+    // T0 : Nov 10 -- today's midnight
+    // T1 : Nov 09 -- yesterday's midnight
+    // T2 : Nov 04 -- last sunday's midnight
+    // T3 : Nov 01 -- 1st day of the month's midnight
+    // -------------------------------------
+    conf.setStrings(strTierPrefix + "Tier.0.Boundary", "0 0 * * *");
+    conf.setStrings(strTierPrefix + "Tier.1.Boundary", "0 0 * * *");
+    conf.setStrings(strTierPrefix + "Tier.2.Boundary", "0 0 * * 0");
+    conf.setStrings(strTierPrefix + "Tier.3.Boundary", "0 0 1 * *");
+    boundaryEquals(new int[] {2012, 11, 10, 12, 11},
+                   new int[] {2012, 11, 10, 0, 0}, 
+                   new int[] {2012, 11, 9, 0, 0},
+                   new int[] {2012, 11, 4, 0, 0},
+                   new int[] {2012, 11, 1, 0, 0});
+
+  }
+
+  /**
+   * 
+   * @throws IOException
+   */
+  public void testCompactionBoundarDefault() throws IOException {
+    conf.setBoolean(strPrefix + strSchema + "IsTierBoundaryFixed", true);
+    Calendar[] expectedCals = new Calendar[numTiers];
+    for (int i = 0; i < numTiers; i++) {
+      expectedCals[i] = Calendar.getInstance();
+    }
+    // every minute
+    testCronExpression("* * * * *", 
+        new int[] {2012, 11, 10, 12, 11}, // the pseudo now
+        new int[] {2012, 11, 10, 12, 10}, 
+        new int[] {2012, 11, 10, 12, 9},
+        new int[] {2012, 11, 10, 12, 8});
+                  
+    // every minute cross hour
+    testCronExpression("* * * * *", 
+        new int[] {2012, 11, 10, 12, 2},
+        new int[] {2012, 11, 10, 12, 1}, 
+        new int[] {2012, 11, 10, 12, 0},
+        new int[] {2012, 11, 10, 11, 59});
+    
+    // every minute cross day
+    testCronExpression("* * * * *", 
+        new int[] {2012, 11, 10, 0, 2},
+        new int[] {2012, 11, 10, 0, 1}, 
+        new int[] {2012, 11, 10, 0, 0},
+        new int[] {2012, 11, 9, 23, 59});
+
+    // every minute cross month
+    testCronExpression("* * * * *", 
+        new int[] {2012, 11, 1, 0, 2},
+        new int[] {2012, 11, 1, 0, 1}, 
+        new int[] {2012, 11, 1, 0, 0},
+        new int[] {2012, 10, 31, 23, 59});
+
+    // every minute cross year
+    testCronExpression("* * * * *", 
+        new int[] {2012, 1, 1, 0, 2},
+        new int[] {2012, 1, 1, 0, 1}, 
+        new int[] {2012, 1, 1, 0, 0},
+        new int[] {2011, 12, 31, 23, 59});
+
+    // every hour 1/2 hr boundary
+    testCronExpression("30 * * * *", 
+        new int[] {2012, 11, 10, 5, 30},
+        new int[] {2012, 11, 10, 4, 30}, 
+        new int[] {2012, 11, 10, 3, 30},
+        new int[] {2012, 11, 10, 2, 30});
+
+    // every hour cross day
+    testCronExpression("1 * * * *", 
+        new int[] {2012, 11, 0, 1, 11},
+        new int[] {2012, 11, 0, 1, 1}, 
+        new int[] {2012, 11, 0, 0, 1},
+        new int[] {2012, 10, 30, 23, 1});
+
+    // every day
+    testCronExpression("1 2 * * *", 
+        new int[] {2012, 11, 10, 12, 11},
+        new int[] {2012, 11, 10, 2, 1}, 
+        new int[] {2012, 11, 9, 2, 1},
+        new int[] {2012, 11, 8, 2, 1});
+
+    // every day at 4am
+    testCronExpression("0 4 * * *", 
+        new int[] {2012, 11, 10, 3, 11},
+        new int[] {2012, 11, 9, 4, 0}, 
+        new int[] {2012, 11, 8, 4, 0},
+        new int[] {2012, 11, 7, 4, 0});
+
+    // every day cross month
+    testCronExpression("1 2 * * *", 
+        new int[] {2012, 1, 1, 3, 11},
+        new int[] {2012, 1, 1, 2, 1}, 
+        new int[] {2011, 12, 31, 2, 1},
+        new int[] {2011, 12, 30, 2, 1});
+
+    // every month
+    testCronExpression("1 2 3 * *", 
+        new int[] {2012, 11, 10, 1, 11},
+        new int[] {2012, 11, 3, 2, 1}, 
+        new int[] {2012, 10, 3, 2, 1},
+        new int[] {2012, 9, 3, 2, 1});
+
+    // every month cross year
+    testCronExpression("1 2 3 * *", 
+        new int[] {2012, 1, 2, 1, 11},
+        new int[] {2011, 12, 3, 2, 1}, 
+        new int[] {2011, 11, 3, 2, 1},
+        new int[] {2011, 10, 3, 2, 1});
+    
+    // every march cross year
+    testCronExpression("1 2 3 3 *", 
+        new int[] {2012, 1, 2, 1, 11},
+        new int[] {2011, 3, 3, 2, 1}, 
+        new int[] {2010, 3, 3, 2, 1},
+        new int[] {2009, 3, 3, 2, 1});
+    
+    // every Thursday per month cross year
+    testCronExpression("1 2 * * 4", 
+        new int[] {2012, 11, 8, 2, 11},
+        new int[] {2012, 11, 8, 2, 1}, 
+        new int[] {2012, 11, 1, 2, 1},
+        new int[] {2012, 10, 25, 2, 1});
+    
+    // last day per month 
+    testCronExpression("1 2 1L * *",
+        new int[] {2012, 11, 8, 2, 11},
+        new int[] {2012, 10, 31, 2, 1}, 
+        new int[] {2012, 9, 30, 2, 1},
+        new int[] {2012, 8, 31, 2, 1});
+
+    // last day bi-monthly
+    testCronExpression("1 2 1L ?/2 *", 
+        new int[] {2012, 11, 8, 2, 11},
+        new int[] {2012, 9, 30, 2, 1}, 
+        new int[] {2012, 7, 31, 2, 1},
+        new int[] {2012, 5, 31, 2, 1});
+
+    // last Friday bi-monthly
+    testCronExpression("1 2 * ?/2 5L",
+        new int[] {2012, 11, 8, 2, 11},
+        new int[] {2012, 9, 28, 2, 1}, 
+        new int[] {2012, 7, 27, 2, 1},
+        new int[] {2012, 5, 25, 2, 1});
+    
+    // last Friday per month
+    testCronExpression("1 2 * * 5L",
+        new int[] {2012, 11, 8, 2, 11},
+        new int[] {2012, 10, 26, 2, 1}, 
+        new int[] {2012, 9, 28, 2, 1},
+        new int[] {2012, 8, 31, 2, 1});
+
+    // last day of the first month per quarter
+    // [1, 4, 7, 10]
+    testCronExpression("1 2 1L */3 *",
+       new int[] {2012, 11, 8, 2, 11},
+       new int[] {2012, 10, 31, 2, 1},
+       new int[] {2012, 7, 31, 2, 1},
+       new int[] {2012, 4, 30, 2, 1});
+
+    // the very first day of every quarter
+    testCronExpression("0 0 1 */3 *",
+        new int[] {2012, 11, 10, 2, 11},
+        new int[] {2012, 10, 1, 0, 0}, 
+        new int[] {2012, 7, 1, 0, 0},
+        new int[] {2012, 4, 1, 0, 0});
+
+    // the very first day of every quarter
+    // NOW: 11/8/2012, 2:11
+    // the first valid month by ?/3 means three months from the current month
+    // [2, 5, 8, 11] 
+    testCronExpression("0 0 1 ?/3 *", 
+        new int[] {2012, 11, 8, 2, 11},
+        new int[] {2012, 8, 1, 0, 0}, 
+        new int[] {2012, 5, 1, 0, 0},
+        new int[] {2012, 2, 1, 0, 0});
+
+    // every mon/wed/fri cross month boundary
+    testCronExpression("1 0 * * 1,3,5", 
+        new int[] {2012, 12, 4, 2, 11},
+        new int[] {2012, 12, 3, 0, 1}, 
+        new int[] {2012, 11, 30, 0, 1},
+        new int[] {2012, 11, 28, 0, 1});
+
+    // same effect as the previous cron expression
+    testCronExpression("1 0 * * 1-5/2", 
+        new int[] {2012, 12, 4, 2, 11},
+        new int[] {2012, 12, 3, 0, 1}, 
+        new int[] {2012, 11, 30, 0, 1},
+        new int[] {2012, 11, 28, 0, 1});
+
+  }
+
+  private void testCronExpression(String cronExpression, int[] currVal,
+      int[]... expectedVals) throws IOException {
+    for (int i = 0; i < expectedVals.length; i++) {
+      String strTier = "Tier." + String.valueOf(i) + ".";
+      this.conf.setStrings(strPrefix + strSchema + strTier + "Boundary",
+          cronExpression);
+    }
+    String strTier = "Tier." + String.valueOf(expectedVals.length) + ".";
+    this.conf.setStrings(strPrefix + strSchema + strTier + "Boundary", "");
+    boundaryEquals(currVal, expectedVals);
+  }
+
+  /**
+   * 
+   * @param currVal: the pseudo now from when the tier boundaries 
+   *                 start to be calculated
+   * @param expectedVals: expected tier boundaries
+   */
+  private void boundaryEquals(int[] currVal, int[]... expectedVals) {
+    int localNumTiers = expectedVals.length + 1;
+    conf.setInt(strPrefix + strSchema + "NumCompactionTiers", localNumTiers);
+    currCal.set(currVal[0], currVal[1] - 1, currVal[2], currVal[3], currVal[4]);
+    String strTier = "Tier." + String.valueOf(localNumTiers - 1) + ".";
+    conf.set(strPrefix + strSchema + strTier + "Boundary", "");
+    for (int i = 0; i < localNumTiers - 1; i++) {
+      expectedCals[i].set(expectedVals[i][0], expectedVals[i][1] - 1,
+          expectedVals[i][2], expectedVals[i][3], expectedVals[i][4]);
+    }
+    expectedCals[localNumTiers - 1].setTimeInMillis(0);
+
+    TierCompactionConfiguration tierConf = new TierCompactionConfiguration(
+        conf, store);
+    Calendar localCal = (Calendar) currCal.clone();
+    TierCompactionConfiguration.CompactionTier tier;
+    if (tierConf.isTierBoundaryFixed()) {
+      for (int i = 0; i < localNumTiers; i++) {
+        tier = tierConf.getCompactionTier(i);
+        localCal = tier.getTierBoundary(localCal);
+        // we can ignore all the numbers below the minute level
+        // since our cron expression starts with the minute
+        assertEquals(expectedCals[i].getTimeInMillis() / 60000, 
+            localCal.getTimeInMillis() / 60000);
+      }
+    }
+  }
+
   @Override
   public void testCompactionRatio() throws IOException {
     conf.setInt(strPrefix + strSchema + "NumCompactionTiers", 1);