You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by st...@apache.org on 2018/04/19 16:18:09 UTC

hive git commit: HIVE-19158: Fix NPE in the HiveMetastore add partition tests (Marta Kuczora, reviewed by Peter Vary and Sahil Takiar)

Repository: hive
Updated Branches:
  refs/heads/master fb22f576d -> 1f25c46a2


HIVE-19158: Fix NPE in the HiveMetastore add partition tests (Marta Kuczora, reviewed by Peter Vary and Sahil Takiar)


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

Branch: refs/heads/master
Commit: 1f25c46a2bf50483e09c756803d78e078dc37b92
Parents: fb22f57
Author: Marta Kuczora <ku...@cloudera.com>
Authored: Thu Apr 19 11:06:45 2018 -0500
Committer: Sahil Takiar <st...@cloudera.com>
Committed: Thu Apr 19 11:06:45 2018 -0500

----------------------------------------------------------------------
 .../hadoop/hive/metastore/HiveMetaStore.java    | 26 +++++-
 .../hive/metastore/HiveMetaStoreClient.java     | 11 ++-
 .../spec/CompositePartitionSpecProxy.java       |  6 +-
 .../spec/PartitionListComposingSpecProxy.java   | 22 ++++-
 .../partition/spec/PartitionSpecProxy.java      |  6 +-
 .../spec/PartitionSpecWithSharedSDProxy.java    |  5 +-
 .../metastore/client/TestAddPartitions.java     | 71 ++++++----------
 .../client/TestAddPartitionsFromPartSpec.java   | 89 ++++++--------------
 8 files changed, 116 insertions(+), 120 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/1f25c46a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index ae9ec5c..9c88cf9 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -3250,6 +3250,12 @@ public class HiveMetaStore extends ThriftHiveMetastore {
                 part.getTableName(), part.toString());
             throw new MetaException(errorMsg);
           }
