You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ga...@apache.org on 2018/03/31 01:25:21 UTC

[02/44] hive git commit: HIVE-18755 Modifications to the metastore for catalogs (Alan Gates, reviewed by Thejas Nair)

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestFunctions.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestFunctions.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestFunctions.java
index d504f34..9857c4e 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestFunctions.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestFunctions.java
@@ -19,8 +19,11 @@
 package org.apache.hadoop.hive.metastore.client;
 
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
 import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
+import org.apache.hadoop.hive.metastore.api.Catalog;
+import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.Function;
 import org.apache.hadoop.hive.metastore.api.FunctionType;
 import org.apache.hadoop.hive.metastore.api.GetAllFunctionsResponse;
@@ -30,10 +33,12 @@ import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.hadoop.hive.metastore.api.ResourceType;
 import org.apache.hadoop.hive.metastore.api.ResourceUri;
+import org.apache.hadoop.hive.metastore.client.builder.CatalogBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.FunctionBuilder;
 import org.apache.hadoop.hive.metastore.minihms.AbstractMetaStoreService;
 import org.apache.thrift.TApplicationException;
+import org.apache.thrift.TException;
 import org.apache.thrift.transport.TTransportException;
 import org.junit.After;
 import org.junit.Assert;
@@ -43,7 +48,11 @@ import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
 
 /**
  * Test class for IMetaStoreClient API. Testing the Function related functions.
@@ -82,27 +91,27 @@ public class TestFunctions extends MetaStoreClientTest {
             .addResourceUri(new ResourceUri(ResourceType.JAR, "hdfs:///tmp/jar1.jar"))
             .addResourceUri(new ResourceUri(ResourceType.FILE, "hdfs:///tmp/file1.txt"))
             .addResourceUri(new ResourceUri(ResourceType.ARCHIVE, "hdfs:///tmp/archive1.tgz"))
-            .build();
+            .build(metaStore.getConf());
     testFunctions[1] =
         new FunctionBuilder()
             .setDbName(DEFAULT_DATABASE)
             .setName("test_function_to_find_2")
             .setClass(TEST_FUNCTION_CLASS)
-            .build();
+            .build(metaStore.getConf());
     testFunctions[2] =
         new FunctionBuilder()
             .setDbName(DEFAULT_DATABASE)
             .setName("test_function_hidden_1")
             .setClass(TEST_FUNCTION_CLASS)
-            .build();
+            .build(metaStore.getConf());
 
-    client.createDatabase(new DatabaseBuilder().setName(OTHER_DATABASE).build());
+    new DatabaseBuilder().setName(OTHER_DATABASE).create(client, metaStore.getConf());
     testFunctions[3] =
         new FunctionBuilder()
             .setDbName(OTHER_DATABASE)
             .setName("test_function_to_find_1")
             .setClass(TEST_FUNCTION_CLASS)
-            .build();
+            .build(metaStore.getConf());
 
     // Create the functions, and reload them from the MetaStore
     for(int i=0; i < testFunctions.length; i++) {
@@ -125,7 +134,6 @@ public class TestFunctions extends MetaStoreClientTest {
 
   /**
    * This test creates and queries a function and then drops it. Good for testing the happy path.
-   * @throws Exception
    */
   @Test
   public void testCreateGetDeleteFunction() throws Exception {
@@ -141,9 +149,7 @@ public class TestFunctions extends MetaStoreClientTest {
             .addResourceUri(new ResourceUri(ResourceType.JAR, "hdfs:///tmp/jar1.jar"))
             .addResourceUri(new ResourceUri(ResourceType.FILE, "hdfs:///tmp/file1.txt"))
             .addResourceUri(new ResourceUri(ResourceType.ARCHIVE, "hdfs:///tmp/archive1.tgz"))
-            .build();
-
-    client.createFunction(function);
+            .create(client, metaStore.getConf());
 
     Function createdFunction = client.getFunction(function.getDbName(),
         function.getFunctionName());
@@ -516,7 +522,7 @@ public class TestFunctions extends MetaStoreClientTest {
             .setOwnerType(PrincipalType.GROUP)
             .setClass("org.apache.hadoop.hive.ql.udf.generic.GenericUDFUpper2")
             .setFunctionType(FunctionType.JAVA)
-            .build();
+            .build(metaStore.getConf());
 
     client.alterFunction(testFunctions[0].getDbName(), testFunctions[0].getFunctionName(),
         newFunction);
@@ -565,7 +571,7 @@ public class TestFunctions extends MetaStoreClientTest {
     return new FunctionBuilder()
             .setName("test_function_2")
             .setClass(TEST_FUNCTION_CLASS)
-            .build();
+            .build(metaStore.getConf());
   }
 
   @Test(expected = MetaException.class)
@@ -797,7 +803,7 @@ public class TestFunctions extends MetaStoreClientTest {
             .setDbName(OTHER_DATABASE)
             .setName("test_function_2")
             .setClass(TEST_FUNCTION_CLASS)
-            .build();
+            .build(metaStore.getConf());
     Function originalFunction = testFunctions[1];
 
     // Test in upper case
@@ -832,4 +838,100 @@ public class TestFunctions extends MetaStoreClientTest {
       // Expected exception
     }
   }
+
+  @Test
+  public void otherCatalog() throws TException {
+    String catName = "functions_catalog";
+    Catalog cat = new CatalogBuilder()
+        .setName(catName)
+        .setLocation(MetaStoreTestUtils.getTestWarehouseDir(catName))
+        .build();
+    client.createCatalog(cat);
+
+    String dbName = "functions_other_catalog_db";
+    Database db = new DatabaseBuilder()
+        .setCatalogName(catName)
+        .setName(dbName)
+        .create(client, metaStore.getConf());
+
+    String functionName = "test_function";
+    Function function =
+        new FunctionBuilder()
+            .inDb(db)
+            .setName(functionName)
+            .setClass(TEST_FUNCTION_CLASS)
+            .setFunctionType(FunctionType.JAVA)
+            .setOwnerType(PrincipalType.ROLE)
+            .setOwner("owner")
+            .setCreateTime(100)
+            .addResourceUri(new ResourceUri(ResourceType.JAR, "hdfs:///tmp/jar1.jar"))
+            .addResourceUri(new ResourceUri(ResourceType.FILE, "hdfs:///tmp/file1.txt"))
+            .addResourceUri(new ResourceUri(ResourceType.ARCHIVE, "hdfs:///tmp/archive1.tgz"))
+            .create(client, metaStore.getConf());
+
+    Function createdFunction = client.getFunction(catName, dbName, functionName);
+    // The createTime will be set on the server side, so the comparison should skip it
+    function.setCreateTime(createdFunction.getCreateTime());
+    Assert.assertEquals("Comparing functions", function, createdFunction);
+
+    String f2Name = "testy_function2";
+    Function f2 = new FunctionBuilder()
+        .inDb(db)
+        .setName(f2Name)
+        .setClass(TEST_FUNCTION_CLASS)
+        .create(client, metaStore.getConf());
+
+    Set<String> functions = new HashSet<>(client.getFunctions(catName, dbName, "test*"));
+    Assert.assertEquals(2, functions.size());
+    Assert.assertTrue(functions.contains(functionName));
+    Assert.assertTrue(functions.contains(f2Name));
+
+    functions = new HashSet<>(client.getFunctions(catName, dbName, "test_*"));
+    Assert.assertEquals(1, functions.size());
+    Assert.assertTrue(functions.contains(functionName));
+    Assert.assertFalse(functions.contains(f2Name));
+
+    client.dropFunction(function.getCatName(), function.getDbName(), function.getFunctionName());
+    try {
+      client.getFunction(function.getCatName(), function.getDbName(), function.getFunctionName());
+      Assert.fail("Expected a NoSuchObjectException to be thrown");
+    } catch (NoSuchObjectException exception) {
+      // Expected exception
+    }
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void addNoSuchCatalog() throws TException {
+    String functionName = "test_function";
+    new FunctionBuilder()
+            .setName(functionName)
+            .setCatName("nosuch")
+            .setDbName(DEFAULT_DATABASE_NAME)
+            .setClass(TEST_FUNCTION_CLASS)
+            .setFunctionType(FunctionType.JAVA)
+            .setOwnerType(PrincipalType.ROLE)
+            .setOwner("owner")
+            .setCreateTime(100)
+            .addResourceUri(new ResourceUri(ResourceType.JAR, "hdfs:///tmp/jar1.jar"))
+            .addResourceUri(new ResourceUri(ResourceType.FILE, "hdfs:///tmp/file1.txt"))
+            .addResourceUri(new ResourceUri(ResourceType.ARCHIVE, "hdfs:///tmp/archive1.tgz"))
+            .create(client, metaStore.getConf());
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void getNoSuchCatalog() throws TException {
+    client.getFunction("nosuch", DEFAULT_DATABASE_NAME, testFunctions[0].getFunctionName());
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void dropNoSuchCatalog() throws TException {
+    client.dropFunction("nosuch", DEFAULT_DATABASE_NAME, testFunctions[0].getFunctionName());
+  }
+
+  @Test
+  public void getFunctionsNoSuchCatalog() throws TException {
+    List<String> functionNames = client.getFunctions("nosuch", DEFAULT_DATABASE_NAME, "*");
+    Assert.assertEquals(0, functionNames.size());
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestGetPartitions.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestGetPartitions.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestGetPartitions.java
index fe5060b..8040728 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestGetPartitions.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestGetPartitions.java
@@ -18,15 +18,22 @@
 
 package org.apache.hadoop.hive.metastore.client;
 
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
 import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
+import org.apache.hadoop.hive.metastore.api.Catalog;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.client.builder.CatalogBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.PartitionBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
@@ -37,6 +44,7 @@ import org.apache.thrift.transport.TTransportException;
 import com.google.common.collect.Lists;
 
 import org.junit.After;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -45,6 +53,7 @@ import org.junit.runners.Parameterized;
 
 import static junit.framework.TestCase.assertNotNull;
 import static junit.framework.TestCase.assertNull;
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
@@ -86,16 +95,15 @@ public class TestGetPartitions extends MetaStoreClientTest {
   }
 
   private void createDB(String dbName) throws TException {
-    Database db = new DatabaseBuilder().
+    new DatabaseBuilder().
             setName(dbName).
-            build();
-    client.createDatabase(db);
+            create(client, metaStore.getConf());
   }
 
 
-  private static Table createTestTable(IMetaStoreClient client, String dbName, String tableName,
+  private Table createTestTable(IMetaStoreClient client, String dbName, String tableName,
                                        List<String> partCols, boolean setPartitionLevelPrivilages)
-          throws Exception {
+          throws TException {
     TableBuilder builder = new TableBuilder()
             .setDbName(dbName)
             .setTableName(tableName)
@@ -103,7 +111,7 @@ public class TestGetPartitions extends MetaStoreClientTest {
             .addCol("name", "string");
 
     partCols.forEach(col -> builder.addPartCol(col, "string"));
-    Table table = builder.build();
+    Table table = builder.build(metaStore.getConf());
 
     if (setPartitionLevelPrivilages) {
       table.putToParameters("PARTITION_LEVEL_PRIVILEGE", "true");
@@ -113,29 +121,29 @@ public class TestGetPartitions extends MetaStoreClientTest {
     return table;
   }
 
-  private static void addPartition(IMetaStoreClient client, Table table, List<String> values)
+  private void addPartition(IMetaStoreClient client, Table table, List<String> values)
           throws TException {
-    PartitionBuilder partitionBuilder = new PartitionBuilder().fromTable(table);
+    PartitionBuilder partitionBuilder = new PartitionBuilder().inTable(table);
     values.forEach(val -> partitionBuilder.addValue(val));
-    client.add_partition(partitionBuilder.build());
+    client.add_partition(partitionBuilder.build(metaStore.getConf()));
   }
 
-  private static void createTable3PartCols1PartGeneric(IMetaStoreClient client, boolean authOn)
-          throws Exception {
+  private void createTable3PartCols1PartGeneric(IMetaStoreClient client, boolean authOn)
+          throws TException {
     Table t = createTestTable(client, DB_NAME, TABLE_NAME, Lists.newArrayList("yyyy", "mm",
             "dd"), authOn);
     addPartition(client, t, Lists.newArrayList("1997", "05", "16"));
   }
 
-  private static void createTable3PartCols1Part(IMetaStoreClient client) throws Exception {
+  private void createTable3PartCols1Part(IMetaStoreClient client) throws TException {
     createTable3PartCols1PartGeneric(client, false);
   }
 
-  private static void createTable3PartCols1PartAuthOn(IMetaStoreClient client) throws Exception {
+  private void createTable3PartCols1PartAuthOn(IMetaStoreClient client) throws TException {
     createTable3PartCols1PartGeneric(client, true);
   }
 
-  private static List<List<String>> createTable4PartColsParts(IMetaStoreClient client) throws
+  private List<List<String>> createTable4PartColsParts(IMetaStoreClient client) throws
           Exception {
     Table t = createTestTable(client, DB_NAME, TABLE_NAME, Lists.newArrayList("yyyy", "mm", "dd"),
             false);
@@ -167,7 +175,6 @@ public class TestGetPartitions extends MetaStoreClientTest {
   /**
    * Testing getPartition(String,String,String) ->
    *         get_partition_by_name(String,String,String).
-   * @throws Exception
    */
   @Test
   public void testGetPartition() throws Exception {
@@ -247,7 +254,6 @@ public class TestGetPartitions extends MetaStoreClientTest {
   /**
    * Testing getPartition(String,String,List(String)) ->
    *         get_partition(String,String,List(String)).
-   * @throws Exception
    */
   @Test
   public void testGetPartitionByValues() throws Exception {
@@ -322,7 +328,6 @@ public class TestGetPartitions extends MetaStoreClientTest {
   /**
    * Testing getPartitionsByNames(String,String,List(String)) ->
    *         get_partitions_by_names(String,String,List(String)).
-   * @throws Exception
    */
   @Test
   public void testGetPartitionsByNames() throws Exception {
@@ -414,7 +419,6 @@ public class TestGetPartitions extends MetaStoreClientTest {
   /**
    * Testing getPartitionWithAuthInfo(String,String,List(String),String,List(String)) ->
    *         get_partition_with_auth(String,String,List(String),String,List(String)).
-   * @throws Exception
    */
   @Test
   public void testGetPartitionWithAuthInfoNoPrivilagesSet() throws Exception {
@@ -516,5 +520,85 @@ public class TestGetPartitions extends MetaStoreClientTest {
             Lists.newArrayList("1997", "05", "16"), "user0", null);
   }
 
+  @Test
+  public void otherCatalog() throws TException {
+    String catName = "get_partition_catalog";
+    Catalog cat = new CatalogBuilder()
+        .setName(catName)
+        .setLocation(MetaStoreTestUtils.getTestWarehouseDir(catName))
+        .build();
+    client.createCatalog(cat);
+
+    String dbName = "get_partition_database_in_other_catalog";
+    Database db = new DatabaseBuilder()
+        .setName(dbName)
+        .setCatalogName(catName)
+        .create(client, metaStore.getConf());
+
+    String tableName = "table_in_other_catalog";
+    Table table = new TableBuilder()
+        .inDb(db)
+        .setTableName(tableName)
+        .addCol("id", "int")
+        .addCol("name", "string")
+        .addPartCol("partcol", "string")
+        .addTableParam("PARTITION_LEVEL_PRIVILEGE", "true")
+        .create(client, metaStore.getConf());
+
+    Partition[] parts = new Partition[5];
+    for (int i = 0; i < parts.length; i++) {
+      parts[i] = new PartitionBuilder()
+          .inTable(table)
+          .addValue("a" + i)
+          .build(metaStore.getConf());
+    }
+    client.add_partitions(Arrays.asList(parts));
+
+    Partition fetched = client.getPartition(catName, dbName, tableName,
+        Collections.singletonList("a0"));
+    Assert.assertEquals(catName, fetched.getCatName());
+    Assert.assertEquals("a0", fetched.getValues().get(0));
+
+    fetched = client.getPartition(catName, dbName, tableName, "partcol=a0");
+    Assert.assertEquals(catName, fetched.getCatName());
+    Assert.assertEquals("a0", fetched.getValues().get(0));
+
+    List<Partition> fetchedParts = client.getPartitionsByNames(catName, dbName, tableName,
+        Arrays.asList("partcol=a0", "partcol=a1"));
+    Assert.assertEquals(2, fetchedParts.size());
+    Set<String> vals = new HashSet<>(fetchedParts.size());
+    for (Partition part : fetchedParts) vals.add(part.getValues().get(0));
+    Assert.assertTrue(vals.contains("a0"));
+    Assert.assertTrue(vals.contains("a1"));
+
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void getPartitionBogusCatalog() throws TException {
+    createTable3PartCols1Part(client);
+    client.getPartition("bogus", DB_NAME, TABLE_NAME, Lists.newArrayList("1997", "05", "16"));
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void getPartitionByNameBogusCatalog() throws TException {
+    createTable3PartCols1Part(client);
+    client.getPartition("bogus", DB_NAME, TABLE_NAME, "yyyy=1997/mm=05/dd=16");
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void getPartitionWithAuthBogusCatalog() throws TException {
+    createTable3PartCols1PartAuthOn(client);
+    client.getPartitionWithAuthInfo("bogus", DB_NAME, TABLE_NAME,
+        Lists.newArrayList("1997", "05", "16"), "user0", Lists.newArrayList("group0"));
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void getPartitionsByNamesBogusCatalog() throws TException {
+    createTable3PartCols1Part(client);
+    client.getPartitionsByNames("bogus", DB_NAME, TABLE_NAME,
+        Collections.singletonList("yyyy=1997/mm=05/dd=16"));
+  }
+
+
 
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestGetTableMeta.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestGetTableMeta.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestGetTableMeta.java
index 4b22a7b..d8448c8 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestGetTableMeta.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestGetTableMeta.java
@@ -19,32 +19,38 @@
 package org.apache.hadoop.hive.metastore.client;
 
 import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
 import com.google.common.collect.ImmutableSet;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
 import org.apache.hadoop.hive.metastore.api.CreationMetadata;
+import org.apache.hadoop.hive.metastore.api.Catalog;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.TableMeta;
+import org.apache.hadoop.hive.metastore.client.builder.CatalogBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
 import org.apache.hadoop.hive.metastore.minihms.AbstractMetaStoreService;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.thrift.TException;
 
 import com.google.common.collect.Lists;
 
 import org.junit.After;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
-import static java.util.stream.Collectors.toSet;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -103,12 +109,10 @@ public class TestGetTableMeta extends MetaStoreClientTest {
   }
 
 
-  private Database createDB(String dbName) throws TException {
-    Database db = new DatabaseBuilder().
+  private void createDB(String dbName) throws TException {
+    new DatabaseBuilder().
             setName(dbName).
-            build();
-    client.createDatabase(db);
-    return db;
+            create(client, metaStore.getConf());
   }
 
 
@@ -122,12 +126,12 @@ public class TestGetTableMeta extends MetaStoreClientTest {
             .setType(type.name());
 
 
-    Table table = builder.build();
+    Table table = builder.build(metaStore.getConf());
 
 
     if (type == TableType.MATERIALIZED_VIEW) {
       CreationMetadata cm = new CreationMetadata(
-          dbName, tableName, ImmutableSet.of());
+          MetaStoreUtils.getDefaultCatalog(metaStore.getConf()), dbName, tableName, ImmutableSet.of());
       table.setCreationMetadata(cm);
     }
 
@@ -156,21 +160,29 @@ public class TestGetTableMeta extends MetaStoreClientTest {
   }
 
   private void assertTableMetas(int[] expected, List<TableMeta> actualTableMetas) {
-    assertEquals("Expected " + expected.length + " but have " + actualTableMetas.size() +
-            " tableMeta(s)", expected.length, actualTableMetas.size());
+    assertTableMetas(expectedMetas, actualTableMetas, expected);
+  }
+
+  private void assertTableMetas(List<TableMeta> actual, int... expected) {
+    assertTableMetas(expectedMetas, actual, expected);
+  }
 
-    Set<TableMeta> metas = actualTableMetas.stream().collect(toSet());
+  private void assertTableMetas(List<TableMeta> fullExpected, List<TableMeta> actual, int... expected) {
+    assertEquals("Expected " + expected.length + " but have " + actual.size() +
+        " tableMeta(s)", expected.length, actual.size());
+
+    Set<TableMeta> metas = new HashSet<>(actual);
     for (int i : expected){
-      assertTrue("Missing " + expectedMetas.get(i), metas.remove(expectedMetas.get(i)));
+      assertTrue("Missing " + fullExpected.get(i), metas.remove(fullExpected.get(i)));
     }
 
     assertTrue("Unexpected tableMeta(s): " + metas, metas.isEmpty());
+
   }
 
   /**
    * Testing getTableMeta(String,String,List(String)) ->
    *         get_table_meta(String,String,List(String)).
-   * @throws Exception
    */
   @Test
   public void testGetTableMeta() throws Exception {
@@ -260,4 +272,55 @@ public class TestGetTableMeta extends MetaStoreClientTest {
     assertTableMetas(new int[]{}, tableMetas);
   }
 
+  @Test
+  public void tablesInDifferentCatalog() throws TException {
+    String catName = "get_table_meta_catalog";
+    Catalog cat = new CatalogBuilder()
+        .setName(catName)
+        .setLocation(MetaStoreTestUtils.getTestWarehouseDir(catName))
+        .build();
+    client.createCatalog(cat);
+
+    String dbName = "db9";
+    // For this one don't specify a location to make sure it gets put in the catalog directory
+    Database db = new DatabaseBuilder()
+        .setName(dbName)
+        .setCatalogName(catName)
+        .create(client, metaStore.getConf());
+
+    String[] tableNames = {"table_in_other_catalog_1", "table_in_other_catalog_2", "random_name"};
+    List<TableMeta> expected = new ArrayList<>(tableNames.length);
+    for (int i = 0; i < tableNames.length; i++) {
+      client.createTable(new TableBuilder()
+          .inDb(db)
+          .setTableName(tableNames[i])
+          .addCol("id", "int")
+          .addCol("name", "string")
+          .build(metaStore.getConf()));
+      expected.add(new TableMeta(dbName, tableNames[i], TableType.MANAGED_TABLE.name()));
+    }
+
+    List<String> types = Collections.singletonList(TableType.MANAGED_TABLE.name());
+    List<TableMeta> actual = client.getTableMeta(catName, dbName, "*", types);
+    assertTableMetas(expected, actual, 0, 1, 2);
+
+    actual = client.getTableMeta(catName, "*", "table_*", types);
+    assertTableMetas(expected, actual, 0, 1);
+
+    actual = client.getTableMeta(dbName, "table_in_other_catalog_*", types);
+    assertTableMetas(expected, actual);
+  }
+
+  @Test
+  public void noSuchCatalog() throws TException {
+    List<TableMeta> tableMetas = client.getTableMeta("nosuchcatalog", "*", "*", Lists.newArrayList());
+    Assert.assertEquals(0, tableMetas.size());
+  }
+
+  @Test
+  public void catalogPatternsDontWork() throws TException {
+    List<TableMeta> tableMetas = client.getTableMeta("h*", "*", "*", Lists.newArrayList());
+    Assert.assertEquals(0, tableMetas.size());
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestListPartitions.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestListPartitions.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestListPartitions.java
index f5e4b8e..a8b6e31 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestListPartitions.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestListPartitions.java
@@ -19,10 +19,14 @@
 package org.apache.hadoop.hive.metastore.client;
 
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
 import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
+import org.apache.hadoop.hive.metastore.api.Catalog;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.MetaException;
@@ -33,9 +37,11 @@ import org.apache.hadoop.hive.metastore.api.PartitionValuesRequest;
 import org.apache.hadoop.hive.metastore.api.PartitionValuesResponse;
 import org.apache.hadoop.hive.metastore.api.PartitionValuesRow;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.client.builder.CatalogBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.PartitionBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.minihms.AbstractMetaStoreService;
 import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
 import org.apache.thrift.TException;
@@ -45,6 +51,8 @@ import org.apache.thrift.transport.TTransportException;
 import com.google.common.collect.Lists;
 
 import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -96,22 +104,21 @@ public class TestListPartitions extends MetaStoreClientTest {
   }
 
   private void createDB(String dbName) throws TException {
-    Database db = new DatabaseBuilder().
+    new DatabaseBuilder().
             setName(dbName).
-            build();
-    client.createDatabase(db);
+            create(client, metaStore.getConf());
   }
 
-  private static Table createTestTable(IMetaStoreClient client, String dbName, String tableName,
+  private Table createTestTable(IMetaStoreClient client, String dbName, String tableName,
                                        List<String> partCols) throws Exception {
 
     return createTestTable(client, dbName, tableName, partCols, false);
   }
 
 
-  private static Table createTestTable(IMetaStoreClient client, String dbName, String tableName,
+  private Table createTestTable(IMetaStoreClient client, String dbName, String tableName,
                                        List<String> partCols, boolean setPartitionLevelPrivilages)
-          throws Exception {
+          throws TException {
     TableBuilder builder = new TableBuilder()
             .setDbName(dbName)
             .setTableName(tableName)
@@ -119,7 +126,7 @@ public class TestListPartitions extends MetaStoreClientTest {
             .addCol("name", "string");
 
     partCols.forEach(col -> builder.addPartCol(col, "string"));
-    Table table = builder.build();
+    Table table = builder.build(metaStore.getConf());
 
     if (setPartitionLevelPrivilages) {
       table.putToParameters("PARTITION_LEVEL_PRIVILEGE", "true");
@@ -129,25 +136,25 @@ public class TestListPartitions extends MetaStoreClientTest {
     return table;
   }
 
-  private static void addPartition(IMetaStoreClient client, Table table, List<String> values)
+  private void addPartition(IMetaStoreClient client, Table table, List<String> values)
           throws TException {
-    PartitionBuilder partitionBuilder = new PartitionBuilder().fromTable(table);
+    PartitionBuilder partitionBuilder = new PartitionBuilder().inTable(table);
     values.forEach(val -> partitionBuilder.addValue(val));
-    client.add_partition(partitionBuilder.build());
+    client.add_partition(partitionBuilder.build(metaStore.getConf()));
   }
 
-  private static void createTable3PartCols1PartGeneric(IMetaStoreClient client, boolean authOn)
-          throws Exception {
+  private void createTable3PartCols1PartGeneric(IMetaStoreClient client, boolean authOn)
+          throws TException {
     Table t = createTestTable(client, DB_NAME, TABLE_NAME, Lists.newArrayList("yyyy", "mm",
             "dd"), authOn);
     addPartition(client, t, Lists.newArrayList("1997", "05", "16"));
   }
 
-  private static void createTable3PartCols1Part(IMetaStoreClient client) throws Exception {
+  private void createTable3PartCols1Part(IMetaStoreClient client) throws TException {
     createTable3PartCols1PartGeneric(client, false);
   }
 
-  private static List<List<String>> createTable4PartColsPartsGeneric(IMetaStoreClient client,
+  private List<List<String>> createTable4PartColsPartsGeneric(IMetaStoreClient client,
                                                                      boolean authOn) throws
           Exception {
     Table t = createTestTable(client, DB_NAME, TABLE_NAME, Lists.newArrayList("yyyy", "mm", "dd"),
@@ -165,12 +172,12 @@ public class TestListPartitions extends MetaStoreClientTest {
     return testValues;
   }
 
-  private static List<List<String>> createTable4PartColsParts(IMetaStoreClient client) throws
+  private List<List<String>> createTable4PartColsParts(IMetaStoreClient client) throws
           Exception {
     return createTable4PartColsPartsGeneric(client, false);
   }
 
-  private static List<List<String>> createTable4PartColsPartsAuthOn(IMetaStoreClient client) throws
+  private List<List<String>> createTable4PartColsPartsAuthOn(IMetaStoreClient client) throws
           Exception {
     return createTable4PartColsPartsGeneric(client, true);
   }
@@ -236,7 +243,6 @@ public class TestListPartitions extends MetaStoreClientTest {
   /**
    * Testing listPartitions(String,String,short) ->
    *         get_partitions(String,String,short).
-   * @throws Exception
    */
   @Test
   public void testListPartitionsAll() throws Exception {
@@ -247,8 +253,11 @@ public class TestListPartitions extends MetaStoreClientTest {
     partitions = client.listPartitions(DB_NAME, TABLE_NAME, (short)1);
     assertPartitionsHaveCorrectValues(partitions, testValues.subList(0, 1));
 
-    partitions = client.listPartitions(DB_NAME, TABLE_NAME, (short)0);
-    assertTrue(partitions.isEmpty());
+    // HIVE-18977
+    if (MetastoreConf.getBoolVar(metaStore.getConf(), MetastoreConf.ConfVars.TRY_DIRECT_SQL)) {
+      partitions = client.listPartitions(DB_NAME, TABLE_NAME, (short) 0);
+      assertTrue(partitions.isEmpty());
+    }
 
   }
 
@@ -293,7 +302,8 @@ public class TestListPartitions extends MetaStoreClientTest {
   public void testListPartitionsAllNullTblName() throws Exception {
     try {
       createTable3PartCols1Part(client);
-      List<Partition> partitions = client.listPartitions(DB_NAME, null, (short)-1);
+      List<Partition> partitions = client.listPartitions(DB_NAME,
+          (String)null, (short)-1);
       fail("Should have thrown exception");
     } catch (NullPointerException | TTransportException e) {
       //TODO: should not throw different exceptions for different HMS deployment types
@@ -316,7 +326,6 @@ public class TestListPartitions extends MetaStoreClientTest {
   /**
    * Testing listPartitions(String,String,List(String),short) ->
    *         get_partitions(String,String,List(String),short).
-   * @throws Exception
    */
   @Test
   public void testListPartitionsByValues() throws Exception {
@@ -388,7 +397,7 @@ public class TestListPartitions extends MetaStoreClientTest {
   @Test(expected = MetaException.class)
   public void testListPartitionsByValuesNullValues() throws Exception {
     createTable3PartCols1Part(client);
-    client.listPartitions(DB_NAME, TABLE_NAME, null, (short)-1);
+    client.listPartitions(DB_NAME, TABLE_NAME, (List<String>)null, (short)-1);
   }
 
 
@@ -396,7 +405,6 @@ public class TestListPartitions extends MetaStoreClientTest {
   /**
    * Testing listPartitionSpecs(String,String,int) ->
    *         get_partitions_pspec(String,String,int).
-   * @throws Exception
    */
   @Test
   public void testListPartitionSpecs() throws Exception {
@@ -408,8 +416,11 @@ public class TestListPartitions extends MetaStoreClientTest {
     partSpecProxy = client.listPartitionSpecs(DB_NAME, TABLE_NAME, 2);
     assertPartitionsSpecProxy(partSpecProxy, testValues.subList(0, 2));
 
-    partSpecProxy = client.listPartitionSpecs(DB_NAME, TABLE_NAME, 0);
-    assertPartitionsSpecProxy(partSpecProxy, testValues.subList(0, 0));
+    // HIVE-18977
+    if (MetastoreConf.getBoolVar(metaStore.getConf(), MetastoreConf.ConfVars.TRY_DIRECT_SQL)) {
+      partSpecProxy = client.listPartitionSpecs(DB_NAME, TABLE_NAME, 0);
+      assertPartitionsSpecProxy(partSpecProxy, testValues.subList(0, 0));
+    }
   }
 
   @Test(expected = NoSuchObjectException.class)
@@ -447,7 +458,7 @@ public class TestListPartitions extends MetaStoreClientTest {
       createTable4PartColsParts(client);
       client.listPartitionSpecs(null, TABLE_NAME,  -1);
       fail("Should have thrown exception");
-    } catch (NullPointerException | TTransportException e) {
+    } catch (MetaException | TTransportException e) {
       //TODO: should not throw different exceptions for different HMS deployment types
     }
   }
@@ -468,7 +479,6 @@ public class TestListPartitions extends MetaStoreClientTest {
   /**
    * Testing listPartitionsWithAuthInfo(String,String,short,String,List(String)) ->
    *         get_partitions_with_auth(String,String,short,String,List(String)).
-   * @throws Exception
    */
   @Test
   public void testListPartitionsWithAuth() throws Exception {
@@ -539,9 +549,10 @@ public class TestListPartitions extends MetaStoreClientTest {
   public void testListPartitionsWithAuthNullTblName() throws Exception {
     try {
       createTable4PartColsParts(client);
-      client.listPartitionsWithAuthInfo(DB_NAME, null, (short)-1, "", Lists.newArrayList());
+      client.listPartitionsWithAuthInfo(DB_NAME, (String)null, (short)-1, "",
+          Lists.newArrayList());
       fail("Should have thrown exception");
-    } catch (AssertionError| TTransportException e) {
+    } catch (MetaException| TTransportException e) {
       //TODO: should not throw different exceptions for different HMS deployment types
     }
   }
@@ -563,7 +574,6 @@ public class TestListPartitions extends MetaStoreClientTest {
   /**
    * Testing listPartitionsWithAuthInfo(String,String,List(String),short,String,List(String)) ->
    *         get_partitions_ps_with_auth(String,String,List(String),short,String,List(String)).
-   * @throws Exception
    */
   @Test
   public void testListPartitionsWithAuthByValues() throws Exception {
@@ -692,7 +702,7 @@ public class TestListPartitions extends MetaStoreClientTest {
   @Test(expected = MetaException.class)
   public void testListPartitionsWithAuthByValuesNullValues() throws Exception {
     createTable4PartColsParts(client);
-    client.listPartitionsWithAuthInfo(DB_NAME, TABLE_NAME, null,
+    client.listPartitionsWithAuthInfo(DB_NAME, TABLE_NAME, (List<String>)null,
             (short)-1, "", Lists.newArrayList());
   }
 
@@ -717,7 +727,6 @@ public class TestListPartitions extends MetaStoreClientTest {
   /**
    * Testing listPartitionsByFilter(String,String,String,short) ->
    *         get_partitions_by_filter(String,String,String,short).
-   * @throws Exception
    */
   @Test
   public void testListPartitionsByFilter() throws Exception {
@@ -736,9 +745,12 @@ public class TestListPartitions extends MetaStoreClientTest {
             "yyyy=\"2017\" OR " + "mm=\"02\"", (short)0);
     assertTrue(partitions.isEmpty());
 
-    partitions = client.listPartitionsByFilter(DB_NAME, TABLE_NAME,
-            "yYyY=\"2017\"", (short)-1);
-    assertPartitionsHaveCorrectValues(partitions, partValues.subList(2, 4));
+    // HIVE-18977
+    if (MetastoreConf.getBoolVar(metaStore.getConf(), MetastoreConf.ConfVars.TRY_DIRECT_SQL)) {
+      partitions = client.listPartitionsByFilter(DB_NAME, TABLE_NAME,
+          "yYyY=\"2017\"", (short) -1);
+      assertPartitionsHaveCorrectValues(partitions, partValues.subList(2, 4));
+    }
 
     partitions = client.listPartitionsByFilter(DB_NAME, TABLE_NAME,
             "yyyy=\"2017\" AND mm=\"99\"", (short)-1);
@@ -822,7 +834,6 @@ public class TestListPartitions extends MetaStoreClientTest {
   /**
    * Testing listPartitionSpecsByFilter(String,String,String,int) ->
    *         get_part_specs_by_filter(String,String,String,int).
-   * @throws Exception
    */
   @Test
   public void testListPartitionsSpecsByFilter() throws Exception {
@@ -844,9 +855,12 @@ public class TestListPartitions extends MetaStoreClientTest {
             "yyyy=\"20177\"", -1);
     assertPartitionsSpecProxy(partSpecProxy, Lists.newArrayList());
 
-    partSpecProxy = client.listPartitionSpecsByFilter(DB_NAME, TABLE_NAME,
-            "yYyY=\"2017\"", -1);
-    assertPartitionsSpecProxy(partSpecProxy, testValues.subList(2, 4));
+    // HIVE-18977
+    if (MetastoreConf.getBoolVar(metaStore.getConf(), MetastoreConf.ConfVars.TRY_DIRECT_SQL)) {
+      partSpecProxy = client.listPartitionSpecsByFilter(DB_NAME, TABLE_NAME,
+          "yYyY=\"2017\"", -1);
+      assertPartitionsSpecProxy(partSpecProxy, testValues.subList(2, 4));
+    }
 
     partSpecProxy = client.listPartitionSpecsByFilter(DB_NAME, TABLE_NAME,
             "yyyy=\"2017\" AND mm=\"99\"", -1);
@@ -919,7 +933,6 @@ public class TestListPartitions extends MetaStoreClientTest {
   /**
    * Testing getNumPartitionsByFilter(String,String,String) ->
    *         get_num_partitions_by_filter(String,String,String).
-   * @throws Exception
    */
   @Test
   public void testGetNumPartitionsByFilter() throws Exception {
@@ -934,8 +947,11 @@ public class TestListPartitions extends MetaStoreClientTest {
     n = client.getNumPartitionsByFilter(DB_NAME, TABLE_NAME, "yyyy=\"20177\"");
     assertEquals(0, n);
 
-    n = client.getNumPartitionsByFilter(DB_NAME, TABLE_NAME, "yYyY=\"2017\"");
-    assertEquals(2, n);
+    // HIVE-18977
+    if (MetastoreConf.getBoolVar(metaStore.getConf(), MetastoreConf.ConfVars.TRY_DIRECT_SQL)) {
+      n = client.getNumPartitionsByFilter(DB_NAME, TABLE_NAME, "yYyY=\"2017\"");
+      assertEquals(2, n);
+    }
 
     n = client.getNumPartitionsByFilter(DB_NAME, TABLE_NAME, "yyyy=\"2017\" AND mm=\"99\"");
     assertEquals(0, n);
@@ -977,7 +993,7 @@ public class TestListPartitions extends MetaStoreClientTest {
       createTable4PartColsParts(client);
       client.getNumPartitionsByFilter(DB_NAME, null, "yyyy=\"2017\"");
       fail("Should have thrown exception");
-    } catch (AssertionError | TTransportException e) {
+    } catch (MetaException | TTransportException e) {
       //TODO: should not throw different exceptions for different HMS deployment types
     }
   }
@@ -1000,7 +1016,6 @@ public class TestListPartitions extends MetaStoreClientTest {
   /**
    * Testing listPartitionNames(String,String,short) ->
    *         get_partition_names(String,String,short).
-   * @throws Exception
    */
   @Test
   public void testListPartitionNames() throws Exception {
@@ -1061,7 +1076,7 @@ public class TestListPartitions extends MetaStoreClientTest {
   public void testListPartitionNamesNullTblName() throws Exception {
     try {
       createTable4PartColsParts(client);
-      client.listPartitionNames(DB_NAME, null, (short)-1);
+      client.listPartitionNames(DB_NAME, (String)null, (short)-1);
       fail("Should have thrown exception");
     } catch (NullPointerException | TTransportException e) {
       //TODO: should not throw different exceptions for different HMS deployment types
@@ -1073,7 +1088,6 @@ public class TestListPartitions extends MetaStoreClientTest {
   /**
    * Testing listPartitionNames(String,String,List(String),short) ->
    *         get_partition_names_ps(String,String,List(String),short).
-   * @throws Exception
    */
   @Test
   public void testListPartitionNamesByValues() throws Exception {
@@ -1175,7 +1189,7 @@ public class TestListPartitions extends MetaStoreClientTest {
   @Test(expected = MetaException.class)
   public void testListPartitionNamesByValuesNullValues() throws Exception {
     createTable4PartColsParts(client);
-    client.listPartitionNames(DB_NAME, TABLE_NAME, null, (short)-1);
+    client.listPartitionNames(DB_NAME, TABLE_NAME, (List<String>)null, (short)-1);
   }
 
 
@@ -1183,7 +1197,6 @@ public class TestListPartitions extends MetaStoreClientTest {
   /**
    * Testing listPartitionValues(PartitionValuesRequest) ->
    *         get_partition_values(PartitionValuesRequest).
-   * @throws Exception
    */
   @Test
   public void testListPartitionValues() throws Exception {
@@ -1319,4 +1332,131 @@ public class TestListPartitions extends MetaStoreClientTest {
     }
   }
 
+  @Test
+  public void otherCatalog() throws TException {
+    String catName = "list_partition_catalog";
+    Catalog cat = new CatalogBuilder()
+        .setName(catName)
+        .setLocation(MetaStoreTestUtils.getTestWarehouseDir(catName))
+        .build();
+    client.createCatalog(cat);
+
+    String dbName = "list_partition_database_in_other_catalog";
+    Database db = new DatabaseBuilder()
+        .setName(dbName)
+        .setCatalogName(catName)
+        .create(client, metaStore.getConf());
+
+    String tableName = "table_in_other_catalog";
+    Table table = new TableBuilder()
+        .inDb(db)
+        .setTableName(tableName)
+        .addCol("id", "int")
+        .addCol("name", "string")
+        .addPartCol("partcol", "string")
+        .create(client, metaStore.getConf());
+
+    Partition[] parts = new Partition[5];
+    for (int i = 0; i < parts.length; i++) {
+      parts[i] = new PartitionBuilder()
+          .inTable(table)
+          .addValue("a" + i)
+          .build(metaStore.getConf());
+    }
+    client.add_partitions(Arrays.asList(parts));
+
+    List<Partition> fetched = client.listPartitions(catName, dbName, tableName, -1);
+    Assert.assertEquals(parts.length, fetched.size());
+    Assert.assertEquals(catName, fetched.get(0).getCatName());
+
+    fetched = client.listPartitions(catName, dbName, tableName,
+        Collections.singletonList("a0"), -1);
+    Assert.assertEquals(1, fetched.size());
+    Assert.assertEquals(catName, fetched.get(0).getCatName());
+
+    PartitionSpecProxy proxy = client.listPartitionSpecs(catName, dbName, tableName, -1);
+    Assert.assertEquals(parts.length, proxy.size());
+    Assert.assertEquals(catName, proxy.getCatName());
+
+    fetched = client.listPartitionsByFilter(catName, dbName, tableName, "partcol=\"a0\"", -1);
+    Assert.assertEquals(1, fetched.size());
+    Assert.assertEquals(catName, fetched.get(0).getCatName());
+
+    proxy = client.listPartitionSpecsByFilter(catName, dbName, tableName, "partcol=\"a0\"", -1);
+    Assert.assertEquals(1, proxy.size());
+    Assert.assertEquals(catName, proxy.getCatName());
+
+    Assert.assertEquals(1, client.getNumPartitionsByFilter(catName, dbName, tableName,
+        "partcol=\"a0\""));
+
+    List<String> names = client.listPartitionNames(catName, dbName, tableName, 57);
+    Assert.assertEquals(parts.length, names.size());
+
+    names = client.listPartitionNames(catName, dbName, tableName, Collections.singletonList("a0"),
+        Short.MAX_VALUE + 1);
+    Assert.assertEquals(1, names.size());
+
+    PartitionValuesRequest rqst = new PartitionValuesRequest(dbName,
+        tableName, Lists.newArrayList(new FieldSchema("partcol", "string", "")));
+    rqst.setCatName(catName);
+    PartitionValuesResponse rsp = client.listPartitionValues(rqst);
+    Assert.assertEquals(5, rsp.getPartitionValuesSize());
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void listPartitionsBogusCatalog() throws TException {
+    createTable3PartCols1Part(client);
+    client.listPartitions("bogus", DB_NAME, TABLE_NAME, -1);
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void listPartitionsWithPartialValuesBogusCatalog() throws TException {
+    createTable3PartCols1Part(client);
+    client.listPartitions("bogus", DB_NAME, TABLE_NAME, Collections.singletonList("a0"), -1);
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void listPartitionsSpecsBogusCatalog() throws TException {
+    createTable3PartCols1Part(client);
+    client.listPartitionSpecs("bogus", DB_NAME, TABLE_NAME, -1);
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void listPartitionsByFilterBogusCatalog() throws TException {
+    createTable3PartCols1Part(client);
+    client.listPartitionsByFilter("bogus", DB_NAME, TABLE_NAME, "partcol=\"a0\"", -1);
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void listPartitionSpecsByFilterBogusCatalog() throws TException {
+    createTable3PartCols1Part(client);
+    client.listPartitionSpecsByFilter("bogus", DB_NAME, TABLE_NAME, "partcol=\"a0\"", -1);
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void getNumPartitionsByFilterBogusCatalog() throws TException {
+    createTable3PartCols1Part(client);
+    client.getNumPartitionsByFilter("bogus", DB_NAME, TABLE_NAME, "partcol=\"a0\"");
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void listPartitionNamesBogusCatalog() throws TException {
+    createTable3PartCols1Part(client);
+    client.listPartitionNames("bogus", DB_NAME, TABLE_NAME, -1);
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void listPartitionNamesPartialValsBogusCatalog() throws TException {
+    createTable3PartCols1Part(client);
+    client.listPartitionNames("bogus", DB_NAME, TABLE_NAME, Collections.singletonList("a0"), -1);
+  }
+
+  @Test(expected = MetaException.class)
+  public void listPartitionValuesBogusCatalog() throws TException {
+    createTable3PartCols1Part(client);
+    PartitionValuesRequest rqst = new PartitionValuesRequest(DB_NAME,
+        TABLE_NAME, Lists.newArrayList(new FieldSchema("partcol", "string", "")));
+    rqst.setCatName("bogus");
+    client.listPartitionValues(rqst);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestNotNullConstraint.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestNotNullConstraint.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestNotNullConstraint.java
new file mode 100644
index 0000000..0c75295
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestNotNullConstraint.java
@@ -0,0 +1,352 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.client;
+
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
+import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
+import org.apache.hadoop.hive.metastore.api.Catalog;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.NotNullConstraintsRequest;
+import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.client.builder.CatalogBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.SQLNotNullConstraintBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
+import org.apache.hadoop.hive.metastore.minihms.AbstractMetaStoreService;
+import org.apache.thrift.TApplicationException;
+import org.apache.thrift.TException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.List;
+
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
+
+@RunWith(Parameterized.class)
+@Category(MetastoreCheckinTest.class)
+public class TestNotNullConstraint extends MetaStoreClientTest {
+  private static final String OTHER_DATABASE = "test_uc_other_database";
+  private static final String OTHER_CATALOG = "test_uc_other_catalog";
+  private static final String DATABASE_IN_OTHER_CATALOG = "test_uc_database_in_other_catalog";
+  private final AbstractMetaStoreService metaStore;
+  private IMetaStoreClient client;
+  private Table[] testTables = new Table[3];
+  private Database inOtherCatalog;
+
+  public TestNotNullConstraint(String name, AbstractMetaStoreService metaStore) throws Exception {
+    this.metaStore = metaStore;
+    this.metaStore.start();
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    // Get new client
+    client = metaStore.getClient();
+
+    // Clean up the database
+    client.dropDatabase(OTHER_DATABASE, true, true, true);
+    // Drop every table in the default database
+    for(String tableName : client.getAllTables(DEFAULT_DATABASE_NAME)) {
+      client.dropTable(DEFAULT_DATABASE_NAME, tableName, true, true, true);
+    }
+
+    client.dropDatabase(OTHER_CATALOG, DATABASE_IN_OTHER_CATALOG, true, true, true);
+    try {
+      client.dropCatalog(OTHER_CATALOG);
+    } catch (NoSuchObjectException e) {
+      // NOP
+    }
+
+    // Clean up trash
+    metaStore.cleanWarehouseDirs();
+
+    new DatabaseBuilder().setName(OTHER_DATABASE).create(client, metaStore.getConf());
+
+    Catalog cat = new CatalogBuilder()
+        .setName(OTHER_CATALOG)
+        .setLocation(MetaStoreTestUtils.getTestWarehouseDir(OTHER_CATALOG))
+        .build();
+    client.createCatalog(cat);
+
+    // For this one don't specify a location to make sure it gets put in the catalog directory
+    inOtherCatalog = new DatabaseBuilder()
+        .setName(DATABASE_IN_OTHER_CATALOG)
+        .setCatalogName(OTHER_CATALOG)
+        .create(client, metaStore.getConf());
+
+    testTables[0] =
+        new TableBuilder()
+            .setTableName("test_table_1")
+            .addCol("col1", "int")
+            .addCol("col2", "varchar(32)")
+            .create(client, metaStore.getConf());
+
+    testTables[1] =
+        new TableBuilder()
+            .setDbName(OTHER_DATABASE)
+            .setTableName("test_table_2")
+            .addCol("col1", "int")
+            .addCol("col2", "varchar(32)")
+            .create(client, metaStore.getConf());
+
+    testTables[2] =
+        new TableBuilder()
+            .inDb(inOtherCatalog)
+            .setTableName("test_table_3")
+            .addCol("col1", "int")
+            .addCol("col2", "varchar(32)")
+            .create(client, metaStore.getConf());
+
+    // Reload tables from the MetaStore
+    for(int i=0; i < testTables.length; i++) {
+      testTables[i] = client.getTable(testTables[i].getCatName(), testTables[i].getDbName(),
+          testTables[i].getTableName());
+    }
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    try {
+      if (client != null) {
+        client.close();
+      }
+    } finally {
+      client = null;
+    }
+  }
+
+  @Test
+  public void createGetDrop() throws TException {
+    Table table = testTables[0];
+    // Make sure get on a table with no key returns empty list
+    NotNullConstraintsRequest rqst =
+        new NotNullConstraintsRequest(table.getCatName(), table.getDbName(), table.getTableName());
+    List<SQLNotNullConstraint> fetched = client.getNotNullConstraints(rqst);
+    Assert.assertTrue(fetched.isEmpty());
+
+    // Single column unnamed primary key in default catalog and database
+    List<SQLNotNullConstraint> nn = new SQLNotNullConstraintBuilder()
+        .onTable(table)
+        .addColumn("col1")
+        .build(metaStore.getConf());
+    client.addNotNullConstraint(nn);
+
+    rqst = new NotNullConstraintsRequest(table.getCatName(), table.getDbName(), table.getTableName());
+    fetched = client.getNotNullConstraints(rqst);
+    Assert.assertEquals(1, fetched.size());
+    Assert.assertEquals(table.getDbName(), fetched.get(0).getTable_db());
+    Assert.assertEquals(table.getTableName(), fetched.get(0).getTable_name());
+    Assert.assertEquals("col1", fetched.get(0).getColumn_name());
+    Assert.assertEquals(table.getTableName() + "_not_null_constraint", fetched.get(0).getNn_name());
+    String table0PkName = fetched.get(0).getNn_name();
+    Assert.assertTrue(fetched.get(0).isEnable_cstr());
+    Assert.assertFalse(fetched.get(0).isValidate_cstr());
+    Assert.assertFalse(fetched.get(0).isRely_cstr());
+    Assert.assertEquals(table.getCatName(), fetched.get(0).getCatName());
+
+    // Drop a primary key
+    client.dropConstraint(table.getCatName(), table.getDbName(),
+        table.getTableName(), table0PkName);
+    rqst = new NotNullConstraintsRequest(table.getCatName(), table.getDbName(), table.getTableName());
+    fetched = client.getNotNullConstraints(rqst);
+    Assert.assertTrue(fetched.isEmpty());
+
+    // Make sure I can add it back
+    client.addNotNullConstraint(nn);
+  }
+
+  @Test
+  public void inOtherCatalog() throws TException {
+    String constraintName = "ocuc";
+    // Table in non 'hive' catalog
+    List<SQLNotNullConstraint> nn = new SQLNotNullConstraintBuilder()
+        .onTable(testTables[2])
+        .addColumn("col1")
+        .setConstraintName(constraintName)
+        .build(metaStore.getConf());
+    client.addNotNullConstraint(nn);
+
+    NotNullConstraintsRequest rqst = new NotNullConstraintsRequest(testTables[2].getCatName(),
+        testTables[2].getDbName(), testTables[2].getTableName());
+    List<SQLNotNullConstraint> fetched = client.getNotNullConstraints(rqst);
+    Assert.assertEquals(1, fetched.size());
+    Assert.assertEquals(testTables[2].getDbName(), fetched.get(0).getTable_db());
+    Assert.assertEquals(testTables[2].getTableName(), fetched.get(0).getTable_name());
+    Assert.assertEquals("col1", fetched.get(0).getColumn_name());
+    Assert.assertEquals(constraintName, fetched.get(0).getNn_name());
+    Assert.assertTrue(fetched.get(0).isEnable_cstr());
+    Assert.assertFalse(fetched.get(0).isValidate_cstr());
+    Assert.assertFalse(fetched.get(0).isRely_cstr());
+    Assert.assertEquals(testTables[2].getCatName(), fetched.get(0).getCatName());
+
+    client.dropConstraint(testTables[2].getCatName(), testTables[2].getDbName(),
+        testTables[2].getTableName(), constraintName);
+    rqst = new NotNullConstraintsRequest(testTables[2].getCatName(), testTables[2].getDbName(),
+        testTables[2].getTableName());
+    fetched = client.getNotNullConstraints(rqst);
+    Assert.assertTrue(fetched.isEmpty());
+  }
+
+  @Test
+  public void createTableWithConstraintsPk() throws TException {
+    String constraintName = "ctwcuc";
+    Table table = new TableBuilder()
+        .setTableName("table_with_constraints")
+        .addCol("col1", "int")
+        .addCol("col2", "varchar(32)")
+        .build(metaStore.getConf());
+
+    List<SQLNotNullConstraint> nn = new SQLNotNullConstraintBuilder()
+        .onTable(table)
+        .addColumn("col1")
+        .setConstraintName(constraintName)
+        .build(metaStore.getConf());
+
+    client.createTableWithConstraints(table, null, null, null, nn, null, null);
+    NotNullConstraintsRequest rqst = new NotNullConstraintsRequest(table.getCatName(),
+        table.getDbName(), table.getTableName());
+    List<SQLNotNullConstraint> fetched = client.getNotNullConstraints(rqst);
+    Assert.assertEquals(1, fetched.size());
+    Assert.assertEquals(table.getDbName(), fetched.get(0).getTable_db());
+    Assert.assertEquals(table.getTableName(), fetched.get(0).getTable_name());
+    Assert.assertEquals("col1", fetched.get(0).getColumn_name());
+    Assert.assertEquals(constraintName, fetched.get(0).getNn_name());
+    Assert.assertTrue(fetched.get(0).isEnable_cstr());
+    Assert.assertFalse(fetched.get(0).isValidate_cstr());
+    Assert.assertFalse(fetched.get(0).isRely_cstr());
+    Assert.assertEquals(table.getCatName(), fetched.get(0).getCatName());
+
+    client.dropConstraint(table.getCatName(), table.getDbName(), table.getTableName(), constraintName);
+    rqst = new NotNullConstraintsRequest(table.getCatName(), table.getDbName(), table.getTableName());
+    fetched = client.getNotNullConstraints(rqst);
+    Assert.assertTrue(fetched.isEmpty());
+
+  }
+
+  @Test
+  public void createTableWithConstraintsPkInOtherCatalog() throws TException {
+    Table table = new TableBuilder()
+        .setTableName("table_in_other_catalog_with_constraints")
+        .inDb(inOtherCatalog)
+        .addCol("col1", "int")
+        .addCol("col2", "varchar(32)")
+        .build(metaStore.getConf());
+
+    List<SQLNotNullConstraint> nn = new SQLNotNullConstraintBuilder()
+        .onTable(table)
+        .addColumn("col1")
+        .build(metaStore.getConf());
+
+    client.createTableWithConstraints(table, null, null, null, nn, null, null);
+    NotNullConstraintsRequest rqst = new NotNullConstraintsRequest(table.getCatName(),
+        table.getDbName(), table.getTableName());
+    List<SQLNotNullConstraint> fetched = client.getNotNullConstraints(rqst);
+    Assert.assertEquals(1, fetched.size());
+    Assert.assertEquals(table.getDbName(), fetched.get(0).getTable_db());
+    Assert.assertEquals(table.getTableName(), fetched.get(0).getTable_name());
+    Assert.assertEquals("col1", fetched.get(0).getColumn_name());
+    Assert.assertEquals(table.getTableName() + "_not_null_constraint", fetched.get(0).getNn_name());
+    String tablePkName = fetched.get(0).getNn_name();
+    Assert.assertTrue(fetched.get(0).isEnable_cstr());
+    Assert.assertFalse(fetched.get(0).isValidate_cstr());
+    Assert.assertFalse(fetched.get(0).isRely_cstr());
+    Assert.assertEquals(table.getCatName(), fetched.get(0).getCatName());
+
+    client.dropConstraint(table.getCatName(), table.getDbName(), table.getTableName(), tablePkName);
+    rqst = new NotNullConstraintsRequest(table.getCatName(), table.getDbName(), table.getTableName());
+    fetched = client.getNotNullConstraints(rqst);
+    Assert.assertTrue(fetched.isEmpty());
+  }
+
+  @Test
+  public void doubleAddNotNullConstraint() throws TException {
+    Table table = testTables[0];
+    // Make sure get on a table with no key returns empty list
+    NotNullConstraintsRequest rqst =
+        new NotNullConstraintsRequest(table.getCatName(), table.getDbName(), table.getTableName());
+    List<SQLNotNullConstraint> fetched = client.getNotNullConstraints(rqst);
+    Assert.assertTrue(fetched.isEmpty());
+
+    // Single column unnamed primary key in default catalog and database
+    List<SQLNotNullConstraint> nn = new SQLNotNullConstraintBuilder()
+        .onTable(table)
+        .addColumn("col1")
+        .build(metaStore.getConf());
+    client.addNotNullConstraint(nn);
+
+    try {
+      nn = new SQLNotNullConstraintBuilder()
+          .onTable(table)
+          .addColumn("col2")
+          .build(metaStore.getConf());
+      client.addNotNullConstraint(nn);
+      Assert.fail();
+    } catch (InvalidObjectException|TApplicationException e) {
+      // NOP
+    }
+  }
+
+  @Test
+  public void addNoSuchTable() throws TException {
+    try {
+      List<SQLNotNullConstraint> nn = new SQLNotNullConstraintBuilder()
+          .setTableName("nosuch")
+          .addColumn("col2")
+          .build(metaStore.getConf());
+      client.addNotNullConstraint(nn);
+      Assert.fail();
+    } catch (InvalidObjectException |TApplicationException e) {
+      // NOP
+    }
+  }
+
+  @Test
+  public void getNoSuchTable() throws TException {
+    NotNullConstraintsRequest rqst =
+        new NotNullConstraintsRequest(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, "nosuch");
+    List<SQLNotNullConstraint> nn = client.getNotNullConstraints(rqst);
+    Assert.assertTrue(nn.isEmpty());
+  }
+
+  @Test
+  public void getNoSuchDb() throws TException {
+    NotNullConstraintsRequest rqst =
+        new NotNullConstraintsRequest(DEFAULT_CATALOG_NAME, "nosuch", testTables[0].getTableName());
+    List<SQLNotNullConstraint> nn = client.getNotNullConstraints(rqst);
+    Assert.assertTrue(nn.isEmpty());
+  }
+
+  @Test
+  public void getNoSuchCatalog() throws TException {
+    NotNullConstraintsRequest rqst =
+        new NotNullConstraintsRequest("nosuch", testTables[0].getDbName(), testTables[0].getTableName());
+    List<SQLNotNullConstraint> nn = client.getNotNullConstraints(rqst);
+    Assert.assertTrue(nn.isEmpty());
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestPrimaryKey.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestPrimaryKey.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestPrimaryKey.java
new file mode 100644
index 0000000..52f5b93
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestPrimaryKey.java
@@ -0,0 +1,465 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.client;
+
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
+import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
+import org.apache.hadoop.hive.metastore.api.Catalog;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.PrimaryKeysRequest;
+import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.client.builder.CatalogBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.SQLPrimaryKeyBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
+import org.apache.hadoop.hive.metastore.minihms.AbstractMetaStoreService;
+import org.apache.thrift.TApplicationException;
+import org.apache.thrift.TException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.List;
+
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
+
+@RunWith(Parameterized.class)
+@Category(MetastoreCheckinTest.class)
+public class TestPrimaryKey extends MetaStoreClientTest {
+  private static final String OTHER_DATABASE = "test_constraints_other_database";
+  private static final String OTHER_CATALOG = "test_constraints_other_catalog";
+  private static final String DATABASE_IN_OTHER_CATALOG = "test_constraints_database_in_other_catalog";
+  private final AbstractMetaStoreService metaStore;
+  private IMetaStoreClient client;
+  private Table[] testTables = new Table[3];
+  private Database inOtherCatalog;
+
+  public TestPrimaryKey(String name, AbstractMetaStoreService metaStore) throws Exception {
+    this.metaStore = metaStore;
+    this.metaStore.start();
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    // Get new client
+    client = metaStore.getClient();
+
+    // Clean up the database
+    client.dropDatabase(OTHER_DATABASE, true, true, true);
+    // Drop every table in the default database
+    for(String tableName : client.getAllTables(DEFAULT_DATABASE_NAME)) {
+      client.dropTable(DEFAULT_DATABASE_NAME, tableName, true, true, true);
+    }
+
+    client.dropDatabase(OTHER_CATALOG, DATABASE_IN_OTHER_CATALOG, true, true, true);
+    try {
+      client.dropCatalog(OTHER_CATALOG);
+    } catch (NoSuchObjectException e) {
+      // NOP
+    }
+
+    // Clean up trash
+    metaStore.cleanWarehouseDirs();
+
+    new DatabaseBuilder().setName(OTHER_DATABASE).create(client, metaStore.getConf());
+
+    Catalog cat = new CatalogBuilder()
+        .setName(OTHER_CATALOG)
+        .setLocation(MetaStoreTestUtils.getTestWarehouseDir(OTHER_CATALOG))
+        .build();
+    client.createCatalog(cat);
+
+    // For this one don't specify a location to make sure it gets put in the catalog directory
+    inOtherCatalog = new DatabaseBuilder()
+        .setName(DATABASE_IN_OTHER_CATALOG)
+        .setCatalogName(OTHER_CATALOG)
+        .create(client, metaStore.getConf());
+
+    testTables[0] =
+        new TableBuilder()
+            .setTableName("test_table_1")
+            .addCol("col1", "int")
+            .addCol("col2", "varchar(32)")
+            .create(client, metaStore.getConf());
+
+    testTables[1] =
+        new TableBuilder()
+            .setDbName(OTHER_DATABASE)
+            .setTableName("test_table_2")
+            .addCol("col1", "int")
+            .addCol("col2", "varchar(32)")
+            .create(client, metaStore.getConf());
+
+    testTables[2] =
+        new TableBuilder()
+            .inDb(inOtherCatalog)
+            .setTableName("test_table_3")
+            .addCol("col1", "int")
+            .addCol("col2", "varchar(32)")
+            .create(client, metaStore.getConf());
+
+    // Reload tables from the MetaStore
+    for(int i=0; i < testTables.length; i++) {
+      testTables[i] = client.getTable(testTables[i].getCatName(), testTables[i].getDbName(),
+          testTables[i].getTableName());
+    }
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    try {
+      if (client != null) {
+        client.close();
+      }
+    } finally {
+      client = null;
+    }
+  }
+
+  @Test
+  public void createGetDrop() throws TException {
+    Table table = testTables[0];
+    // Make sure get on a table with no key returns empty list
+    PrimaryKeysRequest rqst =
+        new PrimaryKeysRequest(table.getDbName(), table.getTableName());
+    rqst.setCatName(table.getCatName());
+    List<SQLPrimaryKey> fetched = client.getPrimaryKeys(rqst);
+    Assert.assertTrue(fetched.isEmpty());
+
+    // Single column unnamed primary key in default catalog and database
+    List<SQLPrimaryKey> pk = new SQLPrimaryKeyBuilder()
+        .onTable(table)
+        .addColumn("col1")
+        .build(metaStore.getConf());
+    client.addPrimaryKey(pk);
+
+    rqst = new PrimaryKeysRequest(table.getDbName(), table.getTableName());
+    rqst.setCatName(table.getCatName());
+    fetched = client.getPrimaryKeys(rqst);
+    Assert.assertEquals(1, fetched.size());
+    Assert.assertEquals(table.getDbName(), fetched.get(0).getTable_db());
+    Assert.assertEquals(table.getTableName(), fetched.get(0).getTable_name());
+    Assert.assertEquals("col1", fetched.get(0).getColumn_name());
+    Assert.assertEquals(1, fetched.get(0).getKey_seq());
+    Assert.assertEquals(table.getTableName() + "_primary_key", fetched.get(0).getPk_name());
+    String table0PkName = fetched.get(0).getPk_name();
+    Assert.assertTrue(fetched.get(0).isEnable_cstr());
+    Assert.assertFalse(fetched.get(0).isValidate_cstr());
+    Assert.assertFalse(fetched.get(0).isRely_cstr());
+    Assert.assertEquals(table.getCatName(), fetched.get(0).getCatName());
+
+    // Drop a primary key
+    client.dropConstraint(table.getCatName(), table.getDbName(),
+        table.getTableName(), table0PkName);
+    rqst = new PrimaryKeysRequest(table.getDbName(), table.getTableName());
+    rqst.setCatName(table.getCatName());
+    fetched = client.getPrimaryKeys(rqst);
+    Assert.assertTrue(fetched.isEmpty());
+
+    // Make sure I can add it back
+    client.addPrimaryKey(pk);
+  }
+
+  @Test
+  public void createGetDrop2Column() throws TException {
+    // Make sure get on a table with no key returns empty list
+    Table table = testTables[1];
+    PrimaryKeysRequest rqst =
+        new PrimaryKeysRequest(table.getDbName(), table.getTableName());
+    rqst.setCatName(table.getCatName());
+    List<SQLPrimaryKey> fetched = client.getPrimaryKeys(rqst);
+    Assert.assertTrue(fetched.isEmpty());
+
+    String constraintName = "cgd2cpk";
+    // Multi-column.  Also covers table in non-default database
+    List<SQLPrimaryKey> pk = new SQLPrimaryKeyBuilder()
+        .onTable(table)
+        .addColumn("col1")
+        .addColumn("col2")
+        .setEnable(false)
+        .setConstraintName(constraintName)
+        .setValidate(true)
+        .setRely(true)
+        .build(metaStore.getConf());
+    client.addPrimaryKey(pk);
+
+    rqst = new PrimaryKeysRequest(table.getDbName(), table.getTableName());
+    rqst.setCatName(table.getCatName());
+    fetched = client.getPrimaryKeys(rqst);
+    Assert.assertEquals(2, fetched.size());
+    Assert.assertEquals(table.getDbName(), fetched.get(0).getTable_db());
+    Assert.assertEquals(table.getTableName(), fetched.get(0).getTable_name());
+    Assert.assertEquals("col1", fetched.get(0).getColumn_name());
+    Assert.assertEquals("col2", fetched.get(1).getColumn_name());
+    Assert.assertEquals(1, fetched.get(0).getKey_seq());
+    Assert.assertEquals(2, fetched.get(1).getKey_seq());
+    Assert.assertEquals(constraintName, fetched.get(0).getPk_name());
+    Assert.assertEquals(fetched.get(0).getPk_name(), fetched.get(1).getPk_name());
+    Assert.assertFalse(fetched.get(0).isEnable_cstr());
+    Assert.assertTrue(fetched.get(0).isValidate_cstr());
+    Assert.assertTrue(fetched.get(0).isRely_cstr());
+    Assert.assertEquals(table.getCatName(), fetched.get(0).getCatName());
+
+    // Drop a named primary key
+    client.dropConstraint(table.getCatName(), table.getDbName(), table.getTableName(), constraintName);
+    rqst = new PrimaryKeysRequest(table.getDbName(), table.getTableName());
+    rqst.setCatName(table.getCatName());
+    fetched = client.getPrimaryKeys(rqst);
+    Assert.assertTrue(fetched.isEmpty());
+
+    // Make sure I can add it back
+    client.addPrimaryKey(pk);
+  }
+
+  @Test
+  public void inOtherCatalog() throws TException {
+    PrimaryKeysRequest rqst =
+        new PrimaryKeysRequest(testTables[2].getDbName(), testTables[2].getTableName());
+    rqst.setCatName(testTables[2].getCatName());
+    List<SQLPrimaryKey> fetched = client.getPrimaryKeys(rqst);
+    Assert.assertTrue(fetched.isEmpty());
+
+    String constraintName = "ocpk";
+    // Table in non 'hive' catalog
+    List<SQLPrimaryKey> pk = new SQLPrimaryKeyBuilder()
+        .onTable(testTables[2])
+        .addColumn("col1")
+        .setConstraintName(constraintName)
+        .build(metaStore.getConf());
+    client.addPrimaryKey(pk);
+
+    rqst = new PrimaryKeysRequest(testTables[2].getDbName(), testTables[2].getTableName());
+    rqst.setCatName(testTables[2].getCatName());
+    fetched = client.getPrimaryKeys(rqst);
+    Assert.assertEquals(1, fetched.size());
+    Assert.assertEquals(testTables[2].getDbName(), fetched.get(0).getTable_db());
+    Assert.assertEquals(testTables[2].getTableName(), fetched.get(0).getTable_name());
+    Assert.assertEquals("col1", fetched.get(0).getColumn_name());
+    Assert.assertEquals(1, fetched.get(0).getKey_seq());
+    Assert.assertEquals(constraintName, fetched.get(0).getPk_name());
+    Assert.assertTrue(fetched.get(0).isEnable_cstr());
+    Assert.assertFalse(fetched.get(0).isValidate_cstr());
+    Assert.assertFalse(fetched.get(0).isRely_cstr());
+    Assert.assertEquals(testTables[2].getCatName(), fetched.get(0).getCatName());
+
+    client.dropConstraint(testTables[2].getCatName(), testTables[2].getDbName(),
+        testTables[2].getTableName(), constraintName);
+    rqst = new PrimaryKeysRequest(testTables[2].getDbName(), testTables[2].getTableName());
+    rqst.setCatName(testTables[2].getCatName());
+    fetched = client.getPrimaryKeys(rqst);
+    Assert.assertTrue(fetched.isEmpty());
+  }
+
+  @Test
+  public void createTableWithConstraintsPk() throws TException {
+    String constraintName = "ctwcpk";
+    Table table = new TableBuilder()
+        .setTableName("table_with_constraints")
+        .addCol("col1", "int")
+        .addCol("col2", "varchar(32)")
+        .build(metaStore.getConf());
+
+    List<SQLPrimaryKey> pk = new SQLPrimaryKeyBuilder()
+        .onTable(table)
+        .addColumn("col1")
+        .setConstraintName(constraintName)
+        .build(metaStore.getConf());
+
+    client.createTableWithConstraints(table, pk, null, null, null, null, null);
+    PrimaryKeysRequest rqst = new PrimaryKeysRequest(table.getDbName(), table.getTableName());
+    rqst.setCatName(table.getCatName());
+    List<SQLPrimaryKey> fetched = client.getPrimaryKeys(rqst);
+    Assert.assertEquals(1, fetched.size());
+    Assert.assertEquals(table.getDbName(), fetched.get(0).getTable_db());
+    Assert.assertEquals(table.getTableName(), fetched.get(0).getTable_name());
+    Assert.assertEquals("col1", fetched.get(0).getColumn_name());
+    Assert.assertEquals(1, fetched.get(0).getKey_seq());
+    Assert.assertEquals(constraintName, fetched.get(0).getPk_name());
+    Assert.assertTrue(fetched.get(0).isEnable_cstr());
+    Assert.assertFalse(fetched.get(0).isValidate_cstr());
+    Assert.assertFalse(fetched.get(0).isRely_cstr());
+    Assert.assertEquals(table.getCatName(), fetched.get(0).getCatName());
+
+    client.dropConstraint(table.getCatName(), table.getDbName(), table.getTableName(), constraintName);
+    rqst = new PrimaryKeysRequest(table.getDbName(), table.getTableName());
+    rqst.setCatName(table.getCatName());
+    fetched = client.getPrimaryKeys(rqst);
+    Assert.assertTrue(fetched.isEmpty());
+
+  }
+
+  @Test
+  public void createTableWithConstraintsPkInOtherCatalog() throws TException {
+    Table table = new TableBuilder()
+        .setTableName("table_in_other_catalog_with_constraints")
+        .inDb(inOtherCatalog)
+        .addCol("col1", "int")
+        .addCol("col2", "varchar(32)")
+        .build(metaStore.getConf());
+
+    List<SQLPrimaryKey> pk = new SQLPrimaryKeyBuilder()
+        .onTable(table)
+        .addColumn("col1")
+        .build(metaStore.getConf());
+
+    client.createTableWithConstraints(table, pk, null, null, null, null, null);
+    PrimaryKeysRequest rqst = new PrimaryKeysRequest(table.getDbName(), table.getTableName());
+    rqst.setCatName(table.getCatName());
+    List<SQLPrimaryKey> fetched = client.getPrimaryKeys(rqst);
+    Assert.assertEquals(1, fetched.size());
+    Assert.assertEquals(table.getDbName(), fetched.get(0).getTable_db());
+    Assert.assertEquals(table.getTableName(), fetched.get(0).getTable_name());
+    Assert.assertEquals("col1", fetched.get(0).getColumn_name());
+    Assert.assertEquals(1, fetched.get(0).getKey_seq());
+    Assert.assertEquals(table.getTableName() + "_primary_key", fetched.get(0).getPk_name());
+    String tablePkName = fetched.get(0).getPk_name();
+    Assert.assertTrue(fetched.get(0).isEnable_cstr());
+    Assert.assertFalse(fetched.get(0).isValidate_cstr());
+    Assert.assertFalse(fetched.get(0).isRely_cstr());
+    Assert.assertEquals(table.getCatName(), fetched.get(0).getCatName());
+
+    client.dropConstraint(table.getCatName(), table.getDbName(), table.getTableName(), tablePkName);
+    rqst = new PrimaryKeysRequest(table.getDbName(), table.getTableName());
+    rqst.setCatName(table.getCatName());
+    fetched = client.getPrimaryKeys(rqst);
+    Assert.assertTrue(fetched.isEmpty());
+  }
+
+  @Test
+  public void doubleAddPrimaryKey() throws TException {
+    Table table = testTables[0];
+    // Make sure get on a table with no key returns empty list
+    PrimaryKeysRequest rqst =
+        new PrimaryKeysRequest(table.getDbName(), table.getTableName());
+    rqst.setCatName(table.getCatName());
+    List<SQLPrimaryKey> fetched = client.getPrimaryKeys(rqst);
+    Assert.assertTrue(fetched.isEmpty());
+
+    // Single column unnamed primary key in default catalog and database
+    List<SQLPrimaryKey> pk = new SQLPrimaryKeyBuilder()
+        .onTable(table)
+        .addColumn("col1")
+        .build(metaStore.getConf());
+    client.addPrimaryKey(pk);
+
+    try {
+      pk = new SQLPrimaryKeyBuilder()
+          .onTable(table)
+          .addColumn("col2")
+          .build(metaStore.getConf());
+      client.addPrimaryKey(pk);
+      Assert.fail();
+    } catch (MetaException e) {
+      Assert.assertTrue(e.getMessage().contains("Primary key already exists for"));
+    }
+  }
+
+  @Test
+  public void addNoSuchTable() throws TException {
+    try {
+      List<SQLPrimaryKey> pk = new SQLPrimaryKeyBuilder()
+          .setTableName("nosuch")
+          .addColumn("col2")
+          .build(metaStore.getConf());
+      client.addPrimaryKey(pk);
+      Assert.fail();
+    } catch (InvalidObjectException|TApplicationException e) {
+      // NOP
+    }
+  }
+
+  @Test
+  public void getNoSuchTable() throws TException {
+    PrimaryKeysRequest rqst =
+        new PrimaryKeysRequest(DEFAULT_DATABASE_NAME, "nosuch");
+    List<SQLPrimaryKey> pk = client.getPrimaryKeys(rqst);
+    Assert.assertTrue(pk.isEmpty());
+  }
+
+  @Test
+  public void getNoSuchDb() throws TException {
+    PrimaryKeysRequest rqst =
+        new PrimaryKeysRequest("nosuch", testTables[0].getTableName());
+    List<SQLPrimaryKey> pk = client.getPrimaryKeys(rqst);
+    Assert.assertTrue(pk.isEmpty());
+  }
+
+  @Test
+  public void getNoSuchCatalog() throws TException {
+    PrimaryKeysRequest rqst =
+        new PrimaryKeysRequest(testTables[0].getTableName(), testTables[0].getTableName());
+    rqst.setCatName("nosuch");
+    List<SQLPrimaryKey> pk = client.getPrimaryKeys(rqst);
+    Assert.assertTrue(pk.isEmpty());
+  }
+
+  @Test
+  public void dropNoSuchConstraint() throws TException {
+    try {
+      client.dropConstraint(testTables[0].getCatName(), testTables[0].getDbName(),
+          testTables[0].getTableName(), "nosuch");
+      Assert.fail();
+    } catch (InvalidObjectException|TApplicationException e) {
+      // NOP
+    }
+
+  }
+
+  @Test
+  public void dropNoSuchTable() throws TException {
+    try {
+      client.dropConstraint(testTables[0].getCatName(), testTables[0].getDbName(),
+          "nosuch", "mypk");
+      Assert.fail();
+    } catch (InvalidObjectException|TApplicationException e) {
+      // NOP
+    }
+  }
+
+  @Test
+  public void dropNoSuchDatabase() throws TException {
+    try {
+      client.dropConstraint(testTables[0].getCatName(), "nosuch",
+          testTables[0].getTableName(), "mypk");
+      Assert.fail();
+    } catch (InvalidObjectException|TApplicationException e) {
+      // NOP
+    }
+  }
+
+  @Test
+  public void dropNoSuchCatalog() throws TException {
+    try {
+      client.dropConstraint("nosuch", testTables[0].getDbName(),
+          testTables[0].getTableName(), "nosuch");
+      Assert.fail();
+    } catch (InvalidObjectException|TApplicationException e) {
+      // NOP
+    }
+  }
+  // TODO no fk across catalogs
+}