You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lens.apache.org by pr...@apache.org on 2015/08/11 14:20:42 UTC

[19/50] [abbrv] incubator-lens git commit: LENS-686: Repetitive add partition with same specs should fail after first time

LENS-686: Repetitive add partition with same specs should fail after first time


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

Branch: refs/heads/current-release-line
Commit: 11593f468d39126ffe6dfc27cb7eb6f3493a13cc
Parents: c5c4f76
Author: Rajat Khandelwal <pr...@apache.org>
Authored: Wed Jul 29 06:56:27 2015 +0530
Committer: Raju Bairishetti <ra...@apache.org>
Committed: Wed Jul 29 06:56:27 2015 +0530

----------------------------------------------------------------------
 .../lens/cube/metadata/CubeMetastoreClient.java |  50 ++--
 .../org/apache/lens/cube/metadata/Storage.java  |   9 +-
 .../api/metastore/CubeMetastoreService.java     |  34 +--
 .../metastore/CubeMetastoreServiceImpl.java     |  64 ++---
 .../apache/lens/server/metastore/JAXBUtils.java |  12 +
 .../server/metastore/MetastoreResource.java     | 246 ++++++++-----------
 .../server/metastore/TestMetastoreService.java  |  30 +++
 7 files changed, 218 insertions(+), 227 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/11593f46/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 0ebc707..f0a0dfe 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
@@ -19,6 +19,8 @@
 
 package org.apache.lens.cube.metadata;
 
+import static org.apache.lens.cube.metadata.MetastoreUtil.getFactOrDimtableStorageTableName;
+
 import java.text.ParseException;
 import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
