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/06/14 01:53:07 UTC

[1/2] hive git commit: HIVE-19871 add_partitions, create constraint calls and create_table_with_constraints do not properly handle client being configured with a non-Hive catalog (Alan Gates reviewed by Daniel Dai)

Repository: hive
Updated Branches:
  refs/heads/branch-3.0 f2ee20447 -> 120a79b07


http://git-wip-us.apache.org/repos/asf/hive/blob/120a79b0/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestNonCatCallsWithCatalog.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestNonCatCallsWithCatalog.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestNonCatCallsWithCatalog.java
deleted file mode 100644
index 55ef885..0000000
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestNonCatCallsWithCatalog.java
+++ /dev/null
@@ -1,1126 +0,0 @@
-/*
- * 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;
-
-import com.google.common.collect.Lists;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.metastore.api.CheckConstraintsRequest;
-import org.apache.hadoop.hive.metastore.api.Database;
-import org.apache.hadoop.hive.metastore.api.DefaultConstraintsRequest;
-import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
-import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.apache.hadoop.hive.metastore.api.ForeignKeysRequest;
-import org.apache.hadoop.hive.metastore.api.Function;
-import org.apache.hadoop.hive.metastore.api.FunctionType;
-import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
-import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
-import org.apache.hadoop.hive.metastore.api.NotNullConstraintsRequest;
-import org.apache.hadoop.hive.metastore.api.Partition;
-import org.apache.hadoop.hive.metastore.api.PartitionValuesRequest;
-import org.apache.hadoop.hive.metastore.api.PartitionValuesResponse;
-import org.apache.hadoop.hive.metastore.api.PrimaryKeysRequest;
-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.api.SQLCheckConstraint;
-import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
-import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
-import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
-import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
-import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
-import org.apache.hadoop.hive.metastore.api.SerDeInfo;
-import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
-import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.api.TableMeta;
-import org.apache.hadoop.hive.metastore.api.UniqueConstraintsRequest;
-import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
-import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
-import org.apache.hadoop.hive.metastore.client.builder.FunctionBuilder;
-import org.apache.hadoop.hive.metastore.client.builder.PartitionBuilder;
-import org.apache.hadoop.hive.metastore.client.builder.SQLCheckConstraintBuilder;
-import org.apache.hadoop.hive.metastore.client.builder.SQLDefaultConstraintBuilder;
-import org.apache.hadoop.hive.metastore.client.builder.SQLForeignKeyBuilder;
-import org.apache.hadoop.hive.metastore.client.builder.SQLNotNullConstraintBuilder;
-import org.apache.hadoop.hive.metastore.client.builder.SQLPrimaryKeyBuilder;
-import org.apache.hadoop.hive.metastore.client.builder.SQLUniqueConstraintBuilder;
-import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
-import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
-import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
-import org.apache.thrift.TException;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.File;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.TreeSet;
-
-import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
-
-public abstract class TestNonCatCallsWithCatalog {
-
-  private static final String OTHER_DATABASE = "non_cat_other_db";
-  private Table[] testTables = new Table[6];
-  private static final String TEST_FUNCTION_CLASS =
-      "org.apache.hadoop.hive.ql.udf.generic.GenericUDFUpper";
-
-  protected Configuration conf;
-
-  protected IMetaStoreClient client;
-  protected abstract IMetaStoreClient getClient() throws Exception;
-  protected abstract String expectedCatalog();
-  protected abstract String expectedBaseDir() throws MetaException;
-
-  @Before
-  public void setUp() throws Exception {
-    conf = MetastoreConf.newMetastoreConf();
-    MetaStoreTestUtils.setConfForStandloneMode(conf);
-
-    // Get new client
-    client = getClient();
-
-    List<String> databases = client.getAllDatabases();
-    for (String db : databases) {
-      if (!DEFAULT_DATABASE_NAME.equals(db)) {
-        client.dropDatabase(db, 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);
-    }
-
-    testTables[0] =
-        new TableBuilder()
-            .setTableName("test_table")
-            .addCol("test_col1", "int")
-            .addCol("test_col2", "int")
-            .addCol("test_col3", "int")
-            .create(client, conf);
-
-    testTables[1] =
-        new TableBuilder()
-            .setTableName("test_view")
-            .addCol("test_col1", "int")
-            .addCol("test_col2", "int")
-            .addCol("test_col3", "int")
-            .setType("VIRTUAL_VIEW")
-            .create(client, conf);
-
-    testTables[2] =
-        new TableBuilder()
-            .setTableName("test_table_to_find_1")
-            .addCol("test_col1", "int")
-            .addCol("test_col2", "int")
-            .addCol("test_col3", "int")
-            .create(client, conf);
-
-    testTables[3] =
-        new TableBuilder()
-            .setTableName("test_partitioned_table")
-            .addCol("test_col1", "int")
-            .addCol("test_col2", "int")
-            .addPartCol("test_part_col", "int")
-            .create(client, conf);
-
-    testTables[4] =
-        new TableBuilder()
-            .setTableName("external_table_for_test")
-            .addCol("test_col", "int")
-            .setLocation(MetaStoreTestUtils.getTestWarehouseDir("/external/table_dir"))
-            .addTableParam("EXTERNAL", "TRUE")
-            .setType("EXTERNAL_TABLE")
-            .create(client, conf);
-
-
-    new DatabaseBuilder().setName(OTHER_DATABASE).create(client, conf);
-
-    testTables[5] =
-        new TableBuilder()
-            .setDbName(OTHER_DATABASE)
-            .setTableName("test_table")
-            .addCol("test_col", "int")
-            .create(client, conf);
-
-    // Create partitions for the partitioned table
-    for(int i=0; i < 3; i++) {
-      new PartitionBuilder()
-          .inTable(testTables[3])
-          .addValue("a" + i)
-          .addToTable(client, conf);
-    }
-
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    try {
-      if (client != null) {
-        client.close();
-      }
-    } finally {
-      client = null;
-    }
-  }
-
-  @Test
-  public void databases() throws TException, URISyntaxException {
-    String[] dbNames = {"db1", "db9"};
-    Database[] dbs = new Database[2];
-    // For this one don't specify a location to make sure it gets put in the catalog directory
-    dbs[0] = new DatabaseBuilder()
-        .setName(dbNames[0])
-        .create(client, conf);
-
-    // For the second one, explicitly set a location to make sure it ends up in the specified place.
-    String db1Location = MetaStoreTestUtils.getTestWarehouseDir(dbNames[1]);
-    dbs[1] = new DatabaseBuilder()
-        .setName(dbNames[1])
-        .setLocation(db1Location)
-        .create(client, conf);
-
-    Database fetched = client.getDatabase(dbNames[0]);
-    String expectedLocation = new File(expectedBaseDir(), dbNames[0] + ".db").toURI().toString();
-    Assert.assertEquals(expectedCatalog(), fetched.getCatalogName());
-    Assert.assertEquals(expectedLocation, fetched.getLocationUri() + "/");
-    String db0Location = new URI(fetched.getLocationUri()).getPath();
-    File dir = new File(db0Location);
-    Assert.assertTrue(dir.exists() && dir.isDirectory());
-    Assert.assertEquals(expectedCatalog(), fetched.getCatalogName());
-
-    fetched = client.getDatabase(dbNames[1]);
-    Assert.assertEquals(new File(db1Location).toURI().toString(), fetched.getLocationUri() + "/");
-    dir = new File(new URI(fetched.getLocationUri()).getPath());
-    Assert.assertTrue(dir.exists() && dir.isDirectory());
-    Assert.assertEquals(expectedCatalog(), fetched.getCatalogName());
-
-    Set<String> fetchedDbs = new HashSet<>(client.getAllDatabases());
-    for (String dbName : dbNames) Assert.assertTrue(fetchedDbs.contains(dbName));
-
-    fetchedDbs = new HashSet<>(client.getDatabases("db*"));
-    Assert.assertEquals(2, fetchedDbs.size());
-    for (String dbName : dbNames) Assert.assertTrue(fetchedDbs.contains(dbName));
-
-    client.dropDatabase(dbNames[0], true, false, false);
-    dir = new File(db0Location);
-    Assert.assertFalse(dir.exists());
-
-    client.dropDatabase(dbNames[1], true, false, false);
-    dir = new File(db1Location);
-    Assert.assertFalse(dir.exists());
-
-    fetchedDbs = new HashSet<>(client.getAllDatabases());
-    for (String dbName : dbNames) Assert.assertFalse(fetchedDbs.contains(dbName));
-  }
-
-  @Test
-  public void tablesCreateDropAlterTruncate() throws TException, URISyntaxException {
-    String dbName = "db_in_other_catalog";
-    // For this one don't specify a location to make sure it gets put in the catalog directory
-    Database db = new DatabaseBuilder()
-        .setName(dbName)
-        .create(client, conf);
-
-    String[] tableNames = new String[4];
-    for (int i = 0; i < tableNames.length; i++) {
-      tableNames[i] = "table_in_other_catalog_" + i;
-      TableBuilder builder = new TableBuilder()
-          .inDb(db)
-          .setTableName(tableNames[i])
-          .addCol("col1_" + i, ColumnType.STRING_TYPE_NAME)
-          .addCol("col2_" + i, ColumnType.INT_TYPE_NAME);
-      // Make one have a non-standard location
-      if (i == 0) builder.setLocation(MetaStoreTestUtils.getTestWarehouseDir(tableNames[i]));
-      // Make one partitioned
-      if (i == 2) builder.addPartCol("pcol1", ColumnType.STRING_TYPE_NAME);
-      // Make one a materialized view
-      /*
-      // TODO HIVE-18991
-      if (i == 3) {
-        builder.setType(TableType.MATERIALIZED_VIEW.name())
-            .setRewriteEnabled(true)
-            .addMaterializedViewReferencedTable(dbName + "." + tableNames[0]);
-      }
-      */
-      client.createTable(builder.build(conf));
-    }
-
-    // Add partitions for the partitioned table
-    String[] partVals = new String[3];
-    Table partitionedTable = client.getTable(dbName, tableNames[2]);
-    for (int i = 0; i < partVals.length; i++) {
-      partVals[i] = "part" + i;
-      new PartitionBuilder()
-          .inTable(partitionedTable)
-          .addValue(partVals[i])
-          .addToTable(client, conf);
-    }
-
-    // Get tables, make sure the locations are correct
-    for (int i = 0; i < tableNames.length; i++) {
-      Table t = client.getTable(dbName, tableNames[i]);
-      Assert.assertEquals(expectedCatalog(), t.getCatName());
-      String expectedLocation = (i < 1) ?
-          new File(MetaStoreTestUtils.getTestWarehouseDir(tableNames[i])).toURI().toString()
-          :
-          new File(expectedBaseDir() + File.separatorChar + dbName + ".db",
-              tableNames[i]).toURI().toString();
-
-      Assert.assertEquals(expectedLocation, t.getSd().getLocation() + "/");
-      File dir = new File(new URI(t.getSd().getLocation()).getPath());
-      Assert.assertTrue(dir.exists() && dir.isDirectory());
-
-    }
-
-    // Make sure getting table in the wrong catalog does not work
-    try {
-      Table t = client.getTable(DEFAULT_DATABASE_NAME, tableNames[0]);
-      Assert.fail();
-    } catch (NoSuchObjectException e) {
-      // NOP
-    }
-
-    // test getAllTables
-    Set<String> fetchedNames = new HashSet<>(client.getAllTables(dbName));
-    Assert.assertEquals(tableNames.length, fetchedNames.size());
-    for (String tableName : tableNames) Assert.assertTrue(fetchedNames.contains(tableName));
-
-    fetchedNames = new HashSet<>(client.getAllTables(DEFAULT_DATABASE_NAME));
-    for (String tableName : tableNames) Assert.assertFalse(fetchedNames.contains(tableName));
-
-    // test getMaterializedViewsForRewriting
-    /* TODO HIVE-18991
-    List<String> materializedViews = client.getMaterializedViewsForRewriting(dbName);
-    Assert.assertEquals(1, materializedViews.size());
-    Assert.assertEquals(tableNames[3], materializedViews.get(0));
-    */
-
-    fetchedNames = new HashSet<>(client.getMaterializedViewsForRewriting(DEFAULT_DATABASE_NAME));
-    Assert.assertFalse(fetchedNames.contains(tableNames[3]));
-
-    // test getTableObjectsByName
-    List<Table> fetchedTables = client.getTableObjectsByName(dbName,
-        Arrays.asList(tableNames[0], tableNames[1]));
-    Assert.assertEquals(2, fetchedTables.size());
-    Collections.sort(fetchedTables);
-    Assert.assertEquals(tableNames[0], fetchedTables.get(0).getTableName());
-    Assert.assertEquals(tableNames[1], fetchedTables.get(1).getTableName());
-
-    fetchedTables = client.getTableObjectsByName(DEFAULT_DATABASE_NAME,
-        Arrays.asList(tableNames[0], tableNames[1]));
-    Assert.assertEquals(0, fetchedTables.size());
-
-    // Test altering the table
-    Table t = client.getTable(dbName, tableNames[0]).deepCopy();
-    t.getParameters().put("test", "test");
-    client.alter_table(dbName, tableNames[0], t);
-    t = client.getTable(dbName, tableNames[0]).deepCopy();
-    Assert.assertEquals("test", t.getParameters().get("test"));
-
-    // Alter a table in the wrong catalog
-    try {
-      client.alter_table(DEFAULT_DATABASE_NAME, tableNames[0], t);
-      Assert.fail();
-    } catch (InvalidOperationException e) {
-      // NOP
-    }
-
-    // Update the metadata for the materialized view
-    /* TODO HIVE-18991
-    CreationMetadata cm = client.getTable(dbName, tableNames[3]).getCreationMetadata();
-    cm.addToTablesUsed(dbName + "." + tableNames[1]);
-    client.updateCreationMetadata(dbName, tableNames[3], cm);
-    */
-
-    List<String> partNames = new ArrayList<>();
-    for (String partVal : partVals) partNames.add("pcol1=" + partVal);
-    // Truncate a table
-    client.truncateTable(dbName, tableNames[0], partNames);
-
-    // Have to do this in reverse order so that we drop the materialized view first.
-    for (int i = tableNames.length - 1; i >= 0; i--) {
-      t = client.getTable(dbName, tableNames[i]);
-      File tableDir = new File(new URI(t.getSd().getLocation()).getPath());
-      Assert.assertTrue(tableDir.exists() && tableDir.isDirectory());
-
-      if (tableNames[i].equalsIgnoreCase(tableNames[0])) {
-        client.dropTable(dbName, tableNames[i], false, false);
-        Assert.assertTrue(tableDir.exists() && tableDir.isDirectory());
-      } else {
-        client.dropTable(dbName, tableNames[i]);
-        Assert.assertFalse(tableDir.exists());
-      }
-    }
-    Assert.assertEquals(0, client.getAllTables(dbName).size());
-  }
-
-  @Test
-  public void tablesGetExists() throws TException {
-    String dbName = "db_in_other_catalog";
-    // For this one don't specify a location to make sure it gets put in the catalog directory
-    Database db = new DatabaseBuilder()
-        .setName(dbName)
-        .create(client, conf);
-
-    String[] tableNames = new String[4];
-    for (int i = 0; i < tableNames.length; i++) {
-      tableNames[i] = "table_in_other_catalog_" + i;
-      new TableBuilder()
-          .inDb(db)
-          .setTableName(tableNames[i])
-          .addCol("col1_" + i, ColumnType.STRING_TYPE_NAME)
-          .addCol("col2_" + i, ColumnType.INT_TYPE_NAME)
-          .create(client, conf);
-    }
-
-    Set<String> tables = new HashSet<>(client.getTables(dbName, "*e_in_other_*"));
-    Assert.assertEquals(4, tables.size());
-    for (String tableName : tableNames) Assert.assertTrue(tables.contains(tableName));
-
-    List<String> fetchedNames = client.getTables(dbName, "*_3");
-    Assert.assertEquals(1, fetchedNames.size());
-    Assert.assertEquals(tableNames[3], fetchedNames.get(0));
-
-    Assert.assertTrue("Table exists", client.tableExists(dbName, tableNames[0]));
-    Assert.assertFalse("Table not exists", client.tableExists(dbName, "non_existing_table"));
-  }
-
-  @Test
-  public void tablesList() throws TException {
-    String dbName = "db_in_other_catalog";
-    // For this one don't specify a location to make sure it gets put in the catalog directory
-    Database db = new DatabaseBuilder()
-        .setName(dbName)
-        .create(client, conf);
-
-    String[] tableNames = new String[4];
-    for (int i = 0; i < tableNames.length; i++) {
-      tableNames[i] = "table_in_other_catalog_" + i;
-      TableBuilder builder = new TableBuilder()
-          .inDb(db)
-          .setTableName(tableNames[i])
-          .addCol("col1_" + i, ColumnType.STRING_TYPE_NAME)
-          .addCol("col2_" + i, ColumnType.INT_TYPE_NAME);
-      if (i == 0) builder.addTableParam("the_key", "the_value");
-      builder.create(client, conf);
-    }
-
-    String filter = hive_metastoreConstants.HIVE_FILTER_FIELD_PARAMS + "the_key=\"the_value\"";
-    List<String> fetchedNames = client.listTableNamesByFilter(dbName, filter, (short)-1);
-    Assert.assertEquals(1, fetchedNames.size());
-    Assert.assertEquals(tableNames[0], fetchedNames.get(0));
-  }
-
-  @Test
-  public void getTableMeta() throws TException {
-    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)
-        .create(client, conf);
-
-    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(conf));
-      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(dbName, "*", types);
-    Assert.assertEquals(new TreeSet<>(expected), new TreeSet<>(actual));
-
-    actual = client.getTableMeta("*", "table_*", types);
-    Assert.assertEquals(expected.subList(0, 2), actual.subList(0, 2));
-
-  }
-
-  @Test
-  public void addPartitions() throws TException {
-    String dbName = "add_partition_database_in_other_catalog";
-    Database db = new DatabaseBuilder()
-        .setName(dbName)
-        .create(client, conf);
-
-    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, conf);
-
-    Partition[] parts = new Partition[5];
-    for (int i = 0; i < parts.length; i++) {
-      parts[i] = new PartitionBuilder()
-          .inTable(table)
-          .addValue("a" + i)
-          .build(conf);
-    }
-    client.add_partition(parts[0]);
-    Assert.assertEquals(2, client.add_partitions(Arrays.asList(parts[1], parts[2])));
-    client.add_partitions(Arrays.asList(parts), true, false);
-
-    for (int i = 0; i < parts.length; i++) {
-      Partition fetched = client.getPartition(dbName, tableName,
-          Collections.singletonList("a" + i));
-      Assert.assertEquals(dbName, fetched.getDbName());
-      Assert.assertEquals(tableName, fetched.getTableName());
-      Assert.assertEquals(expectedCatalog(), fetched.getCatName());
-    }
-
-    client.dropDatabase(dbName, true, true, true);
-  }
-
-  @Test
-  public void getPartitions() throws TException {
-    String dbName = "get_partition_database_in_other_catalog";
-    Database db = new DatabaseBuilder()
-        .setName(dbName)
-        .create(client, conf);
-
-    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, conf);
-
-    Partition[] parts = new Partition[5];
-    for (int i = 0; i < parts.length; i++) {
-      parts[i] = new PartitionBuilder()
-          .inTable(table)
-          .addValue("a" + i)
-          .build(conf);
-    }
-    client.add_partitions(Arrays.asList(parts));
-
-    Partition fetched = client.getPartition(dbName, tableName,
-        Collections.singletonList("a0"));
-    Assert.assertEquals(expectedCatalog(), fetched.getCatName());
-    Assert.assertEquals("a0", fetched.getValues().get(0));
-
-    fetched = client.getPartition(dbName, tableName, "partcol=a0");
-    Assert.assertEquals(expectedCatalog(), fetched.getCatName());
-    Assert.assertEquals("a0", fetched.getValues().get(0));
-
-    List<Partition> fetchedParts = client.getPartitionsByNames(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
-  public void listPartitions() throws TException {
-    String dbName = "list_partition_database_in_other_catalog";
-    Database db = new DatabaseBuilder()
-        .setName(dbName)
-        .create(client, conf);
-
-    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, conf);
-
-    Partition[] parts = new Partition[5];
-    for (int i = 0; i < parts.length; i++) {
-      parts[i] = new PartitionBuilder()
-          .inTable(table)
-          .addValue("a" + i)
-          .build(conf);
-    }
-    client.add_partitions(Arrays.asList(parts));
-
-    List<Partition> fetched = client.listPartitions(dbName, tableName, (short)-1);
-    Assert.assertEquals(parts.length, fetched.size());
-    Assert.assertEquals(expectedCatalog(), fetched.get(0).getCatName());
-
-    fetched = client.listPartitions(dbName, tableName,
-        Collections.singletonList("a0"), (short)-1);
-    Assert.assertEquals(1, fetched.size());
-    Assert.assertEquals(expectedCatalog(), fetched.get(0).getCatName());
-
-    PartitionSpecProxy proxy = client.listPartitionSpecs(dbName, tableName, -1);
-    Assert.assertEquals(parts.length, proxy.size());
-    Assert.assertEquals(expectedCatalog(), proxy.getCatName());
-
-    fetched = client.listPartitionsByFilter(dbName, tableName, "partcol=\"a0\"", (short)-1);
-    Assert.assertEquals(1, fetched.size());
-    Assert.assertEquals(expectedCatalog(), fetched.get(0).getCatName());
-
-    proxy = client.listPartitionSpecsByFilter(dbName, tableName, "partcol=\"a0\"", -1);
-    Assert.assertEquals(1, proxy.size());
-    Assert.assertEquals(expectedCatalog(), proxy.getCatName());
-
-    Assert.assertEquals(1, client.getNumPartitionsByFilter(dbName, tableName,
-        "partcol=\"a0\""));
-
-    List<String> names = client.listPartitionNames(dbName, tableName, (short)57);
-    Assert.assertEquals(parts.length, names.size());
-
-    names = client.listPartitionNames(dbName, tableName, Collections.singletonList("a0"),
-        Short.MAX_VALUE);
-    Assert.assertEquals(1, names.size());
-
-    PartitionValuesRequest rqst = new PartitionValuesRequest(dbName,
-        tableName, Lists.newArrayList(new FieldSchema("partcol", "string", "")));
-    PartitionValuesResponse rsp = client.listPartitionValues(rqst);
-    Assert.assertEquals(5, rsp.getPartitionValuesSize());
-  }
-
-  @Test
-  public void alterPartitions() throws TException {
-    String dbName = "alter_partition_database_in_other_catalog";
-    Database db = new DatabaseBuilder()
-        .setName(dbName)
-        .create(client, conf);
-
-    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, conf);
-
-    Partition[] parts = new Partition[5];
-    for (int i = 0; i < 5; i++) {
-      parts[i] = new PartitionBuilder()
-          .inTable(table)
-          .addValue("a" + i)
-          .setLocation(MetaStoreTestUtils.getTestWarehouseDir("b" + i))
-          .build(conf);
-    }
-    client.add_partitions(Arrays.asList(parts));
-
-    Partition newPart =
-        client.getPartition(dbName, tableName, Collections.singletonList("a0"));
-    newPart.getParameters().put("test_key", "test_value");
-    client.alter_partition(dbName, tableName, newPart);
-
-    Partition fetched =
-        client.getPartition(dbName, tableName, Collections.singletonList("a0"));
-    Assert.assertEquals("test_value", fetched.getParameters().get("test_key"));
-
-    newPart =
-        client.getPartition(dbName, tableName, Collections.singletonList("a1"));
-    newPart.setLastAccessTime(3);
-    Partition newPart1 =
-        client.getPartition(dbName, tableName, Collections.singletonList("a2"));
-    newPart1.getSd().setLocation(MetaStoreTestUtils.getTestWarehouseDir("somewhere"));
-    client.alter_partitions(dbName, tableName, Arrays.asList(newPart, newPart1));
-    fetched =
-        client.getPartition(dbName, tableName, Collections.singletonList("a1"));
-    Assert.assertEquals(3L, fetched.getLastAccessTime());
-    fetched =
-        client.getPartition(dbName, tableName, Collections.singletonList("a2"));
-    Assert.assertTrue(fetched.getSd().getLocation().contains("somewhere"));
-
-    newPart =
-        client.getPartition(dbName, tableName, Collections.singletonList("a4"));
-    newPart.getParameters().put("test_key", "test_value");
-    EnvironmentContext ec = new EnvironmentContext();
-    ec.setProperties(Collections.singletonMap("a", "b"));
-    client.alter_partition(dbName, tableName, newPart, ec);
-    fetched =
-        client.getPartition(dbName, tableName, Collections.singletonList("a4"));
-    Assert.assertEquals("test_value", fetched.getParameters().get("test_key"));
-
-
-    client.dropDatabase(dbName, true, true, true);
-  }
-
-  @Test
-  public void dropPartitions() throws TException {
-    String dbName = "drop_partition_database_in_other_catalog";
-    Database db = new DatabaseBuilder()
-        .setName(dbName)
-        .create(client, conf);
-
-    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, conf);
-
-    Partition[] parts = new Partition[2];
-    for (int i = 0; i < parts.length; i++) {
-      parts[i] = new PartitionBuilder()
-          .inTable(table)
-          .addValue("a" + i)
-          .build(conf);
-    }
-    client.add_partitions(Arrays.asList(parts));
-    List<Partition> fetched = client.listPartitions(dbName, tableName, (short)-1);
-    Assert.assertEquals(parts.length, fetched.size());
-
-    Assert.assertTrue(client.dropPartition(dbName, tableName,
-        Collections.singletonList("a0"), PartitionDropOptions.instance().ifExists(false)));
-    try {
-      client.getPartition(dbName, tableName, Collections.singletonList("a0"));
-      Assert.fail();
-    } catch (NoSuchObjectException e) {
-      // NOP
-    }
-
-    Assert.assertTrue(client.dropPartition(dbName, tableName, "partcol=a1", true));
-    try {
-      client.getPartition(dbName, tableName, Collections.singletonList("a1"));
-      Assert.fail();
-    } catch (NoSuchObjectException e) {
-      // NOP
-    }
-  }
-
-  @Test
-  public void primaryKeyAndForeignKey() throws TException {
-    Table parentTable = testTables[2];
-    Table table = testTables[3];
-    String constraintName = "othercatfk";
-
-    // Single column unnamed primary key in default catalog and database
-    List<SQLPrimaryKey> pk = new SQLPrimaryKeyBuilder()
-        .onTable(parentTable)
-        .addColumn("test_col1")
-        .build(conf);
-    client.addPrimaryKey(pk);
-
-    List<SQLForeignKey> fk = new SQLForeignKeyBuilder()
-        .fromPrimaryKey(pk)
-        .onTable(table)
-        .addColumn("test_col1")
-        .setConstraintName(constraintName)
-        .build(conf);
-    client.addForeignKey(fk);
-
-    PrimaryKeysRequest pkRqst = new PrimaryKeysRequest(parentTable.getDbName(),
-        parentTable.getTableName());
-    pkRqst.setCatName(parentTable.getCatName());
-    List<SQLPrimaryKey> pkFetched = client.getPrimaryKeys(pkRqst);
-    Assert.assertEquals(1, pkFetched.size());
-    Assert.assertEquals(expectedCatalog(), pkFetched.get(0).getCatName());
-    Assert.assertEquals(parentTable.getDbName(), pkFetched.get(0).getTable_db());
-    Assert.assertEquals(parentTable.getTableName(), pkFetched.get(0).getTable_name());
-    Assert.assertEquals("test_col1", pkFetched.get(0).getColumn_name());
-    Assert.assertEquals(1, pkFetched.get(0).getKey_seq());
-    Assert.assertTrue(pkFetched.get(0).isEnable_cstr());
-    Assert.assertFalse(pkFetched.get(0).isValidate_cstr());
-    Assert.assertFalse(pkFetched.get(0).isRely_cstr());
-    Assert.assertEquals(parentTable.getCatName(), pkFetched.get(0).getCatName());
-
-    ForeignKeysRequest rqst = new ForeignKeysRequest(parentTable.getDbName(),
-        parentTable.getTableName(), table.getDbName(), table.getTableName());
-    rqst.setCatName(table.getCatName());
-    List<SQLForeignKey> fetched = client.getForeignKeys(rqst);
-    Assert.assertEquals(1, fetched.size());
-    Assert.assertEquals(table.getDbName(), fetched.get(0).getFktable_db());
-    Assert.assertEquals(table.getTableName(), fetched.get(0).getFktable_name());
-    Assert.assertEquals(expectedCatalog(), fetched.get(0).getCatName());
-    Assert.assertEquals("test_col1", fetched.get(0).getFkcolumn_name());
-    Assert.assertEquals(parentTable.getDbName(), fetched.get(0).getPktable_db());
-    Assert.assertEquals(parentTable.getTableName(), fetched.get(0).getPktable_name());
-    Assert.assertEquals("test_col1", fetched.get(0).getFkcolumn_name());
-    Assert.assertEquals(1, fetched.get(0).getKey_seq());
-    Assert.assertEquals(parentTable.getTableName() + "_primary_key", fetched.get(0).getPk_name());
-    Assert.assertEquals(constraintName, fetched.get(0).getFk_name());
-    String table0FkName = fetched.get(0).getFk_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 foreign key
-    client.dropConstraint(table.getDbName(), table.getTableName(), table0FkName);
-    rqst = new ForeignKeysRequest(parentTable.getDbName(), parentTable.getTableName(),
-        table.getDbName(), table.getTableName());
-    rqst.setCatName(table.getCatName());
-    fetched = client.getForeignKeys(rqst);
-    Assert.assertTrue(fetched.isEmpty());
-  }
-
-  @Test
-  public void notNullConstraint() throws TException {
-    String constraintName = "ocuc";
-    // Table in non 'hive' catalog
-    List<SQLNotNullConstraint> nn = new SQLNotNullConstraintBuilder()
-        .onTable(testTables[2])
-        .addColumn("test_col1")
-        .setConstraintName(constraintName)
-        .build(conf);
-    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("test_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].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 uniqueConstraint() throws TException {
-    String constraintName = "ocuc";
-    // Table in non 'hive' catalog
-    List<SQLUniqueConstraint> uc = new SQLUniqueConstraintBuilder()
-        .onTable(testTables[2])
-        .addColumn("test_col1")
-        .setConstraintName(constraintName)
-        .build(conf);
-    client.addUniqueConstraint(uc);
-
-    UniqueConstraintsRequest rqst = new UniqueConstraintsRequest(testTables[2].getCatName(),
-        testTables[2].getDbName(), testTables[2].getTableName());
-    List<SQLUniqueConstraint> fetched = client.getUniqueConstraints(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("test_col1", fetched.get(0).getColumn_name());
-    Assert.assertEquals(1, fetched.get(0).getKey_seq());
-    Assert.assertEquals(constraintName, fetched.get(0).getUk_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());
-    Assert.assertEquals(expectedCatalog(), fetched.get(0).getCatName());
-
-    client.dropConstraint(testTables[2].getDbName(), testTables[2].getTableName(), constraintName);
-    rqst = new UniqueConstraintsRequest(testTables[2].getCatName(), testTables[2].getDbName(),
-        testTables[2].getTableName());
-    fetched = client.getUniqueConstraints(rqst);
-    Assert.assertTrue(fetched.isEmpty());
-  }
-
-  @Test
-  public void defaultConstraints() throws TException {
-    String constraintName = "ocdv";
-    // Table in non 'hive' catalog
-    List<SQLDefaultConstraint> dv = new SQLDefaultConstraintBuilder()
-        .onTable(testTables[2])
-        .addColumn("test_col1")
-        .setConstraintName(constraintName)
-        .setDefaultVal("empty")
-        .build(conf);
-    client.addDefaultConstraint(dv);
-
-    DefaultConstraintsRequest rqst = new DefaultConstraintsRequest(testTables[2].getCatName(),
-        testTables[2].getDbName(), testTables[2].getTableName());
-    List<SQLDefaultConstraint> fetched = client.getDefaultConstraints(rqst);
-    Assert.assertEquals(1, fetched.size());
-    Assert.assertEquals(expectedCatalog(), fetched.get(0).getCatName());
-    Assert.assertEquals(testTables[2].getDbName(), fetched.get(0).getTable_db());
-    Assert.assertEquals(testTables[2].getTableName(), fetched.get(0).getTable_name());
-    Assert.assertEquals("test_col1", fetched.get(0).getColumn_name());
-    Assert.assertEquals("empty", fetched.get(0).getDefault_value());
-    Assert.assertEquals(constraintName, fetched.get(0).getDc_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].getDbName(), testTables[2].getTableName(), constraintName);
-    rqst = new DefaultConstraintsRequest(testTables[2].getCatName(), testTables[2].getDbName(),
-        testTables[2].getTableName());
-    fetched = client.getDefaultConstraints(rqst);
-    Assert.assertTrue(fetched.isEmpty());
-  }
-
-  @Test
-  public void createTableWithConstraints() throws TException {
-    Table parentTable = testTables[2];
-
-
-    Table table = new TableBuilder()
-        .setTableName("table_in_other_catalog_with_constraints")
-        .addCol("col1", "int")
-        .addCol("col2", "varchar(32)")
-        .addCol("col3", "int")
-        .addCol("col4", "int")
-        .addCol("col5", "int")
-        .addCol("col6", "int")
-        .build(conf);
-
-    List<SQLPrimaryKey> parentPk = new SQLPrimaryKeyBuilder()
-        .onTable(parentTable)
-        .addColumn("test_col1")
-        .build(conf);
-    client.addPrimaryKey(parentPk);
-
-    List<SQLPrimaryKey> pk = new SQLPrimaryKeyBuilder()
-        .onTable(table)
-        .addColumn("col2")
-        .build(conf);
-
-    List<SQLForeignKey> fk = new SQLForeignKeyBuilder()
-        .fromPrimaryKey(parentPk)
-        .onTable(table)
-        .addColumn("col1")
-        .build(conf);
-
-    List<SQLDefaultConstraint> dv = new SQLDefaultConstraintBuilder()
-        .onTable(table)
-        .addColumn("col3")
-        .setDefaultVal(0)
-        .build(conf);
-
-    List<SQLNotNullConstraint> nn = new SQLNotNullConstraintBuilder()
-        .onTable(table)
-        .addColumn("col4")
-        .build(conf);
-
-    List<SQLUniqueConstraint> uc = new SQLUniqueConstraintBuilder()
-        .onTable(table)
-        .addColumn("col5")
-        .build(conf);
-
-    List<SQLCheckConstraint> cc = new SQLCheckConstraintBuilder()
-        .onTable(table)
-        .addColumn("col6")
-        .setCheckExpression("> 0")
-        .build(conf);
-
-    client.createTableWithConstraints(table, pk, fk, uc, nn, dv, cc);
-
-    PrimaryKeysRequest pkRqst = new PrimaryKeysRequest(parentTable.getDbName(),
-        parentTable.getTableName());
-    pkRqst.setCatName(parentTable.getCatName());
-    List<SQLPrimaryKey> pkFetched = client.getPrimaryKeys(pkRqst);
-    Assert.assertEquals(1, pkFetched.size());
-    Assert.assertEquals(expectedCatalog(), pkFetched.get(0).getCatName());
-    Assert.assertEquals(parentTable.getDbName(), pkFetched.get(0).getTable_db());
-    Assert.assertEquals(parentTable.getTableName(), pkFetched.get(0).getTable_name());
-    Assert.assertEquals("test_col1", pkFetched.get(0).getColumn_name());
-    Assert.assertEquals(1, pkFetched.get(0).getKey_seq());
-    Assert.assertTrue(pkFetched.get(0).isEnable_cstr());
-    Assert.assertFalse(pkFetched.get(0).isValidate_cstr());
-    Assert.assertFalse(pkFetched.get(0).isRely_cstr());
-    Assert.assertEquals(parentTable.getCatName(), pkFetched.get(0).getCatName());
-
-    ForeignKeysRequest fkRqst = new ForeignKeysRequest(parentTable.getDbName(), parentTable
-        .getTableName(),
-        table.getDbName(), table.getTableName());
-    fkRqst.setCatName(table.getCatName());
-    List<SQLForeignKey> fkFetched = client.getForeignKeys(fkRqst);
-    Assert.assertEquals(1, fkFetched.size());
-    Assert.assertEquals(expectedCatalog(), fkFetched.get(0).getCatName());
-    Assert.assertEquals(table.getDbName(), fkFetched.get(0).getFktable_db());
-    Assert.assertEquals(table.getTableName(), fkFetched.get(0).getFktable_name());
-    Assert.assertEquals("col1", fkFetched.get(0).getFkcolumn_name());
-    Assert.assertEquals(parentTable.getDbName(), fkFetched.get(0).getPktable_db());
-    Assert.assertEquals(parentTable.getTableName(), fkFetched.get(0).getPktable_name());
-    Assert.assertEquals(1, fkFetched.get(0).getKey_seq());
-    Assert.assertEquals(parentTable.getTableName() + "_primary_key", fkFetched.get(0).getPk_name());
-    Assert.assertTrue(fkFetched.get(0).isEnable_cstr());
-    Assert.assertFalse(fkFetched.get(0).isValidate_cstr());
-    Assert.assertFalse(fkFetched.get(0).isRely_cstr());
-    Assert.assertEquals(table.getCatName(), fkFetched.get(0).getCatName());
-
-    NotNullConstraintsRequest nnRqst = new NotNullConstraintsRequest(table.getCatName(),
-        table.getDbName(), table.getTableName());
-    List<SQLNotNullConstraint> nnFetched = client.getNotNullConstraints(nnRqst);
-    Assert.assertEquals(1, nnFetched.size());
-    Assert.assertEquals(table.getDbName(), nnFetched.get(0).getTable_db());
-    Assert.assertEquals(table.getTableName(), nnFetched.get(0).getTable_name());
-    Assert.assertEquals("col4", nnFetched.get(0).getColumn_name());
-    Assert.assertEquals(table.getTableName() + "_not_null_constraint", nnFetched.get(0).getNn_name());
-    Assert.assertTrue(nnFetched.get(0).isEnable_cstr());
-    Assert.assertFalse(nnFetched.get(0).isValidate_cstr());
-    Assert.assertFalse(nnFetched.get(0).isRely_cstr());
-    Assert.assertEquals(table.getCatName(), nnFetched.get(0).getCatName());
-
-    UniqueConstraintsRequest ucRqst = new UniqueConstraintsRequest(table.getCatName(), table
-        .getDbName(), table.getTableName());
-    List<SQLUniqueConstraint> ucFetched = client.getUniqueConstraints(ucRqst);
-    Assert.assertEquals(1, ucFetched.size());
-    Assert.assertEquals(table.getDbName(), ucFetched.get(0).getTable_db());
-    Assert.assertEquals(table.getTableName(), ucFetched.get(0).getTable_name());
-    Assert.assertEquals("col5", ucFetched.get(0).getColumn_name());
-    Assert.assertEquals(1, ucFetched.get(0).getKey_seq());
-    Assert.assertEquals(table.getTableName() + "_unique_constraint", ucFetched.get(0).getUk_name());
-    Assert.assertTrue(ucFetched.get(0).isEnable_cstr());
-    Assert.assertFalse(ucFetched.get(0).isValidate_cstr());
-    Assert.assertFalse(ucFetched.get(0).isRely_cstr());
-    Assert.assertEquals(table.getCatName(), ucFetched.get(0).getCatName());
-
-    DefaultConstraintsRequest dcRqst = new DefaultConstraintsRequest(table.getCatName(), table
-        .getDbName(), table.getTableName());
-    List<SQLDefaultConstraint> dcFetched = client.getDefaultConstraints(dcRqst);
-    Assert.assertEquals(1, dcFetched.size());
-    Assert.assertEquals(expectedCatalog(), dcFetched.get(0).getCatName());
-    Assert.assertEquals(table.getDbName(), dcFetched.get(0).getTable_db());
-    Assert.assertEquals(table.getTableName(), dcFetched.get(0).getTable_name());
-    Assert.assertEquals("col3", dcFetched.get(0).getColumn_name());
-    Assert.assertEquals("0", dcFetched.get(0).getDefault_value());
-    Assert.assertEquals(table.getTableName() + "_default_value", dcFetched.get(0).getDc_name());
-    Assert.assertTrue(dcFetched.get(0).isEnable_cstr());
-    Assert.assertFalse(dcFetched.get(0).isValidate_cstr());
-    Assert.assertFalse(dcFetched.get(0).isRely_cstr());
-    Assert.assertEquals(table.getCatName(), dcFetched.get(0).getCatName());
-
-    CheckConstraintsRequest ccRqst = new CheckConstraintsRequest(table.getCatName(), table
-        .getDbName(), table.getTableName());
-    List<SQLCheckConstraint> ccFetched = client.getCheckConstraints(ccRqst);
-    Assert.assertEquals(1, ccFetched.size());
-    Assert.assertEquals(expectedCatalog(), ccFetched.get(0).getCatName());
-    Assert.assertEquals(table.getDbName(), ccFetched.get(0).getTable_db());
-    Assert.assertEquals(table.getTableName(), ccFetched.get(0).getTable_name());
-    Assert.assertEquals("col6", ccFetched.get(0).getColumn_name());
-    Assert.assertEquals("> 0", ccFetched.get(0).getCheck_expression());
-    Assert.assertEquals(table.getTableName() + "_check_constraint", ccFetched.get(0).getDc_name());
-    Assert.assertTrue(ccFetched.get(0).isEnable_cstr());
-    Assert.assertFalse(ccFetched.get(0).isValidate_cstr());
-    Assert.assertFalse(ccFetched.get(0).isRely_cstr());
-    Assert.assertEquals(table.getCatName(), ccFetched.get(0).getCatName());
-  }
-
-  @Test
-  public void functions() throws TException {
-    String dbName = "functions_other_catalog_db";
-    Database db = new DatabaseBuilder()
-        .setName(dbName)
-        .create(client, conf);
-
-    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, conf);
-
-    Function createdFunction = client.getFunction(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, conf);
-
-    Set<String> functions = new HashSet<>(client.getFunctions(dbName, "test*"));
-    Assert.assertEquals(2, functions.size());
-    Assert.assertTrue(functions.contains(functionName));
-    Assert.assertTrue(functions.contains(f2Name));
-
-    functions = new HashSet<>(client.getFunctions(dbName, "test_*"));
-    Assert.assertEquals(1, functions.size());
-    Assert.assertTrue(functions.contains(functionName));
-    Assert.assertFalse(functions.contains(f2Name));
-
-    client.dropFunction(function.getDbName(), function.getFunctionName());
-    try {
-      client.getFunction(function.getDbName(), function.getFunctionName());
-      Assert.fail("Expected a NoSuchObjectException to be thrown");
-    } catch (NoSuchObjectException exception) {
-      // Expected exception
-    }
-  }
-
-  // Run a test without the builders.  They make certain default assumptions about catalogs, etc.
-  //  Make sure things still work without those assumptions.
-  @Test
-  public void noBuilders() throws TException {
-    String dbName = "db_no_builder";
-
-    Database db = new Database(dbName, "bla", MetaStoreTestUtils.getTestWarehouseDir(dbName),
-        new HashMap<>());
-    client.createDatabase(db);
-
-    Database fetched = client.getDatabase(dbName);
-    Assert.assertEquals(expectedCatalog(), fetched.getCatalogName());
-
-    String tableName = "now_I_remember_why_I_made_those_builders";
-    List<FieldSchema> cols = Arrays.asList(
-        new FieldSchema("col1", "int", ""),
-        new FieldSchema("col2", "int", "")
-    );
-    List<FieldSchema> partKeys = Collections.singletonList(new FieldSchema("pk1", "string", ""));
-    SerDeInfo serdeInfo = new SerDeInfo("serde", "lib", new HashMap<>());
-    StorageDescriptor sd = new StorageDescriptor(cols, null,
-        "org.apache.hadoop.hive.ql.io.HiveInputFormat",
-        "org.apache.hadoop.hive.ql.io.HiveOutputFormat", false, 0, serdeInfo, new ArrayList<>(),
-        new ArrayList<>(), new HashMap<>());
-    Table table = new Table(tableName, dbName, "me", 0, 0, 0, sd, partKeys, new HashMap<>(),
-        null, null, TableType.MANAGED_TABLE.name());
-    client.createTable(table);
-
-    Table fetchedTable = client.getTable(dbName, tableName);
-    Assert.assertEquals(expectedCatalog(), fetchedTable.getCatName());
-
-    List<String> values = Collections.singletonList("p1");
-    Partition part = new Partition(values, dbName, tableName, 0, 0, sd, new HashMap<>());
-    client.add_partition(part);
-
-    Partition fetchedPart = client.getPartition(dbName, tableName, values);
-    Assert.assertEquals(expectedCatalog(), fetchedPart.getCatName());
-
-    client.dropDatabase(dbName, true, false, true);
-  }
-}


