You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lens.apache.org by sh...@apache.org on 2015/12/30 08:10:32 UTC

[13/50] [abbrv] lens git commit: LENS-865: Add/delete partition throws NPE when a part col doesn't exist in the table

LENS-865: Add/delete partition throws NPE when a part col doesn't exist in the table


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

Branch: refs/heads/LENS-581
Commit: e5691d8d655c94cdd64e0d83028ec59735d73edc
Parents: 09baa12
Author: Rajat Khandelwal <pr...@apache.org>
Authored: Tue Nov 24 12:07:17 2015 +0530
Committer: Rajat Khandelwal <ra...@gmail.com>
Committed: Tue Nov 24 12:07:17 2015 +0530

----------------------------------------------------------------------
 .../java/org/apache/lens/api/APIResult.java     |  10 +-
 lens-api/src/main/resources/lens-errors.conf    |   6 +
 .../lens/cube/error/LensCubeErrorCode.java      |   5 +-
 .../lens/cube/metadata/CubeMetastoreClient.java |  42 +-
 .../cube/metadata/TestCubeMetastoreClient.java  | 404 ++++++++++---------
 .../lens/server/api/error/LensException.java    |   8 +-
 .../server/metastore/MetastoreResource.java     | 384 ++++++++++--------
 .../server/metastore/TestMetastoreService.java  |  40 +-
 8 files changed, 498 insertions(+), 401 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/e5691d8d/lens-api/src/main/java/org/apache/lens/api/APIResult.java
----------------------------------------------------------------------
diff --git a/lens-api/src/main/java/org/apache/lens/api/APIResult.java b/lens-api/src/main/java/org/apache/lens/api/APIResult.java
index 06c608a..0cdee0d 100644
--- a/lens-api/src/main/java/org/apache/lens/api/APIResult.java
+++ b/lens-api/src/main/java/org/apache/lens/api/APIResult.java
@@ -152,11 +152,13 @@ public class APIResult {
   }
 
   private static String extractCause(Throwable e) {
-    String cause = null;
-    while ((cause == null || cause.isEmpty()) && e != null) {
-      cause = e.getMessage();
+    StringBuilder cause = new StringBuilder();
+    String sep = "";
+    while (e != null) {
+      cause.append(sep).append(e.getMessage());
       e = e.getCause();
+      sep = ": ";
     }
-    return cause;
+    return cause.toString();
   }
 }

http://git-wip-us.apache.org/repos/asf/lens/blob/e5691d8d/lens-api/src/main/resources/lens-errors.conf
----------------------------------------------------------------------
diff --git a/lens-api/src/main/resources/lens-errors.conf b/lens-api/src/main/resources/lens-errors.conf
index f50433a..ca8562f 100644
--- a/lens-api/src/main/resources/lens-errors.conf
+++ b/lens-api/src/main/resources/lens-errors.conf
@@ -296,6 +296,12 @@ lensCubeErrorsForMetastore = [
     httpStatusCode = ${BAD_REQUEST}
     errorMsg = "Problem in submitting entity: %s"
   }
+
+  {
+      errorCode = 3102
+      httpStatusCode = ${BAD_REQUEST}
+      errorMsg = "No timeline found for fact=%s, storage=%s, update period=%s, partition column=%s."
+  }
 ]
 
 lensCubeErrors = ${lensCubeErrorsForQuery}${lensCubeErrorsForMetastore}