@@ -189,7 +191,7 @@ public class CubeMetastoreClient {
     if (isFactTable(factOrDimtableName)) {
       return Lists.newArrayList();
     }
-    String storageTableName = MetastoreUtil.getFactOrDimtableStorageTableName(factOrDimtableName, storageName);
+    String storageTableName = getFactOrDimtableStorageTableName(factOrDimtableName, storageName);
     Table storageTable = getTable(storageTableName);
     List<String> timePartCols = getTimePartColNamesOfTable(storageTable);
     List<Partition> latestParts = Lists.newArrayList();
@@ -782,23 +784,26 @@ public class CubeMetastoreClient {
    * @param storageName The storage object
    * @throws HiveException
    */
-  public void addPartition(StoragePartitionDesc partSpec, String storageName) throws HiveException, LensException {
-    addPartitions(Collections.singletonList(partSpec), storageName);
+  public List<Partition> addPartition(StoragePartitionDesc partSpec, String storageName)
+    throws HiveException, LensException {
+    return addPartitions(Collections.singletonList(partSpec), storageName);
   }
 
   /** batch addition */
-  public void addPartitions(List<StoragePartitionDesc> storagePartitionDescs, String storageName)
+  public List<Partition> addPartitions(List<StoragePartitionDesc> storagePartitionDescs, String storageName)
     throws HiveException, LensException {
+    List<Partition> partsAdded = Lists.newArrayList();
     for (Map.Entry<String, Map<UpdatePeriod, List<StoragePartitionDesc>>> group : groupPartitionDescs(
       storagePartitionDescs).entrySet()) {
       String factOrDimtable = group.getKey();
       for (Map.Entry<UpdatePeriod, List<StoragePartitionDesc>> entry : group.getValue().entrySet()) {
-        addPartitions(factOrDimtable, storageName, entry.getKey(), entry.getValue());
+        partsAdded.addAll(addPartitions(factOrDimtable, storageName, entry.getKey(), entry.getValue()));
       }
     }
+    return partsAdded;
   }
 
-  private void addPartitions(String factOrDimTable, String storageName, UpdatePeriod updatePeriod,
+  private List<Partition> addPartitions(String factOrDimTable, String storageName, UpdatePeriod updatePeriod,
     List<StoragePartitionDesc> storagePartitionDescs) throws HiveException, LensException {
     String storageTableName = MetastoreUtil.getStorageTableName(factOrDimTable.trim(),
       Storage.getPrefix(storageName.trim())).toLowerCase();
@@ -810,17 +815,27 @@ public class CubeMetastoreClient {
         latestInfos.put(entry.getKey(),
           getDimTableLatestInfo(storageTableName, entry.getKey(), getTimePartSpecs(entry.getValue()), updatePeriod));
       }
-      getStorage(storageName).addPartitions(getClient(), factOrDimTable, updatePeriod, storagePartitionDescs,
-        latestInfos);
+      List<Partition> partsAdded =
+        getStorage(storageName).addPartitions(getClient(), factOrDimTable, updatePeriod, storagePartitionDescs,
+          latestInfos);
+      ListIterator<Partition> iter = partsAdded.listIterator();
+      while (iter.hasNext()) {
+        if (iter.next().getSpec().values().contains(StorageConstants.LATEST_PARTITION_VALUE)) {
+          iter.remove();
+        }
+      }
       latestLookupCache.add(storageTableName);
+      return partsAdded;
     } else {
       // first update in memory, then add to hive table's partitions. delete is reverse.
       partitionTimelineCache.updateForAddition(factOrDimTable, storageName, updatePeriod,
         getTimePartSpecs(storagePartitionDescs));
       // Adding partition in fact table.
-      getStorage(storageName).addPartitions(getClient(), factOrDimTable, updatePeriod, storagePartitionDescs, null);
+      List<Partition> partsAdded =
+        getStorage(storageName).addPartitions(getClient(), factOrDimTable, updatePeriod, storagePartitionDescs, null);
       // update hive table
       alterTablePartitionCache(MetastoreUtil.getStorageTableName(factOrDimTable, Storage.getPrefix(storageName)));
+      return partsAdded;
     }
   }
 
@@ -1112,7 +1127,7 @@ public class CubeMetastoreClient {
 
   public boolean factPartitionExists(String factName, String storageName, UpdatePeriod updatePeriod,
     Map<String, Date> partitionTimestamp, Map<String, String> partSpec) throws HiveException {
-    String storageTableName = MetastoreUtil.getFactOrDimtableStorageTableName(factName, storageName);
+    String storageTableName = getFactOrDimtableStorageTableName(factName, storageName);
     return partitionExists(storageTableName, updatePeriod, partitionTimestamp, partSpec);
   }
 
@@ -1185,7 +1200,7 @@ public class CubeMetastoreClient {
 
   boolean dimPartitionExists(String dimTblName, String storageName, Map<String, Date> partitionTimestamps)
     throws HiveException {
-    String storageTableName = MetastoreUtil.getFactOrDimtableStorageTableName(dimTblName, storageName);
+    String storageTableName = getFactOrDimtableStorageTableName(dimTblName, storageName);
     return partitionExists(storageTableName, getDimensionTable(dimTblName).getSnapshotDumpPeriods().get(storageName),
       partitionTimestamps);
   }
@@ -1232,7 +1247,10 @@ public class CubeMetastoreClient {
   public Table getHiveTable(String tableName) throws HiveException {
     return getTable(tableName);
   }
-
+  public List<String> getTimePartColNamesOfTable(String tblName, String storageName) throws HiveException {
+    return getTimePartColNamesOfTable(getFactOrDimtableStorageTableName(tblName,
+      storageName));
+  }
   public List<String> getTimePartColNamesOfTable(String storageTableName) throws HiveException {
     return getTimePartColNamesOfTable(getTable(storageTableName));
   }
@@ -1449,7 +1467,7 @@ public class CubeMetastoreClient {
               if (dimTable.getStorages() != null && !dimTable.getStorages().isEmpty()) {
                 for (String storageName : dimTable.getStorages()) {
                   if (dimTable.hasStorageSnapshots(storageName)) {
-                    String storageTableName = MetastoreUtil.getFactOrDimtableStorageTableName(dimTable.getName(),
+                    String storageTableName = getFactOrDimtableStorageTableName(dimTable.getName(),
                       storageName);
                     if (dimLatestPartitionExists(storageTableName,
                       getDimension(dimTable.getDimName()).getTimedDimension())) {
@@ -2046,7 +2064,7 @@ public class CubeMetastoreClient {
   public void dropStorageFromFact(String factName, String storage) throws HiveException {
     CubeFactTable cft = getFactTable(factName);
     cft.dropStorage(storage);
-    dropHiveTable(MetastoreUtil.getFactOrDimtableStorageTableName(factName, storage));
+    dropHiveTable(getFactOrDimtableStorageTableName(factName, storage));
     alterCubeTable(factName, getTable(factName), cft);
     updateFactCache(factName);
   }
@@ -2054,7 +2072,7 @@ public class CubeMetastoreClient {
   // updateFact will be false when fact is fully dropped
   private void dropStorageFromFact(String factName, String storage, boolean updateFact) throws HiveException {
     CubeFactTable cft = getFactTable(factName);
-    dropHiveTable(MetastoreUtil.getFactOrDimtableStorageTableName(factName, storage));
+    dropHiveTable(getFactOrDimtableStorageTableName(factName, storage));
     if (updateFact) {
       cft.dropStorage(storage);
       alterCubeTable(factName, getTable(factName), cft);
@@ -2076,7 +2094,7 @@ public class CubeMetastoreClient {
   // updateDimTbl will be false when dropping dimTbl
   private void dropStorageFromDim(String dimTblName, String storage, boolean updateDimTbl) throws HiveException {
     CubeDimensionTable cdt = getDimensionTable(dimTblName);
-    String storageTableName = MetastoreUtil.getFactOrDimtableStorageTableName(dimTblName, storage);
+    String storageTableName = getFactOrDimtableStorageTableName(dimTblName, storage);
     dropHiveTable(storageTableName);
     latestLookupCache.remove(storageTableName.trim().toLowerCase());
     if (updateDimTbl) {

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/11593f46/lens-cube/src/main/java/org/apache/lens/cube/metadata/Storage.java
----------------------------------------------------------------------
diff --git a/lens-cube/src/main/java/org/apache/lens/cube/metadata/Storage.java b/lens-cube/src/main/java/org/apache/lens/cube/metadata/Storage.java
index 873aa49..437227c 100644
--- a/lens-cube/src/main/java/org/apache/lens/cube/metadata/Storage.java
+++ b/lens-cube/src/main/java/org/apache/lens/cube/metadata/Storage.java
@@ -239,11 +239,11 @@ public abstract class Storage extends AbstractCubeTable implements PartitionMeta
    * @param latestInfo
    * @throws HiveException
    */
-  public void addPartition(Hive client, StoragePartitionDesc addPartitionDesc, LatestInfo latestInfo)
+  public List<Partition> addPartition(Hive client, StoragePartitionDesc addPartitionDesc, LatestInfo latestInfo)
     throws HiveException {
     Map<Map<String, String>, LatestInfo> latestInfos = Maps.newHashMap();
     latestInfos.put(addPartitionDesc.getNonTimePartSpec(), latestInfo);
-    addPartitions(client, addPartitionDesc.getCubeTableName(), addPartitionDesc.getUpdatePeriod(),
+    return addPartitions(client, addPartitionDesc.getCubeTableName(), addPartitionDesc.getUpdatePeriod(),
       Collections.singletonList(addPartitionDesc), latestInfos);
   }
 
@@ -258,7 +258,7 @@ public abstract class Storage extends AbstractCubeTable implements PartitionMeta
    *                              column
    * @throws HiveException
    */
-  public void addPartitions(Hive client, String factOrDimTable, UpdatePeriod updatePeriod,
+  public List<Partition> addPartitions(Hive client, String factOrDimTable, UpdatePeriod updatePeriod,
     List<StoragePartitionDesc> storagePartitionDescs,
     Map<Map<String, String>, LatestInfo> latestInfos) throws HiveException {
     preAddPartitions(storagePartitionDescs);
@@ -348,8 +348,9 @@ public abstract class Storage extends AbstractCubeTable implements PartitionMeta
           }
         }
       }
-      client.createPartitions(addParts);
+      List<Partition> partitionsAdded = client.createPartitions(addParts);
       success = true;
+      return partitionsAdded;
     } finally {
       if (success) {
         commitAddPartitions(storagePartitionDescs);

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/11593f46/lens-server-api/src/main/java/org/apache/lens/server/api/metastore/CubeMetastoreService.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/metastore/CubeMetastoreService.java b/lens-server-api/src/main/java/org/apache/lens/server/api/metastore/CubeMetastoreService.java
index d7a033e..c11fd83 100644
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/metastore/CubeMetastoreService.java
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/metastore/CubeMetastoreService.java
@@ -337,14 +337,14 @@ public interface CubeMetastoreService extends LensService {
    * Get all partitions of a dimension table in a storage
    *
    * @param sessionid  The sessionid
-   * @param dimTblName The dimension table name
-   * @param storage    The storage name
+   * @param dimension The dimension table name
+   * @param storageName    The storage name
    * @param filter     The filter for the list of partitions
    * @return list of {@link XPartition}
    * @throws LensException
    */
-  List<XPartition> getAllPartitionsOfDimTableStorage(LensSessionHandle sessionid, String dimTblName,
-    String storage, String filter) throws LensException;
+  XPartitionList getAllPartitionsOfDimTableStorage(LensSessionHandle sessionid, String dimension,
+    String storageName, String filter) throws LensException;
 
   /**
    * Add partition to dimension table on a storage.
@@ -354,9 +354,10 @@ public interface CubeMetastoreService extends LensService {
    * @param storageName The storage name
    * @param partition   {@link XPartition}
    * @throws LensException
+   * @return number of partitions added. Either 0 or 1
    */
-  void addPartitionToDimStorage(LensSessionHandle sessionid, String dimTblName, String storageName,
-     XPartition partition) throws LensException;
+  int addPartitionToDimStorage(LensSessionHandle sessionid, String dimTblName, String storageName,
+    XPartition partition) throws LensException;
 
   /**
    * Add partitions to dimension table on a storage.
@@ -366,9 +367,10 @@ public interface CubeMetastoreService extends LensService {
    * @param storageName The storage name
    * @param partitions  {@link XPartitionList}
    * @throws LensException
+   * @return Number of partitions added
    */
-  void addPartitionsToDimStorage(LensSessionHandle sessionid, String dimTblName, String storageName,
-     XPartitionList partitions) throws LensException;
+  int addPartitionsToDimStorage(LensSessionHandle sessionid, String dimTblName, String storageName,
+    XPartitionList partitions) throws LensException;
 
   /**
    * Get fact table given by name
@@ -475,13 +477,13 @@ public interface CubeMetastoreService extends LensService {
    *
    * @param sessionid The sessionid
    * @param fact      The fact table name
-   * @param storage   The storage name
+   * @param storageName   The storage name
    * @param filter    The filter for partition listing
    * @return List of {@link XPartition}
    * @throws LensException
    */
-  List<XPartition> getAllPartitionsOfFactStorage(LensSessionHandle sessionid, String fact,
-    String storage, String filter) throws LensException;
+  XPartitionList getAllPartitionsOfFactStorage(LensSessionHandle sessionid, String fact,
+    String storageName, String filter) throws LensException;
 
   /**
    * Add partition to fact on a storage
@@ -491,9 +493,10 @@ public interface CubeMetastoreService extends LensService {
    * @param storageName The storage name
    * @param partition   {@link XPartition}
    * @throws LensException
+   * @return number of partitions added. Either 0 or 1
    */
-  void addPartitionToFactStorage(LensSessionHandle sessionid, String fact, String storageName,
-     XPartition partition) throws LensException;
+  int addPartitionToFactStorage(LensSessionHandle sessionid, String fact, String storageName,
+    XPartition partition) throws LensException;
 
   /**
    * Add partitions to fact on a storage
@@ -503,9 +506,10 @@ public interface CubeMetastoreService extends LensService {
    * @param storageName The storage name
    * @param partitions  {@link XPartitionList}
    * @throws LensException
+   * @return Number of partitions added
    */
-  void addPartitionsToFactStorage(LensSessionHandle sessionid, String fact, String storageName,
-     XPartitionList partitions) throws LensException;
+  int addPartitionsToFactStorage(LensSessionHandle sessionid, String fact, String storageName,
+    XPartitionList partitions) throws LensException;
 
   /**
    * Drop partition from storage with spec specified as comma separated string

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/11593f46/lens-server/src/main/java/org/apache/lens/server/metastore/CubeMetastoreServiceImpl.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/metastore/CubeMetastoreServiceImpl.java b/lens-server/src/main/java/org/apache/lens/server/metastore/CubeMetastoreServiceImpl.java
index cb7a530..37cebfe 100644
--- a/lens-server/src/main/java/org/apache/lens/server/metastore/CubeMetastoreServiceImpl.java
+++ b/lens-server/src/main/java/org/apache/lens/server/metastore/CubeMetastoreServiceImpl.java
@@ -18,6 +18,8 @@
  */
 package org.apache.lens.server.metastore;
 
+import static org.apache.lens.server.metastore.JAXBUtils.*;
+
 import java.util.*;
 
 import javax.ws.rs.BadRequestException;
@@ -692,7 +694,7 @@ public class CubeMetastoreServiceImpl extends BaseLensService implements CubeMet
   }
 
   @Override
-  public List<XPartition> getAllPartitionsOfFactStorage(
+  public XPartitionList getAllPartitionsOfFactStorage(
     LensSessionHandle sessionid, String fact, String storageName,
     String filter) throws LensException {
     try {
@@ -703,16 +705,7 @@ public class CubeMetastoreServiceImpl extends BaseLensService implements CubeMet
         storageName);
       List<Partition> parts = client.getPartitionsByFilter(storageTableName, filter);
       List<String> timePartCols = client.getTimePartColNamesOfTable(storageTableName);
-      if (parts != null) {
-        List<XPartition> result = new ArrayList<XPartition>(parts.size());
-        for (Partition p : parts) {
-          XPartition xp = JAXBUtils.xpartitionFromPartition(p, timePartCols);
-          result.add(xp);
-        }
-        return result;
-      } else {
-        return new ArrayList<>();
-      }
+      return xpartitionListFromPartitionList(parts, timePartCols);
     } catch (HiveException exc) {
       throw new LensException(exc);
     } finally {
@@ -721,14 +714,12 @@ public class CubeMetastoreServiceImpl extends BaseLensService implements CubeMet
   }
 
   @Override
-  public void addPartitionToFactStorage(LensSessionHandle sessionid, String fact, String storageName,
+  public int addPartitionToFactStorage(LensSessionHandle sessionid, String fact, String storageName,
     XPartition partition) throws LensException {
     try {
       acquire(sessionid);
       checkFactStorage(sessionid, fact, storageName);
-      getClient(sessionid).addPartition(
-        JAXBUtils.storagePartSpecFromXPartition(partition),
-        storageName);
+      return getClient(sessionid).addPartition(storagePartSpecFromXPartition(partition), storageName).size();
     } catch (HiveException exc) {
       throw new LensException(exc);
     } finally {
@@ -737,14 +728,12 @@ public class CubeMetastoreServiceImpl extends BaseLensService implements CubeMet
   }
 
   @Override
-  public void addPartitionsToFactStorage(LensSessionHandle sessionid, String fact, String storageName,
+  public int addPartitionsToFactStorage(LensSessionHandle sessionid, String fact, String storageName,
     XPartitionList partitions) throws LensException {
     try {
       acquire(sessionid);
       checkFactStorage(sessionid, fact, storageName);
-      getClient(sessionid).addPartitions(
-        JAXBUtils.storagePartSpecListFromXPartitionList(partitions),
-        storageName);
+      return getClient(sessionid).addPartitions(storagePartSpecListFromXPartitionList(partitions), storageName).size();
     } catch (HiveException exc) {
       throw new LensException(exc);
     } finally {
@@ -766,27 +755,17 @@ public class CubeMetastoreServiceImpl extends BaseLensService implements CubeMet
   }
 
   @Override
-  public List<XPartition> getAllPartitionsOfDimTableStorage(
+  public XPartitionList getAllPartitionsOfDimTableStorage(
     LensSessionHandle sessionid, String dimension, String storageName, String filter)
     throws LensException {
     try {
       acquire(sessionid);
       checkDimensionStorage(sessionid, dimension, storageName);
       CubeMetastoreClient client = getClient(sessionid);
-      String storageTableName = MetastoreUtil.getFactOrDimtableStorageTableName(dimension,
-        storageName);
+      String storageTableName = MetastoreUtil.getFactOrDimtableStorageTableName(dimension, storageName);
       List<Partition> partitions = client.getPartitionsByFilter(storageTableName, filter);
       List<String> timePartCols = client.getTimePartColNamesOfTable(storageTableName);
-      if (partitions != null) {
-        List<XPartition> result = new ArrayList<XPartition>(partitions.size());
-        for (Partition p : partitions) {
-          XPartition xp = JAXBUtils.xpartitionFromPartition(p, timePartCols);
-          result.add(xp);
-        }
-        return result;
-      } else {
-        return new ArrayList<XPartition>();
-      }
+      return xpartitionListFromPartitionList(partitions, timePartCols);
     } catch (HiveException exc) {
       throw new LensException(exc);
     } finally {
@@ -795,15 +774,12 @@ public class CubeMetastoreServiceImpl extends BaseLensService implements CubeMet
   }
 
   @Override
-  public void addPartitionToDimStorage(LensSessionHandle sessionid,
+  public int addPartitionToDimStorage(LensSessionHandle sessionid,
     String dimTblName, String storageName, XPartition partition) throws LensException {
     try {
       acquire(sessionid);
       checkDimensionStorage(sessionid, dimTblName, storageName);
-      getClient(sessionid).addPartition(
-        JAXBUtils.storagePartSpecFromXPartition(partition),
-        storageName);
-      log.info("Added partition for dimension: " + dimTblName + " storage: " + storageName);
+      return getClient(sessionid).addPartition(storagePartSpecFromXPartition(partition), storageName).size();
     } catch (HiveException exc) {
       throw new LensException(exc);
     } finally {
@@ -822,7 +798,7 @@ public class CubeMetastoreServiceImpl extends BaseLensService implements CubeMet
         StorageConstants.getPartFilter(JAXBUtils.getFullPartSpecAsMap(xPartition)));
       JAXBUtils.updatePartitionFromXPartition(existingPartition, xPartition);
       client.updatePartition(tblName, storageName, existingPartition);
-    } catch (HiveException | ClassNotFoundException |InvalidOperationException | UnsupportedOperationException exc) {
+    } catch (HiveException | ClassNotFoundException | InvalidOperationException | UnsupportedOperationException exc) {
       throw new LensException(exc);
     } finally {
       release(sessionid);
@@ -852,14 +828,12 @@ public class CubeMetastoreServiceImpl extends BaseLensService implements CubeMet
   }
 
   @Override
-  public void addPartitionsToDimStorage(LensSessionHandle sessionid,
+  public int addPartitionsToDimStorage(LensSessionHandle sessionid,
     String dimTblName, String storageName, XPartitionList partitions) throws LensException {
     try {
       acquire(sessionid);
       checkDimensionStorage(sessionid, dimTblName, storageName);
-      getClient(sessionid).addPartitions(
-        JAXBUtils.storagePartSpecListFromXPartitionList(partitions),
-        storageName);
+      return getClient(sessionid).addPartitions(storagePartSpecListFromXPartitionList(partitions), storageName).size();
     } catch (HiveException exc) {
       throw new LensException(exc);
     } finally {
@@ -1481,7 +1455,7 @@ public class CubeMetastoreServiceImpl extends BaseLensService implements CubeMet
     boolean isHealthy = true;
     StringBuilder details = new StringBuilder();
 
-    try{
+    try {
       /** Try to issue command on hive **/
       Hive.get(LensServerConf.getHiveConf()).getAllDatabases();
     } catch (HiveException e) {
@@ -1498,8 +1472,8 @@ public class CubeMetastoreServiceImpl extends BaseLensService implements CubeMet
     }
 
     return isHealthy
-        ? new HealthStatus(isHealthy, "Cube metastore service is healthy.")
-        : new HealthStatus(isHealthy, details.toString());
+      ? new HealthStatus(isHealthy, "Cube metastore service is healthy.")
+      : new HealthStatus(isHealthy, details.toString());
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/11593f46/lens-server/src/main/java/org/apache/lens/server/metastore/JAXBUtils.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/metastore/JAXBUtils.java b/lens-server/src/main/java/org/apache/lens/server/metastore/JAXBUtils.java
index 2d0eba2..b7c4b40 100644
--- a/lens-server/src/main/java/org/apache/lens/server/metastore/JAXBUtils.java
+++ b/lens-server/src/main/java/org/apache/lens/server/metastore/JAXBUtils.java
@@ -785,6 +785,18 @@ public final class JAXBUtils {
     return nonTimePartSpec;
   }
 
+  public static XPartitionList xpartitionListFromPartitionList(List<Partition> partitions, List<String> timePartCols)
+    throws HiveException {
+    XPartitionList xPartitionList = new XPartitionList();
+    xPartitionList.getPartition();
+    if (partitions != null) {
+      for (Partition partition : partitions) {
+        xPartitionList.getPartition().add(xpartitionFromPartition(partition, timePartCols));
+      }
+    }
+    return xPartitionList;
+  }
+
   public static XPartition xpartitionFromPartition(Partition p, List<String> timePartCols) throws HiveException {
     XPartition xp = new XPartition();
     xp.setPartitionParameters(new XProperties());

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/11593f46/lens-server/src/main/java/org/apache/lens/server/metastore/MetastoreResource.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/metastore/MetastoreResource.java b/lens-server/src/main/java/org/apache/lens/server/metastore/MetastoreResource.java
index d6ef04d..e7ef804 100644
--- a/lens-server/src/main/java/org/apache/lens/server/metastore/MetastoreResource.java
+++ b/lens-server/src/main/java/org/apache/lens/server/metastore/MetastoreResource.java
@@ -18,6 +18,8 @@
  */
 package org.apache.lens.server.metastore;
 
+import static org.apache.lens.api.APIResult.*;
+
 import java.util.List;
 
 import javax.ws.rs.*;
@@ -25,7 +27,7 @@ import javax.ws.rs.core.MediaType;
 import javax.xml.bind.JAXBElement;
 
 import org.apache.lens.api.*;
-import org.apache.lens.api.APIResult.Status;
+import org.apache.lens.api.APIResult.*;
 import org.apache.lens.api.metastore.*;
 import org.apache.lens.server.LensServices;
 import org.apache.lens.server.api.error.LensException;
@@ -37,7 +39,6 @@ import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.glassfish.jersey.media.multipart.FormDataParam;
 
 import com.google.common.collect.Lists;
-
 import lombok.extern.slf4j.Slf4j;
 
 /**
@@ -49,7 +50,6 @@ import lombok.extern.slf4j.Slf4j;
 @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
 @Slf4j
 public class MetastoreResource {
-  public static final APIResult SUCCESS = new APIResult(Status.SUCCEEDED, "");
   public static final ObjectFactory X_CUBE_OBJECT_FACTORY = new ObjectFactory();
 
   public CubeMetastoreService getSvc() {
@@ -135,11 +135,11 @@ public class MetastoreResource {
     log.info("Set database:{}", dbName);
     try {
       getSvc().setCurrentDatabase(sessionid, dbName);
+      return success();
     } catch (LensException e) {
       log.error("Error changing current database", e);
-      return new APIResult(Status.FAILED, e.getMessage());
+      return failure(e);
     }
-    return SUCCESS;
   }
 
   /**
@@ -160,11 +160,11 @@ public class MetastoreResource {
     log.info("Drop database {} cascade?{}", dbName, cascade);
     try {
       getSvc().dropDatabase(sessionid, dbName, cascade);
+      return success();
     } catch (LensException e) {
       log.error("Error dropping {}", dbName, e);
-      return new APIResult(Status.FAILED, e.getMessage());
+      return failure(e);
     }
-    return SUCCESS;
   }
 
   /**
@@ -186,11 +186,11 @@ public class MetastoreResource {
 
     try {
       getSvc().createDatabase(sessionid, dbName, ignoreIfExisting);
+      return success();
     } catch (LensException e) {
       log.error("Error creating database {}", dbName, e);
-      return new APIResult(Status.FAILED, e.getMessage());
+      return failure(e);
     }
-    return SUCCESS;
   }
 
   /**
@@ -289,8 +289,8 @@ public class MetastoreResource {
   @Path("cubes")
   public APIResult deleteAllCubes(@QueryParam("sessionid") LensSessionHandle sessionid) {
     checkSessionId(sessionid);
-    boolean failed = false;
     List<String> cubeNames = null;
+    LensException exc = null;
     int numDeleted = 0;
     try {
       cubeNames = getSvc().getAllCubeNames(sessionid);
@@ -300,21 +300,9 @@ public class MetastoreResource {
       }
     } catch (LensException e) {
       log.error("Error deleting cubes:", e);
-      failed = true;
-    }
-    if (cubeNames != null && numDeleted == cubeNames.size()) {
-      return new APIResult(Status.SUCCEEDED, "Delete of all "
-        + "cubes is successful");
-    } else {
-      assert (failed);
-      if (numDeleted == 0) {
-        return new APIResult(Status.FAILED, "Delete of all "
-          + "cubes has failed");
-      } else {
-        return new APIResult(Status.PARTIAL, "Delete of all "
-          + "cubes is partial");
-      }
+      exc = e;
     }
+    return successOrPartialOrFailure(numDeleted, cubeNames.size(), exc);
   }
 
   /**
@@ -337,9 +325,9 @@ public class MetastoreResource {
         checkTableNotFound(e, ((XDerivedCube) cube).getParent());
       }
       log.error("Error creating cube {}", cube.getName(), e);
-      return new APIResult(Status.FAILED, e.getMessage());
+      return failure(e);
     }
-    return SUCCESS;
+    return success();
   }
 
 
@@ -379,9 +367,9 @@ public class MetastoreResource {
       }
       checkTableNotFound(e, cube.getName());
       log.error("Error updating cube {}", cube.getName(), e);
-      return new APIResult(Status.FAILED, e.getMessage());
+      return failure(e);
     }
-    return SUCCESS;
+    return success();
   }
 
   /**
@@ -423,9 +411,9 @@ public class MetastoreResource {
     } catch (LensException e) {
       checkTableNotFound(e, cubeName);
       log.error("Error droping cube {}", cubeName, e);
-      return new APIResult(Status.FAILED, e.getMessage());
+      return failure(e);
     }
-    return SUCCESS;
+    return success();
   }
 
   /**
@@ -463,9 +451,9 @@ public class MetastoreResource {
       getSvc().createStorage(sessionid, storage);
     } catch (LensException e) {
       log.error("Error creating storage {}", storage.getName(), e);
-      return new APIResult(Status.FAILED, e.getMessage());
+      return failure(e);
     }
-    return SUCCESS;
+    return success();
   }
 
   /**
@@ -479,8 +467,8 @@ public class MetastoreResource {
   @Path("storages")
   public APIResult deleteAllStorages(@QueryParam("sessionid") LensSessionHandle sessionid) {
     checkSessionId(sessionid);
-    boolean failed = false;
     List<String> storageNames = null;
+    LensException exc = null;
     int numDeleted = 0;
     try {
       storageNames = getSvc().getAllStorageNames(sessionid);
@@ -490,21 +478,10 @@ public class MetastoreResource {
       }
     } catch (LensException e) {
       log.error("Error deleting storages:", e);
-      failed = true;
-    }
-    if (storageNames != null && numDeleted == storageNames.size()) {
-      return new APIResult(Status.SUCCEEDED, "Delete of all "
-        + "storages is successful");
-    } else {
-      assert (failed);
-      if (numDeleted == 0) {
-        return new APIResult(Status.FAILED, "Delete of all "
-          + "storages has failed");
-      } else {
-        return new APIResult(Status.PARTIAL, "Delete of all "
-          + "storages is partial");
-      }
+      exc = e;
     }
+    assert (numDeleted == storageNames.size() || exc != null);
+    return successOrPartialOrFailure(numDeleted, storageNames.size(), exc);
   }
 
   /**
@@ -526,9 +503,9 @@ public class MetastoreResource {
     } catch (LensException e) {
       checkTableNotFound(e, storageName);
       log.error("Error updating storage {}", storageName, e);
-      return new APIResult(Status.FAILED, e.getMessage());
+      return failure(e);
     }
-    return SUCCESS;
+    return success();
   }
 
   /**
@@ -569,9 +546,9 @@ public class MetastoreResource {
     } catch (LensException e) {
       checkTableNotFound(e, storageName);
       log.error("Error dropping storage {}", storageName, e);
-      return new APIResult(Status.FAILED, e.getMessage());
+      return failure(e);
     }
-    return SUCCESS;
+    return success();
   }
 
   /**
@@ -609,9 +586,9 @@ public class MetastoreResource {
       getSvc().createDimension(sessionid, dimension);
     } catch (LensException e) {
       log.error("Error creating dimension {}", dimension.getName(), e);
-      return new APIResult(Status.FAILED, e.getMessage());
+      return failure(e);
     }
-    return SUCCESS;
+    return success();
   }
 
   /**
@@ -625,9 +602,9 @@ public class MetastoreResource {
   @Path("dimensions")
   public APIResult deleteAllDimensions(@QueryParam("sessionid") LensSessionHandle sessionid) {
     checkSessionId(sessionid);
-    boolean failed = false;
     List<String> dimNames = null;
     int numDeleted = 0;
+    LensException exc = null;
     try {
       dimNames = getSvc().getAllDimensionNames(sessionid);
       for (String dimName : dimNames) {
@@ -636,21 +613,9 @@ public class MetastoreResource {
       }
     } catch (LensException e) {
       log.error("Error deleting dimensions:", e);
-      failed = true;
-    }
-    if (dimNames != null && numDeleted == dimNames.size()) {
-      return new APIResult(Status.SUCCEEDED, "Delete of all "
-        + "dimensions is successful");
-    } else {
-      assert (failed);
-      if (numDeleted == 0) {
-        return new APIResult(Status.FAILED, "Delete of all "
-          + "dimensions has failed");
-      } else {
-        return new APIResult(Status.PARTIAL, "Delete of all "
-          + "dimensions is partial");
-      }
+      exc = e;
     }
+    return successOrPartialOrFailure(numDeleted, dimNames.size(), exc);
   }
 
   /**
@@ -672,9 +637,9 @@ public class MetastoreResource {
     } catch (LensException e) {
       checkTableNotFound(e, dimName);
       log.error("Error updating dimension {}", dimName, e);
-      return new APIResult(Status.FAILED, e.getMessage());
+      return failure(e);
     }
-    return SUCCESS;
+    return success();
   }
 
   /**
@@ -715,9 +680,9 @@ public class MetastoreResource {
     } catch (LensException e) {
       checkTableNotFound(e, dimName);
       log.error("Error dropping dimension {}", dimName, e);
-      return new APIResult(Status.FAILED, e.getMessage());
+      return failure(e);
     }
-    return SUCCESS;
+    return success();
   }
 
   /**
@@ -789,9 +754,9 @@ public class MetastoreResource {
   public APIResult deleteAllFacts(@QueryParam("sessionid") LensSessionHandle sessionid,
     @DefaultValue("false") @QueryParam("cascade") boolean cascade) {
     checkSessionId(sessionid);
-    boolean failed = false;
     List<String> factNames = null;
     int numDeleted = 0;
+    LensException exc = null;
     try {
       factNames = getSvc().getAllCubeNames(sessionid);
       for (String factName : factNames) {
@@ -800,21 +765,9 @@ public class MetastoreResource {
       }
     } catch (LensException e) {
       log.error("Error deleting cubes:", e);
-      failed = true;
-    }
-    if (factNames != null && numDeleted == factNames.size()) {
-      return new APIResult(Status.SUCCEEDED, "Delete of all "
-        + "fact tables is successful");
-    } else {
-      assert (failed);
-      if (numDeleted == 0) {
-        return new APIResult(Status.FAILED, "Delete of all "
-          + "fact tables has failed");
-      } else {
-        return new APIResult(Status.PARTIAL, "Delete of all "
-          + "fact tables is partial");
-      }
+      exc = e;
     }
+    return successOrPartialOrFailure(numDeleted, factNames.size(), exc);
   }
 
   /**
@@ -858,9 +811,9 @@ public class MetastoreResource {
       getSvc().createFactTable(sessionid, fact);
     } catch (LensException exc) {
       log.error("Exception creating fact:", exc);
-      return new APIResult(Status.FAILED, exc.getMessage());
+      return failure(exc);
     }
-    return SUCCESS;
+    return success();
   }
 
   /**
@@ -883,9 +836,9 @@ public class MetastoreResource {
     } catch (LensException exc) {
       checkTableNotFound(exc, factName);
       log.error("Error updating fact {}", factName, exc);
-      return new APIResult(Status.FAILED, exc.getMessage());
+      return failure(exc);
     }
-    return SUCCESS;
+    return success();
   }
 
   /**
@@ -909,9 +862,9 @@ public class MetastoreResource {
     } catch (LensException exc) {
       checkTableNotFound(exc, factName);
       log.error("Error dropping fact {}", factName, exc);
-      return new APIResult(Status.FAILED, exc.getMessage());
+      return failure(exc);
     }
-    return SUCCESS;
+    return success();
   }
 
   /**
@@ -953,9 +906,9 @@ public class MetastoreResource {
     } catch (LensException exc) {
       checkTableNotFound(exc, factName);
       log.error("Error dropping storages of fact {}", factName, exc);
-      return new APIResult(Status.FAILED, exc.getMessage());
+      return failure(exc);
     }
-    return SUCCESS;
+    return success();
   }
 
   /**
@@ -978,9 +931,9 @@ public class MetastoreResource {
     } catch (LensException exc) {
       checkTableNotFound(exc, factName);
       log.error("Error adding storage to fact {}", factName, exc);
-      return new APIResult(Status.FAILED, exc.getMessage());
+      return failure(exc);
     }
-    return SUCCESS;
+    return success();
   }
 
   /**
@@ -1004,9 +957,9 @@ public class MetastoreResource {
     } catch (LensException exc) {
       checkTableNotFound(exc, factName);
       log.error("Error dropping storage of fact {}", factName, exc);
-      return new APIResult(Status.FAILED, exc.getMessage());
+      return failure(exc);
     }
-    return SUCCESS;
+    return success();
   }
 
   /**
@@ -1043,10 +996,8 @@ public class MetastoreResource {
     @QueryParam("filter") String filter) throws LensException {
     checkSessionId(sessionid);
     try {
-      List<XPartition> partitions = getSvc().getAllPartitionsOfFactStorage(sessionid, factName, storage, filter);
-      XPartitionList partList = X_CUBE_OBJECT_FACTORY.createXPartitionList();
-      partList.getPartition().addAll(partitions);
-      return X_CUBE_OBJECT_FACTORY.createXPartitionList(partList);
+      return X_CUBE_OBJECT_FACTORY
+        .createXPartitionList(getSvc().getAllPartitionsOfFactStorage(sessionid, factName, storage, filter));
     } catch (LensException exc) {
       checkTableNotFound(exc, factName);
       throw exc;
@@ -1076,9 +1027,9 @@ public class MetastoreResource {
     } catch (LensException exc) {
       log.warn("Got exception while dropping partition.", exc);
       checkTableNotFound(exc, factName);
-      return new APIResult(Status.PARTIAL, exc.getMessage());
+      return partial(exc);
     }
-    return SUCCESS;
+    return success();
   }
 
   /**
@@ -1100,14 +1051,14 @@ public class MetastoreResource {
     checkSessionId(sessionid);
     checkNonNullArgs("Partition is null", partition);
     try {
-      getSvc().addPartitionToFactStorage(sessionid, factName, storage, partition);
+      return successOrPartialOrFailure(getSvc().addPartitionToFactStorage(sessionid, factName, storage, partition), 1);
     } catch (LensException exc) {
       checkTableNotFound(exc, factName);
       log.error("Error adding partition to storage of fact {}:{}", factName, storage, exc);
-      return new APIResult(Status.FAILED, exc.getMessage());
+      return failure(exc);
     }
-    return SUCCESS;
   }
+
   /**
    * updates an existing partition for a storage of fact
    *
@@ -1132,9 +1083,9 @@ public class MetastoreResource {
     } catch (LensException exc) {
       checkTableNotFound(exc, factName);
       log.error("Error adding partition to storage of fact {}:{}", factName, storage, exc);
-      return new APIResult(Status.FAILED, exc.getMessage());
+      return failure(exc);
     }
-    return SUCCESS;
+    return success();
   }
 
   /**
@@ -1156,14 +1107,15 @@ public class MetastoreResource {
     checkSessionId(sessionid);
     checkNonNullArgs("Partition List is null", partitions);
     try {
-      getSvc().addPartitionsToFactStorage(sessionid, factName, storage, partitions);
+      return successOrPartialOrFailure(getSvc().addPartitionsToFactStorage(sessionid, factName, storage, partitions),
+        partitions.getPartition().size());
     } catch (LensException exc) {
       checkTableNotFound(exc, factName);
       log.error("Error adding partition to storage of fact {}:{}", factName, storage, exc);
-      return new APIResult(Status.FAILED, exc.getMessage());
+      return failure(exc);
     }
-    return SUCCESS;
   }
+
   /**
    * Batch Update partitions for a storage of fact
    *
@@ -1188,9 +1140,9 @@ public class MetastoreResource {
     } catch (LensException exc) {
       checkTableNotFound(exc, factName);
       log.error("Error adding partition to storage of fact {}:{}", factName, storage, exc);
-      return new APIResult(Status.FAILED, exc.getMessage());
+      return failure(exc);
     }
-    return SUCCESS;
+    return success();
   }
 
   /**
@@ -1217,9 +1169,9 @@ public class MetastoreResource {
     } catch (LensException e) {
       checkTableNotFound(e, factName);
       log.error("Error dropping partition to storage of fact {}:{}", factName, storage, e);
-      return new APIResult(Status.FAILED, e.getMessage());
+      return failure(e);
     }
-    return SUCCESS;
+    return success();
   }
 
   /**
@@ -1251,9 +1203,9 @@ public class MetastoreResource {
       getSvc().createDimensionTable(sessionid, dimensionTable);
     } catch (LensException exc) {
       log.error("Error creating cube dimension table {}", dimensionTable.getTableName(), exc);
-      return new APIResult(Status.FAILED, exc.getMessage());
+      return failure(exc);
     }
-    return SUCCESS;
+    return success();
   }
 
   /**
@@ -1275,9 +1227,9 @@ public class MetastoreResource {
     } catch (LensException exc) {
       checkTableNotFound(exc, dimTableName);
       log.error("Error updating cube dimension table {}", dimTableName, exc);
-      return new APIResult(Status.FAILED, exc.getMessage());
+      return failure(exc);
     }
-    return SUCCESS;
+    return success();
   }
 
   /**
@@ -1300,9 +1252,9 @@ public class MetastoreResource {
     } catch (LensException e) {
       checkTableNotFound(e, dimension);
       log.error("Error dropping cube dimension table {}", dimension, e);
-      return new APIResult(Status.FAILED, e.getMessage());
+      return failure(e);
     }
-    return SUCCESS;
+    return success();
   }
 
   /**
@@ -1363,9 +1315,9 @@ public class MetastoreResource {
     } catch (LensException e) {
       checkTableNotFound(e, dimTableName);
       log.error("Error creating dimension table storage {}:{}", dimTableName, storageTbl.getStorageName(), e);
-      return new APIResult(Status.FAILED, e.getMessage());
+      return failure(e);
     }
-    return SUCCESS;
+    return success();
   }
 
   /**
@@ -1403,9 +1355,9 @@ public class MetastoreResource {
     } catch (LensException exc) {
       checkTableNotFound(exc, dimTableName);
       log.error("Error dropping storages of dimension table {}", dimTableName, exc);
-      return new APIResult(Status.FAILED, exc.getMessage());
+      return failure(exc);
     }
-    return SUCCESS;
+    return success();
   }
 
   /**
@@ -1428,9 +1380,9 @@ public class MetastoreResource {
     } catch (LensException exc) {
       checkTableNotFound(exc, dimTableName);
       log.error("Error dropping storage of dimension table {}:{}", dimTableName, storage, exc);
-      return new APIResult(Status.FAILED, exc.getMessage());
+      return failure(exc);
     }
-    return SUCCESS;
+    return success();
   }
 
   /**
@@ -1451,10 +1403,8 @@ public class MetastoreResource {
     @QueryParam("filter") String filter)
     throws LensException {
     checkSessionId(sessionid);
-    List<XPartition> partitions = getSvc().getAllPartitionsOfDimTableStorage(sessionid, dimension, storage, filter);
-    XPartitionList partList = X_CUBE_OBJECT_FACTORY.createXPartitionList();
-    partList.getPartition().addAll(partitions);
-    return X_CUBE_OBJECT_FACTORY.createXPartitionList(partList);
+    return X_CUBE_OBJECT_FACTORY
+      .createXPartitionList(getSvc().getAllPartitionsOfDimTableStorage(sessionid, dimension, storage, filter));
   }
 
   /**
@@ -1479,9 +1429,9 @@ public class MetastoreResource {
       getSvc().dropPartitionFromStorageByFilter(sessionid, dimTableName, storage, filter);
     } catch (LensException exc) {
       log.error("Error dropping partition on storage of dimension table {}:{}", dimTableName, storage, exc);
-      return new APIResult(Status.PARTIAL, exc.getMessage());
+      return partial(exc);
     }
-    return SUCCESS;
+    return success();
   }
 
   /**
@@ -1506,9 +1456,9 @@ public class MetastoreResource {
         values);
     } catch (LensException exc) {
       log.error("Error dropping partitions on storage of dimension table {}:{}", dimTableName, storage, exc);
-      return new APIResult(Status.FAILED, exc.getMessage());
+      return failure(exc);
     }
-    return SUCCESS;
+    return success();
   }
 
   /**
@@ -1530,13 +1480,14 @@ public class MetastoreResource {
     checkSessionId(sessionid);
     checkNonNullArgs("Partition is null", partition);
     try {
-      getSvc().addPartitionToDimStorage(sessionid, dimTableName, storage, partition);
+      return successOrPartialOrFailure(getSvc().addPartitionToDimStorage(sessionid, dimTableName, storage, partition),
+        1);
     } catch (LensException exc) {
       log.error("Error adding partition to storage of dimension table {}:{}", dimTableName, storage, exc);
-      return new APIResult(Status.FAILED, exc.getMessage());
+      return failure(exc);
     }
-    return SUCCESS;
   }
+
   /**
    * Updates an existing partition for a storage of dimension
    *
@@ -1560,9 +1511,9 @@ public class MetastoreResource {
       getSvc().updatePartition(sessionid, dimTableName, storage, partition);
     } catch (LensException exc) {
       log.error("Error adding partition to storage of dimension table {}:{}", dimTableName, storage, exc);
-      return new APIResult(Status.FAILED, exc.getMessage());
+      return failure(exc);
     }
-    return SUCCESS;
+    return success();
   }
 
   /**
@@ -1584,13 +1535,14 @@ public class MetastoreResource {
     checkSessionId(sessionid);
     checkNonNullArgs("Partition list is null", partitions);
     try {
-      getSvc().addPartitionsToDimStorage(sessionid, dimTableName, storage, partitions);
+      return successOrPartialOrFailure(getSvc().addPartitionsToDimStorage(sessionid, dimTableName, storage, partitions),
+        partitions.getPartition().size());
     } catch (LensException exc) {
       log.error("Error adding partition to storage of dimension table {}:{}", dimTableName, storage, exc);
-      return new APIResult(Status.FAILED, exc.getMessage());
+      return failure(exc);
     }
-    return SUCCESS;
   }
+
   /**
    * Add new partitions for a storage of dimension
    *
@@ -1614,9 +1566,9 @@ public class MetastoreResource {
       getSvc().updatePartitions(sessionid, dimTableName, storage, partitions);
     } catch (LensException exc) {
       log.error("Error adding partition to storage of dimension table {}:{}", dimTableName, storage, exc);
-      return new APIResult(Status.FAILED, exc.getMessage());
+      return failure(exc);
     }
-    return SUCCESS;
+    return success();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/11593f46/lens-server/src/test/java/org/apache/lens/server/metastore/TestMetastoreService.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/metastore/TestMetastoreService.java b/lens-server/src/test/java/org/apache/lens/server/metastore/TestMetastoreService.java
index 22dfd55..5edbd13 100644
--- a/lens-server/src/test/java/org/apache/lens/server/metastore/TestMetastoreService.java
+++ b/lens-server/src/test/java/org/apache/lens/server/metastore/TestMetastoreService.java
@@ -1999,6 +1999,12 @@ public class TestMetastoreService extends LensJerseyTest {
         .post(Entity.xml(cubeObjectFactory.createXPartition(xp)), APIResult.class);
       assertEquals(partAddResult.getStatus(), Status.SUCCEEDED);
 
+      // add same should fail
+      partAddResult = target().path("metastore/facts/").path(table).path("storages/S2/partition")
+        .queryParam("sessionid", lensSessionId).request(mediaType)
+        .post(Entity.xml(cubeObjectFactory.createXPartition(xp)), APIResult.class);
+      assertEquals(partAddResult.getStatus(), Status.FAILED);
+
       xp.setLocation(xp.getLocation() + "/a/b/c");
       APIResult partUpdateResult = target().path("metastore/facts/").path(table).path("storages/S2/partition")
         .queryParam("sessionid", lensSessionId).request(mediaType)
@@ -2031,6 +2037,14 @@ public class TestMetastoreService extends LensJerseyTest {
       partDate.setSeconds(0);
       partDate.setTime(partDate.getTime() - partDate.getTime() % 1000);
       assertEquals(date.getDate(), partDate);
+      // add two partitions, one of them already added. result should be partial
+      XPartitionList parts = new XPartitionList();
+      parts.getPartition().add(xp);
+      parts.getPartition().add(createPartition(table, DateUtils.addHours(partDate, 1)));
+      partAddResult = target().path("metastore/facts/").path(table).path("storages/S2/partitions")
+        .queryParam("sessionid", lensSessionId).request(mediaType)
+        .post(Entity.xml(cubeObjectFactory.createXPartitionList(parts)), APIResult.class);
+      assertEquals(partAddResult.getStatus(), Status.PARTIAL);
 
       // Drop the partitions
       APIResult dropResult = target().path("metastore/facts").path(table).path("storages/S2/partitions")
@@ -2118,6 +2132,13 @@ public class TestMetastoreService extends LensJerseyTest {
         .post(Entity.xml(cubeObjectFactory.createXPartition(xp)), APIResult.class);
       assertEquals(partAddResult.getStatus(), Status.SUCCEEDED);
 
+      // create call for same
+      partAddResult = target().path("metastore/dimtables/").path(table).path("storages/test/partition")
+        .queryParam("sessionid", lensSessionId).request(mediaType)
+        .post(Entity.xml(cubeObjectFactory.createXPartition(xp)), APIResult.class);
+      assertEquals(partAddResult.getStatus(), Status.FAILED);
+
+
       xp.setLocation(xp.getLocation() + "/a/b/c");
       APIResult partUpdateResult = target().path("metastore/dimtables/").path(table).path("storages/test/partition")
         .queryParam("sessionid", lensSessionId).request(mediaType)
@@ -2163,6 +2184,15 @@ public class TestMetastoreService extends LensJerseyTest {
       assertEquals(latestPartition.getFullPartitionSpec().getPartSpecElement().get(0).getValue(),
         "latest");
 
+      // create should give partial now:
+      XPartition xp2 = createPartition(table, DateUtils.addHours(partDate, 1));
+      XPartitionList parts = new XPartitionList();
+      parts.getPartition().add(xp);
+      parts.getPartition().add(xp2);
+      partAddResult = target().path("metastore/dimtables/").path(table).path("storages/test/partitions")
+        .queryParam("sessionid", lensSessionId).request(mediaType)
+        .post(Entity.xml(cubeObjectFactory.createXPartitionList(parts)), APIResult.class);
+      assertEquals(partAddResult.getStatus(), Status.PARTIAL);
 
       // Drop the partitions
       APIResult dropResult = target().path("metastore/dimtables").path(table).path("storages/test/partitions")