[2/2] hive git commit: HIVE-19871 add_partitions, create constraint calls and create_table_with_constraints do not properly handle client being configured with a non-Hive catalog (Alan Gates reviewed by Daniel Dai)

Posted by ga...@apache.org.
HIVE-19871 add_partitions, create constraint calls and create_table_with_constraints do not properly handle client being configured with a non-Hive catalog (Alan Gates reviewed by Daniel Dai)


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

Branch: refs/heads/branch-3.0
Commit: 120a79b075626fa983786dd3fd71ba07d06450a8
Parents: f2ee204
Author: Alan Gates <ga...@hortonworks.com>
Authored: Wed Jun 13 18:52:24 2018 -0700
Committer: Alan Gates <ga...@hortonworks.com>
Committed: Wed Jun 13 18:52:24 2018 -0700

----------------------------------------------------------------------
 .../hadoop/hive/metastore/HiveMetaStore.java    |   49 +
 .../hive/metastore/HiveMetaStoreClient.java     |    5 +
 .../hive/metastore/NonCatCallsWithCatalog.java  | 1158 ++++++++++++++++++
 .../metastore/TestCatalogNonDefaultClient.java  |    2 +-
 .../metastore/TestCatalogNonDefaultSvr.java     |    4 +-
 .../hive/metastore/TestCatalogOldClient.java    |    2 +-
 .../metastore/TestNonCatCallsWithCatalog.java   | 1126 -----------------
 7 files changed, 1215 insertions(+), 1131 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/120a79b0/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 f9bd64b..d9011c6 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