http://git-wip-us.apache.org/repos/asf/lens/blob/e5691d8d/lens-cube/src/main/java/org/apache/lens/cube/error/LensCubeErrorCode.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/error/LensCubeErrorCode.java b/lens-cube/src/main/java/org/apache/lens/cube/error/LensCubeErrorCode.java
index 24fb80b..6c5dc2f 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/error/LensCubeErrorCode.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/error/LensCubeErrorCode.java
@@ -21,6 +21,7 @@ package org.apache.lens.cube.error;
 import org.apache.lens.server.api.LensErrorInfo;
 
 public enum LensCubeErrorCode {
+  // Error codes less than 3100 are errors encountered while submitting a query
   // Error codes same for drivers
   SYNTAX_ERROR(3001, 0),
   FIELDS_CANNOT_BE_QUERIED_TOGETHER(3002, 0),
@@ -54,7 +55,9 @@ public enum LensCubeErrorCode {
   NO_CANDIDATE_DIM_STORAGE_TABLES(3029, 1300),
   NO_STORAGE_TABLE_AVAIABLE(3030, 1400),
   STORAGE_UNION_DISABLED(3031, 100),
-  ERROR_IN_ENTITY_DEFINITION(3101, 100);
+  // Error codes greater than 3100 are errors while doing a metastore operation.
+  ERROR_IN_ENTITY_DEFINITION(3101, 100),
+  TIMELINE_ABSENT(3102, 100);
 
   public LensErrorInfo getLensErrorInfo() {
     return this.errorInfo;

http://git-wip-us.apache.org/repos/asf/lens/blob/e5691d8d/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeMetastoreClient.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeMetastoreClient.java b/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeMetastoreClient.java
index f945e0f..1f13617 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeMetastoreClient.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/metadata/CubeMetastoreClient.java
@@ -25,6 +25,7 @@ import java.text.ParseException;
 import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
 
+import org.apache.lens.cube.error.LensCubeErrorCode;
 import org.apache.lens.cube.metadata.Storage.LatestInfo;
 import org.apache.lens.cube.metadata.Storage.LatestPartColumnInfo;
 import org.apache.lens.cube.metadata.timeline.PartitionTimeline;
@@ -115,8 +116,7 @@ public class CubeMetastoreClient {
     for (CubeFactTable fact : getAllFacts(cube)) {
       for (String storage : fact.getStorages()) {
         for (UpdatePeriod updatePeriod : fact.getUpdatePeriods().get(storage)) {
-          PartitionTimeline timeline = partitionTimelineCache.get(fact.getName(), storage, updatePeriod,
-            partCol);
+          PartitionTimeline timeline = partitionTimelineCache.get(fact.getName(), storage, updatePeriod, partCol);
           if (timeline != null) {// this storage table is partitioned by partCol or not.
             Date latest = timeline.getLatestDate();
             if (latest != null && latest.after(max)) {
@@ -424,8 +424,8 @@ public class CubeMetastoreClient {
     /** check partition existence in the appropriate timeline if it exists */
     public boolean partitionTimeExists(String name, String storage, UpdatePeriod period, String partCol, Date partSpec)
       throws HiveException, LensException {
-      return get(name, storage, period, partCol) != null && get(name, storage, period, partCol).exists(TimePartition.of(
-        period, partSpec));
+      return get(name, storage, period, partCol) != null
+        && get(name, storage, period, partCol).exists(TimePartition.of(period, partSpec));
     }
 
     /**
@@ -437,12 +437,29 @@ public class CubeMetastoreClient {
       return get(fact, storage) != null && get(fact, storage).get(updatePeriod) != null && get(fact, storage).get(
         updatePeriod).get(partCol) != null ? get(fact, storage).get(updatePeriod).get(partCol) : null;
     }
+    /**
+     * returns the timeline corresponding to fact-storage table, updatePeriod, partCol. throws exception if not
+     * exists, which would most probably mean the combination is incorrect.
+     */
+    public PartitionTimeline getAndFailFast(String fact, String storage, UpdatePeriod updatePeriod, String partCol)
+      throws HiveException, LensException {
+      PartitionTimeline timeline = get(fact, storage, updatePeriod, partCol);
+      if (timeline == null) {
+        throw new LensException(LensCubeErrorCode.TIMELINE_ABSENT.getLensErrorInfo(),
+          fact, storage, updatePeriod, partCol);
+      }
+      return timeline;
+    }
+
 
     /** update partition timeline cache for addition of time partition */
     public void updateForAddition(String cubeTableName, String storageName, UpdatePeriod updatePeriod,
       Map<String, TreeSet<Date>> timePartSpec) throws HiveException, LensException {
+      // fail fast. All part cols mentioned in all partitions should exist.
+      for (String partCol : timePartSpec.keySet()) {
+        getAndFailFast(cubeTableName, storageName, updatePeriod, partCol);
+      }
       for (Map.Entry<String, TreeSet<Date>> entry : timePartSpec.entrySet()) {
-        //Assume timelines has all the time part columns.
         for (Date dt : entry.getValue()) {
           get(cubeTableName, storageName, updatePeriod, entry.getKey()).add(TimePartition.of(updatePeriod, dt));
         }
@@ -452,6 +469,10 @@ public class CubeMetastoreClient {
     /** update partition timeline cache for deletion of time partition */
     public boolean updateForDeletion(String cubeTableName, String storageName, UpdatePeriod updatePeriod,
       Map<String, Date> timePartSpec) throws HiveException, LensException {
+      // fail fast. All part cols mentioned in all partitions should exist.
+      for (String partCol : timePartSpec.keySet()) {
+        getAndFailFast(cubeTableName, storageName, updatePeriod, partCol);
+      }
       boolean updated = false;
       for (Map.Entry<String, Date> entry : timePartSpec.entrySet()) {
         TimePartition part = TimePartition.of(updatePeriod, entry.getValue());
@@ -1209,13 +1230,14 @@ public class CubeMetastoreClient {
     throws HiveException, LensException {
     String storageTableName = MetastoreUtil.getStorageTableName(factOrDimTblName, Storage.getPrefix(storageName));
     if (isDimensionTable(factOrDimTblName)) {
-      return dimLatestPartitionExists(storageTableName, StorageConstants.getLatestPartFilter(latestPartCol));
+      return dimTableLatestPartitionExists(storageTableName);
     } else {
       return !partitionTimelineCache.noPartitionsExist(factOrDimTblName, storageName, latestPartCol);
     }
   }
 
-  private boolean dimLatestPartitionExists(String storageTableName, String latestPartCol) throws HiveException {
+  private boolean dimTableLatestPartitionExistsInMetastore(String storageTableName, String latestPartCol)
+    throws HiveException {
     return partitionExistsByFilter(storageTableName, StorageConstants.getLatestPartFilter(latestPartCol));
   }
 
@@ -1223,10 +1245,6 @@ public class CubeMetastoreClient {
     return latestLookupCache.contains(storageTableName.trim().toLowerCase());
   }
 
-  Partition getLatestPart(String storageTableName, String latestPartCol) throws HiveException {
-    return getLatestPart(storageTableName, latestPartCol, null);
-  }
-
   Partition getLatestPart(String storageTableName, String latestPartCol, Map<String, String> nonTimeParts)
     throws HiveException {
     List<Partition> latestParts =
@@ -1471,7 +1489,7 @@ public class CubeMetastoreClient {
                   if (dimTable.hasStorageSnapshots(storageName)) {
                     String storageTableName = getFactOrDimtableStorageTableName(dimTable.getName(),
                       storageName);
-                    if (dimLatestPartitionExists(storageTableName,
+                    if (dimTableLatestPartitionExistsInMetastore(storageTableName,
                       getDimension(dimTable.getDimName()).getTimedDimension())) {
                       latestLookupCache.add(storageTableName.trim().toLowerCase());
                     }

http://git-wip-us.apache.org/repos/asf/lens/blob/e5691d8d/lens-cube/src/test/java/org/apache/lens/cube/metadata/TestCubeMetastoreClient.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/test/java/org/apache/lens/cube/metadata/TestCubeMetastoreClient.java b/lens-cube/src/test/java/org/apache/lens/cube/metadata/TestCubeMetastoreClient.java
index 6a2dc50..6b6f645 100644
--- a/lens-cube/src/test/java/org/apache/lens/cube/metadata/TestCubeMetastoreClient.java
+++ b/lens-cube/src/test/java/org/apache/lens/cube/metadata/TestCubeMetastoreClient.java
@@ -19,10 +19,16 @@
 
 package org.apache.lens.cube.metadata;
 
+import static org.apache.lens.cube.metadata.UpdatePeriod.DAILY;
+import static org.apache.lens.cube.metadata.UpdatePeriod.HOURLY;
+import static org.apache.lens.cube.metadata.UpdatePeriod.MONTHLY;
+
 import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.fail;
 
 import java.util.*;
 
+import org.apache.lens.cube.error.LensCubeErrorCode;
 import org.apache.lens.cube.metadata.ExprColumn.ExprSpec;
 import org.apache.lens.cube.metadata.ReferencedDimAtrribute.ChainRefCol;
 import org.apache.lens.cube.metadata.timeline.EndsAndHolesPartitionTimeline;
@@ -780,8 +786,8 @@ public class TestCubeMetastoreClient {
     List<String> timePartCols = new ArrayList<String>();
     partCols.add(getDatePartition());
     timePartCols.add(getDatePartitionKey());
-    updates.add(UpdatePeriod.HOURLY);
-    updates.add(UpdatePeriod.DAILY);
+    updates.add(HOURLY);
+    updates.add(DAILY);
     StorageTableDesc s1 = new StorageTableDesc();
     s1.setInputFormat(TextInputFormat.class.getCanonicalName());
     s1.setOutputFormat(HiveIgnoreKeyTextOutputFormat.class.getCanonicalName());
@@ -813,10 +819,20 @@ public class TestCubeMetastoreClient {
 
     Map<String, Date> timeParts = new HashMap<String, Date>();
     timeParts.put(TestCubeMetastoreClient.getDatePartitionKey(), now);
+    timeParts.put("non_existing_part_col", now);
+    // test error on adding invalid partition
     // test partition
-    StoragePartitionDesc partSpec = new StoragePartitionDesc(cubeFact.getName(), timeParts, null, UpdatePeriod.HOURLY);
+    StoragePartitionDesc partSpec = new StoragePartitionDesc(cubeFact.getName(), timeParts, null, HOURLY);
+    try{
+      client.addPartition(partSpec, c1);
+      fail("Add should fail since non_existing_part_col is non-existing");
+    } catch(LensException e){
+      assertEquals(e.getErrorCode(), LensCubeErrorCode.TIMELINE_ABSENT.getLensErrorInfo().getErrorCode());
+    }
+    timeParts.remove("non_existing_part_col");
+    partSpec = new StoragePartitionDesc(cubeFact.getName(), timeParts, null, HOURLY);
     client.addPartition(partSpec, c1);
-    Assert.assertTrue(client.factPartitionExists(cubeFact.getName(), c1, UpdatePeriod.HOURLY, timeParts,
+    Assert.assertTrue(client.factPartitionExists(cubeFact.getName(), c1, HOURLY, timeParts,
       new HashMap<String, String>()));
     Assert.assertTrue(client.latestPartitionExists(cubeFact.getName(), c1,
       TestCubeMetastoreClient.getDatePartitionKey()));
@@ -833,36 +849,36 @@ public class TestCubeMetastoreClient {
     Map<String, Date> timeParts2 = new HashMap<String, Date>();
     timeParts2.put(TestCubeMetastoreClient.getDatePartitionKey(), nowPlus1);
     StoragePartitionDesc partSpec2 =
-      new StoragePartitionDesc(cubeFact.getName(), timeParts2, null, UpdatePeriod.HOURLY);
+      new StoragePartitionDesc(cubeFact.getName(), timeParts2, null, HOURLY);
     partSpec2.setInputFormat(SequenceFileInputFormat.class.getCanonicalName());
     partSpec2.setOutputFormat(HiveIgnoreKeyTextOutputFormat.class.getCanonicalName());
     client.addPartition(partSpec2, c1);
     assertEquals(client.getAllParts(storageTableName).size(), 2);
-    Assert.assertTrue(client.factPartitionExists(cubeFact.getName(), c1, UpdatePeriod.HOURLY, timeParts,
+    Assert.assertTrue(client.factPartitionExists(cubeFact.getName(), c1, HOURLY, timeParts,
       new HashMap<String, String>()));
-    Assert.assertTrue(client.factPartitionExists(cubeFact.getName(), c1, UpdatePeriod.HOURLY, timeParts2,
+    Assert.assertTrue(client.factPartitionExists(cubeFact.getName(), c1, HOURLY, timeParts2,
       new HashMap<String, String>()));
     Assert.assertTrue(client.latestPartitionExists(cubeFact.getName(), c1,
       TestCubeMetastoreClient.getDatePartitionKey()));
     parts = client.getPartitionsByFilter(storageTableName, "dt='latest'");
     assertEquals(parts.size(), 0);
 
-    client.dropPartition(cubeFact.getName(), c1, timeParts2, null, UpdatePeriod.HOURLY);
+    client.dropPartition(cubeFact.getName(), c1, timeParts2, null, HOURLY);
     assertEquals(client.getAllParts(storageTableName).size(), 1);
-    Assert.assertTrue(client.factPartitionExists(cubeFact.getName(), c1, UpdatePeriod.HOURLY, timeParts,
+    Assert.assertTrue(client.factPartitionExists(cubeFact.getName(), c1, HOURLY, timeParts,
       new HashMap<String, String>()));
-    Assert.assertFalse(client.factPartitionExists(cubeFact.getName(), c1, UpdatePeriod.HOURLY, timeParts2,
+    Assert.assertFalse(client.factPartitionExists(cubeFact.getName(), c1, HOURLY, timeParts2,
       new HashMap<String, String>()));
     Assert.assertTrue(client.latestPartitionExists(cubeFact.getName(), c1,
       TestCubeMetastoreClient.getDatePartitionKey()));
     parts = client.getPartitionsByFilter(storageTableName, "dt='latest'");
     assertEquals(parts.size(), 0);
 
-    client.dropPartition(cubeFact.getName(), c1, timeParts, null, UpdatePeriod.HOURLY);
+    client.dropPartition(cubeFact.getName(), c1, timeParts, null, HOURLY);
     assertEquals(client.getAllParts(storageTableName).size(), 0);
-    Assert.assertFalse(client.factPartitionExists(cubeFact.getName(), c1, UpdatePeriod.HOURLY, timeParts,
+    Assert.assertFalse(client.factPartitionExists(cubeFact.getName(), c1, HOURLY, timeParts,
       new HashMap<String, String>()));
-    Assert.assertFalse(client.factPartitionExists(cubeFact.getName(), c1, UpdatePeriod.HOURLY, timeParts2,
+    Assert.assertFalse(client.factPartitionExists(cubeFact.getName(), c1, HOURLY, timeParts2,
       new HashMap<String, String>()));
     Assert.assertFalse(client.latestPartitionExists(cubeFact.getName(), c1,
       TestCubeMetastoreClient.getDatePartitionKey()));
@@ -881,15 +897,15 @@ public class TestCubeMetastoreClient {
 
     Map<String, Set<UpdatePeriod>> updatePeriods = new HashMap<String, Set<UpdatePeriod>>();
     Set<UpdatePeriod> updates = new HashSet<UpdatePeriod>();
-    updates.add(UpdatePeriod.HOURLY);
-    updates.add(UpdatePeriod.DAILY);
+    updates.add(HOURLY);
+    updates.add(DAILY);
 
     ArrayList<FieldSchema> partCols = new ArrayList<FieldSchema>();
     List<String> timePartCols = new ArrayList<String>();
     partCols.add(getDatePartition());
     timePartCols.add(getDatePartitionKey());
-    updates.add(UpdatePeriod.HOURLY);
-    updates.add(UpdatePeriod.DAILY);
+    updates.add(HOURLY);
+    updates.add(DAILY);
     StorageTableDesc s1 = new StorageTableDesc();
     s1.setInputFormat(TextInputFormat.class.getCanonicalName());
     s1.setOutputFormat(HiveIgnoreKeyTextOutputFormat.class.getCanonicalName());
@@ -913,12 +929,12 @@ public class TestCubeMetastoreClient {
     Map<String, String> newProp = new HashMap<String, String>();
     newProp.put("new.prop", "val");
     factTable.addProperties(newProp);
-    factTable.addUpdatePeriod(c1, UpdatePeriod.MONTHLY);
-    factTable.removeUpdatePeriod(c1, UpdatePeriod.HOURLY);
+    factTable.addUpdatePeriod(c1, MONTHLY);
+    factTable.removeUpdatePeriod(c1, HOURLY);
     Set<UpdatePeriod> alterupdates = new HashSet<UpdatePeriod>();
-    alterupdates.add(UpdatePeriod.HOURLY);
-    alterupdates.add(UpdatePeriod.DAILY);
-    alterupdates.add(UpdatePeriod.MONTHLY);
+    alterupdates.add(HOURLY);
+    alterupdates.add(DAILY);
+    alterupdates.add(MONTHLY);
     factTable.alterStorage(c2, alterupdates);
 
     client.alterCubeFactTable(factName, factTable, storageTables);
@@ -928,11 +944,11 @@ public class TestCubeMetastoreClient {
 
     Assert.assertTrue(altered.weight() == 100L);
     Assert.assertTrue(altered.getProperties().get("new.prop").equals("val"));
-    Assert.assertTrue(altered.getUpdatePeriods().get(c1).contains(UpdatePeriod.MONTHLY));
-    Assert.assertFalse(altered.getUpdatePeriods().get(c1).contains(UpdatePeriod.HOURLY));
-    Assert.assertTrue(altered.getUpdatePeriods().get(c2).contains(UpdatePeriod.MONTHLY));
-    Assert.assertTrue(altered.getUpdatePeriods().get(c2).contains(UpdatePeriod.DAILY));
-    Assert.assertTrue(altered.getUpdatePeriods().get(c2).contains(UpdatePeriod.HOURLY));
+    Assert.assertTrue(altered.getUpdatePeriods().get(c1).contains(MONTHLY));
+    Assert.assertFalse(altered.getUpdatePeriods().get(c1).contains(HOURLY));
+    Assert.assertTrue(altered.getUpdatePeriods().get(c2).contains(MONTHLY));
+    Assert.assertTrue(altered.getUpdatePeriods().get(c2).contains(DAILY));
+    Assert.assertTrue(altered.getUpdatePeriods().get(c2).contains(HOURLY));
     Assert.assertTrue(altered.getCubeName().equalsIgnoreCase(CUBE_NAME.toLowerCase()));
     boolean contains = false;
     for (FieldSchema column : altered.getColumns()) {
@@ -1026,8 +1042,8 @@ public class TestCubeMetastoreClient {
 
     Map<String, Set<UpdatePeriod>> updatePeriods = new HashMap<String, Set<UpdatePeriod>>();
     Set<UpdatePeriod> updates = new HashSet<UpdatePeriod>();
-    updates.add(UpdatePeriod.HOURLY);
-    updates.add(UpdatePeriod.DAILY);
+    updates.add(HOURLY);
+    updates.add(DAILY);
     FieldSchema testDtPart = new FieldSchema("mydate", "string", "date part");
     ArrayList<FieldSchema> partCols = new ArrayList<FieldSchema>();
     List<String> timePartCols = new ArrayList<String>();
@@ -1071,9 +1087,9 @@ public class TestCubeMetastoreClient {
     Map<String, Date> timeParts = new HashMap<String, Date>();
     timeParts.put(TestCubeMetastoreClient.getDatePartitionKey(), now);
     timeParts.put(testDtPart.getName(), testDt);
-    StoragePartitionDesc partSpec = new StoragePartitionDesc(cubeFact.getName(), timeParts, null, UpdatePeriod.HOURLY);
+    StoragePartitionDesc partSpec = new StoragePartitionDesc(cubeFact.getName(), timeParts, null, HOURLY);
     client.addPartition(partSpec, c1);
-    Assert.assertTrue(client.factPartitionExists(cubeFact.getName(), c1, UpdatePeriod.HOURLY, timeParts,
+    Assert.assertTrue(client.factPartitionExists(cubeFact.getName(), c1, HOURLY, timeParts,
       new HashMap<String, String>()));
     Assert.assertTrue(client.latestPartitionExists(cubeFact.getName(), c1, testDtPart.getName()));
     Assert.assertTrue(client.latestPartitionExists(cubeFact.getName(), c1,
@@ -1086,9 +1102,9 @@ public class TestCubeMetastoreClient {
     parts = client.getPartitionsByFilter(storageTableName, testDtPart.getName() + "='latest'");
     assertEquals(parts.size(), 0);
 
-    client.dropPartition(cubeFact.getName(), c1, timeParts, null, UpdatePeriod.HOURLY);
+    client.dropPartition(cubeFact.getName(), c1, timeParts, null, HOURLY);
     assertEquals(client.getAllParts(storageTableName).size(), 0);
-    Assert.assertFalse(client.factPartitionExists(cubeFact.getName(), c1, UpdatePeriod.HOURLY, timeParts,
+    Assert.assertFalse(client.factPartitionExists(cubeFact.getName(), c1, HOURLY, timeParts,
       new HashMap<String, String>()));
     Assert.assertFalse(client.latestPartitionExists(cubeFact.getName(), c1,
       TestCubeMetastoreClient.getDatePartitionKey()));
@@ -1108,8 +1124,8 @@ public class TestCubeMetastoreClient {
 
     Map<String, Set<UpdatePeriod>> updatePeriods = new HashMap<String, Set<UpdatePeriod>>();
     Set<UpdatePeriod> updates = new HashSet<UpdatePeriod>();
-    updates.add(UpdatePeriod.HOURLY);
-    updates.add(UpdatePeriod.DAILY);
+    updates.add(HOURLY);
+    updates.add(DAILY);
     FieldSchema itPart = new FieldSchema("it", "string", "date part");
     FieldSchema etPart = new FieldSchema("et", "string", "date part");
     ArrayList<FieldSchema> partCols = new ArrayList<FieldSchema>();
@@ -1158,35 +1174,35 @@ public class TestCubeMetastoreClient {
 
     Table c1Table = client.getHiveTable(c1TableName);
     Table c2Table = client.getHiveTable(c2TableName);
-    c2Table.getParameters().put(MetastoreUtil.getPartitionTimelineStorageClassKey(UpdatePeriod.HOURLY,
+    c2Table.getParameters().put(MetastoreUtil.getPartitionTimelineStorageClassKey(HOURLY,
       getDatePartitionKey()), StoreAllPartitionTimeline.class.getCanonicalName());
-    c2Table.getParameters().put(MetastoreUtil.getPartitionTimelineStorageClassKey(UpdatePeriod.HOURLY,
+    c2Table.getParameters().put(MetastoreUtil.getPartitionTimelineStorageClassKey(HOURLY,
       itPart.getName()), StoreAllPartitionTimeline.class.getCanonicalName());
-    c2Table.getParameters().put(MetastoreUtil.getPartitionTimelineStorageClassKey(UpdatePeriod.HOURLY,
+    c2Table.getParameters().put(MetastoreUtil.getPartitionTimelineStorageClassKey(HOURLY,
       etPart.getName()), StoreAllPartitionTimeline.class.getCanonicalName());
     client.pushHiveTable(c2Table);
 
     // same before insertion.
-    assertSameTimelines(factName, storages, UpdatePeriod.HOURLY, partColNames);
+    assertSameTimelines(factName, storages, HOURLY, partColNames);
     EndsAndHolesPartitionTimeline timelineDt = ((EndsAndHolesPartitionTimeline) client.partitionTimelineCache.get(
-      factName, c1, UpdatePeriod.HOURLY, getDatePartitionKey()));
+      factName, c1, HOURLY, getDatePartitionKey()));
     EndsAndHolesPartitionTimeline timelineIt = ((EndsAndHolesPartitionTimeline) client.partitionTimelineCache.get(
-      factName, c1, UpdatePeriod.HOURLY, itPart.getName()));
+      factName, c1, HOURLY, itPart.getName()));
     EndsAndHolesPartitionTimeline timelineEt = ((EndsAndHolesPartitionTimeline) client.partitionTimelineCache.get(
-      factName, c1, UpdatePeriod.HOURLY, etPart.getName()));
+      factName, c1, HOURLY, etPart.getName()));
     StoreAllPartitionTimeline timelineDtC2 = ((StoreAllPartitionTimeline) client.partitionTimelineCache.get(
-      factName, c2, UpdatePeriod.HOURLY, getDatePartitionKey()));
+      factName, c2, HOURLY, getDatePartitionKey()));
     StoreAllPartitionTimeline timelineItC2 = ((StoreAllPartitionTimeline) client.partitionTimelineCache.get(
-      factName, c2, UpdatePeriod.HOURLY, itPart.getName()));
+      factName, c2, HOURLY, itPart.getName()));
     StoreAllPartitionTimeline timelineEtC2 = ((StoreAllPartitionTimeline) client.partitionTimelineCache.get(
-      factName, c2, UpdatePeriod.HOURLY, etPart.getName()));
+      factName, c2, HOURLY, etPart.getName()));
 
     Map<String, Date> timeParts1 = new HashMap<String, Date>();
     timeParts1.put(TestCubeMetastoreClient.getDatePartitionKey(), now);
     timeParts1.put(itPart.getName(), now);
     timeParts1.put(etPart.getName(), now);
     StoragePartitionDesc partSpec1 = new StoragePartitionDesc(cubeFact.getName(), timeParts1, null,
-      UpdatePeriod.HOURLY);
+      HOURLY);
 
     Map<String, Date> timeParts2 = new HashMap<String, Date>();
     timeParts2.put(TestCubeMetastoreClient.getDatePartitionKey(), now);
@@ -1194,19 +1210,19 @@ public class TestCubeMetastoreClient {
     Map<String, String> nonTimeSpec = new HashMap<String, String>();
     nonTimeSpec.put(itPart.getName(), "default");
     final StoragePartitionDesc partSpec2 = new StoragePartitionDesc(cubeFact.getName(), timeParts2, nonTimeSpec,
-      UpdatePeriod.HOURLY);
+      HOURLY);
 
     Map<String, Date> timeParts3 = new HashMap<String, Date>();
     timeParts3.put(TestCubeMetastoreClient.getDatePartitionKey(), now);
     timeParts3.put(etPart.getName(), now);
     final StoragePartitionDesc partSpec3 = new StoragePartitionDesc(cubeFact.getName(), timeParts3, nonTimeSpec,
-      UpdatePeriod.HOURLY);
+      HOURLY);
 
     client.addPartitions(Arrays.asList(partSpec1, partSpec2, partSpec3), c1);
     client.addPartitions(Arrays.asList(partSpec1, partSpec2, partSpec3), c2);
-    PartitionTimeline timeline1Temp = client.partitionTimelineCache.get(factName, c1, UpdatePeriod.HOURLY,
+    PartitionTimeline timeline1Temp = client.partitionTimelineCache.get(factName, c1, HOURLY,
       getDatePartitionKey());
-    PartitionTimeline timeline2Temp = client.partitionTimelineCache.get(factName, c2, UpdatePeriod.HOURLY,
+    PartitionTimeline timeline2Temp = client.partitionTimelineCache.get(factName, c2, HOURLY,
       getDatePartitionKey());
 
     assertEquals(timeline1Temp.getClass(), EndsAndHolesPartitionTimeline.class);
@@ -1215,11 +1231,11 @@ public class TestCubeMetastoreClient {
     assertEquals(client.getAllParts(c1TableName).size(), 3);
     assertEquals(client.getAllParts(c2TableName).size(), 3);
 
-    assertSameTimelines(factName, storages, UpdatePeriod.HOURLY, partColNames);
+    assertSameTimelines(factName, storages, HOURLY, partColNames);
 
-    assertTimeline(timelineDt, timelineDtC2, UpdatePeriod.HOURLY, now, now);
-    assertTimeline(timelineEt, timelineEtC2, UpdatePeriod.HOURLY, now, nowPlus1);
-    assertTimeline(timelineIt, timelineItC2, UpdatePeriod.HOURLY, now, now);
+    assertTimeline(timelineDt, timelineDtC2, HOURLY, now, now);
+    assertTimeline(timelineEt, timelineEtC2, HOURLY, now, nowPlus1);
+    assertTimeline(timelineIt, timelineItC2, HOURLY, now, now);
 
     Assert.assertTrue(client.latestPartitionExists(cubeFact.getName(), c1,
       TestCubeMetastoreClient.getDatePartitionKey()));
@@ -1234,7 +1250,7 @@ public class TestCubeMetastoreClient {
     timeParts4.put(itPart.getName(), nowPlus1);
     timeParts4.put(etPart.getName(), nowMinus1);
     final StoragePartitionDesc partSpec4 = new StoragePartitionDesc(cubeFact.getName(), timeParts4, null,
-      UpdatePeriod.HOURLY);
+      HOURLY);
 
 
     Map<String, Date> timeParts5 = new HashMap<String, Date>();
@@ -1242,7 +1258,7 @@ public class TestCubeMetastoreClient {
     timeParts5.put(itPart.getName(), nowMinus1);
     timeParts5.put(etPart.getName(), nowMinus2);
     final StoragePartitionDesc partSpec5 = new StoragePartitionDesc(cubeFact.getName(), timeParts5, null,
-      UpdatePeriod.HOURLY);
+      HOURLY);
 
     client.addPartitions(Arrays.asList(partSpec4, partSpec5), c1);
     client.addPartitions(Arrays.asList(partSpec4, partSpec5), c2);
@@ -1250,11 +1266,11 @@ public class TestCubeMetastoreClient {
     assertEquals(client.getAllParts(c1TableName).size(), 5);
     assertEquals(client.getAllParts(c2TableName).size(), 5);
 
-    assertSameTimelines(factName, storages, UpdatePeriod.HOURLY, partColNames);
+    assertSameTimelines(factName, storages, HOURLY, partColNames);
 
-    assertTimeline(timelineDt, timelineDtC2, UpdatePeriod.HOURLY, now, nowPlus1);
-    assertTimeline(timelineEt, timelineEtC2, UpdatePeriod.HOURLY, nowMinus2, nowPlus1);
-    assertTimeline(timelineIt, timelineItC2, UpdatePeriod.HOURLY, nowMinus1, nowPlus1);
+    assertTimeline(timelineDt, timelineDtC2, HOURLY, now, nowPlus1);
+    assertTimeline(timelineEt, timelineEtC2, HOURLY, nowMinus2, nowPlus1);
+    assertTimeline(timelineIt, timelineItC2, HOURLY, nowMinus1, nowPlus1);
 
     Assert.assertTrue(client.latestPartitionExists(cubeFact.getName(), c1,
       TestCubeMetastoreClient.getDatePartitionKey()));
@@ -1266,7 +1282,7 @@ public class TestCubeMetastoreClient {
     timeParts6.put(itPart.getName(), nowMinus1);
     timeParts6.put(etPart.getName(), nowMinus2);
     final StoragePartitionDesc partSpec6 = new StoragePartitionDesc(cubeFact.getName(), timeParts6, null,
-      UpdatePeriod.HOURLY);
+      HOURLY);
 
     client.addPartition(partSpec6, c1);
     client.addPartition(partSpec6, c2);
@@ -1274,18 +1290,18 @@ public class TestCubeMetastoreClient {
     assertEquals(client.getAllParts(c1TableName).size(), 6);
     assertEquals(client.getAllParts(c2TableName).size(), 6);
 
-    assertSameTimelines(factName, storages, UpdatePeriod.HOURLY, partColNames);
+    assertSameTimelines(factName, storages, HOURLY, partColNames);
 
-    assertTimeline(timelineDt, timelineDtC2, UpdatePeriod.HOURLY, nowMinus2, nowPlus1, nowMinus1);
-    assertTimeline(timelineEt, timelineEtC2, UpdatePeriod.HOURLY, nowMinus2, nowPlus1);
-    assertTimeline(timelineIt, timelineItC2, UpdatePeriod.HOURLY, nowMinus1, nowPlus1);
+    assertTimeline(timelineDt, timelineDtC2, HOURLY, nowMinus2, nowPlus1, nowMinus1);
+    assertTimeline(timelineEt, timelineEtC2, HOURLY, nowMinus2, nowPlus1);
+    assertTimeline(timelineIt, timelineItC2, HOURLY, nowMinus1, nowPlus1);
 
     Map<String, Date> timeParts7 = new HashMap<String, Date>();
     timeParts7.put(TestCubeMetastoreClient.getDatePartitionKey(), nowMinus5);
     timeParts7.put(itPart.getName(), nowMinus5);
     timeParts7.put(etPart.getName(), nowMinus5);
     final StoragePartitionDesc partSpec7 = new StoragePartitionDesc(cubeFact.getName(), timeParts7, null,
-      UpdatePeriod.HOURLY);
+      HOURLY);
 
     client.addPartition(partSpec7, c1);
     client.addPartition(partSpec7, c2);
@@ -1312,14 +1328,14 @@ public class TestCubeMetastoreClient {
     client.updatePartitions(factName, c2, c2Parts);
     assertSamePartitions(client.getAllParts(c2TableName), c2Parts);
 
-    assertSameTimelines(factName, storages, UpdatePeriod.HOURLY, partColNames);
+    assertSameTimelines(factName, storages, HOURLY, partColNames);
 
-    assertTimeline(timelineDt, timelineDtC2, UpdatePeriod.HOURLY, nowMinus5, nowPlus1, nowMinus4, nowMinus3, nowMinus1);
-    assertTimeline(timelineEt, timelineEtC2, UpdatePeriod.HOURLY, nowMinus5, nowPlus1, nowMinus4, nowMinus3);
-    assertTimeline(timelineIt, timelineItC2, UpdatePeriod.HOURLY, nowMinus5, nowPlus1, nowMinus4, nowMinus3, nowMinus2);
+    assertTimeline(timelineDt, timelineDtC2, HOURLY, nowMinus5, nowPlus1, nowMinus4, nowMinus3, nowMinus1);
+    assertTimeline(timelineEt, timelineEtC2, HOURLY, nowMinus5, nowPlus1, nowMinus4, nowMinus3);
+    assertTimeline(timelineIt, timelineItC2, HOURLY, nowMinus5, nowPlus1, nowMinus4, nowMinus3, nowMinus2);
 
     assertNoPartitionNamedLatest(c1TableName, partColNames);
-    assertSameTimelines(factName, storages, UpdatePeriod.HOURLY, partColNames);
+    assertSameTimelines(factName, storages, HOURLY, partColNames);
     assertEquals(Hive.get(client.getConf()).getTable(c1TableName).getParameters().get(
       MetastoreUtil.getPartitionTimelineCachePresenceKey()), "true");
     assertEquals(Hive.get(client.getConf()).getTable(c2TableName).getParameters().get(
@@ -1327,15 +1343,15 @@ public class TestCubeMetastoreClient {
 
     // alter tables and see timeline still exists
     client.alterCubeFactTable(factName, cubeFact, storageTables);
-    assertSameTimelines(factName, storages, UpdatePeriod.HOURLY, partColNames);
+    assertSameTimelines(factName, storages, HOURLY, partColNames);
     assertEquals(Hive.get(client.getConf()).getTable(c1TableName).getParameters().get(
       MetastoreUtil.getPartitionTimelineCachePresenceKey()), "true");
     assertEquals(Hive.get(client.getConf()).getTable(c2TableName).getParameters().get(
       MetastoreUtil.getPartitionTimelineCachePresenceKey()), "true");
 
 
-    client.dropPartition(cubeFact.getName(), c1, timeParts5, null, UpdatePeriod.HOURLY);
-    client.dropPartition(cubeFact.getName(), c2, timeParts5, null, UpdatePeriod.HOURLY);
+    client.dropPartition(cubeFact.getName(), c1, timeParts5, null, HOURLY);
+    client.dropPartition(cubeFact.getName(), c2, timeParts5, null, HOURLY);
     assertEquals(client.getAllParts(c1TableName).size(), 6);
     Assert.assertTrue(client.latestPartitionExists(cubeFact.getName(), c1,
       TestCubeMetastoreClient.getDatePartitionKey()));
@@ -1343,15 +1359,15 @@ public class TestCubeMetastoreClient {
     Assert.assertTrue(client.latestPartitionExists(cubeFact.getName(), c1, etPart.getName()));
 
     assertNoPartitionNamedLatest(c1TableName, partColNames);
-    assertSameTimelines(factName, storages, UpdatePeriod.HOURLY, partColNames);
+    assertSameTimelines(factName, storages, HOURLY, partColNames);
 
-    assertTimeline(timelineDt, timelineDtC2, UpdatePeriod.HOURLY, nowMinus5, now, nowMinus4, nowMinus3, nowMinus1);
-    assertTimeline(timelineEt, timelineEtC2, UpdatePeriod.HOURLY, nowMinus5, nowPlus1, nowMinus4, nowMinus3);
-    assertTimeline(timelineIt, timelineItC2, UpdatePeriod.HOURLY, nowMinus5, nowPlus1, nowMinus4, nowMinus3, nowMinus2);
+    assertTimeline(timelineDt, timelineDtC2, HOURLY, nowMinus5, now, nowMinus4, nowMinus3, nowMinus1);
+    assertTimeline(timelineEt, timelineEtC2, HOURLY, nowMinus5, nowPlus1, nowMinus4, nowMinus3);
+    assertTimeline(timelineIt, timelineItC2, HOURLY, nowMinus5, nowPlus1, nowMinus4, nowMinus3, nowMinus2);
 
 
-    client.dropPartition(cubeFact.getName(), c1, timeParts7, null, UpdatePeriod.HOURLY);
-    client.dropPartition(cubeFact.getName(), c2, timeParts7, null, UpdatePeriod.HOURLY);
+    client.dropPartition(cubeFact.getName(), c1, timeParts7, null, HOURLY);
+    client.dropPartition(cubeFact.getName(), c2, timeParts7, null, HOURLY);
     assertEquals(client.getAllParts(c1TableName).size(), 5);
     Assert.assertTrue(client.latestPartitionExists(cubeFact.getName(), c1,
       TestCubeMetastoreClient.getDatePartitionKey()));
@@ -1359,14 +1375,14 @@ public class TestCubeMetastoreClient {
     Assert.assertTrue(client.latestPartitionExists(cubeFact.getName(), c1, etPart.getName()));
 
     assertNoPartitionNamedLatest(c1TableName, partColNames);
-    assertSameTimelines(factName, storages, UpdatePeriod.HOURLY, partColNames);
-    assertTimeline(timelineDt, timelineDtC2, UpdatePeriod.HOURLY, nowMinus2, now, nowMinus1);
-    assertTimeline(timelineEt, timelineEtC2, UpdatePeriod.HOURLY, nowMinus2, nowPlus1);
-    assertTimeline(timelineIt, timelineItC2, UpdatePeriod.HOURLY, nowMinus1, nowPlus1);
+    assertSameTimelines(factName, storages, HOURLY, partColNames);
+    assertTimeline(timelineDt, timelineDtC2, HOURLY, nowMinus2, now, nowMinus1);
+    assertTimeline(timelineEt, timelineEtC2, HOURLY, nowMinus2, nowPlus1);
+    assertTimeline(timelineIt, timelineItC2, HOURLY, nowMinus1, nowPlus1);
 
 
-    client.dropPartition(cubeFact.getName(), c1, timeParts2, nonTimeSpec, UpdatePeriod.HOURLY);
-    client.dropPartition(cubeFact.getName(), c2, timeParts2, nonTimeSpec, UpdatePeriod.HOURLY);
+    client.dropPartition(cubeFact.getName(), c1, timeParts2, nonTimeSpec, HOURLY);
+    client.dropPartition(cubeFact.getName(), c2, timeParts2, nonTimeSpec, HOURLY);
     assertEquals(client.getAllParts(c1TableName).size(), 4);
     Assert.assertTrue(client.latestPartitionExists(cubeFact.getName(), c1,
       TestCubeMetastoreClient.getDatePartitionKey()));
@@ -1374,36 +1390,36 @@ public class TestCubeMetastoreClient {
     Assert.assertTrue(client.latestPartitionExists(cubeFact.getName(), c1, etPart.getName()));
 
     assertNoPartitionNamedLatest(c1TableName, partColNames);
-    assertSameTimelines(factName, storages, UpdatePeriod.HOURLY, partColNames);
-    assertTimeline(timelineDt, timelineDtC2, UpdatePeriod.HOURLY, nowMinus2, now, nowMinus1);
-    assertTimeline(timelineEt, timelineEtC2, UpdatePeriod.HOURLY, nowMinus2, now);
-    assertTimeline(timelineIt, timelineItC2, UpdatePeriod.HOURLY, nowMinus1, nowPlus1);
+    assertSameTimelines(factName, storages, HOURLY, partColNames);
+    assertTimeline(timelineDt, timelineDtC2, HOURLY, nowMinus2, now, nowMinus1);
+    assertTimeline(timelineEt, timelineEtC2, HOURLY, nowMinus2, now);
+    assertTimeline(timelineIt, timelineItC2, HOURLY, nowMinus1, nowPlus1);
 
-    client.dropPartition(cubeFact.getName(), c1, timeParts4, null, UpdatePeriod.HOURLY);
-    client.dropPartition(cubeFact.getName(), c2, timeParts4, null, UpdatePeriod.HOURLY);
+    client.dropPartition(cubeFact.getName(), c1, timeParts4, null, HOURLY);
+    client.dropPartition(cubeFact.getName(), c2, timeParts4, null, HOURLY);
     assertEquals(client.getAllParts(c1TableName).size(), 3);
 
     assertNoPartitionNamedLatest(c1TableName, partColNames);
-    assertSameTimelines(factName, storages, UpdatePeriod.HOURLY, partColNames);
-    assertTimeline(timelineDt, timelineDtC2, UpdatePeriod.HOURLY, nowMinus2, now, nowMinus1);
-    assertTimeline(timelineEt, timelineEtC2, UpdatePeriod.HOURLY, nowMinus2, now, nowMinus1);
-    assertTimeline(timelineIt, timelineItC2, UpdatePeriod.HOURLY, nowMinus1, now);
-    client.dropPartition(cubeFact.getName(), c1, timeParts3, nonTimeSpec, UpdatePeriod.HOURLY);
-    client.dropPartition(cubeFact.getName(), c2, timeParts3, nonTimeSpec, UpdatePeriod.HOURLY);
-    assertSameTimelines(factName, storages, UpdatePeriod.HOURLY, partColNames);
-    assertTimeline(timelineDt, timelineDtC2, UpdatePeriod.HOURLY, nowMinus2, now, nowMinus1);
-    assertTimeline(timelineEt, timelineEtC2, UpdatePeriod.HOURLY, nowMinus2, now, nowMinus1);
-    assertTimeline(timelineIt, timelineItC2, UpdatePeriod.HOURLY, nowMinus1, now);
-
-    client.dropPartition(cubeFact.getName(), c1, timeParts6, null, UpdatePeriod.HOURLY);
-    client.dropPartition(cubeFact.getName(), c2, timeParts6, null, UpdatePeriod.HOURLY);
-    assertSameTimelines(factName, storages, UpdatePeriod.HOURLY, partColNames);
-    assertTimeline(timelineDt, timelineDtC2, UpdatePeriod.HOURLY, now, now);
-    assertTimeline(timelineEt, timelineEtC2, UpdatePeriod.HOURLY, now, now);
-    assertTimeline(timelineIt, timelineItC2, UpdatePeriod.HOURLY, now, now);
-    client.dropPartition(cubeFact.getName(), c1, timeParts1, null, UpdatePeriod.HOURLY);
-    client.dropPartition(cubeFact.getName(), c2, timeParts1, null, UpdatePeriod.HOURLY);
-    assertSameTimelines(factName, storages, UpdatePeriod.HOURLY, partColNames);
+    assertSameTimelines(factName, storages, HOURLY, partColNames);
+    assertTimeline(timelineDt, timelineDtC2, HOURLY, nowMinus2, now, nowMinus1);
+    assertTimeline(timelineEt, timelineEtC2, HOURLY, nowMinus2, now, nowMinus1);
+    assertTimeline(timelineIt, timelineItC2, HOURLY, nowMinus1, now);
+    client.dropPartition(cubeFact.getName(), c1, timeParts3, nonTimeSpec, HOURLY);
+    client.dropPartition(cubeFact.getName(), c2, timeParts3, nonTimeSpec, HOURLY);
+    assertSameTimelines(factName, storages, HOURLY, partColNames);
+    assertTimeline(timelineDt, timelineDtC2, HOURLY, nowMinus2, now, nowMinus1);
+    assertTimeline(timelineEt, timelineEtC2, HOURLY, nowMinus2, now, nowMinus1);
+    assertTimeline(timelineIt, timelineItC2, HOURLY, nowMinus1, now);
+
+    client.dropPartition(cubeFact.getName(), c1, timeParts6, null, HOURLY);
+    client.dropPartition(cubeFact.getName(), c2, timeParts6, null, HOURLY);
+    assertSameTimelines(factName, storages, HOURLY, partColNames);
+    assertTimeline(timelineDt, timelineDtC2, HOURLY, now, now);
+    assertTimeline(timelineEt, timelineEtC2, HOURLY, now, now);
+    assertTimeline(timelineIt, timelineItC2, HOURLY, now, now);
+    client.dropPartition(cubeFact.getName(), c1, timeParts1, null, HOURLY);
+    client.dropPartition(cubeFact.getName(), c2, timeParts1, null, HOURLY);
+    assertSameTimelines(factName, storages, HOURLY, partColNames);
     Assert.assertTrue(timelineDt.isEmpty());
     Assert.assertTrue(timelineEt.isEmpty());
     Assert.assertTrue(timelineIt.isEmpty());
@@ -1529,8 +1545,8 @@ public class TestCubeMetastoreClient {
 
     Map<String, Set<UpdatePeriod>> updatePeriods = new HashMap<String, Set<UpdatePeriod>>();
     Set<UpdatePeriod> updates = new HashSet<UpdatePeriod>();
-    updates.add(UpdatePeriod.HOURLY);
-    updates.add(UpdatePeriod.DAILY);
+    updates.add(HOURLY);
+    updates.add(DAILY);
     ArrayList<FieldSchema> partCols = new ArrayList<FieldSchema>();
     List<String> timePartCols = new ArrayList<String>();
     partCols.add(getDatePartition());
@@ -1566,9 +1582,9 @@ public class TestCubeMetastoreClient {
     // test partition
     Map<String, Date> timeParts = new HashMap<String, Date>();
     timeParts.put(TestCubeMetastoreClient.getDatePartitionKey(), now);
-    StoragePartitionDesc partSpec = new StoragePartitionDesc(cubeFact.getName(), timeParts, null, UpdatePeriod.HOURLY);
+    StoragePartitionDesc partSpec = new StoragePartitionDesc(cubeFact.getName(), timeParts, null, HOURLY);
     client.addPartition(partSpec, c1);
-    Assert.assertTrue(client.factPartitionExists(cubeFact.getName(), c1, UpdatePeriod.HOURLY, timeParts,
+    Assert.assertTrue(client.factPartitionExists(cubeFact.getName(), c1, HOURLY, timeParts,
       new HashMap<String, String>()));
     Assert.assertTrue(client.latestPartitionExists(cubeFact.getName(), c1,
       TestCubeMetastoreClient.getDatePartitionKey()));
@@ -1577,8 +1593,8 @@ public class TestCubeMetastoreClient {
     assertEquals(parts.size(), 0);
     assertEquals(client.getAllParts(storageTableName).size(), 1);
 
-    client.dropPartition(cubeFact.getName(), c1, timeParts, null, UpdatePeriod.HOURLY);
-    Assert.assertFalse(client.factPartitionExists(cubeFact.getName(), c1, UpdatePeriod.HOURLY, timeParts,
+    client.dropPartition(cubeFact.getName(), c1, timeParts, null, HOURLY);
+    Assert.assertFalse(client.factPartitionExists(cubeFact.getName(), c1, HOURLY, timeParts,
       new HashMap<String, String>()));
     Assert.assertFalse(client.latestPartitionExists(cubeFact.getName(), c1,
       TestCubeMetastoreClient.getDatePartitionKey()));
@@ -1603,8 +1619,8 @@ public class TestCubeMetastoreClient {
 
     Map<String, Set<UpdatePeriod>> updatePeriods = new HashMap<String, Set<UpdatePeriod>>();
     Set<UpdatePeriod> updates = new HashSet<UpdatePeriod>();
-    updates.add(UpdatePeriod.HOURLY);
-    updates.add(UpdatePeriod.DAILY);
+    updates.add(HOURLY);
+    updates.add(DAILY);
     ArrayList<FieldSchema> partCols = new ArrayList<FieldSchema>();
     List<String> timePartCols = new ArrayList<String>();
     partCols.add(getDatePartition());
@@ -1647,9 +1663,9 @@ public class TestCubeMetastoreClient {
     timeParts.put(TestCubeMetastoreClient.getDatePartitionKey(), now);
     // test partition
     StoragePartitionDesc sPartSpec =
-      new StoragePartitionDesc(cubeFactWithParts.getName(), timeParts, partSpec, UpdatePeriod.HOURLY);
+      new StoragePartitionDesc(cubeFactWithParts.getName(), timeParts, partSpec, HOURLY);
     client.addPartition(sPartSpec, c1);
-    Assert.assertTrue(client.factPartitionExists(cubeFactWithParts.getName(), c1, UpdatePeriod.HOURLY, timeParts,
+    Assert.assertTrue(client.factPartitionExists(cubeFactWithParts.getName(), c1, HOURLY, timeParts,
       partSpec));
     Assert.assertTrue(client.latestPartitionExists(cubeFactWithParts.getName(), c1,
       TestCubeMetastoreClient.getDatePartitionKey()));
@@ -1659,8 +1675,8 @@ public class TestCubeMetastoreClient {
     List<Partition> parts = client.getPartitionsByFilter(storageTableName, "dt='latest'");
     assertEquals(parts.size(), 0);
 
-    client.dropPartition(cubeFactWithParts.getName(), c1, timeParts, partSpec, UpdatePeriod.HOURLY);
-    Assert.assertFalse(client.factPartitionExists(cubeFactWithParts.getName(), c1, UpdatePeriod.HOURLY, timeParts,
+    client.dropPartition(cubeFactWithParts.getName(), c1, timeParts, partSpec, HOURLY);
+    Assert.assertFalse(client.factPartitionExists(cubeFactWithParts.getName(), c1, HOURLY, timeParts,
       partSpec));
     Assert.assertFalse(client.latestPartitionExists(cubeFactWithParts.getName(), c1,
       TestCubeMetastoreClient.getDatePartitionKey()));
@@ -1685,8 +1701,8 @@ public class TestCubeMetastoreClient {
 
     Map<String, Set<UpdatePeriod>> updatePeriods = new HashMap<String, Set<UpdatePeriod>>();
     Set<UpdatePeriod> updates = new HashSet<UpdatePeriod>();
-    updates.add(UpdatePeriod.HOURLY);
-    updates.add(UpdatePeriod.DAILY);
+    updates.add(HOURLY);
+    updates.add(DAILY);
     FieldSchema testDtPart = new FieldSchema("mydate", "string", "date part");
     ArrayList<FieldSchema> partCols = new ArrayList<FieldSchema>();
     List<String> timePartCols = new ArrayList<String>();
@@ -1734,9 +1750,9 @@ public class TestCubeMetastoreClient {
     timeParts.put(testDtPart.getName(), testDt);
     // test partition
     StoragePartitionDesc sPartSpec =
-      new StoragePartitionDesc(cubeFactWithParts.getName(), timeParts, partSpec, UpdatePeriod.HOURLY);
+      new StoragePartitionDesc(cubeFactWithParts.getName(), timeParts, partSpec, HOURLY);
     client.addPartition(sPartSpec, c1);
-    Assert.assertTrue(client.factPartitionExists(cubeFactWithParts.getName(), c1, UpdatePeriod.HOURLY, timeParts,
+    Assert.assertTrue(client.factPartitionExists(cubeFactWithParts.getName(), c1, HOURLY, timeParts,
       partSpec));
     Assert.assertTrue(client.latestPartitionExists(cubeFactWithParts.getName(), c1,
       TestCubeMetastoreClient.getDatePartitionKey()));
@@ -1747,8 +1763,8 @@ public class TestCubeMetastoreClient {
 
     assertNoPartitionNamedLatest(storageTableName, "dt", testDtPart.getName());
 
-    client.dropPartition(cubeFactWithParts.getName(), c1, timeParts, partSpec, UpdatePeriod.HOURLY);
-    Assert.assertFalse(client.factPartitionExists(cubeFactWithParts.getName(), c1, UpdatePeriod.HOURLY, timeParts,
+    client.dropPartition(cubeFactWithParts.getName(), c1, timeParts, partSpec, HOURLY);
+    Assert.assertFalse(client.factPartitionExists(cubeFactWithParts.getName(), c1, HOURLY, timeParts,
       partSpec));
     Assert.assertFalse(client.latestPartitionExists(cubeFactWithParts.getName(), c1, testDtPart.getName()));
     Assert.assertFalse(client.latestPartitionExists(cubeFactWithParts.getName(), c1,
@@ -1774,8 +1790,8 @@ public class TestCubeMetastoreClient {
 
     Map<String, Set<UpdatePeriod>> updatePeriods = new HashMap<String, Set<UpdatePeriod>>();
     Set<UpdatePeriod> updates = new HashSet<UpdatePeriod>();
-    updates.add(UpdatePeriod.HOURLY);
-    updates.add(UpdatePeriod.DAILY);
+    updates.add(HOURLY);
+    updates.add(DAILY);
     ArrayList<FieldSchema> partCols = new ArrayList<FieldSchema>();
     List<String> timePartCols = new ArrayList<String>();
     partCols.add(getDatePartition());
@@ -1823,9 +1839,9 @@ public class TestCubeMetastoreClient {
     timeParts.put(TestCubeMetastoreClient.getDatePartitionKey(), now);
     // test partition
     StoragePartitionDesc sPartSpec =
-      new StoragePartitionDesc(cubeFactWithTwoStorages.getName(), timeParts, partSpec, UpdatePeriod.HOURLY);
+      new StoragePartitionDesc(cubeFactWithTwoStorages.getName(), timeParts, partSpec, HOURLY);
     client.addPartition(sPartSpec, c1);
-    Assert.assertTrue(client.factPartitionExists(cubeFactWithTwoStorages.getName(), c1, UpdatePeriod.HOURLY, timeParts,
+    Assert.assertTrue(client.factPartitionExists(cubeFactWithTwoStorages.getName(), c1, HOURLY, timeParts,
       partSpec));
     Assert.assertTrue(client.latestPartitionExists(cubeFactWithTwoStorages.getName(), c1,
       TestCubeMetastoreClient.getDatePartitionKey()));
@@ -1835,9 +1851,9 @@ public class TestCubeMetastoreClient {
     assertNoPartitionNamedLatest(storageTableName, "dt");
 
     StoragePartitionDesc sPartSpec2 =
-      new StoragePartitionDesc(cubeFactWithTwoStorages.getName(), timeParts, null, UpdatePeriod.HOURLY);
+      new StoragePartitionDesc(cubeFactWithTwoStorages.getName(), timeParts, null, HOURLY);
     client.addPartition(sPartSpec2, c2);
-    Assert.assertTrue(client.factPartitionExists(cubeFactWithTwoStorages.getName(), c2, UpdatePeriod.HOURLY, timeParts,
+    Assert.assertTrue(client.factPartitionExists(cubeFactWithTwoStorages.getName(), c2, HOURLY, timeParts,
       new HashMap<String, String>()));
     Assert.assertTrue(client.latestPartitionExists(cubeFactWithTwoStorages.getName(), c2,
       TestCubeMetastoreClient.getDatePartitionKey()));
@@ -1846,15 +1862,15 @@ public class TestCubeMetastoreClient {
 
     assertNoPartitionNamedLatest(storageTableName2, "dt");
 
-    client.dropPartition(cubeFactWithTwoStorages.getName(), c1, timeParts, partSpec, UpdatePeriod.HOURLY);
-    Assert.assertFalse(client.factPartitionExists(cubeFactWithTwoStorages.getName(), c1, UpdatePeriod.HOURLY,
+    client.dropPartition(cubeFactWithTwoStorages.getName(), c1, timeParts, partSpec, HOURLY);
+    Assert.assertFalse(client.factPartitionExists(cubeFactWithTwoStorages.getName(), c1, HOURLY,
       timeParts, partSpec));
     Assert.assertFalse(client.latestPartitionExists(cubeFactWithTwoStorages.getName(), c1,
       TestCubeMetastoreClient.getDatePartitionKey()));
     assertEquals(client.getAllParts(storageTableName).size(), 0);
 
-    client.dropPartition(cubeFactWithTwoStorages.getName(), c2, timeParts, null, UpdatePeriod.HOURLY);
-    Assert.assertFalse(client.factPartitionExists(cubeFactWithTwoStorages.getName(), c2, UpdatePeriod.HOURLY,
+    client.dropPartition(cubeFactWithTwoStorages.getName(), c2, timeParts, null, HOURLY);
+    Assert.assertFalse(client.factPartitionExists(cubeFactWithTwoStorages.getName(), c2, HOURLY,
       timeParts, new HashMap<String, String>()));
     Assert.assertFalse(client.latestPartitionExists(cubeFactWithTwoStorages.getName(), c2,
       TestCubeMetastoreClient.getDatePartitionKey()));
@@ -1881,7 +1897,7 @@ public class TestCubeMetastoreClient {
     s1.setOutputFormat(HiveIgnoreKeyTextOutputFormat.class.getCanonicalName());
     s1.setPartCols(partCols);
     s1.setTimePartCols(timePartCols);
-    dumpPeriods.put(c1, UpdatePeriod.HOURLY);
+    dumpPeriods.put(c1, HOURLY);
 
     Map<String, StorageTableDesc> storageTables = new HashMap<String, StorageTableDesc>();
     storageTables.put(c1, s1);
@@ -1913,7 +1929,7 @@ public class TestCubeMetastoreClient {
     // test partition
     Map<String, Date> timeParts = new HashMap<String, Date>();
     timeParts.put(TestCubeMetastoreClient.getDatePartitionKey(), now);
-    StoragePartitionDesc sPartSpec = new StoragePartitionDesc(cubeDim.getName(), timeParts, null, UpdatePeriod.HOURLY);
+    StoragePartitionDesc sPartSpec = new StoragePartitionDesc(cubeDim.getName(), timeParts, null, HOURLY);
     client.addPartition(sPartSpec, c1);
     Assert.assertTrue(client.dimPartitionExists(cubeDim.getName(), c1, timeParts));
     Assert
@@ -1924,9 +1940,9 @@ public class TestCubeMetastoreClient {
     assertEquals(1, parts.size());
     assertEquals(TextInputFormat.class.getCanonicalName(), parts.get(0).getInputFormatClass().getCanonicalName());
     assertEquals(parts.get(0).getParameters().get(MetastoreUtil.getLatestPartTimestampKey("dt")),
-      UpdatePeriod.HOURLY.format().format(now));
+      HOURLY.format().format(now));
 
-    client.dropPartition(cubeDim.getName(), c1, timeParts, null, UpdatePeriod.HOURLY);
+    client.dropPartition(cubeDim.getName(), c1, timeParts, null, HOURLY);
     Assert.assertFalse(client.dimPartitionExists(cubeDim.getName(), c1, timeParts));
     Assert.assertFalse(client.latestPartitionExists(cubeDim.getName(), c1,
       TestCubeMetastoreClient.getDatePartitionKey()));
@@ -1954,7 +1970,7 @@ public class TestCubeMetastoreClient {
     s1.setOutputFormat(HiveIgnoreKeyTextOutputFormat.class.getCanonicalName());
     s1.setPartCols(partCols);
     s1.setTimePartCols(timePartCols);
-    dumpPeriods.put(c1, UpdatePeriod.HOURLY);
+    dumpPeriods.put(c1, HOURLY);
 
     Map<String, StorageTableDesc> storageTables = new HashMap<String, StorageTableDesc>();
     storageTables.put(c1, s1);
@@ -1993,7 +2009,7 @@ public class TestCubeMetastoreClient {
     Assert.assertFalse(client.dimTableLatestPartitionExists(storageTableName));
     Map<String, Date> timeParts = new HashMap<String, Date>();
     timeParts.put(TestCubeMetastoreClient.getDatePartitionKey(), now);
-    StoragePartitionDesc sPartSpec = new StoragePartitionDesc(cubeDim.getName(), timeParts, null, UpdatePeriod.HOURLY);
+    StoragePartitionDesc sPartSpec = new StoragePartitionDesc(cubeDim.getName(), timeParts, null, HOURLY);
     client.addPartition(sPartSpec, c1);
     Assert.assertTrue(client.dimPartitionExists(cubeDim.getName(), c1, timeParts));
     Assert
@@ -2005,7 +2021,7 @@ public class TestCubeMetastoreClient {
     assertEquals(TextInputFormat.class.getCanonicalName(), parts.get(0).getInputFormatClass().getCanonicalName());
     Assert.assertFalse(parts.get(0).getCols().contains(newcol));
     assertEquals(parts.get(0).getParameters().get(MetastoreUtil.getLatestPartTimestampKey("dt")),
-      UpdatePeriod.HOURLY.format().format(now));
+      HOURLY.format().format(now));
 
     // Partition with different schema
     cubeDim.alterColumn(newcol);
@@ -2014,7 +2030,7 @@ public class TestCubeMetastoreClient {
     Map<String, Date> timeParts2 = new HashMap<String, Date>();
     timeParts2.put(TestCubeMetastoreClient.getDatePartitionKey(), nowPlus1);
     StoragePartitionDesc sPartSpec2 =
-      new StoragePartitionDesc(cubeDim.getName(), timeParts2, null, UpdatePeriod.HOURLY);
+      new StoragePartitionDesc(cubeDim.getName(), timeParts2, null, HOURLY);
     sPartSpec2.setInputFormat(SequenceFileInputFormat.class.getCanonicalName());
     sPartSpec2.setOutputFormat(HiveIgnoreKeyTextOutputFormat.class.getCanonicalName());
     client.addPartition(sPartSpec2, c1);
@@ -2029,11 +2045,11 @@ public class TestCubeMetastoreClient {
       .getCanonicalName());
     Assert.assertTrue(parts.get(0).getCols().contains(newcol));
     assertEquals(parts.get(0).getParameters().get(MetastoreUtil.getLatestPartTimestampKey("dt")),
-      UpdatePeriod.HOURLY.format().format(nowPlus1));
+      HOURLY.format().format(nowPlus1));
 
-    client.dropPartition(cubeDim.getName(), c1, timeParts2, null, UpdatePeriod.HOURLY);
-    Assert.assertTrue(client.dimPartitionExists(cubeDim.getName(), c1, timeParts));
-    Assert.assertFalse(client.dimPartitionExists(cubeDim.getName(), c1, timeParts2));
+    client.dropPartition(cubeDim.getName(), c1, timeParts, null, HOURLY);
+    Assert.assertTrue(client.dimPartitionExists(cubeDim.getName(), c1, timeParts2));
+    Assert.assertFalse(client.dimPartitionExists(cubeDim.getName(), c1, timeParts));
     Assert
       .assertTrue(client.latestPartitionExists(cubeDim.getName(), c1, TestCubeMetastoreClient.getDatePartitionKey()));
     Assert.assertTrue(client.dimTableLatestPartitionExists(storageTableName));
@@ -2041,12 +2057,12 @@ public class TestCubeMetastoreClient {
     assertEquals(1, parts.size());
     assertEquals(TextInputFormat.class.getCanonicalName(), parts.get(0).getInputFormatClass().getCanonicalName());
     assertEquals(parts.get(0).getParameters().get(MetastoreUtil.getLatestPartTimestampKey("dt")),
-      UpdatePeriod.HOURLY.format().format(now));
+      HOURLY.format().format(nowPlus1));
     assertEquals(client.getAllParts(storageTableName).size(), 2);
 
-    client.dropPartition(cubeDim.getName(), c1, timeParts, null, UpdatePeriod.HOURLY);
-    Assert.assertFalse(client.dimPartitionExists(cubeDim.getName(), c1, timeParts));
+    client.dropPartition(cubeDim.getName(), c1, timeParts2, null, HOURLY);
     Assert.assertFalse(client.dimPartitionExists(cubeDim.getName(), c1, timeParts2));
+    Assert.assertFalse(client.dimPartitionExists(cubeDim.getName(), c1, timeParts));
     Assert.assertFalse(client.latestPartitionExists(cubeDim.getName(), c1,
       TestCubeMetastoreClient.getDatePartitionKey()));
     assertEquals(client.getAllParts(storageTableName).size(), 0);
@@ -2094,14 +2110,14 @@ public class TestCubeMetastoreClient {
 
     timeParts.put(TestCubeMetastoreClient.getDatePartitionKey(), now);
     nonTimeParts.put("region", "asia");
-    StoragePartitionDesc sPartSpec = new StoragePartitionDesc(dimName, timeParts, nonTimeParts, UpdatePeriod.HOURLY);
+    StoragePartitionDesc sPartSpec = new StoragePartitionDesc(dimName, timeParts, nonTimeParts, HOURLY);
     client.addPartition(sPartSpec, c3);
     expectedLatestValues.put("asia", now);
     assertLatestForRegions(storageTableName, expectedLatestValues);
 
     timeParts.put(TestCubeMetastoreClient.getDatePartitionKey(), nowMinus1);
     nonTimeParts.put("region", "africa");
-    sPartSpec = new StoragePartitionDesc(dimName, timeParts, nonTimeParts, UpdatePeriod.HOURLY);
+    sPartSpec = new StoragePartitionDesc(dimName, timeParts, nonTimeParts, HOURLY);
     client.addPartition(sPartSpec, c3);
     expectedLatestValues.put("asia", now);
     expectedLatestValues.put("africa", nowMinus1);
@@ -2109,7 +2125,7 @@ public class TestCubeMetastoreClient {
 
     timeParts.put(TestCubeMetastoreClient.getDatePartitionKey(), nowPlus1);
     nonTimeParts.put("region", "africa");
-    sPartSpec = new StoragePartitionDesc(dimName, timeParts, nonTimeParts, UpdatePeriod.HOURLY);
+    sPartSpec = new StoragePartitionDesc(dimName, timeParts, nonTimeParts, HOURLY);
     client.addPartition(sPartSpec, c3);
     expectedLatestValues.put("asia", now);
     expectedLatestValues.put("africa", nowPlus1);
@@ -2117,19 +2133,19 @@ public class TestCubeMetastoreClient {
 
     timeParts.put(TestCubeMetastoreClient.getDatePartitionKey(), nowPlus3);
     nonTimeParts.put("region", "asia");
-    sPartSpec = new StoragePartitionDesc(dimName, timeParts, nonTimeParts, UpdatePeriod.HOURLY);
+    sPartSpec = new StoragePartitionDesc(dimName, timeParts, nonTimeParts, HOURLY);
     client.addPartition(sPartSpec, c3);
     expectedLatestValues.put("asia", nowPlus3);
     expectedLatestValues.put("africa", nowPlus1);
     assertLatestForRegions(storageTableName, expectedLatestValues);
 
-    client.dropPartition(dimName, c3, timeParts, nonTimeParts, UpdatePeriod.HOURLY);
+    client.dropPartition(dimName, c3, timeParts, nonTimeParts, HOURLY);
     expectedLatestValues.put("asia", now);
     expectedLatestValues.put("africa", nowPlus1);
     assertLatestForRegions(storageTableName, expectedLatestValues);
 
     timeParts.put(TestCubeMetastoreClient.getDatePartitionKey(), now);
-    client.dropPartition(dimName, c3, timeParts, nonTimeParts, UpdatePeriod.HOURLY);
+    client.dropPartition(dimName, c3, timeParts, nonTimeParts, HOURLY);
     expectedLatestValues.remove("asia");
     assertLatestForRegions(storageTableName, expectedLatestValues);
 
@@ -2148,7 +2164,7 @@ public class TestCubeMetastoreClient {
     assertEquals(parts.size(), expectedLatestValues.size());
     for (Partition part : parts) {
       assertEquals(MetastoreUtil.getLatestTimeStampOfDimtable(part, getDatePartitionKey()),
-        TimePartition.of(UpdatePeriod.HOURLY, expectedLatestValues.get(part.getSpec().get("region"))).getDate());
+        TimePartition.of(HOURLY, expectedLatestValues.get(part.getSpec().get("region"))).getDate());
     }
   }
 
@@ -2181,7 +2197,7 @@ public class TestCubeMetastoreClient {
     s1.setTimePartCols(timePartCols);
     String[] partColNames = new String[]{getDatePartitionKey(), itPart.getName(), etPart.getName()};
     Map<String, UpdatePeriod> dumpPeriods = new HashMap<String, UpdatePeriod>();
-    dumpPeriods.put(c1, UpdatePeriod.HOURLY);
+    dumpPeriods.put(c1, HOURLY);
 
     Map<String, StorageTableDesc> storageTables = new HashMap<String, StorageTableDesc>();
     storageTables.put(c1, s1);
@@ -2218,7 +2234,7 @@ public class TestCubeMetastoreClient {
     timeParts1.put(itPart.getName(), now);
     timeParts1.put(etPart.getName(), now);
     StoragePartitionDesc partSpec1 = new StoragePartitionDesc(cubeDim.getName(), timeParts1, null,
-      UpdatePeriod.HOURLY);
+      HOURLY);
 
     Map<String, Date> timeParts2 = new HashMap<String, Date>();
     timeParts2.put(TestCubeMetastoreClient.getDatePartitionKey(), now);
@@ -2226,27 +2242,27 @@ public class TestCubeMetastoreClient {
     Map<String, String> nonTimeSpec = new HashMap<String, String>();
     nonTimeSpec.put(itPart.getName(), "default");
     final StoragePartitionDesc partSpec2 = new StoragePartitionDesc(cubeDim.getName(), timeParts2, nonTimeSpec,
-      UpdatePeriod.HOURLY);
+      HOURLY);
 
     Map<String, Date> timeParts3 = new HashMap<String, Date>();
     timeParts3.put(TestCubeMetastoreClient.getDatePartitionKey(), now);
     timeParts3.put(etPart.getName(), now);
     final StoragePartitionDesc partSpec3 = new StoragePartitionDesc(cubeDim.getName(), timeParts3, nonTimeSpec,
-      UpdatePeriod.HOURLY);
+      HOURLY);
 
     client.addPartitions(Arrays.asList(partSpec1, partSpec2, partSpec3), c1);
     String c1TableName = MetastoreUtil.getFactOrDimtableStorageTableName(cubeDim.getName(), c1);
     assertEquals(client.getAllParts(c1TableName).size(), 8);
 
-    assertEquals(getLatestValues(c1TableName, UpdatePeriod.HOURLY, partColNames, null),
-      toPartitionArray(UpdatePeriod.HOURLY, now, now, nowPlus1));
+    assertEquals(getLatestValues(c1TableName, HOURLY, partColNames, null),
+      toPartitionArray(HOURLY, now, now, nowPlus1));
 
     Map<String, Date> timeParts4 = new HashMap<String, Date>();
     timeParts4.put(TestCubeMetastoreClient.getDatePartitionKey(), now);
     timeParts4.put(itPart.getName(), nowPlus1);
     timeParts4.put(etPart.getName(), nowMinus1);
     final StoragePartitionDesc partSpec4 = new StoragePartitionDesc(cubeDim.getName(), timeParts4, null,
-      UpdatePeriod.HOURLY);
+      HOURLY);
 
 
     Map<String, Date> timeParts5 = new HashMap<String, Date>();
@@ -2254,19 +2270,19 @@ public class TestCubeMetastoreClient {
     timeParts5.put(itPart.getName(), nowMinus1);
     timeParts5.put(etPart.getName(), nowMinus2);
     final StoragePartitionDesc partSpec5 = new StoragePartitionDesc(cubeDim.getName(), timeParts5, null,
-      UpdatePeriod.HOURLY);
+      HOURLY);
 
     client.addPartitions(Arrays.asList(partSpec4, partSpec5), c1);
 
     assertEquals(client.getAllParts(c1TableName).size(), 10);
-    assertEquals(getLatestValues(c1TableName, UpdatePeriod.HOURLY, partColNames, null),
-      toPartitionArray(UpdatePeriod.HOURLY, nowPlus1, nowPlus1, nowPlus1));
+    assertEquals(getLatestValues(c1TableName, HOURLY, partColNames, null),
+      toPartitionArray(HOURLY, nowPlus1, nowPlus1, nowPlus1));
     Map<String, Date> timeParts6 = new HashMap<String, Date>();
     timeParts6.put(TestCubeMetastoreClient.getDatePartitionKey(), nowMinus2);
     timeParts6.put(itPart.getName(), nowMinus1);
     timeParts6.put(etPart.getName(), nowMinus2);
     final StoragePartitionDesc partSpec6 = new StoragePartitionDesc(cubeDim.getName(), timeParts6, null,
-      UpdatePeriod.HOURLY);
+      HOURLY);
 
     client.addPartition(partSpec6, c1);
 
@@ -2278,38 +2294,38 @@ public class TestCubeMetastoreClient {
     timeParts7.put(itPart.getName(), nowMinus5);
     timeParts7.put(etPart.getName(), nowMinus5);
     final StoragePartitionDesc partSpec7 = new StoragePartitionDesc(cubeDim.getName(), timeParts7, null,
-      UpdatePeriod.HOURLY);
+      HOURLY);
 
     client.addPartition(partSpec7, c1);
     assertEquals(client.getAllParts(c1TableName).size(), 12);
-    assertEquals(getLatestValues(c1TableName, UpdatePeriod.HOURLY, partColNames, null),
-      toPartitionArray(UpdatePeriod.HOURLY, nowPlus1, nowPlus1, nowPlus1));
+    assertEquals(getLatestValues(c1TableName, HOURLY, partColNames, null),
+      toPartitionArray(HOURLY, nowPlus1, nowPlus1, nowPlus1));
 
-    client.dropPartition(cubeDim.getName(), c1, timeParts5, null, UpdatePeriod.HOURLY);
+    client.dropPartition(cubeDim.getName(), c1, timeParts5, null, HOURLY);
     assertEquals(client.getAllParts(c1TableName).size(), 11);
-    assertEquals(getLatestValues(c1TableName, UpdatePeriod.HOURLY, partColNames, null),
-      toPartitionArray(UpdatePeriod.HOURLY, now, nowPlus1, nowPlus1));
+    assertEquals(getLatestValues(c1TableName, HOURLY, partColNames, null),
+      toPartitionArray(HOURLY, now, nowPlus1, nowPlus1));
 
-    client.dropPartition(cubeDim.getName(), c1, timeParts7, null, UpdatePeriod.HOURLY);
+    client.dropPartition(cubeDim.getName(), c1, timeParts7, null, HOURLY);
     assertEquals(client.getAllParts(c1TableName).size(), 10);
-    assertEquals(getLatestValues(c1TableName, UpdatePeriod.HOURLY, partColNames, null),
-      toPartitionArray(UpdatePeriod.HOURLY, now, nowPlus1, nowPlus1));
+    assertEquals(getLatestValues(c1TableName, HOURLY, partColNames, null),
+      toPartitionArray(HOURLY, now, nowPlus1, nowPlus1));
 
-    client.dropPartition(cubeDim.getName(), c1, timeParts2, nonTimeSpec, UpdatePeriod.HOURLY);
+    client.dropPartition(cubeDim.getName(), c1, timeParts2, nonTimeSpec, HOURLY);
     assertEquals(client.getAllParts(c1TableName).size(), 9);
-    assertEquals(getLatestValues(c1TableName, UpdatePeriod.HOURLY, partColNames, null),
-      toPartitionArray(UpdatePeriod.HOURLY, now, nowPlus1, now));
+    assertEquals(getLatestValues(c1TableName, HOURLY, partColNames, null),
+      toPartitionArray(HOURLY, now, nowPlus1, now));
 
-    client.dropPartition(cubeDim.getName(), c1, timeParts4, null, UpdatePeriod.HOURLY);
+    client.dropPartition(cubeDim.getName(), c1, timeParts4, null, HOURLY);
     assertEquals(client.getAllParts(c1TableName).size(), 8);
-    assertEquals(getLatestValues(c1TableName, UpdatePeriod.HOURLY, partColNames, null),
-      toPartitionArray(UpdatePeriod.HOURLY, now, now, now));
+    assertEquals(getLatestValues(c1TableName, HOURLY, partColNames, null),
+      toPartitionArray(HOURLY, now, now, now));
 
-    client.dropPartition(cubeDim.getName(), c1, timeParts3, nonTimeSpec, UpdatePeriod.HOURLY);
+    client.dropPartition(cubeDim.getName(), c1, timeParts3, nonTimeSpec, HOURLY);
     assertEquals(client.getAllParts(c1TableName).size(), 5);
-    client.dropPartition(cubeDim.getName(), c1, timeParts6, null, UpdatePeriod.HOURLY);
+    client.dropPartition(cubeDim.getName(), c1, timeParts6, null, HOURLY);
     assertEquals(client.getAllParts(c1TableName).size(), 4);
-    client.dropPartition(cubeDim.getName(), c1, timeParts1, null, UpdatePeriod.HOURLY);
+    client.dropPartition(cubeDim.getName(), c1, timeParts1, null, HOURLY);
     assertEquals(client.getAllParts(c1TableName).size(), 0);
     assertNoPartitionNamedLatest(c1TableName, partColNames);
   }
@@ -2331,7 +2347,7 @@ public class TestCubeMetastoreClient {
     s1.setOutputFormat(HiveIgnoreKeyTextOutputFormat.class.getCanonicalName());
     s1.setPartCols(partCols);
     s1.setTimePartCols(timePartCols);
-    dumpPeriods.put(c1, UpdatePeriod.HOURLY);
+    dumpPeriods.put(c1, HOURLY);
 
     Map<String, StorageTableDesc> storageTables = new HashMap<String, StorageTableDesc>();
     storageTables.put(c1, s1);
@@ -2418,7 +2434,7 @@ public class TestCubeMetastoreClient {
     s2.setInputFormat(TextInputFormat.class.getCanonicalName());
     s2.setOutputFormat(HiveIgnoreKeyTextOutputFormat.class.getCanonicalName());
     client.addStorage(dimTable, c2, null, s2);
-    client.addStorage(dimTable, c3, UpdatePeriod.DAILY, s1);
+    client.addStorage(dimTable, c3, DAILY, s1);
     Assert.assertTrue(client.tableExists(MetastoreUtil.getFactOrDimtableStorageTableName(dimTblName, c2)));
     Assert.assertTrue(client.tableExists(MetastoreUtil.getFactOrDimtableStorageTableName(dimTblName, c3)));
     CubeDimensionTable altered3 = client.getDimensionTable(dimTblName);
@@ -2501,7 +2517,7 @@ public class TestCubeMetastoreClient {
     s1.setOutputFormat(HiveIgnoreKeyTextOutputFormat.class.getCanonicalName());
     s1.setPartCols(partCols);
     s1.setTimePartCols(timePartCols);
-    dumpPeriods.put(c1, UpdatePeriod.HOURLY);
+    dumpPeriods.put(c1, HOURLY);
 
     StorageTableDesc s2 = new StorageTableDesc();
     s2.setInputFormat(TextInputFormat.class.getCanonicalName());

http://git-wip-us.apache.org/repos/asf/lens/blob/e5691d8d/lens-server-api/src/main/java/org/apache/lens/server/api/error/LensException.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/error/LensException.java b/lens-server-api/src/main/java/org/apache/lens/server/api/error/LensException.java
index ac1c558..a1ffeb6 100644
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/error/LensException.java
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/error/LensException.java
@@ -55,6 +55,7 @@ public class LensException extends Exception implements Comparable<LensException
 
   @Getter
   private final LensErrorInfo errorInfo;
+  private String formattedErrorMsg;
 
   public int getErrorCode() {
     return errorInfo.getErrorCode();
@@ -202,12 +203,17 @@ public class LensException extends Exception implements Comparable<LensException
 
   private LensErrorTO buildLensErrorTO(final ErrorCollection errorCollection, final LensError lensError) {
 
-    final String formattedErrorMsg = getFormattedErrorMsg(lensError);
+    formattedErrorMsg = getFormattedErrorMsg(lensError);
     final String stackTrace = getStackTraceString();
     return buildLensErrorTO(errorCollection, formattedErrorMsg, stackTrace);
   }
 
   @Override
+  public String getMessage() {
+    return formattedErrorMsg != null ? formattedErrorMsg : super.getMessage();
+  }
+
+  @Override
   public boolean equals(final Object o) {
 
     if (this == o) {