+          if (part.getValues() == null || part.getValues().isEmpty()) {
+            throw new MetaException("The partition values cannot be null or empty.");
+          }
+          if (part.getValues().contains(null)) {
+            throw new MetaException("Partition value cannot be null.");
+          }
 
           boolean shouldAdd = startAddPartition(ms, part, ifNotExists);
           if (!shouldAdd) {
@@ -3410,7 +3416,10 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     public int add_partitions(final List<Partition> parts) throws MetaException,
         InvalidObjectException, AlreadyExistsException {
       startFunction("add_partition");
-      if (parts.size() == 0) {
+      if (parts == null) {
+        throw new MetaException("Partition list cannot be null.");
+      }
+      if (parts.isEmpty()) {
         return 0;
       }
 
@@ -3471,6 +3480,9 @@ public class HiveMetaStore extends ThriftHiveMetastore {
                                           boolean ifNotExists)
         throws TException {
       boolean success = false;
+      if (dbName == null || tblName == null) {
+        throw new MetaException("The database and table name cannot be null.");
+      }
       // Ensures that the list doesn't have dups, and keeps track of directories we have created.
       final Map<PartValEqWrapperLite, Boolean> addedPartitions = new ConcurrentHashMap<>();
       PartitionSpecProxy partitionSpecProxy = PartitionSpecProxy.Factory.get(partSpecs);
@@ -3496,12 +3508,18 @@ public class HiveMetaStore extends ThriftHiveMetastore {
           // will be created if the list contains an invalid partition.
           final Partition part = partitionIterator.getCurrent();
 
+          if (part.getDbName() == null || part.getTableName() == null) {
+            throw new MetaException("The database and table name must be set in the partition.");
+          }
           if (!part.getTableName().equalsIgnoreCase(tblName) || !part.getDbName().equalsIgnoreCase(dbName)) {
             String errorMsg = String.format(
                 "Partition does not belong to target table %s.%s. It belongs to the table %s.%s : %s",
                 dbName, tblName, part.getDbName(), part.getTableName(), part.toString());
             throw new MetaException(errorMsg);
           }
+          if (part.getValues() == null || part.getValues().isEmpty()) {
+            throw new MetaException("The partition values cannot be null or empty.");
+          }
 
           boolean shouldAdd = startAddPartition(ms, part, ifNotExists);
           if (!shouldAdd) {
@@ -3733,6 +3751,9 @@ public class HiveMetaStore extends ThriftHiveMetastore {
 
         firePreEvent(new PreAddPartitionEvent(tbl, part, this));
 
+        if (part.getValues() == null || part.getValues().isEmpty()) {
+          throw new MetaException("The partition values cannot be null or empty.");
+        }
         boolean shouldAdd = startAddPartition(ms, part, false);
         assert shouldAdd; // start would throw if it already existed here
         boolean madeDir = createLocationForAddedPartition(tbl, part);
@@ -3789,6 +3810,9 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         final Partition part, EnvironmentContext envContext)
         throws InvalidObjectException, AlreadyExistsException,
         MetaException {
+      if (part == null) {
+        throw new MetaException("Partition cannot be null.");
+      }
       startTableFunction("add_partition",
           part.getCatName(), part.getDbName(), part.getTableName());
       Partition ret = null;

http://git-wip-us.apache.org/repos/asf/hive/blob/1f25c46a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index 7ba053d..4f686a0 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -649,7 +649,7 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
 
   public Partition add_partition(Partition new_part, EnvironmentContext envContext)
       throws TException {
-    if (!new_part.isSetCatName()) new_part.setCatName(getDefaultCatalog(conf));
+    if (new_part != null && !new_part.isSetCatName()) new_part.setCatName(getDefaultCatalog(conf));
     Partition p = client.add_partition_with_environment_context(new_part, envContext);
     return deepCopy(p);
   }
@@ -664,6 +664,9 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
    */
   @Override
   public int add_partitions(List<Partition> new_parts) throws TException {
+    if (new_parts == null || new_parts.contains(null)) {
+      throw new MetaException("Partitions cannot be null.");
+    }
     if (new_parts != null && !new_parts.isEmpty() && !new_parts.get(0).isSetCatName()) {
       final String defaultCat = getDefaultCatalog(conf);
       new_parts.forEach(p -> p.setCatName(defaultCat));
@@ -674,6 +677,9 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   @Override
   public List<Partition> add_partitions(
       List<Partition> parts, boolean ifNotExists, boolean needResults) throws TException {
+    if (parts == null || parts.contains(null)) {
+      throw new MetaException("Partitions cannot be null.");
+    }
     if (parts.isEmpty()) {
       return needResults ? new ArrayList<>() : null;
     }
@@ -688,6 +694,9 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
 
   @Override
   public int add_partitions_pspec(PartitionSpecProxy partitionSpec) throws TException {
+    if (partitionSpec == null) {
+      throw new MetaException("PartitionSpec cannot be null.");
+    }
     if (partitionSpec.getCatName() == null) partitionSpec.setCatName(getDefaultCatalog(conf));
     return client.add_partitions_pspec(partitionSpec.toPartitionSpec());
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/1f25c46a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/CompositePartitionSpecProxy.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/CompositePartitionSpecProxy.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/CompositePartitionSpecProxy.java
index 92813b9..91d790a 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/CompositePartitionSpecProxy.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/CompositePartitionSpecProxy.java
@@ -40,7 +40,7 @@ public class CompositePartitionSpecProxy extends PartitionSpecProxy {
   private List<PartitionSpecProxy> partitionSpecProxies;
   private int size = 0;
 
-  protected CompositePartitionSpecProxy(List<PartitionSpec> partitionSpecs) {
+  protected CompositePartitionSpecProxy(List<PartitionSpec> partitionSpecs) throws MetaException {
     this.partitionSpecs = partitionSpecs;
     if (partitionSpecs.isEmpty()) {
       catName = null;
@@ -63,13 +63,13 @@ public class CompositePartitionSpecProxy extends PartitionSpecProxy {
   }
 
   @Deprecated
-  protected CompositePartitionSpecProxy(String dbName, String tableName, List<PartitionSpec> partitionSpecs) {
+  protected CompositePartitionSpecProxy(String dbName, String tableName, List<PartitionSpec> partitionSpecs) throws MetaException {
     this(DEFAULT_CATALOG_NAME, dbName, tableName, partitionSpecs);
 
   }
 
   protected CompositePartitionSpecProxy(String catName, String dbName, String tableName,
-                                        List<PartitionSpec> partitionSpecs) {
+                                        List<PartitionSpec> partitionSpecs) throws MetaException {
     this.catName = catName;
     this.dbName = dbName;
     this.tableName = tableName;

http://git-wip-us.apache.org/repos/asf/hive/blob/1f25c46a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionListComposingSpecProxy.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionListComposingSpecProxy.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionListComposingSpecProxy.java
index 6bd29d0..585b8fd 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionListComposingSpecProxy.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionListComposingSpecProxy.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.metastore.partition.spec;
 
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.PartitionListComposingSpec;
 import org.apache.hadoop.hive.metastore.api.PartitionSpec;
 
 import java.util.Arrays;
@@ -33,9 +34,24 @@ public class PartitionListComposingSpecProxy extends PartitionSpecProxy {
 
   private PartitionSpec partitionSpec;
 
-  protected PartitionListComposingSpecProxy(PartitionSpec partitionSpec) {
+  protected PartitionListComposingSpecProxy(PartitionSpec partitionSpec) throws MetaException {
     assert partitionSpec.isSetPartitionList()
         : "Partition-list should have been set.";
+    PartitionListComposingSpec partitionList = partitionSpec.getPartitionList();
+    if (partitionList == null || partitionList.getPartitions() == null) {
+      throw new MetaException("The partition list cannot be null.");
+    }
+    for (Partition partition : partitionList.getPartitions()) {
+      if (partition == null) {
+        throw new MetaException("Partition cannot be null.");
+      }
+      if (partition.getValues() == null || partition.getValues().isEmpty()) {
+        throw new MetaException("The partition value list cannot be null or empty.");
+      }
+      if (partition.getValues().contains(null)) {
+        throw new MetaException("Partition value cannot be null.");
+      }
+    }
     this.partitionSpec = partitionSpec;
   }
 
@@ -102,6 +118,10 @@ public class PartitionListComposingSpecProxy extends PartitionSpecProxy {
       throw new MetaException("No common root-path. Can't replace root-path!");
     }
 
+    if (newRootPath == null) {
+      throw new MetaException("Root path cannot be null.");
+    }
+
     for (Partition partition : partitionSpec.getPartitionList().getPartitions()) {
       String location = partition.getSd().getLocation();
       if (location.startsWith(oldRootPath)) {

http://git-wip-us.apache.org/repos/asf/hive/blob/1f25c46a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionSpecProxy.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionSpecProxy.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionSpecProxy.java
index ff2dea1..1866446 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionSpecProxy.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionSpecProxy.java
@@ -100,8 +100,9 @@ public abstract class PartitionSpecProxy {
      * Factory method. Construct PartitionSpecProxy from raw PartitionSpec.
      * @param partSpec Raw PartitionSpec from the Thrift API.
      * @return PartitionSpecProxy instance.
+     * @throws MetaException
      */
-    public static PartitionSpecProxy get(PartitionSpec partSpec) {
+    public static PartitionSpecProxy get(PartitionSpec partSpec) throws MetaException {
 
       if (partSpec == null) {
         return null;
@@ -123,8 +124,9 @@ public abstract class PartitionSpecProxy {
      * Factory method to construct CompositePartitionSpecProxy.
      * @param partitionSpecs List of raw PartitionSpecs.
      * @return A CompositePartitionSpecProxy instance.
+     * @throws MetaException
      */
-    public static PartitionSpecProxy get(List<PartitionSpec> partitionSpecs) {
+    public static PartitionSpecProxy get(List<PartitionSpec> partitionSpecs) throws MetaException {
       return new CompositePartitionSpecProxy(partitionSpecs);
     }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/1f25c46a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionSpecWithSharedSDProxy.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionSpecWithSharedSDProxy.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionSpecWithSharedSDProxy.java
index 61e00ea..5b46206 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionSpecWithSharedSDProxy.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionSpecWithSharedSDProxy.java
@@ -38,8 +38,11 @@ public class PartitionSpecWithSharedSDProxy extends PartitionSpecProxy {
 
   private PartitionSpec partitionSpec;
 
-  public PartitionSpecWithSharedSDProxy(PartitionSpec partitionSpec) {
+  public PartitionSpecWithSharedSDProxy(PartitionSpec partitionSpec) throws MetaException {
     assert partitionSpec.isSetSharedSDPartitionSpec();
+    if (partitionSpec.getSharedSDPartitionSpec().getSd() == null) {
+      throw new MetaException("The shared storage descriptor must be set.");
+    }
     this.partitionSpec = partitionSpec;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/1f25c46a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestAddPartitions.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestAddPartitions.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestAddPartitions.java
index f8497c7..88064d9 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestAddPartitions.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestAddPartitions.java
@@ -46,7 +46,6 @@ import org.apache.hadoop.hive.metastore.client.builder.PartitionBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
 import org.apache.hadoop.hive.metastore.minihms.AbstractMetaStoreService;
 import org.apache.thrift.TException;
-import org.apache.thrift.transport.TTransportException;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -512,7 +511,7 @@ public class TestAddPartitions extends MetaStoreClientTest {
   @Test(expected = MetaException.class)
   public void testAddPartitionNoPartColOnTable() throws Exception {
 
-    Table origTable = new TableBuilder()
+    new TableBuilder()
         .setDbName(DB_NAME)
         .setTableName(TABLE_NAME)
         .addCol("test_id", "int", "test col id")
@@ -575,27 +574,19 @@ public class TestAddPartitions extends MetaStoreClientTest {
     client.add_partition(partition);
   }
 
-  @Test
+  @Test(expected = MetaException.class)
   public void testAddPartitionNullPartition() throws Exception {
-    try {
-      client.add_partition(null);
-      Assert.fail("Exception should have been thrown.");
-    } catch (TTransportException | NullPointerException e) {
-      // TODO: NPE should not be thrown.
-    }
+
+    client.add_partition(null);
   }
 
-  @Test
-  public void testAddPartitionNullValue() throws Exception {
+  @Test(expected = MetaException.class)
+  public void testAddPartitionNullValues() throws Exception {
 
     createTable();
     Partition partition = buildPartition(DB_NAME, TABLE_NAME, null);
-    try {
-      client.add_partition(partition);
-    } catch (NullPointerException e) {
-      // TODO: This works different in remote and embedded mode.
-      // In embedded mode, no exception happens.
-    }
+    partition.setValues(null);
+    client.add_partition(partition);
   }
 
   @Test
@@ -698,14 +689,10 @@ public class TestAddPartitions extends MetaStoreClientTest {
     verifyPartitionAttributesDefaultValues(part, table.getSd().getLocation());
   }
 
-  @Test
+  @Test(expected = MetaException.class)
   public void testAddPartitionsNullList() throws Exception {
-    try {
-      client.add_partitions(null);
-      Assert.fail("Exception should have been thrown.");
-    } catch (TTransportException | NullPointerException e) {
-      // TODO: NPE should not be thrown
-    }
+
+    client.add_partitions(null);
   }
 
   @Test
@@ -1159,31 +1146,23 @@ public class TestAddPartitions extends MetaStoreClientTest {
     client.add_partitions(partitions);
   }
 
-  @Test
+  @Test(expected = MetaException.class)
   public void testAddPartitionsNullPartition() throws Exception {
-    try {
-      List<Partition> partitions = new ArrayList<>();
-      partitions.add(null);
-      client.add_partitions(partitions);
-      Assert.fail("Exception should have been thrown.");
-    } catch (TTransportException | NullPointerException e) {
-      // TODO: NPE should not be thrown
-    }
+
+    List<Partition> partitions = new ArrayList<>();
+    partitions.add(null);
+    client.add_partitions(partitions);
   }
 
-  @Test
-  public void testAddPartitionsNullValue() throws Exception {
+  @Test(expected = MetaException.class)
+  public void testAddPartitionsNullValues() throws Exception {
 
     createTable();
     Partition partition = buildPartition(DB_NAME, TABLE_NAME, null);
+    partition.setValues(null);
     List<Partition> partitions = new ArrayList<>();
     partitions.add(partition);
-    try {
-      client.add_partitions(partitions);
-    } catch (NullPointerException e) {
-      // TODO: This works different in remote and embedded mode.
-      // In embedded mode, no exception happens.
-    }
+    client.add_partitions(partitions);
   }
 
   @Test
@@ -1313,9 +1292,9 @@ public class TestAddPartitions extends MetaStoreClientTest {
     verifyPartition(table, "year=2016/month=march", Lists.newArrayList("2016", "march"), 3);
   }
 
-  @Test(expected = NullPointerException.class)
+  @Test(expected = MetaException.class)
   public void testAddPartsNullList() throws Exception {
-    // TODO: NPE should not be thrown
+
     client.add_partitions(null, false, false);
   }
 
@@ -1429,9 +1408,9 @@ public class TestAddPartitions extends MetaStoreClientTest {
     Assert.assertTrue(partitionNames.contains("year=2017"));
   }
 
-  @Test(expected = NullPointerException.class)
+  @Test(expected = MetaException.class)
   public void testAddPartsNullPartition() throws Exception {
-    // TODO: NPE should not be thrown
+
     List<Partition> partitions = new ArrayList<>();
     partitions.add(null);
     client.add_partitions(partitions, false, false);
@@ -1452,7 +1431,7 @@ public class TestAddPartitions extends MetaStoreClientTest {
 
   private Table createTable(String dbName, String tableName, List<FieldSchema> partCols,
       String location) throws Exception {
-    Table table = new TableBuilder()
+    new TableBuilder()
         .setDbName(dbName)
         .setTableName(tableName)
         .addCol("test_id", "int", "test col id")

http://git-wip-us.apache.org/repos/asf/hive/blob/1f25c46a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestAddPartitionsFromPartSpec.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestAddPartitionsFromPartSpec.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestAddPartitionsFromPartSpec.java
index fc0c60f..debcd0e 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestAddPartitionsFromPartSpec.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestAddPartitionsFromPartSpec.java
@@ -27,7 +27,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
-import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
 import org.apache.hadoop.hive.metastore.api.MetaException;
@@ -45,7 +44,6 @@ import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
 import org.apache.hadoop.hive.metastore.minihms.AbstractMetaStoreService;
 import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
 import org.apache.thrift.TException;
-import org.apache.thrift.transport.TTransportException;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -170,10 +168,9 @@ public class TestAddPartitionsFromPartSpec extends MetaStoreClientTest {
 
   // TODO add tests for partitions in other catalogs
 
-  @Test(expected = NullPointerException.class)
+  @Test(expected = MetaException.class)
   public void testAddPartitionSpecNullSpec() throws Exception {
 
-    // TODO: NPE should not be thrown.
     client.add_partitions_pspec(null);
   }
 
@@ -186,51 +183,36 @@ public class TestAddPartitionsFromPartSpec extends MetaStoreClientTest {
     client.add_partitions_pspec(partitionSpec);
   }
 
-  @Test
+  @Test(expected = MetaException.class)
   public void testAddPartitionSpecNullPartList() throws Exception {
 
     createTable();
     List<Partition> partitions = null;
     PartitionSpecProxy partitionSpec = buildPartitionSpec(DB_NAME, TABLE_NAME, null, partitions);
-    try {
-      client.add_partitions_pspec(partitionSpec);
-      Assert.fail("Exception should have been thrown.");
-    } catch (NullPointerException | TTransportException e) {
-      // TODO: NPE should not be thrown.
-    }
+    client.add_partitions_pspec(partitionSpec);
   }
 
-  @Test
+  @Test(expected = MetaException.class)
   public void testAddPartitionSpecNoDB() throws Exception {
 
     createTable();
     Partition partition = buildPartition(DB_NAME, TABLE_NAME, DEFAULT_YEAR_VALUE);
     PartitionSpecProxy partitionSpecProxy =
         buildPartitionSpec(null, TABLE_NAME, null, Lists.newArrayList(partition));
-    try {
-      client.add_partitions_pspec(partitionSpecProxy);
-      Assert.fail("Exception should have been thrown.");
-    } catch (NullPointerException | TTransportException e) {
-      // TODO: NPE should not be thrown.
-    }
+    client.add_partitions_pspec(partitionSpecProxy);
   }
 
-  @Test
+  @Test(expected = MetaException.class)
   public void testAddPartitionSpecNoTable() throws Exception {
 
     createTable();
     Partition partition = buildPartition(DB_NAME, TABLE_NAME, DEFAULT_YEAR_VALUE);
     PartitionSpecProxy partitionSpecProxy =
         buildPartitionSpec(DB_NAME, null, null, Lists.newArrayList(partition));
-    try {
-      client.add_partitions_pspec(partitionSpecProxy);
-      Assert.fail("Exception should have been thrown.");
-    } catch (NullPointerException | TTransportException e) {
-      // TODO: NPE should not be thrown.
-    }
+    client.add_partitions_pspec(partitionSpecProxy);
   }
 
-  @Test
+  @Test(expected = MetaException.class)
   public void testAddPartitionSpecNoDBAndTableInPartition() throws Exception {
 
     createTable();
@@ -239,12 +221,7 @@ public class TestAddPartitionsFromPartSpec extends MetaStoreClientTest {
     partition.setTableName(null);
     PartitionSpecProxy partitionSpecProxy =
         buildPartitionSpec(DB_NAME, TABLE_NAME, null, Lists.newArrayList(partition));
-    try {
-      client.add_partitions_pspec(partitionSpecProxy);
-      Assert.fail("Exception should have been thrown.");
-    } catch (NullPointerException | TTransportException e) {
-      // TODO: NPE should not be thrown.
-    }
+    client.add_partitions_pspec(partitionSpecProxy);
   }
 
   @Test
@@ -346,7 +323,7 @@ public class TestAddPartitionsFromPartSpec extends MetaStoreClientTest {
     }
   }
 
-  @Test
+  @Test(expected = MetaException.class)
   public void testAddPartitionSpecNullPart() throws Exception {
 
     createTable();
@@ -357,11 +334,7 @@ public class TestAddPartitionsFromPartSpec extends MetaStoreClientTest {
     partitions.add(partition2);
     PartitionSpecProxy partitionSpecProxy =
         buildPartitionSpec(DB_NAME, TABLE_NAME, null, partitions);
-    try {
-      client.add_partitions_pspec(partitionSpecProxy);
-    } catch (NullPointerException e) {
-      // TODO: NPE should not be thrown.
-    }
+    client.add_partitions_pspec(partitionSpecProxy);
   }
 
   @Test
@@ -457,7 +430,7 @@ public class TestAddPartitionsFromPartSpec extends MetaStoreClientTest {
     client.add_partitions_pspec(partitionSpecProxy);
   }
 
-  @Test(expected = NullPointerException.class)
+  @Test(expected = MetaException.class)
   public void testAddPartitionSpecChangeRootPathToNull() throws Exception {
 
     Table table = createTable();
@@ -467,7 +440,6 @@ public class TestAddPartitionsFromPartSpec extends MetaStoreClientTest {
         buildPartitionSpec(DB_NAME, TABLE_NAME, rootPath, Lists.newArrayList(partition));
     partitionSpecProxy.setRootLocation(null);
     client.add_partitions_pspec(partitionSpecProxy);
-    // TODO: NPE should not be thrown.
   }
 
   @Test(expected = MetaException.class)
@@ -596,7 +568,7 @@ public class TestAddPartitionsFromPartSpec extends MetaStoreClientTest {
     client.add_partitions_pspec(partitionSpecProxy);
   }
 
-  @Test
+  @Test(expected = MetaException.class)
   public void testAddPartitionSpecWithSharedSDNullSd() throws Exception {
 
     createTable();
@@ -604,12 +576,7 @@ public class TestAddPartitionsFromPartSpec extends MetaStoreClientTest {
     StorageDescriptor sd = null;
     PartitionSpecProxy partitionSpecProxy =
         buildPartitionSpecWithSharedSD(Lists.newArrayList(partition), sd);
-    try {
-      client.add_partitions_pspec(partitionSpecProxy);
-      Assert.fail("Exception should have been thrown.");
-    } catch (NullPointerException | TTransportException e) {
-      // TODO: NPE should not be thrown.
-    }
+    client.add_partitions_pspec(partitionSpecProxy);
   }
 
   @Test(expected = MetaException.class)
@@ -734,7 +701,7 @@ public class TestAddPartitionsFromPartSpec extends MetaStoreClientTest {
     Assert.assertNull(part.getSd());
   }
 
-  @Test
+  @Test(expected=MetaException.class)
   public void testAddPartitionSpecWithSharedSDNoValue() throws Exception {
 
     Table table = createTable();
@@ -743,12 +710,7 @@ public class TestAddPartitionsFromPartSpec extends MetaStoreClientTest {
     String location = table.getSd().getLocation() + "/nullValueTest/";
     PartitionSpecProxy partitionSpecProxy =
         buildPartitionSpecWithSharedSD(Lists.newArrayList(partition), buildSD(location));
-    try {
-      client.add_partitions_pspec(partitionSpecProxy);
-      Assert.fail("Exception should have been thrown.");
-    } catch (NullPointerException | TTransportException e) {
-      // TODO: NPE should not be thrown.
-    }
+    client.add_partitions_pspec(partitionSpecProxy);
   }
 
   @Test(expected=MetaException.class)
@@ -767,18 +729,15 @@ public class TestAddPartitionsFromPartSpec extends MetaStoreClientTest {
     client.add_partitions_pspec(partitionSpecProxy);
   }
 
-  @Test
-  public void testAddPartitionSpecNullValue() throws Exception {
+  @Test(expected = MetaException.class)
+  public void testAddPartitionSpecNullValues() throws Exception {
 
     createTable();
     Partition partition = buildPartition(DB_NAME, TABLE_NAME, null);
+    partition.setValues(null);
     PartitionSpecProxy partitionSpecProxy =
         buildPartitionSpec(DB_NAME, TABLE_NAME, null, Lists.newArrayList(partition));
-    try {
-      client.add_partitions_pspec(partitionSpecProxy);
-    } catch (NullPointerException e) {
-      // TODO: NPE should not be thrown
-    }
+    client.add_partitions_pspec(partitionSpecProxy);
   }
 
   @Test
@@ -938,7 +897,7 @@ public class TestAddPartitionsFromPartSpec extends MetaStoreClientTest {
 
   // Helper methods
   private void createDB(String dbName) throws TException {
-    Database db = new DatabaseBuilder().setName(dbName).create(client, metaStore.getConf());
+    new DatabaseBuilder().setName(dbName).create(client, metaStore.getConf());
   }
 
   private Table createTable() throws Exception {
@@ -948,7 +907,7 @@ public class TestAddPartitionsFromPartSpec extends MetaStoreClientTest {
 
   private Table createTable(String dbName, String tableName, List<FieldSchema> partCols,
       String location) throws Exception {
-    Table table = new TableBuilder()
+    new TableBuilder()
         .setDbName(dbName)
         .setTableName(tableName)
         .addCol("test_id", "int", "test col id")
@@ -1073,7 +1032,7 @@ public class TestAddPartitionsFromPartSpec extends MetaStoreClientTest {
   }
 
   private PartitionSpecProxy buildPartitionSpec(String dbName, String tableName, String rootPath,
-      List<Partition> partitions) {
+      List<Partition> partitions) throws MetaException {
 
     PartitionSpec partitionSpec = new PartitionSpec();
     partitionSpec.setDbName(dbName);
@@ -1104,7 +1063,7 @@ public class TestAddPartitionsFromPartSpec extends MetaStoreClientTest {
   }
 
   private PartitionSpecProxy buildPartitionSpecWithSharedSD(List<PartitionWithoutSD> partitions,
-      StorageDescriptor sd) {
+      StorageDescriptor sd) throws MetaException {
 
     PartitionSpec partitionSpec = new PartitionSpec();
     partitionSpec.setDbName(DB_NAME);