@@ -1823,6 +1823,25 @@ public class HiveMetaStore extends ThriftHiveMetastore {
             && checkConstraints == null) {
           ms.createTable(tbl);
         } else {
+          // Check that constraints have catalog name properly set first
+          if (primaryKeys != null && !primaryKeys.isEmpty() && !primaryKeys.get(0).isSetCatName()) {
+            for (SQLPrimaryKey pkcol : primaryKeys) pkcol.setCatName(tbl.getCatName());
+          }
+          if (foreignKeys != null && !foreignKeys.isEmpty() && !foreignKeys.get(0).isSetCatName()) {
+            for (SQLForeignKey fkcol : foreignKeys) fkcol.setCatName(tbl.getCatName());
+          }
+          if (uniqueConstraints != null && !uniqueConstraints.isEmpty() && !uniqueConstraints.get(0).isSetCatName()) {
+            for (SQLUniqueConstraint uccol : uniqueConstraints) uccol.setCatName(tbl.getCatName());
+          }
+          if (notNullConstraints != null && !notNullConstraints.isEmpty() && !notNullConstraints.get(0).isSetCatName()) {
+            for (SQLNotNullConstraint nncol : notNullConstraints) nncol.setCatName(tbl.getCatName());
+          }
+          if (defaultConstraints != null && !defaultConstraints.isEmpty() && !defaultConstraints.get(0).isSetCatName()) {
+            for (SQLDefaultConstraint dccol : defaultConstraints) dccol.setCatName(tbl.getCatName());
+          }
+          if (checkConstraints != null && !checkConstraints.isEmpty() && !checkConstraints.get(0).isSetCatName()) {
+            for (SQLCheckConstraint cccol : checkConstraints) cccol.setCatName(tbl.getCatName());
+          }
           // Set constraint name if null before sending to listener
           List<String> constraintNames = ms.createTableWithConstraints(tbl, primaryKeys, foreignKeys,
               uniqueConstraints, notNullConstraints, defaultConstraints, checkConstraints);
@@ -1833,6 +1852,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
               if (primaryKeys.get(i).getPk_name() == null) {
                 primaryKeys.get(i).setPk_name(constraintNames.get(i));
               }
+              if (!primaryKeys.get(i).isSetCatName()) primaryKeys.get(i).setCatName(tbl.getCatName());
             }
           }
           int foreignKeySize = 0;
@@ -1842,6 +1862,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
               if (foreignKeys.get(i).getFk_name() == null) {
                 foreignKeys.get(i).setFk_name(constraintNames.get(primaryKeySize + i));
               }
+              if (!foreignKeys.get(i).isSetCatName()) foreignKeys.get(i).setCatName(tbl.getCatName());
             }
           }
           int uniqueConstraintSize = 0;
@@ -1851,6 +1872,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
               if (uniqueConstraints.get(i).getUk_name() == null) {
                 uniqueConstraints.get(i).setUk_name(constraintNames.get(primaryKeySize + foreignKeySize + i));
               }
+              if (!uniqueConstraints.get(i).isSetCatName()) uniqueConstraints.get(i).setCatName(tbl.getCatName());
             }
           }
           int notNullConstraintSize =  0;
@@ -1859,6 +1881,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
               if (notNullConstraints.get(i).getNn_name() == null) {
                 notNullConstraints.get(i).setNn_name(constraintNames.get(primaryKeySize + foreignKeySize + uniqueConstraintSize + i));
               }
+              if (!notNullConstraints.get(i).isSetCatName()) notNullConstraints.get(i).setCatName(tbl.getCatName());
             }
           }
           int defaultConstraintSize =  0;
@@ -1868,6 +1891,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
                 defaultConstraints.get(i).setDc_name(constraintNames.get(primaryKeySize + foreignKeySize
                     + uniqueConstraintSize + notNullConstraintSize + i));
               }
+              if (!defaultConstraints.get(i).isSetCatName()) defaultConstraints.get(i).setCatName(tbl.getCatName());
             }
           }
           if (checkConstraints!= null) {
@@ -1878,6 +1902,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
                                                                              + defaultConstraintSize
                                                                            + notNullConstraintSize + i));
               }
+              if (!checkConstraints.get(i).isSetCatName()) checkConstraints.get(i).setCatName(tbl.getCatName());
             }
           }
         }
@@ -2060,6 +2085,10 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       startFunction("add_primary_key", ": " + constraintName);
       boolean success = false;
       Exception ex = null;
+      if (!primaryKeyCols.isEmpty() && !primaryKeyCols.get(0).isSetCatName()) {
+        String defaultCat = getDefaultCatalog(conf);
+        primaryKeyCols.forEach(pk -> pk.setCatName(defaultCat));
+      }
       RawStore ms = getMS();
       try {
         ms.openTransaction();
@@ -2112,6 +2141,10 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       startFunction("add_foreign_key", ": " + constraintName);
       boolean success = false;
       Exception ex = null;
+      if (!foreignKeyCols.isEmpty() && !foreignKeyCols.get(0).isSetCatName()) {
+        String defaultCat = getDefaultCatalog(conf);
+        foreignKeyCols.forEach(pk -> pk.setCatName(defaultCat));
+      }
       RawStore ms = getMS();
       try {
         ms.openTransaction();
@@ -2164,6 +2197,10 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       startFunction("add_unique_constraint", ": " + constraintName);
       boolean success = false;
       Exception ex = null;
+      if (!uniqueConstraintCols.isEmpty() && !uniqueConstraintCols.get(0).isSetCatName()) {
+        String defaultCat = getDefaultCatalog(conf);
+        uniqueConstraintCols.forEach(pk -> pk.setCatName(defaultCat));
+      }
       RawStore ms = getMS();
       try {
         ms.openTransaction();
@@ -2216,6 +2253,10 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       startFunction("add_not_null_constraint", ": " + constraintName);
       boolean success = false;
       Exception ex = null;
+      if (!notNullConstraintCols.isEmpty() && !notNullConstraintCols.get(0).isSetCatName()) {
+        String defaultCat = getDefaultCatalog(conf);
+        notNullConstraintCols.forEach(pk -> pk.setCatName(defaultCat));
+      }
       RawStore ms = getMS();
       try {
         ms.openTransaction();
@@ -2268,6 +2309,10 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       startFunction("add_default_constraint", ": " + constraintName);
       boolean success = false;
       Exception ex = null;
+      if (!defaultConstraintCols.isEmpty() && !defaultConstraintCols.get(0).isSetCatName()) {
+        String defaultCat = getDefaultCatalog(conf);
+        defaultConstraintCols.forEach(pk -> pk.setCatName(defaultCat));
+      }
       RawStore ms = getMS();
       try {
         ms.openTransaction();
@@ -2321,6 +2366,10 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       startFunction("add_check_constraint", ": " + constraintName);
       boolean success = false;
       Exception ex = null;
+      if (!checkConstraintCols.isEmpty() && !checkConstraintCols.get(0).isSetCatName()) {
+        String defaultCat = getDefaultCatalog(conf);
+        checkConstraintCols.forEach(pk -> pk.setCatName(defaultCat));
+      }
       RawStore ms = getMS();
       try {
         ms.openTransaction();

http://git-wip-us.apache.org/repos/asf/hive/blob/120a79b0/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 23cf7e4..0cb68dd 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
@@ -677,6 +677,11 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
       return needResults ? new ArrayList<>() : null;
     }
     Partition part = parts.get(0);
+    // Have to set it for each partition too
+    if (!part.isSetCatName()) {
+      final String defaultCat = getDefaultCatalog(conf);
+      parts.forEach(p -> p.setCatName(defaultCat));
+    }
     AddPartitionsRequest req = new AddPartitionsRequest(
         part.getDbName(), part.getTableName(), parts, ifNotExists);
     req.setCatName(part.isSetCatName() ? part.getCatName() : getDefaultCatalog(conf));

http://git-wip-us.apache.org/repos/asf/hive/blob/120a79b0/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/NonCatCallsWithCatalog.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/NonCatCallsWithCatalog.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/NonCatCallsWithCatalog.java
new file mode 100644
index 0000000..f750ca2
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/NonCatCallsWithCatalog.java
@@ -0,0 +1,1158 @@
+/*
+ * 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;
+
+import com.google.common.collect.Lists;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.CheckConstraintsRequest;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.DefaultConstraintsRequest;
+import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.ForeignKeysRequest;
+import org.apache.hadoop.hive.metastore.api.Function;
+import org.apache.hadoop.hive.metastore.api.FunctionType;
+import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.NotNullConstraintsRequest;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.PartitionValuesRequest;
+import org.apache.hadoop.hive.metastore.api.PartitionValuesResponse;
+import org.apache.hadoop.hive.metastore.api.PrimaryKeysRequest;
+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.api.SQLCheckConstraint;
+import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
+import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
+import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TableMeta;
+import org.apache.hadoop.hive.metastore.api.UniqueConstraintsRequest;
+import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
+import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.FunctionBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.PartitionBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.SQLCheckConstraintBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.SQLDefaultConstraintBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.SQLForeignKeyBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.SQLNotNullConstraintBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.SQLPrimaryKeyBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.SQLUniqueConstraintBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
+import org.apache.thrift.TException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
+
+public abstract class NonCatCallsWithCatalog {
+
+  private static final String OTHER_DATABASE = "non_cat_other_db";
+  private Table[] testTables = new Table[6];
+  private static final String TEST_FUNCTION_CLASS =
+      "org.apache.hadoop.hive.ql.udf.generic.GenericUDFUpper";
+
+  protected Configuration conf;
+  protected IMetaStoreClient client;
+
+  protected abstract IMetaStoreClient getClient() throws Exception;
+  protected abstract String expectedCatalog();
+  protected abstract String expectedBaseDir() throws MetaException;
+
+  @Before
+  public void setUp() throws Exception {
+    conf = MetastoreConf.newMetastoreConf();
+    MetaStoreTestUtils.setConfForStandloneMode(conf);
+
+    // Get new client
+    client = getClient();
+
+    List<String> databases = client.getAllDatabases();
+    for (String db : databases) {
+      if (!DEFAULT_DATABASE_NAME.equals(db)) {
+        client.dropDatabase(db, 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);
+    }
+
+    Database db = new DatabaseBuilder().setName(OTHER_DATABASE).build(conf);
+    db.unsetCatalogName();
+    client.createDatabase(db);
+
+
+    testTables[0] =
+        new TableBuilder()
+            .setTableName("test_table")
+            .addCol("test_col1", "int")
+            .addCol("test_col2", "int")
+            .addCol("test_col3", "int")
+            .build(conf);
+
+    testTables[1] =
+        new TableBuilder()
+            .setTableName("test_view")
+            .addCol("test_col1", "int")
+            .addCol("test_col2", "int")
+            .addCol("test_col3", "int")
+            .setType("VIRTUAL_VIEW")
+            .build(conf);
+
+    testTables[2] =
+        new TableBuilder()
+            .setTableName("test_table_to_find_1")
+            .addCol("test_col1", "int")
+            .addCol("test_col2", "int")
+            .addCol("test_col3", "int")
+            .build(conf);
+
+    testTables[3] =
+        new TableBuilder()
+            .setTableName("test_partitioned_table")
+            .addCol("test_col1", "int")
+            .addCol("test_col2", "int")
+            .addPartCol("test_part_col", "int")
+            .build(conf);
+
+    testTables[4] =
+        new TableBuilder()
+            .setTableName("external_table_for_test")
+            .addCol("test_col", "int")
+            .setLocation(MetaStoreTestUtils.getTestWarehouseDir("/external/table_dir"))
+            .addTableParam("EXTERNAL", "TRUE")
+            .setType("EXTERNAL_TABLE")
+            .build(conf);
+
+    testTables[5] =
+        new TableBuilder()
+            .setDbName(OTHER_DATABASE)
+            .setTableName("test_table")
+            .addCol("test_col", "int")
+            .build(conf);
+
+    for (Table t : testTables) {
+      t.unsetCatName();
+      client.createTable(t);
+    }
+
+    // Create partitions for the partitioned table
+    for(int i=0; i < 3; i++) {
+      Partition p = new PartitionBuilder()
+          .inTable(testTables[3])
+          .addValue("a" + i)
+          .build(conf);
+      p.unsetCatName();
+      client.add_partition(p);
+    }
+
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    try {
+      if (client != null) {
+        client.close();
+      }
+    } finally {
+      client = null;
+    }
+  }
+
+  @Test
+  public void databases() throws TException, URISyntaxException {
+    String[] dbNames = {"db1", "db9"};
+    Database[] dbs = new Database[2];
+    // For this one don't specify a location to make sure it gets put in the catalog directory
+    dbs[0] = new DatabaseBuilder()
+        .setName(dbNames[0])
+        .build(conf);
+
+    // For the second one, explicitly set a location to make sure it ends up in the specified place.
+    String db1Location = MetaStoreTestUtils.getTestWarehouseDir(dbNames[1]);
+    dbs[1] = new DatabaseBuilder()
+        .setName(dbNames[1])
+        .setLocation(db1Location)
+        .build(conf);
+
+    for (Database db : dbs) {
+      db.unsetCatalogName();
+      client.createDatabase(db);
+    }
+
+    Database fetched = client.getDatabase(dbNames[0]);
+    String expectedLocation = new File(expectedBaseDir(), dbNames[0] + ".db").toURI().toString();
+    Assert.assertEquals(expectedCatalog(), fetched.getCatalogName());
+    Assert.assertEquals(expectedLocation, fetched.getLocationUri() + "/");
+    String db0Location = new URI(fetched.getLocationUri()).getPath();
+    File dir = new File(db0Location);
+    Assert.assertTrue(dir.exists() && dir.isDirectory());
+    Assert.assertEquals(expectedCatalog(), fetched.getCatalogName());
+
+    fetched = client.getDatabase(dbNames[1]);
+    Assert.assertEquals(new File(db1Location).toURI().toString(), fetched.getLocationUri() + "/");
+    dir = new File(new URI(fetched.getLocationUri()).getPath());
+    Assert.assertTrue(dir.exists() && dir.isDirectory());
+    Assert.assertEquals(expectedCatalog(), fetched.getCatalogName());
+
+    Set<String> fetchedDbs = new HashSet<>(client.getAllDatabases());
+    for (String dbName : dbNames) Assert.assertTrue(fetchedDbs.contains(dbName));
+
+    fetchedDbs = new HashSet<>(client.getDatabases("db*"));
+    Assert.assertEquals(2, fetchedDbs.size());
+    for (String dbName : dbNames) Assert.assertTrue(fetchedDbs.contains(dbName));
+
+    client.dropDatabase(dbNames[0], true, false, false);
+    dir = new File(db0Location);
+    Assert.assertFalse(dir.exists());
+
+    client.dropDatabase(dbNames[1], true, false, false);
+    dir = new File(db1Location);
+    Assert.assertFalse(dir.exists());
+
+    fetchedDbs = new HashSet<>(client.getAllDatabases());
+    for (String dbName : dbNames) Assert.assertFalse(fetchedDbs.contains(dbName));
+  }
+
+  @Test
+  public void tablesCreateDropAlterTruncate() throws TException, URISyntaxException {
+    String dbName = "db_in_other_catalog";
+    // For this one don't specify a location to make sure it gets put in the catalog directory
+    Database db = new DatabaseBuilder()
+        .setName(dbName)
+        .build(conf);
+    db.unsetCatalogName();
+    client.createDatabase(db);
+
+    String[] tableNames = new String[4];
+    for (int i = 0; i < tableNames.length; i++) {
+      tableNames[i] = "table_in_other_catalog_" + i;
+      TableBuilder builder = new TableBuilder()
+          .inDb(db)
+          .setTableName(tableNames[i])
+          .addCol("col1_" + i, ColumnType.STRING_TYPE_NAME)
+          .addCol("col2_" + i, ColumnType.INT_TYPE_NAME);
+      // Make one have a non-standard location
+      if (i == 0) builder.setLocation(MetaStoreTestUtils.getTestWarehouseDir(tableNames[i]));
+      // Make one partitioned
+      if (i == 2) builder.addPartCol("pcol1", ColumnType.STRING_TYPE_NAME);
+      // Make one a materialized view
+      /*
+      // TODO HIVE-18991
+      if (i == 3) {
+        builder.setType(TableType.MATERIALIZED_VIEW.name())
+            .setRewriteEnabled(true)
+            .addMaterializedViewReferencedTable(dbName + "." + tableNames[0]);
+      }
+      */
+      Table t = builder.build(conf);
+      t.unsetCatName();
+      client.createTable(t);
+    }
+
+    // Add partitions for the partitioned table
+    String[] partVals = new String[3];
+    Table partitionedTable = client.getTable(dbName, tableNames[2]);
+    for (int i = 0; i < partVals.length; i++) {
+      partVals[i] = "part" + i;
+      Partition p = new PartitionBuilder()
+          .inTable(partitionedTable)
+          .addValue(partVals[i])
+          .build(conf);
+      p.unsetCatName();
+      client.add_partition(p);
+    }
+
+    // Get tables, make sure the locations are correct
+    for (int i = 0; i < tableNames.length; i++) {
+      Table t = client.getTable(dbName, tableNames[i]);
+      Assert.assertEquals(expectedCatalog(), t.getCatName());
+      String expectedLocation = (i < 1) ?
+          new File(MetaStoreTestUtils.getTestWarehouseDir(tableNames[i])).toURI().toString()
+          :
+          new File(expectedBaseDir() + File.separatorChar + dbName + ".db",
+              tableNames[i]).toURI().toString();
+
+      Assert.assertEquals(expectedLocation, t.getSd().getLocation() + "/");
+      File dir = new File(new URI(t.getSd().getLocation()).getPath());
+      Assert.assertTrue(dir.exists() && dir.isDirectory());
+
+    }
+
+    // Make sure getting table in the wrong catalog does not work
+    try {
+      Table t = client.getTable(DEFAULT_DATABASE_NAME, tableNames[0]);
+      Assert.fail();
+    } catch (NoSuchObjectException e) {
+      // NOP
+    }
+
+    // test getAllTables
+    Set<String> fetchedNames = new HashSet<>(client.getAllTables(dbName));
+    Assert.assertEquals(tableNames.length, fetchedNames.size());
+    for (String tableName : tableNames) Assert.assertTrue(fetchedNames.contains(tableName));
+
+    fetchedNames = new HashSet<>(client.getAllTables(DEFAULT_DATABASE_NAME));
+    for (String tableName : tableNames) Assert.assertFalse(fetchedNames.contains(tableName));
+
+    // test getMaterializedViewsForRewriting
+    /* TODO HIVE-18991
+    List<String> materializedViews = client.getMaterializedViewsForRewriting(dbName);
+    Assert.assertEquals(1, materializedViews.size());
+    Assert.assertEquals(tableNames[3], materializedViews.get(0));
+    */
+
+    fetchedNames = new HashSet<>(client.getMaterializedViewsForRewriting(DEFAULT_DATABASE_NAME));
+    Assert.assertFalse(fetchedNames.contains(tableNames[3]));
+
+    // test getTableObjectsByName
+    List<Table> fetchedTables = client.getTableObjectsByName(dbName,
+        Arrays.asList(tableNames[0], tableNames[1]));
+    Assert.assertEquals(2, fetchedTables.size());
+    Collections.sort(fetchedTables);
+    Assert.assertEquals(tableNames[0], fetchedTables.get(0).getTableName());
+    Assert.assertEquals(tableNames[1], fetchedTables.get(1).getTableName());
+
+    fetchedTables = client.getTableObjectsByName(DEFAULT_DATABASE_NAME,
+        Arrays.asList(tableNames[0], tableNames[1]));
+    Assert.assertEquals(0, fetchedTables.size());
+
+    // Test altering the table
+    Table t = client.getTable(dbName, tableNames[0]).deepCopy();
+    t.getParameters().put("test", "test");
+    client.alter_table(dbName, tableNames[0], t);
+    t = client.getTable(dbName, tableNames[0]).deepCopy();
+    Assert.assertEquals("test", t.getParameters().get("test"));
+
+    // Alter a table in the wrong catalog
+    try {
+      client.alter_table(DEFAULT_DATABASE_NAME, tableNames[0], t);
+      Assert.fail();
+    } catch (InvalidOperationException e) {
+      // NOP
+    }
+
+    // Update the metadata for the materialized view
+    /* TODO HIVE-18991
+    CreationMetadata cm = client.getTable(dbName, tableNames[3]).getCreationMetadata();
+    cm.addToTablesUsed(dbName + "." + tableNames[1]);
+    client.updateCreationMetadata(dbName, tableNames[3], cm);
+    */
+
+    List<String> partNames = new ArrayList<>();
+    for (String partVal : partVals) partNames.add("pcol1=" + partVal);
+    // Truncate a table
+    client.truncateTable(dbName, tableNames[0], partNames);
+
+    // Have to do this in reverse order so that we drop the materialized view first.
+    for (int i = tableNames.length - 1; i >= 0; i--) {
+      t = client.getTable(dbName, tableNames[i]);
+      File tableDir = new File(new URI(t.getSd().getLocation()).getPath());
+      Assert.assertTrue(tableDir.exists() && tableDir.isDirectory());
+
+      if (tableNames[i].equalsIgnoreCase(tableNames[0])) {
+        client.dropTable(dbName, tableNames[i], false, false);
+        Assert.assertTrue(tableDir.exists() && tableDir.isDirectory());
+      } else {
+        client.dropTable(dbName, tableNames[i]);
+        Assert.assertFalse(tableDir.exists());
+      }
+    }
+    Assert.assertEquals(0, client.getAllTables(dbName).size());
+  }
+
+  @Test
+  public void tablesGetExists() throws TException {
+    String dbName = "db_in_other_catalog";
+    // For this one don't specify a location to make sure it gets put in the catalog directory
+    Database db = new DatabaseBuilder()
+        .setName(dbName)
+        .build(conf);
+    db.unsetCatalogName();
+    client.createDatabase(db);
+
+    String[] tableNames = new String[4];
+    for (int i = 0; i < tableNames.length; i++) {
+      tableNames[i] = "table_in_other_catalog_" + i;
+      Table table = new TableBuilder()
+          .inDb(db)
+          .setTableName(tableNames[i])
+          .addCol("col1_" + i, ColumnType.STRING_TYPE_NAME)
+          .addCol("col2_" + i, ColumnType.INT_TYPE_NAME)
+          .build(conf);
+      table.unsetCatName();
+      client.createTable(table);
+    }
+
+    Set<String> tables = new HashSet<>(client.getTables(dbName, "*e_in_other_*"));
+    Assert.assertEquals(4, tables.size());
+    for (String tableName : tableNames) Assert.assertTrue(tables.contains(tableName));
+
+    List<String> fetchedNames = client.getTables(dbName, "*_3");
+    Assert.assertEquals(1, fetchedNames.size());
+    Assert.assertEquals(tableNames[3], fetchedNames.get(0));
+
+    Assert.assertTrue("Table exists", client.tableExists(dbName, tableNames[0]));
+    Assert.assertFalse("Table not exists", client.tableExists(dbName, "non_existing_table"));
+  }
+
+  @Test
+  public void tablesList() throws TException {
+    String dbName = "db_in_other_catalog";
+    // For this one don't specify a location to make sure it gets put in the catalog directory
+    Database db = new DatabaseBuilder()
+        .setName(dbName)
+        .build(conf);
+    db.unsetCatalogName();
+    client.createDatabase(db);
+
+    String[] tableNames = new String[4];
+    for (int i = 0; i < tableNames.length; i++) {
+      tableNames[i] = "table_in_other_catalog_" + i;
+      TableBuilder builder = new TableBuilder()
+          .inDb(db)
+          .setTableName(tableNames[i])
+          .addCol("col1_" + i, ColumnType.STRING_TYPE_NAME)
+          .addCol("col2_" + i, ColumnType.INT_TYPE_NAME);
+      if (i == 0) builder.addTableParam("the_key", "the_value");
+      Table table = builder.build(conf);
+      table.unsetCatName();
+      client.createTable(table);
+    }
+
+    String filter = hive_metastoreConstants.HIVE_FILTER_FIELD_PARAMS + "the_key=\"the_value\"";
+    List<String> fetchedNames = client.listTableNamesByFilter(dbName, filter, (short)-1);
+    Assert.assertEquals(1, fetchedNames.size());
+    Assert.assertEquals(tableNames[0], fetchedNames.get(0));
+  }
+
+  @Test
+  public void getTableMeta() throws TException {
+    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)
+        .build(conf);
+    db.unsetCatalogName();
+    client.createDatabase(db);
+
+    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++) {
+      Table table = new TableBuilder()
+          .inDb(db)
+          .setTableName(tableNames[i])
+          .addCol("id", "int")
+          .addCol("name", "string")
+          .build(conf);
+      table.unsetCatName();
+      client.createTable(table);
+      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(dbName, "*", types);
+    Assert.assertEquals(new TreeSet<>(expected), new TreeSet<>(actual));
+
+    actual = client.getTableMeta("*", "table_*", types);
+    Assert.assertEquals(expected.subList(0, 2), actual.subList(0, 2));
+
+  }
+
+  @Test
+  public void addPartitions() throws TException {
+    String dbName = "add_partition_database_in_other_catalog";
+    Database db = new DatabaseBuilder()
+        .setName(dbName)
+        .build(conf);
+    db.unsetCatalogName();
+    client.createDatabase(db);
+
+    String tableName = "table_in_other_catalog";
+    Table table = new TableBuilder()
+        .inDb(db)
+        .setTableName(tableName)
+        .addCol("id", "int")
+        .addCol("name", "string")
+        .addPartCol("partcol", "string")
+        .build(conf);
+    table.unsetCatName();
+    client.createTable(table);
+
+    Partition[] parts = new Partition[5];
+    for (int i = 0; i < parts.length; i++) {
+      parts[i] = new PartitionBuilder()
+          .inTable(table)
+          .addValue("a" + i)
+          .build(conf);
+      parts[i].unsetCatName();
+    }
+    client.add_partition(parts[0]);
+    Assert.assertEquals(2, client.add_partitions(Arrays.asList(parts[1], parts[2])));
+    client.add_partitions(Arrays.asList(parts[3], parts[4]), true, false);
+
+    for (int i = 0; i < parts.length; i++) {
+      Partition fetched = client.getPartition(dbName, tableName,
+          Collections.singletonList("a" + i));
+      Assert.assertEquals(dbName, fetched.getDbName());
+      Assert.assertEquals(tableName, fetched.getTableName());
+      Assert.assertEquals(expectedCatalog(), fetched.getCatName());
+    }
+
+    client.dropDatabase(dbName, true, true, true);
+  }
+
+  @Test
+  public void getPartitions() throws TException {
+    String dbName = "get_partition_database_in_other_catalog";
+    Database db = new DatabaseBuilder()
+        .setName(dbName)
+        .build(conf);
+    db.unsetCatalogName();
+    client.createDatabase(db);
+
+    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")
+        .build(conf);
+    table.unsetCatName();
+    client.createTable(table);
+
+    Partition[] parts = new Partition[5];
+    for (int i = 0; i < parts.length; i++) {
+      parts[i] = new PartitionBuilder()
+          .inTable(table)
+          .addValue("a" + i)
+          .build(conf);
+      parts[i].unsetCatName();
+    }
+    client.add_partitions(Arrays.asList(parts));
+
+    Partition fetched = client.getPartition(dbName, tableName,
+        Collections.singletonList("a0"));
+    Assert.assertEquals(expectedCatalog(), fetched.getCatName());
+    Assert.assertEquals("a0", fetched.getValues().get(0));
+
+    fetched = client.getPartition(dbName, tableName, "partcol=a0");
+    Assert.assertEquals(expectedCatalog(), fetched.getCatName());
+    Assert.assertEquals("a0", fetched.getValues().get(0));
+
+    List<Partition> fetchedParts = client.getPartitionsByNames(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
+  public void listPartitions() throws TException {
+    String dbName = "list_partition_database_in_other_catalog";
+    Database db = new DatabaseBuilder()
+        .setName(dbName)
+        .build(conf);
+    db.unsetCatalogName();
+    client.createDatabase(db);
+
+    String tableName = "table_in_other_catalog";
+    Table table = new TableBuilder()
+        .inDb(db)
+        .setTableName(tableName)
+        .addCol("id", "int")
+        .addCol("name", "string")
+        .addPartCol("partcol", "string")
+        .build(conf);
+    table.unsetCatName();
+    client.createTable(table);
+
+    Partition[] parts = new Partition[5];
+    for (int i = 0; i < parts.length; i++) {
+      parts[i] = new PartitionBuilder()
+          .inTable(table)
+          .addValue("a" + i)
+          .build(conf);
+      parts[i].unsetCatName();
+    }
+    client.add_partitions(Arrays.asList(parts));
+
+    List<Partition> fetched = client.listPartitions(dbName, tableName, (short)-1);
+    Assert.assertEquals(parts.length, fetched.size());
+    Assert.assertEquals(expectedCatalog(), fetched.get(0).getCatName());
+
+    fetched = client.listPartitions(dbName, tableName,
+        Collections.singletonList("a0"), (short)-1);
+    Assert.assertEquals(1, fetched.size());
+    Assert.assertEquals(expectedCatalog(), fetched.get(0).getCatName());
+
+    PartitionSpecProxy proxy = client.listPartitionSpecs(dbName, tableName, -1);
+    Assert.assertEquals(parts.length, proxy.size());
+    Assert.assertEquals(expectedCatalog(), proxy.getCatName());
+
+    fetched = client.listPartitionsByFilter(dbName, tableName, "partcol=\"a0\"", (short)-1);
+    Assert.assertEquals(1, fetched.size());
+    Assert.assertEquals(expectedCatalog(), fetched.get(0).getCatName());
+
+    proxy = client.listPartitionSpecsByFilter(dbName, tableName, "partcol=\"a0\"", -1);
+    Assert.assertEquals(1, proxy.size());
+    Assert.assertEquals(expectedCatalog(), proxy.getCatName());
+
+    Assert.assertEquals(1, client.getNumPartitionsByFilter(dbName, tableName,
+        "partcol=\"a0\""));
+
+    List<String> names = client.listPartitionNames(dbName, tableName, (short)57);
+    Assert.assertEquals(parts.length, names.size());
+
+    names = client.listPartitionNames(dbName, tableName, Collections.singletonList("a0"),
+        Short.MAX_VALUE);
+    Assert.assertEquals(1, names.size());
+
+    PartitionValuesRequest rqst = new PartitionValuesRequest(dbName,
+        tableName, Lists.newArrayList(new FieldSchema("partcol", "string", "")));
+    PartitionValuesResponse rsp = client.listPartitionValues(rqst);
+    Assert.assertEquals(5, rsp.getPartitionValuesSize());
+  }
+
+  @Test
+  public void alterPartitions() throws TException {
+    String dbName = "alter_partition_database_in_other_catalog";
+    Database db = new DatabaseBuilder()
+        .setName(dbName)
+        .build(conf);
+    db.unsetCatalogName();
+    client.createDatabase(db);
+
+    String tableName = "table_in_other_catalog";
+    Table table = new TableBuilder()
+        .inDb(db)
+        .setTableName(tableName)
+        .addCol("id", "int")
+        .addCol("name", "string")
+        .addPartCol("partcol", "string")
+        .build(conf);
+    table.unsetCatName();
+    client.createTable(table);
+
+    Partition[] parts = new Partition[5];
+    for (int i = 0; i < 5; i++) {
+      parts[i] = new PartitionBuilder()
+          .inTable(table)
+          .addValue("a" + i)
+          .setLocation(MetaStoreTestUtils.getTestWarehouseDir("b" + i))
+          .build(conf);
+      parts[i].unsetCatName();
+    }
+    client.add_partitions(Arrays.asList(parts));
+
+    Partition newPart =
+        client.getPartition(dbName, tableName, Collections.singletonList("a0"));
+    newPart.getParameters().put("test_key", "test_value");
+    client.alter_partition(dbName, tableName, newPart);
+
+    Partition fetched =
+        client.getPartition(dbName, tableName, Collections.singletonList("a0"));
+    Assert.assertEquals("test_value", fetched.getParameters().get("test_key"));
+
+    newPart =
+        client.getPartition(dbName, tableName, Collections.singletonList("a1"));
+    newPart.setLastAccessTime(3);
+    Partition newPart1 =
+        client.getPartition(dbName, tableName, Collections.singletonList("a2"));
+    newPart1.getSd().setLocation(MetaStoreTestUtils.getTestWarehouseDir("somewhere"));
+    client.alter_partitions(dbName, tableName, Arrays.asList(newPart, newPart1));
+    fetched =
+        client.getPartition(dbName, tableName, Collections.singletonList("a1"));
+    Assert.assertEquals(3L, fetched.getLastAccessTime());
+    fetched =
+        client.getPartition(dbName, tableName, Collections.singletonList("a2"));
+    Assert.assertTrue(fetched.getSd().getLocation().contains("somewhere"));
+
+    newPart =
+        client.getPartition(dbName, tableName, Collections.singletonList("a4"));
+    newPart.getParameters().put("test_key", "test_value");
+    EnvironmentContext ec = new EnvironmentContext();
+    ec.setProperties(Collections.singletonMap("a", "b"));
+    client.alter_partition(dbName, tableName, newPart, ec);
+    fetched =
+        client.getPartition(dbName, tableName, Collections.singletonList("a4"));
+    Assert.assertEquals("test_value", fetched.getParameters().get("test_key"));
+
+
+    client.dropDatabase(dbName, true, true, true);
+  }
+
+  @Test
+  public void dropPartitions() throws TException {
+    String dbName = "drop_partition_database_in_other_catalog";
+    Database db = new DatabaseBuilder()
+        .setName(dbName)
+        .build(conf);
+    db.unsetCatalogName();
+    client.createDatabase(db);
+
+    String tableName = "table_in_other_catalog";
+    Table table = new TableBuilder()
+        .inDb(db)
+        .setTableName(tableName)
+        .addCol("id", "int")
+        .addCol("name", "string")
+        .addPartCol("partcol", "string")
+        .build(conf);
+    table.unsetCatName();
+    client.createTable(table);
+
+    Partition[] parts = new Partition[2];
+    for (int i = 0; i < parts.length; i++) {
+      parts[i] = new PartitionBuilder()
+          .inTable(table)
+          .addValue("a" + i)
+          .build(conf);
+      parts[i].unsetCatName();
+    }
+    client.add_partitions(Arrays.asList(parts));
+    List<Partition> fetched = client.listPartitions(dbName, tableName, (short)-1);
+    Assert.assertEquals(parts.length, fetched.size());
+
+    Assert.assertTrue(client.dropPartition(dbName, tableName,
+        Collections.singletonList("a0"), PartitionDropOptions.instance().ifExists(false)));
+    try {
+      client.getPartition(dbName, tableName, Collections.singletonList("a0"));
+      Assert.fail();
+    } catch (NoSuchObjectException e) {
+      // NOP
+    }
+
+    Assert.assertTrue(client.dropPartition(dbName, tableName, "partcol=a1", true));
+    try {
+      client.getPartition(dbName, tableName, Collections.singletonList("a1"));
+      Assert.fail();
+    } catch (NoSuchObjectException e) {
+      // NOP
+    }
+  }
+
+  @Test
+  public void primaryKeyAndForeignKey() throws TException {
+    Table parentTable = testTables[2];
+    Table table = testTables[3];
+    String constraintName = "othercatfk";
+
+    // Single column unnamed primary key in default catalog and database
+    List<SQLPrimaryKey> pk = new SQLPrimaryKeyBuilder()
+        .onTable(parentTable)
+        .addColumn("test_col1")
+        .build(conf);
+    client.addPrimaryKey(pk);
+
+    List<SQLForeignKey> fk = new SQLForeignKeyBuilder()
+        .fromPrimaryKey(pk)
+        .onTable(table)
+        .addColumn("test_col1")
+        .setConstraintName(constraintName)
+        .build(conf);
+    client.addForeignKey(fk);
+
+    PrimaryKeysRequest pkRqst = new PrimaryKeysRequest(parentTable.getDbName(),
+        parentTable.getTableName());
+    pkRqst.setCatName(parentTable.getCatName());
+    List<SQLPrimaryKey> pkFetched = client.getPrimaryKeys(pkRqst);
+    Assert.assertEquals(1, pkFetched.size());
+    Assert.assertEquals(expectedCatalog(), pkFetched.get(0).getCatName());
+    Assert.assertEquals(parentTable.getDbName(), pkFetched.get(0).getTable_db());
+    Assert.assertEquals(parentTable.getTableName(), pkFetched.get(0).getTable_name());
+    Assert.assertEquals("test_col1", pkFetched.get(0).getColumn_name());
+    Assert.assertEquals(1, pkFetched.get(0).getKey_seq());
+    Assert.assertTrue(pkFetched.get(0).isEnable_cstr());
+    Assert.assertFalse(pkFetched.get(0).isValidate_cstr());
+    Assert.assertFalse(pkFetched.get(0).isRely_cstr());
+    Assert.assertEquals(parentTable.getCatName(), pkFetched.get(0).getCatName());
+
+    ForeignKeysRequest rqst = new ForeignKeysRequest(parentTable.getDbName(),
+        parentTable.getTableName(), table.getDbName(), table.getTableName());
+    rqst.setCatName(table.getCatName());
+    List<SQLForeignKey> fetched = client.getForeignKeys(rqst);
+    Assert.assertEquals(1, fetched.size());
+    Assert.assertEquals(table.getDbName(), fetched.get(0).getFktable_db());
+    Assert.assertEquals(table.getTableName(), fetched.get(0).getFktable_name());
+    Assert.assertEquals(expectedCatalog(), fetched.get(0).getCatName());
+    Assert.assertEquals("test_col1", fetched.get(0).getFkcolumn_name());
+    Assert.assertEquals(parentTable.getDbName(), fetched.get(0).getPktable_db());
+    Assert.assertEquals(parentTable.getTableName(), fetched.get(0).getPktable_name());
+    Assert.assertEquals("test_col1", fetched.get(0).getFkcolumn_name());
+    Assert.assertEquals(1, fetched.get(0).getKey_seq());
+    Assert.assertEquals(parentTable.getTableName() + "_primary_key", fetched.get(0).getPk_name());
+    Assert.assertEquals(constraintName, fetched.get(0).getFk_name());
+    String table0FkName = fetched.get(0).getFk_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 foreign key
+    client.dropConstraint(table.getDbName(), table.getTableName(), table0FkName);
+    rqst = new ForeignKeysRequest(parentTable.getDbName(), parentTable.getTableName(),
+        table.getDbName(), table.getTableName());
+    rqst.setCatName(table.getCatName());
+    fetched = client.getForeignKeys(rqst);
+    Assert.assertTrue(fetched.isEmpty());
+  }
+
+  @Test
+  public void notNullConstraint() throws TException {
+    String constraintName = "ocuc";
+    // Table in non 'hive' catalog
+    List<SQLNotNullConstraint> nn = new SQLNotNullConstraintBuilder()
+        .onTable(testTables[2])
+        .addColumn("test_col1")
+        .setConstraintName(constraintName)
+        .build(conf);
+    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("test_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].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 uniqueConstraint() throws TException {
+    String constraintName = "ocuc";
+    // Table in non 'hive' catalog
+    List<SQLUniqueConstraint> uc = new SQLUniqueConstraintBuilder()
+        .onTable(testTables[2])
+        .addColumn("test_col1")
+        .setConstraintName(constraintName)
+        .build(conf);
+    client.addUniqueConstraint(uc);
+
+    UniqueConstraintsRequest rqst = new UniqueConstraintsRequest(testTables[2].getCatName(),
+        testTables[2].getDbName(), testTables[2].getTableName());
+    List<SQLUniqueConstraint> fetched = client.getUniqueConstraints(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("test_col1", fetched.get(0).getColumn_name());
+    Assert.assertEquals(1, fetched.get(0).getKey_seq());
+    Assert.assertEquals(constraintName, fetched.get(0).getUk_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());
+    Assert.assertEquals(expectedCatalog(), fetched.get(0).getCatName());
+
+    client.dropConstraint(testTables[2].getDbName(), testTables[2].getTableName(), constraintName);
+    rqst = new UniqueConstraintsRequest(testTables[2].getCatName(), testTables[2].getDbName(),
+        testTables[2].getTableName());
+    fetched = client.getUniqueConstraints(rqst);
+    Assert.assertTrue(fetched.isEmpty());
+  }
+
+  @Test
+  public void defaultConstraints() throws TException {
+    String constraintName = "ocdv";
+    // Table in non 'hive' catalog
+    List<SQLDefaultConstraint> dv = new SQLDefaultConstraintBuilder()
+        .onTable(testTables[2])
+        .addColumn("test_col1")
+        .setConstraintName(constraintName)
+        .setDefaultVal("empty")
+        .build(conf);
+    client.addDefaultConstraint(dv);
+
+    DefaultConstraintsRequest rqst = new DefaultConstraintsRequest(testTables[2].getCatName(),
+        testTables[2].getDbName(), testTables[2].getTableName());
+    List<SQLDefaultConstraint> fetched = client.getDefaultConstraints(rqst);
+    Assert.assertEquals(1, fetched.size());
+    Assert.assertEquals(expectedCatalog(), fetched.get(0).getCatName());
+    Assert.assertEquals(testTables[2].getDbName(), fetched.get(0).getTable_db());
+    Assert.assertEquals(testTables[2].getTableName(), fetched.get(0).getTable_name());
+    Assert.assertEquals("test_col1", fetched.get(0).getColumn_name());
+    Assert.assertEquals("empty", fetched.get(0).getDefault_value());
+    Assert.assertEquals(constraintName, fetched.get(0).getDc_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].getDbName(), testTables[2].getTableName(), constraintName);
+    rqst = new DefaultConstraintsRequest(testTables[2].getCatName(), testTables[2].getDbName(),
+        testTables[2].getTableName());
+    fetched = client.getDefaultConstraints(rqst);
+    Assert.assertTrue(fetched.isEmpty());
+  }
+
+  @Test
+  public void createTableWithConstraints() throws TException {
+    Table parentTable = testTables[2];
+
+
+    Table table = new TableBuilder()
+        .setTableName("table_in_other_catalog_with_constraints")
+        .addCol("col1", "int")
+        .addCol("col2", "varchar(32)")
+        .addCol("col3", "int")
+        .addCol("col4", "int")
+        .addCol("col5", "int")
+        .addCol("col6", "int")
+        .build(conf);
+    table.unsetCatName();
+
+    List<SQLPrimaryKey> parentPk = new SQLPrimaryKeyBuilder()
+        .onTable(parentTable)
+        .addColumn("test_col1")
+        .build(conf);
+    for (SQLPrimaryKey pkcol : parentPk) pkcol.unsetCatName();
+    client.addPrimaryKey(parentPk);
+
+    List<SQLPrimaryKey> pk = new SQLPrimaryKeyBuilder()
+        .onTable(table)
+        .addColumn("col2")
+        .build(conf);
+    for (SQLPrimaryKey pkcol : pk) pkcol.unsetCatName();
+
+    List<SQLForeignKey> fk = new SQLForeignKeyBuilder()
+        .fromPrimaryKey(parentPk)
+        .onTable(table)
+        .addColumn("col1")
+        .build(conf);
+    for (SQLForeignKey fkcol : fk) fkcol.unsetCatName();
+
+    List<SQLDefaultConstraint> dv = new SQLDefaultConstraintBuilder()
+        .onTable(table)
+        .addColumn("col3")
+        .setDefaultVal(0)
+        .build(conf);
+    for (SQLDefaultConstraint dccol : dv) dccol.unsetCatName();
+
+    List<SQLNotNullConstraint> nn = new SQLNotNullConstraintBuilder()
+        .onTable(table)
+        .addColumn("col4")
+        .build(conf);
+    for (SQLNotNullConstraint nncol : nn) nncol.unsetCatName();
+
+    List<SQLUniqueConstraint> uc = new SQLUniqueConstraintBuilder()
+        .onTable(table)
+        .addColumn("col5")
+        .build(conf);
+    for (SQLUniqueConstraint uccol : uc) uccol.unsetCatName();
+
+    List<SQLCheckConstraint> cc = new SQLCheckConstraintBuilder()
+        .onTable(table)
+        .addColumn("col6")
+        .setCheckExpression("> 0")
+        .build(conf);
+    for (SQLCheckConstraint cccol : cc) cccol.unsetCatName();
+
+    client.createTableWithConstraints(table, pk, fk, uc, nn, dv, cc);
+
+    PrimaryKeysRequest pkRqst = new PrimaryKeysRequest(parentTable.getDbName(),
+        parentTable.getTableName());
+    pkRqst.setCatName(parentTable.getCatName());
+    List<SQLPrimaryKey> pkFetched = client.getPrimaryKeys(pkRqst);
+    Assert.assertEquals(1, pkFetched.size());
+    Assert.assertEquals(expectedCatalog(), pkFetched.get(0).getCatName());
+    Assert.assertEquals(parentTable.getDbName(), pkFetched.get(0).getTable_db());
+    Assert.assertEquals(parentTable.getTableName(), pkFetched.get(0).getTable_name());
+    Assert.assertEquals("test_col1", pkFetched.get(0).getColumn_name());
+    Assert.assertEquals(1, pkFetched.get(0).getKey_seq());
+    Assert.assertTrue(pkFetched.get(0).isEnable_cstr());
+    Assert.assertFalse(pkFetched.get(0).isValidate_cstr());
+    Assert.assertFalse(pkFetched.get(0).isRely_cstr());
+    Assert.assertEquals(parentTable.getCatName(), pkFetched.get(0).getCatName());
+
+    ForeignKeysRequest fkRqst = new ForeignKeysRequest(parentTable.getDbName(), parentTable
+        .getTableName(),
+        table.getDbName(), table.getTableName());
+    fkRqst.setCatName(table.getCatName());
+    List<SQLForeignKey> fkFetched = client.getForeignKeys(fkRqst);
+    Assert.assertEquals(1, fkFetched.size());
+    Assert.assertEquals(expectedCatalog(), fkFetched.get(0).getCatName());
+    Assert.assertEquals(table.getDbName(), fkFetched.get(0).getFktable_db());
+    Assert.assertEquals(table.getTableName(), fkFetched.get(0).getFktable_name());
+    Assert.assertEquals("col1", fkFetched.get(0).getFkcolumn_name());
+    Assert.assertEquals(parentTable.getDbName(), fkFetched.get(0).getPktable_db());
+    Assert.assertEquals(parentTable.getTableName(), fkFetched.get(0).getPktable_name());
+    Assert.assertEquals(1, fkFetched.get(0).getKey_seq());
+    Assert.assertEquals(parentTable.getTableName() + "_primary_key", fkFetched.get(0).getPk_name());
+    Assert.assertTrue(fkFetched.get(0).isEnable_cstr());
+    Assert.assertFalse(fkFetched.get(0).isValidate_cstr());
+    Assert.assertFalse(fkFetched.get(0).isRely_cstr());
+    Assert.assertEquals(table.getCatName(), fkFetched.get(0).getCatName());
+
+    NotNullConstraintsRequest nnRqst = new NotNullConstraintsRequest(table.getCatName(),
+        table.getDbName(), table.getTableName());
+    List<SQLNotNullConstraint> nnFetched = client.getNotNullConstraints(nnRqst);
+    Assert.assertEquals(1, nnFetched.size());
+    Assert.assertEquals(table.getDbName(), nnFetched.get(0).getTable_db());
+    Assert.assertEquals(table.getTableName(), nnFetched.get(0).getTable_name());
+    Assert.assertEquals("col4", nnFetched.get(0).getColumn_name());
+    Assert.assertEquals(table.getTableName() + "_not_null_constraint", nnFetched.get(0).getNn_name());
+    Assert.assertTrue(nnFetched.get(0).isEnable_cstr());
+    Assert.assertFalse(nnFetched.get(0).isValidate_cstr());
+    Assert.assertFalse(nnFetched.get(0).isRely_cstr());
+    Assert.assertEquals(table.getCatName(), nnFetched.get(0).getCatName());
+
+    UniqueConstraintsRequest ucRqst = new UniqueConstraintsRequest(table.getCatName(), table
+        .getDbName(), table.getTableName());
+    List<SQLUniqueConstraint> ucFetched = client.getUniqueConstraints(ucRqst);
+    Assert.assertEquals(1, ucFetched.size());
+    Assert.assertEquals(table.getDbName(), ucFetched.get(0).getTable_db());
+    Assert.assertEquals(table.getTableName(), ucFetched.get(0).getTable_name());
+    Assert.assertEquals("col5", ucFetched.get(0).getColumn_name());
+    Assert.assertEquals(1, ucFetched.get(0).getKey_seq());
+    Assert.assertEquals(table.getTableName() + "_unique_constraint", ucFetched.get(0).getUk_name());
+    Assert.assertTrue(ucFetched.get(0).isEnable_cstr());
+    Assert.assertFalse(ucFetched.get(0).isValidate_cstr());
+    Assert.assertFalse(ucFetched.get(0).isRely_cstr());
+    Assert.assertEquals(table.getCatName(), ucFetched.get(0).getCatName());
+
+    DefaultConstraintsRequest dcRqst = new DefaultConstraintsRequest(table.getCatName(), table
+        .getDbName(), table.getTableName());
+    List<SQLDefaultConstraint> dcFetched = client.getDefaultConstraints(dcRqst);
+    Assert.assertEquals(1, dcFetched.size());
+    Assert.assertEquals(expectedCatalog(), dcFetched.get(0).getCatName());
+    Assert.assertEquals(table.getDbName(), dcFetched.get(0).getTable_db());
+    Assert.assertEquals(table.getTableName(), dcFetched.get(0).getTable_name());
+    Assert.assertEquals("col3", dcFetched.get(0).getColumn_name());
+    Assert.assertEquals("0", dcFetched.get(0).getDefault_value());
+    Assert.assertEquals(table.getTableName() + "_default_value", dcFetched.get(0).getDc_name());
+    Assert.assertTrue(dcFetched.get(0).isEnable_cstr());
+    Assert.assertFalse(dcFetched.get(0).isValidate_cstr());
+    Assert.assertFalse(dcFetched.get(0).isRely_cstr());
+    Assert.assertEquals(table.getCatName(), dcFetched.get(0).getCatName());
+
+    CheckConstraintsRequest ccRqst = new CheckConstraintsRequest(table.getCatName(), table
+        .getDbName(), table.getTableName());
+    List<SQLCheckConstraint> ccFetched = client.getCheckConstraints(ccRqst);
+    Assert.assertEquals(1, ccFetched.size());
+    Assert.assertEquals(expectedCatalog(), ccFetched.get(0).getCatName());
+    Assert.assertEquals(table.getDbName(), ccFetched.get(0).getTable_db());
+    Assert.assertEquals(table.getTableName(), ccFetched.get(0).getTable_name());
+    Assert.assertEquals("col6", ccFetched.get(0).getColumn_name());
+    Assert.assertEquals("> 0", ccFetched.get(0).getCheck_expression());
+    Assert.assertEquals(table.getTableName() + "_check_constraint", ccFetched.get(0).getDc_name());
+    Assert.assertTrue(ccFetched.get(0).isEnable_cstr());
+    Assert.assertFalse(ccFetched.get(0).isValidate_cstr());
+    Assert.assertFalse(ccFetched.get(0).isRely_cstr());
+    Assert.assertEquals(table.getCatName(), ccFetched.get(0).getCatName());
+  }
+
+  @Test
+  public void functions() throws TException {
+    String dbName = "functions_other_catalog_db";
+    Database db = new DatabaseBuilder()
+        .setName(dbName)
+        .build(conf);
+    db.unsetCatalogName();
+    client.createDatabase(db);
+
+    String functionName = "test_function";
+    Function function =
+        new FunctionBuilder()
+            .inDb(db)
+            .setName(functionName)
+            .setClass(TEST_FUNCTION_CLASS)
+            .setFunctionType(FunctionType.JAVA)
+            .setOwnerType(PrincipalType.ROLE)
+            .setOwner("owner")
+            .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(conf);
+    function.unsetCatName();
+    client.createFunction(function);
+
+    Function createdFunction = client.getFunction(dbName, functionName);
+    // Creation time will be set by server and not us.
+    Assert.assertEquals(function.getFunctionName(), createdFunction.getFunctionName());
+    Assert.assertEquals(function.getDbName(), createdFunction.getDbName());
+    Assert.assertEquals(expectedCatalog(), createdFunction.getCatName());
+    Assert.assertEquals(function.getClassName(), createdFunction.getClassName());
+    Assert.assertEquals(function.getOwnerName(), createdFunction.getOwnerName());
+    Assert.assertEquals(function.getOwnerType(), createdFunction.getOwnerType());
+    Assert.assertEquals(function.getFunctionType(), createdFunction.getFunctionType());
+    Assert.assertEquals(function.getResourceUris(), createdFunction.getResourceUris());
+
+    String f2Name = "testy_function2";
+    Function f2 = new FunctionBuilder()
+        .inDb(db)
+        .setName(f2Name)
+        .setClass(TEST_FUNCTION_CLASS)
+        .build(conf);
+    f2.unsetCatName();
+    client.createFunction(f2);
+
+    Set<String> functions = new HashSet<>(client.getFunctions(dbName, "test*"));
+    Assert.assertEquals(2, functions.size());
+    Assert.assertTrue(functions.contains(functionName));
+    Assert.assertTrue(functions.contains(f2Name));
+
+    functions = new HashSet<>(client.getFunctions(dbName, "test_*"));
+    Assert.assertEquals(1, functions.size());
+    Assert.assertTrue(functions.contains(functionName));
+    Assert.assertFalse(functions.contains(f2Name));
+
+    client.dropFunction(function.getDbName(), function.getFunctionName());
+    try {
+      client.getFunction(function.getDbName(), function.getFunctionName());
+      Assert.fail("Expected a NoSuchObjectException to be thrown");
+    } catch (NoSuchObjectException exception) {
+      // Expected exception
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/120a79b0/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestCatalogNonDefaultClient.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestCatalogNonDefaultClient.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestCatalogNonDefaultClient.java
index dfe05e9..550b107 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestCatalogNonDefaultClient.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestCatalogNonDefaultClient.java
@@ -30,7 +30,7 @@ import org.junit.After;
  * This tests metastore client calls that do not specify a catalog but with the config on the
  * client set to go to a non-default catalog.
  */
-public class TestCatalogNonDefaultClient extends TestNonCatCallsWithCatalog {
+public class TestCatalogNonDefaultClient extends NonCatCallsWithCatalog {
 
   final private String catName = "non_default_catalog";
   private String catLocation;

http://git-wip-us.apache.org/repos/asf/hive/blob/120a79b0/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestCatalogNonDefaultSvr.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestCatalogNonDefaultSvr.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestCatalogNonDefaultSvr.java
index 13c8723..cf909ac 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestCatalogNonDefaultSvr.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestCatalogNonDefaultSvr.java
@@ -17,12 +17,10 @@
  */
 package org.apache.hadoop.hive.metastore;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.api.Catalog;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.client.builder.CatalogBuilder;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
-import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
 import org.apache.thrift.TException;
 import org.junit.After;
 
@@ -30,7 +28,7 @@ import org.junit.After;
  * This tests metastore client calls that do not specify a catalog but with the config on the
  * server set to go to a non-default catalog.
  */
-public class TestCatalogNonDefaultSvr extends TestNonCatCallsWithCatalog {
+public class TestCatalogNonDefaultSvr extends NonCatCallsWithCatalog {
 
   final private String catName = "non_default_svr_catalog";
   private String catLocation;

http://git-wip-us.apache.org/repos/asf/hive/blob/120a79b0/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestCatalogOldClient.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestCatalogOldClient.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestCatalogOldClient.java
index bb57b85..fc996c8 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestCatalogOldClient.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestCatalogOldClient.java
@@ -25,7 +25,7 @@ import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
  * This tests calls with an older client, to make sure that if the client supplies no catalog
  * information the server still does the right thing.  I assumes the default catalog
  */
-public class TestCatalogOldClient extends TestNonCatCallsWithCatalog {
+public class TestCatalogOldClient extends NonCatCallsWithCatalog {
 
   @Override
   protected IMetaStoreClient getClient() throws MetaException {