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

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

Repository: hive
Updated Branches:
  refs/heads/master 510960268 -> ba8a99e11


http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesCreateDropAlterTruncate.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesCreateDropAlterTruncate.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesCreateDropAlterTruncate.java
index 61ac483..fe2d758 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesCreateDropAlterTruncate.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesCreateDropAlterTruncate.java
@@ -20,9 +20,15 @@ package org.apache.hadoop.hive.metastore.client;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.hadoop.hive.metastore.ColumnType;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
+import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
+import org.apache.hadoop.hive.metastore.api.Catalog;
+import org.apache.hadoop.hive.metastore.api.CreationMetadata;
+import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
@@ -34,11 +40,15 @@ import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.SkewedInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.UnknownDBException;
+import org.apache.hadoop.hive.metastore.client.builder.CatalogBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.PartitionBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.minihms.AbstractMetaStoreService;
+import org.apache.thrift.TApplicationException;
+import org.apache.thrift.TException;
 import org.apache.thrift.protocol.TProtocolException;
 import org.apache.thrift.transport.TTransportException;
 import org.junit.After;
@@ -50,10 +60,20 @@ import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
+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.Map;
+import java.util.Set;
+
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
 
 /**
  * Test class for IMetaStoreClient API. Testing the Table related functions for metadata
@@ -78,7 +98,7 @@ public class TestTablesCreateDropAlterTruncate extends MetaStoreClientTest {
   public static void startMetaStores() {
     Map<MetastoreConf.ConfVars, String> msConf = new HashMap<MetastoreConf.ConfVars, String>();
     // Enable trash, so it can be tested
-    Map<String, String> extraConf = new HashMap<String, String>();
+    Map<String, String> extraConf = new HashMap<>();
     extraConf.put("fs.trash.checkpoint.interval", "30");  // FS_TRASH_CHECKPOINT_INTERVAL_KEY
     extraConf.put("fs.trash.interval", "30");             // FS_TRASH_INTERVAL_KEY (hadoop-2)
     startMetaStores(msConf, extraConf);
@@ -101,74 +121,62 @@ public class TestTablesCreateDropAlterTruncate extends MetaStoreClientTest {
 
     testTables[0] =
         new TableBuilder()
-            .setDbName(DEFAULT_DATABASE)
             .setTableName("test_table")
             .addCol("test_col", "int")
-            .build();
+            .create(client, metaStore.getConf());
 
     testTables[1] =
         new TableBuilder()
-            .setDbName(DEFAULT_DATABASE)
             .setTableName("test_view")
             .addCol("test_col", "int")
             .setType("VIRTUAL_VIEW")
-            .build();
+            .create(client, metaStore.getConf());
 
     testTables[2] =
         new TableBuilder()
-            .setDbName(DEFAULT_DATABASE)
             .setTableName("test_table_to_find_1")
             .addCol("test_col", "int")
-            .build();
+            .create(client, metaStore.getConf());
 
     testTables[3] =
         new TableBuilder()
-            .setDbName(DEFAULT_DATABASE)
             .setTableName("test_partitioned_table")
             .addCol("test_col1", "int")
             .addCol("test_col2", "int")
             .addPartCol("test_part_col", "int")
-            .build();
+            .create(client, metaStore.getConf());
 
     testTables[4] =
         new TableBuilder()
-            .setDbName(DEFAULT_DATABASE)
             .setTableName("external_table_for_test")
             .addCol("test_col", "int")
             .setLocation(metaStore.getWarehouseRoot() + "/external/table_dir")
             .addTableParam("EXTERNAL", "TRUE")
             .setType("EXTERNAL_TABLE")
-            .build();
+            .create(client, metaStore.getConf());
 
 
-    client.createDatabase(new DatabaseBuilder().setName(OTHER_DATABASE).build());
+    new DatabaseBuilder().setName(OTHER_DATABASE).create(client, metaStore.getConf());
 
     testTables[5] =
         new TableBuilder()
             .setDbName(OTHER_DATABASE)
             .setTableName("test_table")
             .addCol("test_col", "int")
-            .build();
-
-    // Create the tables in the MetaStore
-    for(int i=0; i < testTables.length; i++) {
-      client.createTable(testTables[i]);
-    }
+            .create(client, metaStore.getConf());
 
     // Create partitions for the partitioned table
     for(int i=0; i < 3; i++) {
-      Partition partition =
-          new PartitionBuilder()
-              .fromTable(testTables[3])
+      new PartitionBuilder()
+              .inTable(testTables[3])
               .addValue("a" + i)
-              .build();
-      client.add_partition(partition);
+              .addToTable(client, metaStore.getConf());
     }
     // Add data files to the partitioned table
     List<Partition> partitions =
         client.listPartitions(testTables[3].getDbName(), testTables[3].getTableName(), (short)-1);
     for(Partition partition : partitions) {
-      Path dataFile = new Path(partition.getSd().getLocation().toString() + "/dataFile");
+      Path dataFile = new Path(partition.getSd().getLocation() + "/dataFile");
       metaStore.createFile(dataFile, "100");
     }
 
@@ -177,7 +185,7 @@ public class TestTablesCreateDropAlterTruncate extends MetaStoreClientTest {
       testTables[i] = client.getTable(testTables[i].getDbName(), testTables[i].getTableName());
       if (testTables[i].getPartitionKeys().isEmpty()) {
         if (testTables[i].getSd().getLocation() != null) {
-          Path dataFile = new Path(testTables[i].getSd().getLocation().toString() + "/dataFile");
+          Path dataFile = new Path(testTables[i].getSd().getLocation() + "/dataFile");
           metaStore.createFile(dataFile, "100");
         }
       }
@@ -199,7 +207,6 @@ public class TestTablesCreateDropAlterTruncate extends MetaStoreClientTest {
 
   /**
    * This test creates and queries a table and then drops it. Good for testing the happy path
-   * @throws Exception
    */
   @Test
   public void testCreateGetDeleteTable() throws Exception {
@@ -237,7 +244,7 @@ public class TestTablesCreateDropAlterTruncate extends MetaStoreClientTest {
   public void testCreateTableDefaultValues() throws Exception {
     Table table = new Table();
     StorageDescriptor sd = new StorageDescriptor();
-    List<FieldSchema> cols = new ArrayList<FieldSchema>();
+    List<FieldSchema> cols = new ArrayList<>();
 
     table.setDbName(DEFAULT_DATABASE);
     table.setTableName("test_table_2");
@@ -309,7 +316,7 @@ public class TestTablesCreateDropAlterTruncate extends MetaStoreClientTest {
   public void testCreateTableDefaultLocationInSpecificDatabase() throws Exception {
     Table table = new Table();
     StorageDescriptor sd = new StorageDescriptor();
-    List<FieldSchema> cols = new ArrayList<FieldSchema>();
+    List<FieldSchema> cols = new ArrayList<>();
 
     table.setDbName(OTHER_DATABASE);
     table.setTableName("test_table_2");
@@ -329,7 +336,7 @@ public class TestTablesCreateDropAlterTruncate extends MetaStoreClientTest {
   public void testCreateTableDefaultValuesView() throws Exception {
     Table table = new Table();
     StorageDescriptor sd = new StorageDescriptor();
-    List<FieldSchema> cols = new ArrayList<FieldSchema>();
+    List<FieldSchema> cols = new ArrayList<>();
 
     table.setDbName(DEFAULT_DATABASE);
     table.setTableName("test_table_2");
@@ -343,7 +350,6 @@ public class TestTablesCreateDropAlterTruncate extends MetaStoreClientTest {
     Table createdTable = client.getTable(table.getDbName(), table.getTableName());
 
     // No location should be created for views
-    StorageDescriptor createdSd = createdTable.getSd();
     Assert.assertNull("Storage descriptor location should be null",
         createdTable.getSd().getLocation());
   }
@@ -390,10 +396,9 @@ public class TestTablesCreateDropAlterTruncate extends MetaStoreClientTest {
 
   private Table getNewTable() throws MetaException {
     return new TableBuilder()
-               .setDbName(DEFAULT_DATABASE)
                .setTableName("test_table_with_invalid_sd")
                .addCol("test_col", "int")
-               .build();
+               .build(metaStore.getConf());
   }
 
   @Test(expected = MetaException.class)
@@ -604,7 +609,7 @@ public class TestTablesCreateDropAlterTruncate extends MetaStoreClientTest {
   @Test
   public void testTruncateTableUnpartitioned() throws Exception {
     // Unpartitioned table
-    Path dataFile = new Path(testTables[0].getSd().getLocation().toString() + "/dataFile");
+    Path dataFile = new Path(testTables[0].getSd().getLocation() + "/dataFile");
     client.truncateTable(testTables[0].getDbName(), testTables[0].getTableName(), null);
     Assert.assertTrue("Location should exist",
         metaStore.isPathExists(new Path(testTables[0].getSd().getLocation())));
@@ -615,7 +620,7 @@ public class TestTablesCreateDropAlterTruncate extends MetaStoreClientTest {
   @Test
   public void testTruncateTablePartitioned() throws Exception {
     // Partitioned table - delete specific partitions a0, a2
-    List<String> partitionsToDelete = new ArrayList<String>();
+    List<String> partitionsToDelete = new ArrayList<>();
     partitionsToDelete.add("test_part_col=a0");
     partitionsToDelete.add("test_part_col=a2");
     client.truncateTable(partitionedTable.getDbName(), partitionedTable.getTableName(),
@@ -626,7 +631,7 @@ public class TestTablesCreateDropAlterTruncate extends MetaStoreClientTest {
         client.listPartitions(partitionedTable.getDbName(), partitionedTable.getTableName(),
             (short)-1);
     for(Partition partition : partitions) {
-      Path dataFile = new Path(partition.getSd().getLocation().toString() + "/dataFile");
+      Path dataFile = new Path(partition.getSd().getLocation() + "/dataFile");
       if (partition.getValues().contains("a0") || partition.getValues().contains("a2")) {
         // a0, a2 should be empty
         Assert.assertFalse("DataFile should be removed", metaStore.isPathExists(dataFile));
@@ -648,7 +653,7 @@ public class TestTablesCreateDropAlterTruncate extends MetaStoreClientTest {
         client.listPartitions(partitionedTable.getDbName(), partitionedTable.getTableName(),
             (short)-1);
     for(Partition partition : partitions) {
-      Path dataFile = new Path(partition.getSd().getLocation().toString() + "/dataFile");
+      Path dataFile = new Path(partition.getSd().getLocation() + "/dataFile");
       Assert.assertFalse("Every dataFile should be removed", metaStore.isPathExists(dataFile));
     }
   }
@@ -704,7 +709,7 @@ public class TestTablesCreateDropAlterTruncate extends MetaStoreClientTest {
     Assert.assertEquals("New directory should be set", new Path(metaStore.getWarehouseRoot()
         + "/" + alteredTable.getTableName()), new Path(alteredTable.getSd().getLocation()));
 
-    Path dataFile = new Path(alteredTable.getSd().getLocation().toString() + "/dataFile");
+    Path dataFile = new Path(alteredTable.getSd().getLocation() + "/dataFile");
     Assert.assertTrue("New directory should contain data", metaStore.isPathExists(dataFile));
 
     // The following data should be changed
@@ -731,7 +736,7 @@ public class TestTablesCreateDropAlterTruncate extends MetaStoreClientTest {
     Assert.assertEquals("New directory should be set", new Path(metaStore.getWarehouseRoot()
         + "/" + alteredTable.getDbName() + ".db/" + alteredTable.getTableName()),
         new Path(alteredTable.getSd().getLocation()));
-    Path dataFile = new Path(alteredTable.getSd().getLocation().toString() + "/dataFile");
+    Path dataFile = new Path(alteredTable.getSd().getLocation() + "/dataFile");
     Assert.assertTrue("New directory should contain data", metaStore.isPathExists(dataFile));
 
     // The following data should be changed, other data should be the same
@@ -755,7 +760,7 @@ public class TestTablesCreateDropAlterTruncate extends MetaStoreClientTest {
     Table alteredTable = client.getTable(newTable.getDbName(), newTable.getTableName());
     Assert.assertEquals("New location should be the same", originalTable.getSd().getLocation(),
         alteredTable.getSd().getLocation());
-    Path dataFile = new Path(alteredTable.getSd().getLocation().toString() + "/dataFile");
+    Path dataFile = new Path(alteredTable.getSd().getLocation() + "/dataFile");
     Assert.assertTrue("The location should contain data", metaStore.isPathExists(dataFile));
 
     // The extra parameters will be added on server side, so check that the required ones are
@@ -782,7 +787,7 @@ public class TestTablesCreateDropAlterTruncate extends MetaStoreClientTest {
         metaStore.isPathExists(new Path(originalTable.getSd().getLocation())));
     Assert.assertEquals("New location should be the new one", newTable.getSd().getLocation(),
         alteredTable.getSd().getLocation());
-    Path dataFile = new Path(alteredTable.getSd().getLocation().toString() + "/dataFile");
+    Path dataFile = new Path(alteredTable.getSd().getLocation() + "/dataFile");
     Assert.assertFalse("The location should not contain data", metaStore.isPathExists(dataFile));
 
     // The extra parameters will be added on server side, so check that the required ones are
@@ -833,6 +838,7 @@ public class TestTablesCreateDropAlterTruncate extends MetaStoreClientTest {
     Assert.assertEquals("The table data should be the same", newTable, alteredTable);
   }
 
+  @SuppressWarnings("deprecation")
   @Test
   public void testAlterTableCascade() throws Exception {
     Table originalTable = partitionedTable;
@@ -1069,6 +1075,255 @@ public class TestTablesCreateDropAlterTruncate extends MetaStoreClientTest {
     }
   }
 
+  @Test
+  public void tablesInOtherCatalogs() throws TException, URISyntaxException {
+    String catName = "create_etc_tables_in_other_catalogs";
+    Catalog cat = new CatalogBuilder()
+        .setName(catName)
+        .setLocation(MetaStoreTestUtils.getTestWarehouseDir(catName))
+        .build();
+    client.createCatalog(cat);
+
+    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)
+        .setCatalogName(catName)
+        .create(client, metaStore.getConf());
+
+    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
+      if (i == 3) {
+        builder.setType(TableType.MATERIALIZED_VIEW.name())
+            .setRewriteEnabled(true)
+            .addMaterializedViewReferencedTable(dbName + "." + tableNames[0]);
+      }
+      client.createTable(builder.build(metaStore.getConf()));
+    }
+
+    // Add partitions for the partitioned table
+    String[] partVals = new String[3];
+    Table partitionedTable = client.getTable(catName, dbName, tableNames[2]);
+    for (int i = 0; i < partVals.length; i++) {
+      partVals[i] = "part" + i;
+      new PartitionBuilder()
+          .inTable(partitionedTable)
+          .addValue(partVals[i])
+          .addToTable(client, metaStore.getConf());
+    }
+
+    // Get tables, make sure the locations are correct
+    for (int i = 0; i < tableNames.length; i++) {
+      Table t = client.getTable(catName, dbName, tableNames[i]);
+      Assert.assertEquals(catName, t.getCatName());
+      String expectedLocation = (i < 1) ?
+        new File(MetaStoreTestUtils.getTestWarehouseDir(tableNames[i])).toURI().toString()
+        :
+        new File(cat.getLocationUri() + 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(catName, 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
+    List<String> materializedViews = client.getMaterializedViewsForRewriting(catName, 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(catName, 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(catName, dbName, tableNames[0]).deepCopy();
+    t.getParameters().put("test", "test");
+    client.alter_table(catName, dbName, tableNames[0], t);
+    t = client.getTable(catName, 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
+    CreationMetadata cm = client.getTable(catName, dbName, tableNames[3]).getCreationMetadata();
+    cm.addToTablesUsed(dbName + "." + tableNames[1]);
+    client.updateCreationMetadata(catName, dbName, tableNames[3], cm);
+
+    List<String> partNames = new ArrayList<>();
+    for (String partVal : partVals) partNames.add("pcol1=" + partVal);
+    // Truncate a table
+    client.truncateTable(catName, dbName, tableNames[0], partNames);
+
+    // Truncate a table in the wrong catalog
+    try {
+      client.truncateTable(DEFAULT_DATABASE_NAME, tableNames[0], partNames);
+      Assert.fail();
+    } catch (NoSuchObjectException|TApplicationException e) {
+      // NOP
+    }
+
+    // Drop a table from the wrong catalog
+    try {
+      client.dropTable(DEFAULT_DATABASE_NAME, tableNames[0], true, false);
+      Assert.fail();
+    } catch (NoSuchObjectException|TApplicationException e) {
+      // NOP
+    }
+
+    // Should ignore the failure
+    client.dropTable(DEFAULT_DATABASE_NAME, tableNames[0], false, true);
+
+    // 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(catName, 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(catName, dbName, tableNames[i], false, false);
+        Assert.assertTrue(tableDir.exists() && tableDir.isDirectory());
+      } else {
+        client.dropTable(catName, dbName, tableNames[i]);
+        Assert.assertFalse(tableDir.exists());
+      }
+    }
+    Assert.assertEquals(0, client.getAllTables(catName, dbName).size());
+  }
+
+  @Test(expected = InvalidObjectException.class)
+  public void createTableInBogusCatalog() throws TException {
+    new TableBuilder()
+        .setCatName("nosuch")
+        .setTableName("doomed")
+        .addCol("col1", ColumnType.STRING_TYPE_NAME)
+        .addCol("col2", ColumnType.INT_TYPE_NAME)
+        .create(client, metaStore.getConf());
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void getTableInBogusCatalog() throws TException {
+    client.getTable("nosuch", testTables[0].getDbName(), testTables[0].getTableName());
+  }
+
+  @Test
+  public void getAllTablesInBogusCatalog() throws TException {
+    List<String> names = client.getAllTables("nosuch", testTables[0].getDbName());
+    Assert.assertTrue(names.isEmpty());
+  }
+
+  @Test(expected = UnknownDBException.class)
+  public void getTableObjectsByNameBogusCatalog() throws TException {
+    client.getTableObjectsByName("nosuch", testTables[0].getDbName(),
+        Arrays.asList(testTables[0].getTableName(), testTables[1].getTableName()));
+  }
+
+  @Test
+  public void getMaterializedViewsInBogusCatalog() throws TException {
+    List<String> names = client.getMaterializedViewsForRewriting("nosuch", DEFAULT_DATABASE_NAME);
+    Assert.assertTrue(names.isEmpty());
+  }
+
+  @Test(expected = InvalidOperationException.class)
+  public void alterTableBogusCatalog() throws TException {
+    Table t = testTables[0].deepCopy();
+    t.getParameters().put("a", "b");
+    client.alter_table("nosuch", t.getDbName(), t.getTableName(), t);
+  }
+
+  @Test(expected = InvalidOperationException.class)
+  public void moveTablesBetweenCatalogsOnAlter() throws TException {
+    String catName = "move_table_between_catalogs_on_alter";
+    Catalog cat = new CatalogBuilder()
+        .setName(catName)
+        .setLocation(MetaStoreTestUtils.getTestWarehouseDir(catName))
+        .build();
+    client.createCatalog(cat);
+
+    String dbName = "a_db";
+    // For this one don't specify a location to make sure it gets put in the catalog directory
+    Database db = new DatabaseBuilder()
+        .setName(dbName)
+        .setCatalogName(catName)
+        .create(client, metaStore.getConf());
+
+    String tableName = "non_movable_table";
+    Table before = new TableBuilder()
+        .inDb(db)
+        .setTableName(tableName)
+        .addCol("col1", ColumnType.STRING_TYPE_NAME)
+        .addCol("col2", ColumnType.INT_TYPE_NAME)
+        .create(client, metaStore.getConf());
+    Table after = before.deepCopy();
+    after.setCatName(DEFAULT_CATALOG_NAME);
+    client.alter_table(catName, dbName, tableName, after);
+
+  }
+
+  @Test
+  public void truncateTableBogusCatalog() throws TException {
+    try {
+      List<String> partNames = client.listPartitionNames(partitionedTable.getDbName(),
+          partitionedTable.getTableName(), (short) -1);
+      client.truncateTable("nosuch", partitionedTable.getDbName(), partitionedTable.getTableName(),
+          partNames);
+      Assert.fail(); // For reasons I don't understand and am too lazy to debug at the moment the
+      // NoSuchObjectException gets swallowed by a TApplicationException in remote mode.
+    } catch (TApplicationException|NoSuchObjectException e) {
+      //NOP
+    }
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void dropTableBogusCatalog() throws TException {
+    client.dropTable("nosuch", testTables[0].getDbName(), testTables[0].getTableName(), true, false);
+  }
+
   /**
    * Creates a Table with all of the parameters set. The temporary table is available only on HS2
    * server, so do not use it.
@@ -1105,6 +1360,6 @@ public class TestTablesCreateDropAlterTruncate extends MetaStoreClientTest {
                .addSerdeParam("serdeParam", "serdeParamValue")
                .addTableParam("tableParam", "tableParamValue")
                .addStorageDescriptorParam("sdParam", "sdParamValue")
-               .build();
+               .build(metaStore.getConf());
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesGetExists.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesGetExists.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesGetExists.java
index a1716ce..0de7f87 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesGetExists.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesGetExists.java
@@ -18,16 +18,22 @@
 
 package org.apache.hadoop.hive.metastore.client;
 
+import org.apache.hadoop.hive.metastore.ColumnType;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
 import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
+import org.apache.hadoop.hive.metastore.api.Catalog;
+import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.UnknownDBException;
+import org.apache.hadoop.hive.metastore.client.builder.CatalogBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
 import org.apache.hadoop.hive.metastore.minihms.AbstractMetaStoreService;
+import org.apache.thrift.TException;
 import org.apache.thrift.protocol.TProtocolException;
 import org.apache.thrift.transport.TTransportException;
 import org.junit.After;
@@ -39,7 +45,11 @@ import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
 import java.util.ArrayList;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
 
 /**
  * Test class for IMetaStoreClient API. Testing the Table related functions for metadata
@@ -78,7 +88,7 @@ public class TestTablesGetExists extends MetaStoreClientTest {
             .setDbName(DEFAULT_DATABASE)
             .setTableName("test_table")
             .addCol("test_col", "int")
-            .build();
+            .create(client, metaStore.getConf());
 
     testTables[1] =
         new TableBuilder()
@@ -86,14 +96,14 @@ public class TestTablesGetExists extends MetaStoreClientTest {
             .setTableName("test_view")
             .addCol("test_col", "int")
             .setType("VIEW")
-            .build();
+            .create(client, metaStore.getConf());
 
     testTables[2] =
         new TableBuilder()
             .setDbName(DEFAULT_DATABASE)
             .setTableName("test_table_to_find_1")
             .addCol("test_col", "int")
-            .build();
+            .create(client, metaStore.getConf());
 
     testTables[3] =
         new TableBuilder()
@@ -101,39 +111,35 @@ public class TestTablesGetExists extends MetaStoreClientTest {
             .setTableName("test_table_to_find_2")
             .addCol("test_col", "int")
             .setType("VIEW")
-            .build();
+            .create(client, metaStore.getConf());
 
     testTables[4] =
         new TableBuilder()
             .setDbName(DEFAULT_DATABASE)
             .setTableName("test_table_hidden_1")
             .addCol("test_col", "int")
-            .build();
+            .create(client, metaStore.getConf());
 
-    client.createDatabase(new DatabaseBuilder().setName(OTHER_DATABASE).build());
+    new DatabaseBuilder().setName(OTHER_DATABASE).create(client, metaStore.getConf());
 
     testTables[5] =
         new TableBuilder()
             .setDbName(OTHER_DATABASE)
             .setTableName("test_table")
             .addCol("test_col", "int")
-            .build();
+            .create(client, metaStore.getConf());
 
     testTables[6] =
         new TableBuilder()
             .setDbName(OTHER_DATABASE)
             .setTableName("test_table_to_find_3")
             .addCol("test_col", "int")
-            .build();
-
-    // Create the tables in the MetaStore
-    for(int i=0; i < testTables.length; i++) {
-      client.createTable(testTables[i]);
-    }
+            .create(client, metaStore.getConf());
 
     // Reload tables from the MetaStore
     for(int i=0; i < testTables.length; i++) {
-      testTables[i] = client.getTable(testTables[i].getDbName(), testTables[i].getTableName());
+      testTables[i] = client.getTable(testTables[i].getCatName(), testTables[i].getDbName(),
+          testTables[i].getTableName());
     }
   }
 
@@ -153,12 +159,12 @@ public class TestTablesGetExists extends MetaStoreClientTest {
     Table table = testTables[0];
 
     // Test in upper case
-    Table resultUpper = client.getTable(table.getDbName().toUpperCase(),
-        table.getTableName().toUpperCase());
+    Table resultUpper = client.getTable(table.getCatName().toUpperCase(),
+        table.getDbName().toUpperCase(), table.getTableName().toUpperCase());
     Assert.assertEquals("Comparing tables", table, resultUpper);
 
     // Test in mixed case
-    Table resultMix = client.getTable("DeFaUlt", "tEsT_TabLE");
+    Table resultMix = client.getTable("hIvE", "DeFaUlt", "tEsT_TabLE");
     Assert.assertEquals("Comparing tables", table, resultMix);
   }
 
@@ -222,7 +228,7 @@ public class TestTablesGetExists extends MetaStoreClientTest {
     }
 
     // Drop one table, see what remains
-    client.dropTable(testTables[1].getDbName(), testTables[1].getTableName());
+    client.dropTable(testTables[1].getCatName(), testTables[1].getDbName(), testTables[1] .getTableName());
     tables = client.getAllTables(DEFAULT_DATABASE);
     Assert.assertEquals("All tables size", 4, tables.size());
     for(Table table : testTables) {
@@ -274,7 +280,7 @@ public class TestTablesGetExists extends MetaStoreClientTest {
     Assert.assertEquals("No such table size", 0, tables.size());
 
     // Look for tables without pattern
-    tables = client.getTables(DEFAULT_DATABASE, null);
+    tables = client.getTables(DEFAULT_DATABASE, (String)null);
     Assert.assertEquals("No such functions size", 5, tables.size());
 
     // Look for tables with empty pattern
@@ -305,8 +311,9 @@ public class TestTablesGetExists extends MetaStoreClientTest {
     // Using the second table, since a table called "test_table" exists in both databases
     Table table = testTables[1];
 
-    Assert.assertTrue("Table exists", client.tableExists(table.getDbName(), table.getTableName()));
-    Assert.assertFalse("Table not exists", client.tableExists(table.getDbName(),
+    Assert.assertTrue("Table exists", client.tableExists(table.getCatName(), table.getDbName(),
+        table.getTableName()));
+    Assert.assertFalse("Table not exists", client.tableExists(table.getCatName(), table.getDbName(),
         "non_existing_table"));
 
     // No such database
@@ -323,11 +330,11 @@ public class TestTablesGetExists extends MetaStoreClientTest {
     Table table = testTables[0];
 
     // Test in upper case
-    Assert.assertTrue("Table exists", client.tableExists(table.getDbName().toUpperCase(),
-        table.getTableName().toUpperCase()));
+    Assert.assertTrue("Table exists", client.tableExists(table.getCatName().toUpperCase(),
+        table.getDbName().toUpperCase(), table.getTableName().toUpperCase()));
 
     // Test in mixed case
-    Assert.assertTrue("Table exists", client.tableExists("DeFaUlt", "tEsT_TabLE"));
+    Assert.assertTrue("Table exists", client.tableExists("hIVe", "DeFaUlt", "tEsT_TabLE"));
   }
 
   @Test
@@ -360,7 +367,7 @@ public class TestTablesGetExists extends MetaStoreClientTest {
 
   @Test
   public void testGetTableObjectsByName() throws Exception {
-    List<String> tableNames = new ArrayList<String>();
+    List<String> tableNames = new ArrayList<>();
     tableNames.add(testTables[0].getTableName());
     tableNames.add(testTables[1].getTableName());
     List<Table> tables = client.getTableObjectsByName(DEFAULT_DATABASE, tableNames);
@@ -374,17 +381,17 @@ public class TestTablesGetExists extends MetaStoreClientTest {
     }
 
     // Test with empty array
-    tables = client.getTableObjectsByName(DEFAULT_DATABASE, new ArrayList<String>());
+    tables = client.getTableObjectsByName(DEFAULT_DATABASE, new ArrayList<>());
     Assert.assertEquals("Found tables", 0, tables.size());
 
     // Test with table name which does not exists
-    tableNames = new ArrayList<String>();
+    tableNames = new ArrayList<>();
     tableNames.add("no_such_table");
-    client.getTableObjectsByName(testTables[0].getDbName(), tableNames);
+    client.getTableObjectsByName(testTables[0].getCatName(), testTables[0].getDbName(), tableNames);
     Assert.assertEquals("Found tables", 0, tables.size());
 
     // Test with table name which does not exists in the given database
-    tableNames = new ArrayList<String>();
+    tableNames = new ArrayList<>();
     tableNames.add(testTables[0].getTableName());
     client.getTableObjectsByName(OTHER_DATABASE, tableNames);
     Assert.assertEquals("Found tables", 0, tables.size());
@@ -396,23 +403,24 @@ public class TestTablesGetExists extends MetaStoreClientTest {
     Table table = testTables[0];
 
     // Test in upper case
-    List<String> tableNames = new ArrayList<String>();
+    List<String> tableNames = new ArrayList<>();
     tableNames.add(testTables[0].getTableName().toUpperCase());
-    List<Table> tables = client.getTableObjectsByName(table.getDbName().toUpperCase(), tableNames);
+    List<Table> tables = client.getTableObjectsByName(table.getCatName().toUpperCase(),
+        table.getDbName().toUpperCase(), tableNames);
     Assert.assertEquals("Found tables", 1, tables.size());
     Assert.assertEquals("Comparing tables", table, tables.get(0));
 
     // Test in mixed case
-    tableNames = new ArrayList<String>();
+    tableNames = new ArrayList<>();
     tableNames.add("tEsT_TabLE");
-    tables = client.getTableObjectsByName("DeFaUlt", tableNames);
+    tables = client.getTableObjectsByName("HiVe", "DeFaUlt", tableNames);
     Assert.assertEquals("Found tables", 1, tables.size());
     Assert.assertEquals("Comparing tables", table, tables.get(0));
   }
 
   @Test(expected = UnknownDBException.class)
   public void testGetTableObjectsByNameNoSuchDatabase() throws Exception {
-    List<String> tableNames = new ArrayList<String>();
+    List<String> tableNames = new ArrayList<>();
     tableNames.add(testTables[0].getTableName());
 
     client.getTableObjectsByName("no_such_database", tableNames);
@@ -421,7 +429,7 @@ public class TestTablesGetExists extends MetaStoreClientTest {
   @Test
   public void testGetTableObjectsByNameNullDatabase() throws Exception {
     try {
-      List<String> tableNames = new ArrayList<String>();
+      List<String> tableNames = new ArrayList<>();
       tableNames.add(OTHER_DATABASE);
 
       client.getTableObjectsByName(null, tableNames);
@@ -448,4 +456,55 @@ public class TestTablesGetExists extends MetaStoreClientTest {
       // Expected exception - Remote MetaStore
     }
   }
+
+  // Tests for getTable in other catalogs are covered in TestTablesCreateDropAlterTruncate.
+  @Test
+  public void otherCatalog() throws TException {
+    String catName = "get_exists_tables_in_other_catalogs";
+    Catalog cat = new CatalogBuilder()
+        .setName(catName)
+        .setLocation(MetaStoreTestUtils.getTestWarehouseDir(catName))
+        .build();
+    client.createCatalog(cat);
+
+    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)
+        .setCatalogName(catName)
+        .create(client, metaStore.getConf());
+
+    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, metaStore.getConf());
+    }
+
+    Set<String> tables = new HashSet<>(client.getTables(catName, dbName, "*e_in_other_*"));
+    Assert.assertEquals(4, tables.size());
+    for (String tableName : tableNames) Assert.assertTrue(tables.contains(tableName));
+
+    List<String> fetchedNames = client.getTables(catName, dbName, "*_3");
+    Assert.assertEquals(1, fetchedNames.size());
+    Assert.assertEquals(tableNames[3], fetchedNames.get(0));
+
+    Assert.assertTrue("Table exists", client.tableExists(catName, dbName, tableNames[0]));
+    Assert.assertFalse("Table not exists", client.tableExists(catName, dbName, "non_existing_table"));
+  }
+
+  @Test
+  public void getTablesBogusCatalog() throws TException {
+    Assert.assertEquals(0, client.getTables("nosuch", DEFAULT_DATABASE_NAME, "*_to_find_*").size());
+  }
+
+  @Test
+  public void tableExistsBogusCatalog() throws TException {
+    Assert.assertFalse(client.tableExists("nosuch", testTables[0].getDbName(),
+        testTables[0].getTableName()));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesList.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesList.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesList.java
index 7e4a59f..00e9104 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesList.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesList.java
@@ -18,16 +18,22 @@
 
 package org.apache.hadoop.hive.metastore.client;
 
+import org.apache.hadoop.hive.metastore.ColumnType;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
 import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
+import org.apache.hadoop.hive.metastore.api.Catalog;
+import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
+import org.apache.hadoop.hive.metastore.client.builder.CatalogBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
 import org.apache.hadoop.hive.metastore.minihms.AbstractMetaStoreService;
+import org.apache.thrift.TException;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -38,6 +44,8 @@ import org.junit.runners.Parameterized;
 
 import java.util.List;
 
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
+
 /**
  * Test class for IMetaStoreClient API. Testing the Table related functions for metadata
  * querying like getting one, or multiple tables, and table name lists.
@@ -78,7 +86,7 @@ public class TestTablesList extends MetaStoreClientTest {
             .setOwner("Owner1")
             .setLastAccessTime(1000)
             .addTableParam("param1", "value1")
-            .build();
+            .create(client, metaStore.getConf());
 
     testTables[1] =
         new TableBuilder()
@@ -88,7 +96,7 @@ public class TestTablesList extends MetaStoreClientTest {
             .setOwner("Owner1")
             .setLastAccessTime(2000)
             .addTableParam("param1", "value2")
-            .build();
+            .create(client, metaStore.getConf());
 
     testTables[2] =
         new TableBuilder()
@@ -98,7 +106,7 @@ public class TestTablesList extends MetaStoreClientTest {
             .setOwner("Owner2")
             .setLastAccessTime(1000)
             .addTableParam("param1", "value2")
-            .build();
+            .create(client, metaStore.getConf());
 
     testTables[3] =
         new TableBuilder()
@@ -108,7 +116,7 @@ public class TestTablesList extends MetaStoreClientTest {
             .setOwner("Owner3")
             .setLastAccessTime(3000)
             .addTableParam("param1", "value2")
-            .build();
+            .create(client, metaStore.getConf());
 
     testTables[4] =
         new TableBuilder()
@@ -118,16 +126,16 @@ public class TestTablesList extends MetaStoreClientTest {
             .setOwner("Tester")
             .setLastAccessTime(2500)
             .addTableParam("param1", "value4")
-            .build();
+            .create(client, metaStore.getConf());
 
     testTables[5] =
         new TableBuilder()
             .setDbName(DEFAULT_DATABASE)
             .setTableName("filter_test_table_5")
             .addCol("test_col", "int")
-            .build();
+            .create(client, metaStore.getConf());
 
-    client.createDatabase(new DatabaseBuilder().setName(OTHER_DATABASE).build());
+    new DatabaseBuilder().setName(OTHER_DATABASE).create(client, metaStore.getConf());
 
     testTables[6] =
         new TableBuilder()
@@ -137,16 +145,12 @@ public class TestTablesList extends MetaStoreClientTest {
             .setOwner("Owner1")
             .setLastAccessTime(1000)
             .addTableParam("param1", "value1")
-            .build();
-
-    // Create the tables in the MetaStore
-    for(int i=0; i < testTables.length; i++) {
-      client.createTable(testTables[i]);
-    }
+            .create(client, metaStore.getConf());
 
     // Reload tables from the MetaStore
     for(int i=0; i < testTables.length; i++) {
-      testTables[i] = client.getTable(testTables[i].getDbName(), testTables[i].getTableName());
+      testTables[i] = client.getTable(testTables[i].getCatName(), testTables[i].getDbName(),
+          testTables[i].getTableName());
     }
   }
 
@@ -268,4 +272,45 @@ public class TestTablesList extends MetaStoreClientTest {
   public void testListTableNamesByFilterInvalidFilter() throws Exception {
     client.listTableNamesByFilter(DEFAULT_DATABASE, "invalid filter", (short)-1);
   }
+
+  @Test
+  public void otherCatalogs() throws TException {
+    String catName = "list_tables_in_other_catalogs";
+    Catalog cat = new CatalogBuilder()
+        .setName(catName)
+        .setLocation(MetaStoreTestUtils.getTestWarehouseDir(catName))
+        .build();
+    client.createCatalog(cat);
+
+    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)
+        .setCatalogName(catName)
+        .create(client, metaStore.getConf());
+
+    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, metaStore.getConf());
+    }
+
+    String filter = hive_metastoreConstants.HIVE_FILTER_FIELD_PARAMS + "the_key=\"the_value\"";
+    List<String> fetchedNames = client.listTableNamesByFilter(catName, dbName, filter, (short)-1);
+    Assert.assertEquals(1, fetchedNames.size());
+    Assert.assertEquals(tableNames[0], fetchedNames.get(0));
+  }
+
+  @Test(expected = UnknownDBException.class)
+  public void listTablesBogusCatalog() throws TException {
+    String filter = hive_metastoreConstants.HIVE_FILTER_FIELD_PARAMS + "the_key=\"the_value\"";
+    List<String> fetchedNames = client.listTableNamesByFilter("", DEFAULT_DATABASE_NAME,
+        filter, (short)-1);
+  }
 }

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

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/minihms/AbstractMetaStoreService.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/minihms/AbstractMetaStoreService.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/minihms/AbstractMetaStoreService.java
index f2c8fe4..709085d 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/minihms/AbstractMetaStoreService.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/minihms/AbstractMetaStoreService.java
@@ -166,4 +166,8 @@ public abstract class AbstractMetaStoreService {
    */
   public void stop() {
   }
+
+  public Configuration getConf() {
+    return configuration;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/tools/TestSchemaToolForMetastore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/tools/TestSchemaToolForMetastore.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/tools/TestSchemaToolForMetastore.java
index 409ddc5..fa7057f 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/tools/TestSchemaToolForMetastore.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/tools/TestSchemaToolForMetastore.java
@@ -99,7 +99,8 @@ public class TestSchemaToolForMetastore {
     // Test valid case
     String[] scripts = new String[] {
         "insert into SEQUENCE_TABLE values('org.apache.hadoop.hive.metastore.model.MDatabase', 100);",
-        "insert into DBS values(99, 'test db1', 'hdfs:///tmp', 'db1', 'test', 'test');"
+        "insert into CTLGS values(37, 'mycat', 'my description', 'hdfs://tmp');",
+        "insert into DBS values(99, 'test db1', 'hdfs:///tmp', 'db1', 'test', 'test', 'mycat');"
     };
     File scriptFile = generateTestScript(scripts);
     schemaTool.runSqlLine(scriptFile.getPath());
@@ -111,7 +112,7 @@ public class TestSchemaToolForMetastore {
         "delete from SEQUENCE_TABLE;",
         "delete from DBS;",
         "insert into SEQUENCE_TABLE values('org.apache.hadoop.hive.metastore.model.MDatabase', 100);",
-        "insert into DBS values(102, 'test db1', 'hdfs:///tmp', 'db1', 'test', 'test');"
+        "insert into DBS values(102, 'test db1', 'hdfs:///tmp', 'db1', 'test', 'test', 'mycat');"
     };
     scriptFile = generateTestScript(scripts);
     schemaTool.runSqlLine(scriptFile.getPath());
@@ -217,6 +218,7 @@ public class TestSchemaToolForMetastore {
   public void testSchemaInit() throws Exception {
     IMetaStoreSchemaInfo metastoreSchemaInfo = MetaStoreSchemaInfoFactory.get(conf,
         System.getProperty("test.tmp.dir", "target/tmp"), "derby");
+    LOG.info("Starting testSchemaInit");
     schemaTool.doInit(metastoreSchemaInfo.getHiveSchemaVersion());
     schemaTool.verifySchemaVersion();
   }
@@ -296,11 +298,18 @@ public class TestSchemaToolForMetastore {
     System.setOut(outPrintStream);
 
     // Upgrade schema from 0.7.0 to latest
-    schemaTool.doUpgrade("1.2.0");
+    Exception caught = null;
+    try {
+      schemaTool.doUpgrade("1.2.0");
+    } catch (Exception e) {
+      caught = e;
+    }
 
     LOG.info("stdout is " + stdout.toString());
     LOG.info("stderr is " + stderr.toString());
 
+    if (caught != null) Assert.fail(caught.getMessage());
+
     // Verify that the schemaTool ran pre-upgrade scripts and ignored errors
     Assert.assertTrue(stderr.toString().contains(invalidPreUpgradeScript));
     Assert.assertTrue(stderr.toString().contains("foo"));
@@ -329,8 +338,9 @@ public class TestSchemaToolForMetastore {
 
  // Test valid case
     String[] scripts = new String[] {
-         "insert into DBS values(2, 'my db', 'hdfs://myhost.com:8020/user/hive/warehouse/mydb', 'mydb', 'public', 'role');",
-         "insert into DBS values(7, 'db with bad port', 'hdfs://myhost.com:8020/', 'haDB', 'public', 'role');",
+         "insert into CTLGS values (1, 'mycat', 'mydescription', 'hdfs://myhost.com:8020/user/hive/warehouse');",
+         "insert into DBS values(2, 'my db', 'hdfs://myhost.com:8020/user/hive/warehouse/mydb', 'mydb', 'public', 'role', 'mycat');",
+         "insert into DBS values(7, 'db with bad port', 'hdfs://myhost.com:8020/', 'haDB', 'public', 'role', 'mycat');",
          "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (1,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/user/hive/warehouse/mydb',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null);",
          "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (2,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/user/admin/2015_11_18',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null);",
          "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (3,null,'org.apache.hadoop.mapred.TextInputFormat','N','N',null,-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null);",
@@ -357,10 +367,10 @@ public class TestSchemaToolForMetastore {
         "delete from TBLS;",
         "delete from SDS;",
         "delete from DBS;",
-        "insert into DBS values(2, 'my db', '/user/hive/warehouse/mydb', 'mydb', 'public', 'role');",
-        "insert into DBS values(4, 'my db2', 'hdfs://myhost.com:8020', '', 'public', 'role');",
-        "insert into DBS values(6, 'db with bad port', 'hdfs://myhost.com:8020:', 'zDB', 'public', 'role');",
-        "insert into DBS values(7, 'db with bad port', 'hdfs://mynameservice.com/', 'haDB', 'public', 'role');",
+        "insert into DBS values(2, 'my db', '/user/hive/warehouse/mydb', 'mydb', 'public', 'role', 'mycat');",
+        "insert into DBS values(4, 'my db2', 'hdfs://myhost.com:8020', '', 'public', 'role', 'mycat');",
+        "insert into DBS values(6, 'db with bad port', 'hdfs://myhost.com:8020:', 'zDB', 'public', 'role', 'mycat');",
+        "insert into DBS values(7, 'db with bad port', 'hdfs://mynameservice.com/', 'haDB', 'public', 'role', 'mycat');",
         "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (1,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://yourhost.com:8020/user/hive/warehouse/mydb',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null);",
         "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (2,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','file:///user/admin/2015_11_18',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null);",
         "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (2 ,1435255431,2,0 ,'hive',0,1,'mytal','MANAGED_TABLE',NULL,NULL,'n');",
@@ -457,7 +467,8 @@ public class TestSchemaToolForMetastore {
   // Insert the records in DB to simulate a hive table
   private void createTestHiveTableSchemas() throws IOException {
      String[] scripts = new String[] {
-          "insert into DBS values(2, 'my db', 'hdfs://myhost.com:8020/user/hive/warehouse/mydb', 'mydb', 'public', 'role');",
+          "insert into CTLGS values (1, 'mycat', 'my description', 'hdfs://myhost.com:8020/user/hive/warehouse');",
+          "insert into DBS values(2, 'my db', 'hdfs://myhost.com:8020/user/hive/warehouse/mydb', 'mydb', 'public', 'role', 'mycat');",
           "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (1,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/user/hive/warehouse/mydb',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null);",
           "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (2,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/user/admin/2015_11_18',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null);",
           "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (2 ,1435255431,2,0 ,'hive',0,1,'mytal','MANAGED_TABLE',NULL,NULL,'n');",

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/resources/log4j2.properties
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/resources/log4j2.properties b/standalone-metastore/src/test/resources/log4j2.properties
index db8a550..365687e 100644
--- a/standalone-metastore/src/test/resources/log4j2.properties
+++ b/standalone-metastore/src/test/resources/log4j2.properties
@@ -8,64 +8,28 @@
 #
 #     http://www.apache.org/licenses/LICENSE-2.0
 #
-# 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.
+# 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.
 
-status = INFO
-name = MetastoreLog4j2
-packages = org.apache.hadoop.hive.metastore
+name=PropertiesConfig
+property.filename = logs
+appenders = console
 
-# list of properties
-property.metastore.log.level = INFO
-property.metastore.root.logger = DRFA
-property.metastore.log.dir = ${sys:java.io.tmpdir}/${sys:user.name}
-property.metastore.log.file = metastore.log
-property.hive.perflogger.log.level = INFO
-
-# list of all appenders
-appenders = console, DRFA
-
-# console appender
 appender.console.type = Console
-appender.console.name = console
-appender.console.target = SYSTEM_ERR
+appender.console.name = STDOUT
 appender.console.layout.type = PatternLayout
-appender.console.layout.pattern = %d{ISO8601} %5p [%t] %c{2}: %m%n
-
-# daily rolling file appender
-appender.DRFA.type = RollingRandomAccessFile
-appender.DRFA.name = DRFA
-appender.DRFA.fileName = ${sys:metastore.log.dir}/${sys:metastore.log.file}
-# Use %pid in the filePattern to append <process-id>@<host-name> to the filename if you want separate log files for different CLI session
-appender.DRFA.filePattern = ${sys:metastore.log.dir}/${sys:metastore.log.file}.%d{yyyy-MM-dd}
-appender.DRFA.layout.type = PatternLayout
-appender.DRFA.layout.pattern = %d{ISO8601} %5p [%t] %c{2}: %m%n
-appender.DRFA.policies.type = Policies
-appender.DRFA.policies.time.type = TimeBasedTriggeringPolicy
-appender.DRFA.policies.time.interval = 1
-appender.DRFA.policies.time.modulate = true
-appender.DRFA.strategy.type = DefaultRolloverStrategy
-appender.DRFA.strategy.max = 30
-
-# list of all loggers
-loggers = DataNucleus, Datastore, JPOX, PerfLogger
-
-logger.DataNucleus.name = DataNucleus
-logger.DataNucleus.level = INFO
-
-logger.Datastore.name = Datastore
-logger.Datastore.level = INFO
-
-logger.JPOX.name = JPOX
-logger.JPOX.level = INFO
+appender.console.layout.pattern = [%-5level] %d{yyyy-MM-dd HH:mm:ss.SSS} [%t] %c{1} - %msg%n
 
-logger.PerfLogger.name = org.apache.hadoop.hive.ql.log.PerfLogger
-logger.PerfLogger.level = ${sys:hive.perflogger.log.level}
+loggers=file
+logger.file.name=guru.springframework.blog.log4j2properties
+logger.file.level = debug
+logger.file.appenderRefs = file
+logger.file.appenderRef.file.ref = LOGFILE
 
-# root logger
-rootLogger.level = ${sys:metastore.log.level}
-rootLogger.appenderRefs = root
-rootLogger.appenderRef.root.ref = ${sys:metastore.root.logger}
+rootLogger.level = debug
+rootLogger.appenderRefs = stdout
+rootLogger.appenderRef.stdout.ref = STDOUT


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
index fd157b2..faeeea0 100644
--- a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
+++ b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
@@ -570,6 +570,7 @@ class SQLPrimaryKey:
    - enable_cstr
    - validate_cstr
    - rely_cstr
+   - catName
   """
 
   thrift_spec = (
@@ -582,9 +583,10 @@ class SQLPrimaryKey:
     (6, TType.BOOL, 'enable_cstr', None, None, ), # 6
     (7, TType.BOOL, 'validate_cstr', None, None, ), # 7
     (8, TType.BOOL, 'rely_cstr', None, None, ), # 8
+    (9, TType.STRING, 'catName', None, None, ), # 9
   )
 
-  def __init__(self, table_db=None, table_name=None, column_name=None, key_seq=None, pk_name=None, enable_cstr=None, validate_cstr=None, rely_cstr=None,):
+  def __init__(self, table_db=None, table_name=None, column_name=None, key_seq=None, pk_name=None, enable_cstr=None, validate_cstr=None, rely_cstr=None, catName=None,):
     self.table_db = table_db
     self.table_name = table_name
     self.column_name = column_name
@@ -593,6 +595,7 @@ class SQLPrimaryKey:
     self.enable_cstr = enable_cstr
     self.validate_cstr = validate_cstr
     self.rely_cstr = rely_cstr
+    self.catName = catName
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -643,6 +646,11 @@ class SQLPrimaryKey:
           self.rely_cstr = iprot.readBool()
         else:
           iprot.skip(ftype)
+      elif fid == 9:
+        if ftype == TType.STRING:
+          self.catName = iprot.readString()
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -685,6 +693,10 @@ class SQLPrimaryKey:
       oprot.writeFieldBegin('rely_cstr', TType.BOOL, 8)
       oprot.writeBool(self.rely_cstr)
       oprot.writeFieldEnd()
+    if self.catName is not None:
+      oprot.writeFieldBegin('catName', TType.STRING, 9)
+      oprot.writeString(self.catName)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -702,6 +714,7 @@ class SQLPrimaryKey:
     value = (value * 31) ^ hash(self.enable_cstr)
     value = (value * 31) ^ hash(self.validate_cstr)
     value = (value * 31) ^ hash(self.rely_cstr)
+    value = (value * 31) ^ hash(self.catName)
     return value
 
   def __repr__(self):
@@ -732,6 +745,7 @@ class SQLForeignKey:
    - enable_cstr
    - validate_cstr
    - rely_cstr
+   - catName
   """
 
   thrift_spec = (
@@ -750,9 +764,10 @@ class SQLForeignKey:
     (12, TType.BOOL, 'enable_cstr', None, None, ), # 12
     (13, TType.BOOL, 'validate_cstr', None, None, ), # 13
     (14, TType.BOOL, 'rely_cstr', None, None, ), # 14
+    (15, TType.STRING, 'catName', None, None, ), # 15
   )
 
-  def __init__(self, pktable_db=None, pktable_name=None, pkcolumn_name=None, fktable_db=None, fktable_name=None, fkcolumn_name=None, key_seq=None, update_rule=None, delete_rule=None, fk_name=None, pk_name=None, enable_cstr=None, validate_cstr=None, rely_cstr=None,):
+  def __init__(self, pktable_db=None, pktable_name=None, pkcolumn_name=None, fktable_db=None, fktable_name=None, fkcolumn_name=None, key_seq=None, update_rule=None, delete_rule=None, fk_name=None, pk_name=None, enable_cstr=None, validate_cstr=None, rely_cstr=None, catName=None,):
     self.pktable_db = pktable_db
     self.pktable_name = pktable_name
     self.pkcolumn_name = pkcolumn_name
@@ -767,6 +782,7 @@ class SQLForeignKey:
     self.enable_cstr = enable_cstr
     self.validate_cstr = validate_cstr
     self.rely_cstr = rely_cstr
+    self.catName = catName
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -847,6 +863,11 @@ class SQLForeignKey:
           self.rely_cstr = iprot.readBool()
         else:
           iprot.skip(ftype)
+      elif fid == 15:
+        if ftype == TType.STRING:
+          self.catName = iprot.readString()
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -913,6 +934,10 @@ class SQLForeignKey:
       oprot.writeFieldBegin('rely_cstr', TType.BOOL, 14)
       oprot.writeBool(self.rely_cstr)
       oprot.writeFieldEnd()
+    if self.catName is not None:
+      oprot.writeFieldBegin('catName', TType.STRING, 15)
+      oprot.writeString(self.catName)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -936,6 +961,7 @@ class SQLForeignKey:
     value = (value * 31) ^ hash(self.enable_cstr)
     value = (value * 31) ^ hash(self.validate_cstr)
     value = (value * 31) ^ hash(self.rely_cstr)
+    value = (value * 31) ^ hash(self.catName)
     return value
 
   def __repr__(self):
@@ -952,6 +978,7 @@ class SQLForeignKey:
 class SQLUniqueConstraint:
   """
   Attributes:
+   - catName
    - table_db
    - table_name
    - column_name
@@ -964,17 +991,19 @@ class SQLUniqueConstraint:
 
   thrift_spec = (
     None, # 0
-    (1, TType.STRING, 'table_db', None, None, ), # 1
-    (2, TType.STRING, 'table_name', None, None, ), # 2
-    (3, TType.STRING, 'column_name', None, None, ), # 3
-    (4, TType.I32, 'key_seq', None, None, ), # 4
-    (5, TType.STRING, 'uk_name', None, None, ), # 5
-    (6, TType.BOOL, 'enable_cstr', None, None, ), # 6
-    (7, TType.BOOL, 'validate_cstr', None, None, ), # 7
-    (8, TType.BOOL, 'rely_cstr', None, None, ), # 8
+    (1, TType.STRING, 'catName', None, None, ), # 1
+    (2, TType.STRING, 'table_db', None, None, ), # 2
+    (3, TType.STRING, 'table_name', None, None, ), # 3
+    (4, TType.STRING, 'column_name', None, None, ), # 4
+    (5, TType.I32, 'key_seq', None, None, ), # 5
+    (6, TType.STRING, 'uk_name', None, None, ), # 6
+    (7, TType.BOOL, 'enable_cstr', None, None, ), # 7
+    (8, TType.BOOL, 'validate_cstr', None, None, ), # 8
+    (9, TType.BOOL, 'rely_cstr', None, None, ), # 9
   )
 
-  def __init__(self, table_db=None, table_name=None, column_name=None, key_seq=None, uk_name=None, enable_cstr=None, validate_cstr=None, rely_cstr=None,):
+  def __init__(self, catName=None, table_db=None, table_name=None, column_name=None, key_seq=None, uk_name=None, enable_cstr=None, validate_cstr=None, rely_cstr=None,):
+    self.catName = catName
     self.table_db = table_db
     self.table_name = table_name
     self.column_name = column_name
@@ -995,40 +1024,45 @@ class SQLUniqueConstraint:
         break
       if fid == 1:
         if ftype == TType.STRING:
-          self.table_db = iprot.readString()
+          self.catName = iprot.readString()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.STRING:
-          self.table_name = iprot.readString()
+          self.table_db = iprot.readString()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.STRING:
-          self.column_name = iprot.readString()
+          self.table_name = iprot.readString()
         else:
           iprot.skip(ftype)
       elif fid == 4:
+        if ftype == TType.STRING:
+          self.column_name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
         if ftype == TType.I32:
           self.key_seq = iprot.readI32()
         else:
           iprot.skip(ftype)
-      elif fid == 5:
+      elif fid == 6:
         if ftype == TType.STRING:
           self.uk_name = iprot.readString()
         else:
           iprot.skip(ftype)
-      elif fid == 6:
+      elif fid == 7:
         if ftype == TType.BOOL:
           self.enable_cstr = iprot.readBool()
         else:
           iprot.skip(ftype)
-      elif fid == 7:
+      elif fid == 8:
         if ftype == TType.BOOL:
           self.validate_cstr = iprot.readBool()
         else:
           iprot.skip(ftype)
-      elif fid == 8:
+      elif fid == 9:
         if ftype == TType.BOOL:
           self.rely_cstr = iprot.readBool()
         else:
@@ -1043,36 +1077,40 @@ class SQLUniqueConstraint:
       oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
       return
     oprot.writeStructBegin('SQLUniqueConstraint')
+    if self.catName is not None:
+      oprot.writeFieldBegin('catName', TType.STRING, 1)
+      oprot.writeString(self.catName)
+      oprot.writeFieldEnd()
     if self.table_db is not None:
-      oprot.writeFieldBegin('table_db', TType.STRING, 1)
+      oprot.writeFieldBegin('table_db', TType.STRING, 2)
       oprot.writeString(self.table_db)
       oprot.writeFieldEnd()
     if self.table_name is not None:
-      oprot.writeFieldBegin('table_name', TType.STRING, 2)
+      oprot.writeFieldBegin('table_name', TType.STRING, 3)
       oprot.writeString(self.table_name)
       oprot.writeFieldEnd()
     if self.column_name is not None:
-      oprot.writeFieldBegin('column_name', TType.STRING, 3)
+      oprot.writeFieldBegin('column_name', TType.STRING, 4)
       oprot.writeString(self.column_name)
       oprot.writeFieldEnd()
     if self.key_seq is not None:
-      oprot.writeFieldBegin('key_seq', TType.I32, 4)
+      oprot.writeFieldBegin('key_seq', TType.I32, 5)
       oprot.writeI32(self.key_seq)
       oprot.writeFieldEnd()
     if self.uk_name is not None:
-      oprot.writeFieldBegin('uk_name', TType.STRING, 5)
+      oprot.writeFieldBegin('uk_name', TType.STRING, 6)
       oprot.writeString(self.uk_name)
       oprot.writeFieldEnd()
     if self.enable_cstr is not None:
-      oprot.writeFieldBegin('enable_cstr', TType.BOOL, 6)
+      oprot.writeFieldBegin('enable_cstr', TType.BOOL, 7)
       oprot.writeBool(self.enable_cstr)
       oprot.writeFieldEnd()
     if self.validate_cstr is not None:
-      oprot.writeFieldBegin('validate_cstr', TType.BOOL, 7)
+      oprot.writeFieldBegin('validate_cstr', TType.BOOL, 8)
       oprot.writeBool(self.validate_cstr)
       oprot.writeFieldEnd()
     if self.rely_cstr is not None:
-      oprot.writeFieldBegin('rely_cstr', TType.BOOL, 8)
+      oprot.writeFieldBegin('rely_cstr', TType.BOOL, 9)
       oprot.writeBool(self.rely_cstr)
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -1084,6 +1122,7 @@ class SQLUniqueConstraint:
 
   def __hash__(self):
     value = 17
+    value = (value * 31) ^ hash(self.catName)
     value = (value * 31) ^ hash(self.table_db)
     value = (value * 31) ^ hash(self.table_name)
     value = (value * 31) ^ hash(self.column_name)
@@ -1108,6 +1147,7 @@ class SQLUniqueConstraint:
 class SQLNotNullConstraint:
   """
   Attributes:
+   - catName
    - table_db
    - table_name
    - column_name
@@ -1119,16 +1159,18 @@ class SQLNotNullConstraint:
 
   thrift_spec = (
     None, # 0
-    (1, TType.STRING, 'table_db', None, None, ), # 1
-    (2, TType.STRING, 'table_name', None, None, ), # 2
-    (3, TType.STRING, 'column_name', None, None, ), # 3
-    (4, TType.STRING, 'nn_name', None, None, ), # 4
-    (5, TType.BOOL, 'enable_cstr', None, None, ), # 5
-    (6, TType.BOOL, 'validate_cstr', None, None, ), # 6
-    (7, TType.BOOL, 'rely_cstr', None, None, ), # 7
+    (1, TType.STRING, 'catName', None, None, ), # 1
+    (2, TType.STRING, 'table_db', None, None, ), # 2
+    (3, TType.STRING, 'table_name', None, None, ), # 3
+    (4, TType.STRING, 'column_name', None, None, ), # 4
+    (5, TType.STRING, 'nn_name', None, None, ), # 5
+    (6, TType.BOOL, 'enable_cstr', None, None, ), # 6
+    (7, TType.BOOL, 'validate_cstr', None, None, ), # 7
+    (8, TType.BOOL, 'rely_cstr', None, None, ), # 8
   )
 
-  def __init__(self, table_db=None, table_name=None, column_name=None, nn_name=None, enable_cstr=None, validate_cstr=None, rely_cstr=None,):
+  def __init__(self, catName=None, table_db=None, table_name=None, column_name=None, nn_name=None, enable_cstr=None, validate_cstr=None, rely_cstr=None,):
+    self.catName = catName
     self.table_db = table_db
     self.table_name = table_name
     self.column_name = column_name
@@ -1148,35 +1190,40 @@ class SQLNotNullConstraint:
         break
       if fid == 1:
         if ftype == TType.STRING:
-          self.table_db = iprot.readString()
+          self.catName = iprot.readString()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.STRING:
-          self.table_name = iprot.readString()
+          self.table_db = iprot.readString()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.STRING:
-          self.column_name = iprot.readString()
+          self.table_name = iprot.readString()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.STRING:
-          self.nn_name = iprot.readString()
+          self.column_name = iprot.readString()
         else:
           iprot.skip(ftype)
       elif fid == 5:
+        if ftype == TType.STRING:
+          self.nn_name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 6:
         if ftype == TType.BOOL:
           self.enable_cstr = iprot.readBool()
         else:
           iprot.skip(ftype)
-      elif fid == 6:
+      elif fid == 7:
         if ftype == TType.BOOL:
           self.validate_cstr = iprot.readBool()
         else:
           iprot.skip(ftype)
-      elif fid == 7:
+      elif fid == 8:
         if ftype == TType.BOOL:
           self.rely_cstr = iprot.readBool()
         else:
@@ -1191,32 +1238,36 @@ class SQLNotNullConstraint:
       oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
       return
     oprot.writeStructBegin('SQLNotNullConstraint')
+    if self.catName is not None:
+      oprot.writeFieldBegin('catName', TType.STRING, 1)
+      oprot.writeString(self.catName)
+      oprot.writeFieldEnd()
     if self.table_db is not None:
-      oprot.writeFieldBegin('table_db', TType.STRING, 1)
+      oprot.writeFieldBegin('table_db', TType.STRING, 2)
       oprot.writeString(self.table_db)
       oprot.writeFieldEnd()
     if self.table_name is not None:
-      oprot.writeFieldBegin('table_name', TType.STRING, 2)
+      oprot.writeFieldBegin('table_name', TType.STRING, 3)
       oprot.writeString(self.table_name)
       oprot.writeFieldEnd()
     if self.column_name is not None:
-      oprot.writeFieldBegin('column_name', TType.STRING, 3)
+      oprot.writeFieldBegin('column_name', TType.STRING, 4)
       oprot.writeString(self.column_name)
       oprot.writeFieldEnd()
     if self.nn_name is not None:
-      oprot.writeFieldBegin('nn_name', TType.STRING, 4)
+      oprot.writeFieldBegin('nn_name', TType.STRING, 5)
       oprot.writeString(self.nn_name)
       oprot.writeFieldEnd()
     if self.enable_cstr is not None:
-      oprot.writeFieldBegin('enable_cstr', TType.BOOL, 5)
+      oprot.writeFieldBegin('enable_cstr', TType.BOOL, 6)
       oprot.writeBool(self.enable_cstr)
       oprot.writeFieldEnd()
     if self.validate_cstr is not None:
-      oprot.writeFieldBegin('validate_cstr', TType.BOOL, 6)
+      oprot.writeFieldBegin('validate_cstr', TType.BOOL, 7)
       oprot.writeBool(self.validate_cstr)
       oprot.writeFieldEnd()
     if self.rely_cstr is not None:
-      oprot.writeFieldBegin('rely_cstr', TType.BOOL, 7)
+      oprot.writeFieldBegin('rely_cstr', TType.BOOL, 8)
       oprot.writeBool(self.rely_cstr)
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -1228,6 +1279,7 @@ class SQLNotNullConstraint:
 
   def __hash__(self):
     value = 17
+    value = (value * 31) ^ hash(self.catName)
     value = (value * 31) ^ hash(self.table_db)
     value = (value * 31) ^ hash(self.table_name)
     value = (value * 31) ^ hash(self.column_name)
@@ -1251,6 +1303,7 @@ class SQLNotNullConstraint:
 class SQLDefaultConstraint:
   """
   Attributes:
+   - catName
    - table_db
    - table_name
    - column_name
@@ -1263,17 +1316,19 @@ class SQLDefaultConstraint:
 
   thrift_spec = (
     None, # 0
-    (1, TType.STRING, 'table_db', None, None, ), # 1
-    (2, TType.STRING, 'table_name', None, None, ), # 2
-    (3, TType.STRING, 'column_name', None, None, ), # 3
-    (4, TType.STRING, 'default_value', None, None, ), # 4
-    (5, TType.STRING, 'dc_name', None, None, ), # 5
-    (6, TType.BOOL, 'enable_cstr', None, None, ), # 6
-    (7, TType.BOOL, 'validate_cstr', None, None, ), # 7
-    (8, TType.BOOL, 'rely_cstr', None, None, ), # 8
+    (1, TType.STRING, 'catName', None, None, ), # 1
+    (2, TType.STRING, 'table_db', None, None, ), # 2
+    (3, TType.STRING, 'table_name', None, None, ), # 3
+    (4, TType.STRING, 'column_name', None, None, ), # 4
+    (5, TType.STRING, 'default_value', None, None, ), # 5
+    (6, TType.STRING, 'dc_name', None, None, ), # 6
+    (7, TType.BOOL, 'enable_cstr', None, None, ), # 7
+    (8, TType.BOOL, 'validate_cstr', None, None, ), # 8
+    (9, TType.BOOL, 'rely_cstr', None, None, ), # 9
   )
 
-  def __init__(self, table_db=None, table_name=None, column_name=None, default_value=None, dc_name=None, enable_cstr=None, validate_cstr=None, rely_cstr=None,):
+  def __init__(self, catName=None, table_db=None, table_name=None, column_name=None, default_value=None, dc_name=None, enable_cstr=None, validate_cstr=None, rely_cstr=None,):
+    self.catName = catName
     self.table_db = table_db
     self.table_name = table_name
     self.column_name = column_name
@@ -1294,40 +1349,45 @@ class SQLDefaultConstraint:
         break
       if fid == 1:
         if ftype == TType.STRING:
-          self.table_db = iprot.readString()
+          self.catName = iprot.readString()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.STRING:
-          self.table_name = iprot.readString()
+          self.table_db = iprot.readString()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.STRING:
-          self.column_name = iprot.readString()
+          self.table_name = iprot.readString()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.STRING:
-          self.default_value = iprot.readString()
+          self.column_name = iprot.readString()
         else:
           iprot.skip(ftype)
       elif fid == 5:
         if ftype == TType.STRING:
-          self.dc_name = iprot.readString()
+          self.default_value = iprot.readString()
         else:
           iprot.skip(ftype)
       elif fid == 6:
+        if ftype == TType.STRING:
+          self.dc_name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 7:
         if ftype == TType.BOOL:
           self.enable_cstr = iprot.readBool()
         else:
           iprot.skip(ftype)
-      elif fid == 7:
+      elif fid == 8:
         if ftype == TType.BOOL:
           self.validate_cstr = iprot.readBool()
         else:
           iprot.skip(ftype)
-      elif fid == 8:
+      elif fid == 9:
         if ftype == TType.BOOL:
           self.rely_cstr = iprot.readBool()
         else:
@@ -1342,36 +1402,40 @@ class SQLDefaultConstraint:
       oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
       return
     oprot.writeStructBegin('SQLDefaultConstraint')
+    if self.catName is not None:
+      oprot.writeFieldBegin('catName', TType.STRING, 1)
+      oprot.writeString(self.catName)
+      oprot.writeFieldEnd()
     if self.table_db is not None:
-      oprot.writeFieldBegin('table_db', TType.STRING, 1)
+      oprot.writeFieldBegin('table_db', TType.STRING, 2)
       oprot.writeString(self.table_db)
       oprot.writeFieldEnd()
     if self.table_name is not None:
-      oprot.writeFieldBegin('table_name', TType.STRING, 2)
+      oprot.writeFieldBegin('table_name', TType.STRING, 3)
       oprot.writeString(self.table_name)
       oprot.writeFieldEnd()
     if self.column_name is not None:
-      oprot.writeFieldBegin('column_name', TType.STRING, 3)
+      oprot.writeFieldBegin('column_name', TType.STRING, 4)
       oprot.writeString(self.column_name)
       oprot.writeFieldEnd()
     if self.default_value is not None:
-      oprot.writeFieldBegin('default_value', TType.STRING, 4)
+      oprot.writeFieldBegin('default_value', TType.STRING, 5)
       oprot.writeString(self.default_value)
       oprot.writeFieldEnd()
     if self.dc_name is not None:
-      oprot.writeFieldBegin('dc_name', TType.STRING, 5)
+      oprot.writeFieldBegin('dc_name', TType.STRING, 6)
       oprot.writeString(self.dc_name)
       oprot.writeFieldEnd()
     if self.enable_cstr is not None:
-      oprot.writeFieldBegin('enable_cstr', TType.BOOL, 6)
+      oprot.writeFieldBegin('enable_cstr', TType.BOOL, 7)
       oprot.writeBool(self.enable_cstr)
       oprot.writeFieldEnd()
     if self.validate_cstr is not None:
-      oprot.writeFieldBegin('validate_cstr', TType.BOOL, 7)
+      oprot.writeFieldBegin('validate_cstr', TType.BOOL, 8)
       oprot.writeBool(self.validate_cstr)
       oprot.writeFieldEnd()
     if self.rely_cstr is not None:
-      oprot.writeFieldBegin('rely_cstr', TType.BOOL, 8)
+      oprot.writeFieldBegin('rely_cstr', TType.BOOL, 9)
       oprot.writeBool(self.rely_cstr)
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -1383,6 +1447,7 @@ class SQLDefaultConstraint:
 
   def __hash__(self):
     value = 17
+    value = (value * 31) ^ hash(self.catName)
     value = (value * 31) ^ hash(self.table_db)
     value = (value * 31) ^ hash(self.table_name)
     value = (value * 31) ^ hash(self.column_name)
@@ -1407,6 +1472,7 @@ class SQLDefaultConstraint:
 class SQLCheckConstraint:
   """
   Attributes:
+   - catName
    - table_db
    - table_name
    - column_name
@@ -1419,17 +1485,19 @@ class SQLCheckConstraint:
 
   thrift_spec = (
     None, # 0
-    (1, TType.STRING, 'table_db', None, None, ), # 1
-    (2, TType.STRING, 'table_name', None, None, ), # 2
-    (3, TType.STRING, 'column_name', None, None, ), # 3
-    (4, TType.STRING, 'check_expression', None, None, ), # 4
-    (5, TType.STRING, 'dc_name', None, None, ), # 5
-    (6, TType.BOOL, 'enable_cstr', None, None, ), # 6
-    (7, TType.BOOL, 'validate_cstr', None, None, ), # 7
-    (8, TType.BOOL, 'rely_cstr', None, None, ), # 8
+    (1, TType.STRING, 'catName', None, None, ), # 1
+    (2, TType.STRING, 'table_db', None, None, ), # 2
+    (3, TType.STRING, 'table_name', None, None, ), # 3
+    (4, TType.STRING, 'column_name', None, None, ), # 4
+    (5, TType.STRING, 'check_expression', None, None, ), # 5
+    (6, TType.STRING, 'dc_name', None, None, ), # 6
+    (7, TType.BOOL, 'enable_cstr', None, None, ), # 7
+    (8, TType.BOOL, 'validate_cstr', None, None, ), # 8
+    (9, TType.BOOL, 'rely_cstr', None, None, ), # 9
   )
 
-  def __init__(self, table_db=None, table_name=None, column_name=None, check_expression=None, dc_name=None, enable_cstr=None, validate_cstr=None, rely_cstr=None,):
+  def __init__(self, catName=None, table_db=None, table_name=None, column_name=None, check_expression=None, dc_name=None, enable_cstr=None, validate_cstr=None, rely_cstr=None,):
+    self.catName = catName
     self.table_db = table_db
     self.table_name = table_name
     self.column_name = column_name
@@ -1450,40 +1518,45 @@ class SQLCheckConstraint:
         break
       if fid == 1:
         if ftype == TType.STRING:
-          self.table_db = iprot.readString()
+          self.catName = iprot.readString()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.STRING:
-          self.table_name = iprot.readString()
+          self.table_db = iprot.readString()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.STRING:
-          self.column_name = iprot.readString()
+          self.table_name = iprot.readString()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.STRING:
-          self.check_expression = iprot.readString()
+          self.column_name = iprot.readString()
         else:
           iprot.skip(ftype)
       elif fid == 5:
         if ftype == TType.STRING:
-          self.dc_name = iprot.readString()
+          self.check_expression = iprot.readString()
         else:
           iprot.skip(ftype)
       elif fid == 6:
+        if ftype == TType.STRING:
+          self.dc_name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 7:
         if ftype == TType.BOOL:
           self.enable_cstr = iprot.readBool()
         else:
           iprot.skip(ftype)
-      elif fid == 7:
+      elif fid == 8:
         if ftype == TType.BOOL:
           self.validate_cstr = iprot.readBool()
         else:
           iprot.skip(ftype)
-      elif fid == 8:
+      elif fid == 9:
         if ftype == TType.BOOL:
           self.rely_cstr = iprot.readBool()
         else:
@@ -1498,36 +1571,40 @@ class SQLCheckConstraint:
       oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
       return
     oprot.writeStructBegin('SQLCheckConstraint')
+    if self.catName is not None:
+      oprot.writeFieldBegin('catName', TType.STRING, 1)
+      oprot.writeString(self.catName)
+      oprot.writeFieldEnd()
     if self.table_db is not None:
-      oprot.writeFieldBegin('table_db', TType.STRING, 1)
+      oprot.writeFieldBegin('table_db', TType.STRING, 2)
       oprot.writeString(self.table_db)
       oprot.writeFieldEnd()
     if self.table_name is not None:
-      oprot.writeFieldBegin('table_name', TType.STRING, 2)
+      oprot.writeFieldBegin('table_name', TType.STRING, 3)
       oprot.writeString(self.table_name)
       oprot.writeFieldEnd()
     if self.column_name is not None:
-      oprot.writeFieldBegin('column_name', TType.STRING, 3)
+      oprot.writeFieldBegin('column_name', TType.STRING, 4)
       oprot.writeString(self.column_name)
       oprot.writeFieldEnd()
     if self.check_expression is not None:
-      oprot.writeFieldBegin('check_expression', TType.STRING, 4)
+      oprot.writeFieldBegin('check_expression', TType.STRING, 5)
       oprot.writeString(self.check_expression)
       oprot.writeFieldEnd()
     if self.dc_name is not None:
-      oprot.writeFieldBegin('dc_name', TType.STRING, 5)
+      oprot.writeFieldBegin('dc_name', TType.STRING, 6)
       oprot.writeString(self.dc_name)
       oprot.writeFieldEnd()
     if self.enable_cstr is not None:
-      oprot.writeFieldBegin('enable_cstr', TType.BOOL, 6)
+      oprot.writeFieldBegin('enable_cstr', TType.BOOL, 7)
       oprot.writeBool(self.enable_cstr)
       oprot.writeFieldEnd()
     if self.validate_cstr is not None:
-      oprot.writeFieldBegin('validate_cstr', TType.BOOL, 7)
+      oprot.writeFieldBegin('validate_cstr', TType.BOOL, 8)
       oprot.writeBool(self.validate_cstr)
       oprot.writeFieldEnd()
     if self.rely_cstr is not None:
-      oprot.writeFieldBegin('rely_cstr', TType.BOOL, 8)
+      oprot.writeFieldBegin('rely_cstr', TType.BOOL, 9)
       oprot.writeBool(self.rely_cstr)
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -1539,6 +1616,7 @@ class SQLCheckConstraint:
 
   def __hash__(self):
     value = 17
+    value = (value * 31) ^ hash(self.catName)
     value = (value * 31) ^ hash(self.table_db)
     value = (value * 31) ^ hash(self.table_name)
     value = (value * 31) ^ hash(self.column_name)
@@ -1681,6 +1759,7 @@ class HiveObjectRef:
    - objectName
    - partValues
    - columnName
+   - catName
   """
 
   thrift_spec = (
@@ -1690,14 +1769,16 @@ class HiveObjectRef:
     (3, TType.STRING, 'objectName', None, None, ), # 3
     (4, TType.LIST, 'partValues', (TType.STRING,None), None, ), # 4
     (5, TType.STRING, 'columnName', None, None, ), # 5
+    (6, TType.STRING, 'catName', None, None, ), # 6
   )
 
-  def __init__(self, objectType=None, dbName=None, objectName=None, partValues=None, columnName=None,):
+  def __init__(self, objectType=None, dbName=None, objectName=None, partValues=None, columnName=None, catName=None,):
     self.objectType = objectType
     self.dbName = dbName
     self.objectName = objectName
     self.partValues = partValues
     self.columnName = columnName
+    self.catName = catName
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -1738,6 +1819,11 @@ class HiveObjectRef:
           self.columnName = iprot.readString()
         else:
           iprot.skip(ftype)
+      elif fid == 6:
+        if ftype == TType.STRING:
+          self.catName = iprot.readString()
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -1771,6 +1857,10 @@ class HiveObjectRef:
       oprot.writeFieldBegin('columnName', TType.STRING, 5)
       oprot.writeString(self.columnName)
       oprot.writeFieldEnd()
+    if self.catName is not None:
+      oprot.writeFieldBegin('catName', TType.STRING, 6)
+      oprot.writeString(self.catName)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -1785,6 +1875,7 @@ class HiveObjectRef:
     value = (value * 31) ^ hash(self.objectName)
     value = (value * 31) ^ hash(self.partValues)
     value = (value * 31) ^ hash(self.columnName)
+    value = (value * 31) ^ hash(self.catName)
     return value
 
   def __repr__(self):
@@ -3143,16 +3234,12 @@ class GrantRevokeRoleResponse:
   def __ne__(self, other):
     return not (self == other)
 
-class Database:
+class Catalog:
   """
   Attributes:
    - name
    - description
    - locationUri
-   - parameters
-   - privileges
-   - ownerName
-   - ownerType
   """
 
   thrift_spec = (
@@ -3160,20 +3247,12 @@ class Database:
     (1, TType.STRING, 'name', None, None, ), # 1
     (2, TType.STRING, 'description', None, None, ), # 2
     (3, TType.STRING, 'locationUri', None, None, ), # 3
-    (4, TType.MAP, 'parameters', (TType.STRING,None,TType.STRING,None), None, ), # 4
-    (5, TType.STRUCT, 'privileges', (PrincipalPrivilegeSet, PrincipalPrivilegeSet.thrift_spec), None, ), # 5
-    (6, TType.STRING, 'ownerName', None, None, ), # 6
-    (7, TType.I32, 'ownerType', None, None, ), # 7
   )
 
-  def __init__(self, name=None, description=None, locationUri=None, parameters=None, privileges=None, ownerName=None, ownerType=None,):
+  def __init__(self, name=None, description=None, locationUri=None,):
     self.name = name
     self.description = description
     self.locationUri = locationUri
-    self.parameters = parameters
-    self.privileges = privileges
-    self.ownerName = ownerName
-    self.ownerType = ownerType
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -3199,33 +3278,6 @@ class Database:
           self.locationUri = iprot.readString()
         else:
           iprot.skip(ftype)
-      elif fid == 4:
-        if ftype == TType.MAP:
-          self.parameters = {}
-          (_ktype84, _vtype85, _size83 ) = iprot.readMapBegin()
-          for _i87 in xrange(_size83):
-            _key88 = iprot.readString()
-            _val89 = iprot.readString()
-            self.parameters[_key88] = _val89
-          iprot.readMapEnd()
-        else:
-          iprot.skip(ftype)
-      elif fid == 5:
-        if ftype == TType.STRUCT:
-          self.privileges = PrincipalPrivilegeSet()
-          self.privileges.read(iprot)
-        else:
-          iprot.skip(ftype)
-      elif fid == 6:
-        if ftype == TType.STRING:
-          self.ownerName = iprot.readString()
-        else:
-          iprot.skip(ftype)
-      elif fid == 7:
-        if ftype == TType.I32:
-          self.ownerType = iprot.readI32()
-        else:
-          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -3235,7 +3287,7 @@ class Database:
     if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
       oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
       return
-    oprot.writeStructBegin('Database')
+    oprot.writeStructBegin('Catalog')
     if self.name is not None:
       oprot.writeFieldBegin('name', TType.STRING, 1)
       oprot.writeString(self.name)
@@ -3248,26 +3300,6 @@ class Database:
       oprot.writeFieldBegin('locationUri', TType.STRING, 3)
       oprot.writeString(self.locationUri)
       oprot.writeFieldEnd()
-    if self.parameters is not None:
-      oprot.writeFieldBegin('parameters', TType.MAP, 4)
-      oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.parameters))
-      for kiter90,viter91 in self.parameters.items():
-        oprot.writeString(kiter90)
-        oprot.writeString(viter91)
-      oprot.writeMapEnd()
-      oprot.writeFieldEnd()
-    if self.privileges is not None:
-      oprot.writeFieldBegin('privileges', TType.STRUCT, 5)
-      self.privileges.write(oprot)
-      oprot.writeFieldEnd()
-    if self.ownerName is not None:
-      oprot.writeFieldBegin('ownerName', TType.STRING, 6)
-      oprot.writeString(self.ownerName)
-      oprot.writeFieldEnd()
-    if self.ownerType is not None:
-      oprot.writeFieldBegin('ownerType', TType.I32, 7)
-      oprot.writeI32(self.ownerType)
-      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -3280,10 +3312,6 @@ class Database:
     value = (value * 31) ^ hash(self.name)
     value = (value * 31) ^ hash(self.description)
     value = (value * 31) ^ hash(self.locationUri)
-    value = (value * 31) ^ hash(self.parameters)
-    value = (value * 31) ^ hash(self.privileges)
-    value = (value * 31) ^ hash(self.ownerName)
-    value = (value * 31) ^ hash(self.ownerType)
     return value
 
   def __repr__(self):
@@ -3297,37 +3325,19 @@ class Database:
   def __ne__(self, other):
     return not (self == other)
 
-class SerDeInfo:
+class CreateCatalogRequest:
   """
   Attributes:
-   - name
-   - serializationLib
-   - parameters
-   - description
-   - serializerClass
-   - deserializerClass
-   - serdeType
+   - catalog
   """
 
   thrift_spec = (
     None, # 0
-    (1, TType.STRING, 'name', None, None, ), # 1
-    (2, TType.STRING, 'serializationLib', None, None, ), # 2
-    (3, TType.MAP, 'parameters', (TType.STRING,None,TType.STRING,None), None, ), # 3
-    (4, TType.STRING, 'description', None, None, ), # 4
-    (5, TType.STRING, 'serializerClass', None, None, ), # 5
-    (6, TType.STRING, 'deserializerClass', None, None, ), # 6
-    (7, TType.I32, 'serdeType', None, None, ), # 7
+    (1, TType.STRUCT, 'catalog', (Catalog, Catalog.thrift_spec), None, ), # 1
   )
 
-  def __init__(self, name=None, serializationLib=None, parameters=None, description=None, serializerClass=None, deserializerClass=None, serdeType=None,):
-    self.name = name
-    self.serializationLib = serializationLib
-    self.parameters = parameters
-    self.description = description
-    self.serializerClass = serializerClass
-    self.deserializerClass = deserializerClass
-    self.serdeType = serdeType
+  def __init__(self, catalog=None,):
+    self.catalog = catalog
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -3339,44 +3349,9 @@ class SerDeInfo:
       if ftype == TType.STOP:
         break
       if fid == 1:
-        if ftype == TType.STRING:
-          self.name = iprot.readString()
-        else:
-          iprot.skip(ftype)
-      elif fid == 2:
-        if ftype == TType.STRING:
-          self.serializationLib = iprot.readString()
-        else:
-          iprot.skip(ftype)
-      elif fid == 3:
-        if ftype == TType.MAP:
-          self.parameters = {}
-          (_ktype93, _vtype94, _size92 ) = iprot.readMapBegin()
-          for _i96 in xrange(_size92):
-            _key97 = iprot.readString()
-            _val98 = iprot.readString()
-            self.parameters[_key97] = _val98
-          iprot.readMapEnd()
-        else:
-          iprot.skip(ftype)
-      elif fid == 4:
-        if ftype == TType.STRING:
-          self.description = iprot.readString()
-        else:
-          iprot.skip(ftype)
-      elif fid == 5:
-        if ftype == TType.STRING:
-          self.serializerClass = iprot.readString()
-        else:
-          iprot.skip(ftype)
-      elif fid == 6:
-        if ftype == TType.STRING:
-          self.deserializerClass = iprot.readString()
-        else:
-          iprot.skip(ftype)
-      elif fid == 7:
-        if ftype == TType.I32:
-          self.serdeType = iprot.readI32()
+        if ftype == TType.STRUCT:
+          self.catalog = Catalog()
+          self.catalog.read(iprot)
         else:
           iprot.skip(ftype)
       else:
@@ -3388,8 +3363,563 @@ class SerDeInfo:
     if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
       oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
       return
-    oprot.writeStructBegin('SerDeInfo')
-    if self.name is not None:
+    oprot.writeStructBegin('CreateCatalogRequest')
+    if self.catalog is not None:
+      oprot.writeFieldBegin('catalog', TType.STRUCT, 1)
+      self.catalog.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.catalog)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class GetCatalogRequest:
+  """
+  Attributes:
+   - name
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'name', None, None, ), # 1
+  )
+
+  def __init__(self, name=None,):
+    self.name = name
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('GetCatalogRequest')
+    if self.name is not None:
+      oprot.writeFieldBegin('name', TType.STRING, 1)
+      oprot.writeString(self.name)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.name)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class GetCatalogResponse:
+  """
+  Attributes:
+   - catalog
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'catalog', (Catalog, Catalog.thrift_spec), None, ), # 1
+  )
+
+  def __init__(self, catalog=None,):
+    self.catalog = catalog
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.catalog = Catalog()
+          self.catalog.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('GetCatalogResponse')
+    if self.catalog is not None:
+      oprot.writeFieldBegin('catalog', TType.STRUCT, 1)
+      self.catalog.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.catalog)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class GetCatalogsResponse:
+  """
+  Attributes:
+   - names
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.LIST, 'names', (TType.STRING,None), None, ), # 1
+  )
+
+  def __init__(self, names=None,):
+    self.names = names
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.LIST:
+          self.names = []
+          (_etype86, _size83) = iprot.readListBegin()
+          for _i87 in xrange(_size83):
+            _elem88 = iprot.readString()
+            self.names.append(_elem88)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('GetCatalogsResponse')
+    if self.names is not None:
+      oprot.writeFieldBegin('names', TType.LIST, 1)
+      oprot.writeListBegin(TType.STRING, len(self.names))
+      for iter89 in self.names:
+        oprot.writeString(iter89)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.names)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class DropCatalogRequest:
+  """
+  Attributes:
+   - name
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'name', None, None, ), # 1
+  )
+
+  def __init__(self, name=None,):
+    self.name = name
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('DropCatalogRequest')
+    if self.name is not None:
+      oprot.writeFieldBegin('name', TType.STRING, 1)
+      oprot.writeString(self.name)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.name)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class Database:
+  """
+  Attributes:
+   - name
+   - description
+   - locationUri
+   - parameters
+   - privileges
+   - ownerName
+   - ownerType
+   - catalogName
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'name', None, None, ), # 1
+    (2, TType.STRING, 'description', None, None, ), # 2
+    (3, TType.STRING, 'locationUri', None, None, ), # 3
+    (4, TType.MAP, 'parameters', (TType.STRING,None,TType.STRING,None), None, ), # 4
+    (5, TType.STRUCT, 'privileges', (PrincipalPrivilegeSet, PrincipalPrivilegeSet.thrift_spec), None, ), # 5
+    (6, TType.STRING, 'ownerName', None, None, ), # 6
+    (7, TType.I32, 'ownerType', None, None, ), # 7
+    (8, TType.STRING, 'catalogName', None, None, ), # 8
+  )
+
+  def __init__(self, name=None, description=None, locationUri=None, parameters=None, privileges=None, ownerName=None, ownerType=None, catalogName=None,):
+    self.name = name
+    self.description = description
+    self.locationUri = locationUri
+    self.parameters = parameters
+    self.privileges = privileges
+    self.ownerName = ownerName
+    self.ownerType = ownerType
+    self.catalogName = catalogName
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.description = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRING:
+          self.locationUri = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.MAP:
+          self.parameters = {}
+          (_ktype91, _vtype92, _size90 ) = iprot.readMapBegin()
+          for _i94 in xrange(_size90):
+            _key95 = iprot.readString()
+            _val96 = iprot.readString()
+            self.parameters[_key95] = _val96
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.STRUCT:
+          self.privileges = PrincipalPrivilegeSet()
+          self.privileges.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 6:
+        if ftype == TType.STRING:
+          self.ownerName = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 7:
+        if ftype == TType.I32:
+          self.ownerType = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      elif fid == 8:
+        if ftype == TType.STRING:
+          self.catalogName = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('Database')
+    if self.name is not None:
+      oprot.writeFieldBegin('name', TType.STRING, 1)
+      oprot.writeString(self.name)
+      oprot.writeFieldEnd()
+    if self.description is not None:
+      oprot.writeFieldBegin('description', TType.STRING, 2)
+      oprot.writeString(self.description)
+      oprot.writeFieldEnd()
+    if self.locationUri is not None:
+      oprot.writeFieldBegin('locationUri', TType.STRING, 3)
+      oprot.writeString(self.locationUri)
+      oprot.writeFieldEnd()
+    if self.parameters is not None:
+      oprot.writeFieldBegin('parameters', TType.MAP, 4)
+      oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.parameters))
+      for kiter97,viter98 in self.parameters.items():
+        oprot.writeString(kiter97)
+        oprot.writeString(viter98)
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
+    if self.privileges is not None:
+      oprot.writeFieldBegin('privileges', TType.STRUCT, 5)
+      self.privileges.write(oprot)
+      oprot.writeFieldEnd()
+    if self.ownerName is not None:
+      oprot.writeFieldBegin('ownerName', TType.STRING, 6)
+      oprot.writeString(self.ownerName)
+      oprot.writeFieldEnd()
+    if self.ownerType is not None:
+      oprot.writeFieldBegin('ownerType', TType.I32, 7)
+      oprot.writeI32(self.ownerType)
+      oprot.writeFieldEnd()
+    if self.catalogName is not None:
+      oprot.writeFieldBegin('catalogName', TType.STRING, 8)
+      oprot.writeString(self.catalogName)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.name)
+    value = (value * 31) ^ hash(self.description)
+    value = (value * 31) ^ hash(self.locationUri)
+    value = (value * 31) ^ hash(self.parameters)
+    value = (value * 31) ^ hash(self.privileges)
+    value = (value * 31) ^ hash(self.ownerName)
+    value = (value * 31) ^ hash(self.ownerType)
+    value = (value * 31) ^ hash(self.catalogName)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class SerDeInfo:
+  """
+  Attributes:
+   - name
+   - serializationLib
+   - parameters
+   - description
+   - serializerClass
+   - deserializerClass
+   - serdeType
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'name', None, None, ), # 1
+    (2, TType.STRING, 'serializationLib', None, None, ), # 2
+    (3, TType.MAP, 'parameters', (TType.STRING,None,TType.STRING,None), None, ), # 3
+    (4, TType.STRING, 'description', None, None, ), # 4
+    (5, TType.STRING, 'serializerClass', None, None, ), # 5
+    (6, TType.STRING, 'deserializerClass', None, None, ), # 6
+    (7, TType.I32, 'serdeType', None, None, ), # 7
+  )
+
+  def __init__(self, name=None, serializationLib=None, parameters=None, description=None, serializerClass=None, deserializerClass=None, serdeType=None,):
+    self.name = name
+    self.serializationLib = serializationLib
+    self.parameters = parameters
+    self.description = description
+    self.serializerClass = serializerClass
+    self.deserializerClass = deserializerClass
+    self.serdeType = serdeType
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.serializationLib = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.MAP:
+          self.parameters = {}
+          (_ktype100, _vtype101, _size99 ) = iprot.readMapBegin()
+          for _i103 in xrange(_size99):
+            _key104 = iprot.readString()
+            _val105 = iprot.readString()
+            self.parameters[_key104] = _val105
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.STRING:
+          self.description = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.STRING:
+          self.serializerClass = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 6:
+        if ftype == TType.STRING:
+          self.deserializerClass = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 7:
+        if ftype == TType.I32:
+          self.serdeType = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('SerDeInfo')
+    if self.name is not None:
       oprot.writeFieldBegin('name', TType.STRING, 1)
       oprot.writeString(self.name)
       oprot.writeFieldEnd()
@@ -3400,9 +3930,9 @@ class SerDeInfo:
     if self.parameters is not None:
       oprot.writeFieldBegin('parameters', TType.MAP, 3)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.parameters))
-      for kiter99,viter100 in self.parameters.items():
-        oprot.writeString(kiter99)
-        oprot.writeString(viter100)
+      for kiter106,viter107 in self.parameters.items():
+        oprot.writeString(kiter106)
+        oprot.writeString(viter107)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.description is not None:
@@ -3560,41 +4090,41 @@ class SkewedInfo:
       if fid == 1:
         if ftype == TType.LIST:
           self.skewedColNames = []
-          (_etype104, _size101) = iprot.readListBegin()
-          for _i105 in xrange(_size101):
-            _elem106 = iprot.readString()
-            self.skewedColNames.append(_elem106)
+          (_etype111, _size108) = iprot.readListBegin()
+          for _i112 in xrange(_size108):
+            _elem113 = iprot.readString()
+            self.skewedColNames.append(_elem113)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.LIST:
           self.skewedColValues = []
-          (_etype110, _size107) = iprot.readListBegin()
-          for _i111 in xrange(_size107):
-            _elem112 = []
-            (_etype116, _size113) = iprot.readListBegin()
-            for _i117 in xrange(_size113):
-              _elem118 = iprot.readString()
-              _elem112.append(_elem118)
+          (_etype117, _size114) = iprot.readListBegin()
+          for _i118 in xrange(_size114):
+            _elem119 = []
+            (_etype123, _size120) = iprot.readListBegin()
+            for _i124 in xrange(_size120):
+              _elem125 = iprot.readString()
+              _elem119.append(_elem125)
             iprot.readListEnd()
-            self.skewedColValues.append(_elem112)
+            self.skewedColValues.append(_elem119)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.MAP:
           self.skewedColValueLocationMaps = {}
-          (_ktype120, _vtype121, _size119 ) = iprot.readMapBegin()
-          for _i123 in xrange(_size119):
-            _key124 = []
-            (_etype129, _size126) = iprot.readListBegin()
-            for _i130 in xrange(_size126):
-              _elem131 = iprot.readString()
-              _key124.append(_elem131)
+          (_ktype127, _vtype128, _size126 ) = iprot.readMapBegin()
+          for _i130 in xrange(_size126):
+            _key131 = []
+            (_etype136, _size133) = iprot.readListBegin()
+            for _i137 in xrange(_size133):
+              _elem138 = iprot.readString()
+              _key131.append(_elem138)
             iprot.readListEnd()
-            _val125 = iprot.readString()
-            self.skewedColValueLocationMaps[_key124] = _val125
+            _val132 = iprot.readString()
+            self.skewedColValueLocationMaps[_key131] = _val132
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -3611,29 +4141,29 @@ class SkewedInfo:
     if self.skewedColNames is not None:
       oprot.writeFieldBegin('skewedColNames', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.skewedColNames))
-      for iter132 in self.skewedColNames:
-        oprot.writeString(iter132)
+      for iter139 in self.skewedColNames:
+        oprot.writeString(iter139)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.skewedColValues is not None:
       oprot.writeFieldBegin('skewedColValues', TType.LIST, 2)
       oprot.writeListBegin(TType.LIST, len(self.skewedColValues))
-      for iter133 in self.skewedColValues:
-        oprot.writeListBegin(TType.STRING, len(iter133))
-        for iter134 in iter133:
-          oprot.writeString(iter134)
+      for iter140 in self.skewedColValues:
+        oprot.writeListBegin(TType.STRING, len(iter140))
+        for iter141 in iter140:
+          oprot.writeString(iter141)
         oprot.writeListEnd()
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.skewedColValueLocationMaps is not None:
       oprot.writeFieldBegin('skewedColValueLocationMaps', TType.MAP, 3)
       oprot.writeMapBegin(TType.LIST, TType.STRING, len(self.skewedColValueLocationMaps))
-      for kiter135,viter136 in self.skewedColValueLocationMaps.items():
-        oprot.writeListBegin(TType.STRING, len(kiter135))
-        for iter137 in kiter135:
-          oprot.writeString(iter137)
+      for kiter142,viter143 in self.skewedColValueLocationMaps.items():
+        oprot.writeListBegin(TType.STRING, len(kiter142))
+        for iter144 in kiter142:
+          oprot.writeString(iter144)
         oprot.writeListEnd()
-        oprot.writeString(viter136)
+        oprot.writeString(viter143)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -3720,11 +4250,11 @@ class StorageDescriptor:
       if fid == 1:
         if ftype == TType.LIST:
           self.cols = []
-          (_etype141, _size138) = iprot.readListBegin()
-          for _i142 in xrange(_size138):
-            _elem143 = FieldSchema()
-            _elem143.read(iprot)
-            self.cols.append(_elem143)
+          (_etype148, _size145) = iprot.readListBegin()
+          for _i149 in xrange(_size145):
+            _elem150 = FieldSchema()
+            _elem150.read(iprot)
+            self.cols.append(_elem150)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -3762,32 +4292,32 @@ class StorageDescriptor:
       elif fid == 8:
         if ftype == TType.LIST:
           self.bucketCols = []
-          (_etype147, _size144) = iprot.readListBegin()
-          for _i148 in xrange(_size144):
-            _elem149 = iprot.readString()
-            self.bucketCols.append(_elem149)
+          (_etype154, _size151) = iprot.readListBegin()
+          for _i155 in xrange(_size151):
+            _elem156 = iprot.readString()
+            self.bucketCols.append(_elem156)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 9:
         if ftype == TType.LIST:
           self.sortCols = []
-          (_etype153, _size150) = iprot.readListBegin()
-          for _i154 in xrange(_size150):
-            _elem155 = Order()
-            _elem155.read(iprot)
-            self.sortCols.append(_elem155)
+          (_etype160, _size157) = iprot.readListBegin()
+          for _i161 in xrange(_size157):
+            _elem162 = Order()
+            _elem162.read(iprot)
+            self.sortCols.append(_elem162)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 10:
         if ftype == TType.MAP:
           self.parameters = {}
-          (_ktype157, _vtype158, _size156 ) = iprot.readMapBegin()
-          for _i160 in xrange(_size156):
-            _key161 = iprot.readString()
-            _val162 = iprot.readString()
-            self.parameters[_key161] = _val162
+          (_ktype164, _vtype165, _size163 ) = iprot.readMapBegin()
+          for _i167 in xrange(_size163):
+            _key168 = iprot.readString()
+            _val169 = iprot.readString()
+            self.parameters[_key168] = _val169
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -3815,8 +4345,8 @@ class StorageDescriptor:
     if self.cols is not None:
       oprot.writeFieldBegin('cols', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.cols))
-      for iter163 in self.cols:
-        iter163.write(oprot)
+      for iter170 in self.cols:
+        iter170.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.location is not None:
@@ -3846,23 +4376,23 @@ class StorageDescriptor:
     if self.bucketCols is not None:
       oprot.writeFieldBegin('bucketCols', TType.LIST, 8)
       oprot.writeListBegin(TType.STRING, len(self.bucketCols))
-      for iter164 in self.bucketCols:
-        oprot.writeString(iter164)
+      for iter171 in self.bucketCols:
+        oprot.writeString(iter171)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.sortCols is not None:
       oprot.writeFieldBegin('sortCols', TType.LIST, 9)
       oprot.writeListBegin(TType.STRUCT, len(self.sortCols))
-      for iter165 in self.sortCols:
-        iter165.write(oprot)
+      for iter172 in self.sortCols:
+        iter172.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.parameters is not None:
       oprot.writeFieldBegin('parameters', TType.MAP, 10)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.parameters))
-      for kiter166,viter167 in self.parameters.items():
-        oprot.writeString(kiter166)
-        oprot.writeString(viter167)
+      for kiter173,viter174 in self.parameters.items():
+        oprot.writeString(kiter173)
+        oprot.writeString(viter174)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.skewedInfo is not None:
@@ -3926,6 +4456,7 @@ class Table:
    - temporary
    - rewriteEnabled
    - creationMetadata
+   - catName
   """
 
   thrift_spec = (
@@ -3946,9 +4477,10 @@ class Table:
     (14, TType.BOOL, 'temporary', None, False, ), # 14
     (15, TType.BOOL, 'rewriteEnabled', None, None, ), # 15
     (16, TType.STRUCT, 'creationMetadata', (CreationMetadata, CreationMetadata.thrift_spec), None, ), # 16
+    (17, TType.STRING, 'catName', None, None, ), # 17
   )
 
-  def __init__(self, tableName=None, dbName=None, owner=None, createTime=None, lastAccessTime=None, retention=None, sd=None, partitionKeys=None, parameters=None, viewOriginalText=None, viewExpandedText=None, tableType=None, privileges=None, temporary=thrift_spec[14][4], rewriteEnabled=None, creationMetadata=None,):
+  def __init__(self, tableName=None, dbName=None, owner=None, createTime=None, lastAccessTime=None, retention=None, sd=None, partitionKeys=None, parameters=None, viewOriginalText=None, viewExpandedText=None, tableType=None, privileges=None, temporary=thrift_spec[14][4], rewriteEnabled=None, creationMetadata=None, catName=None,):
     self.tableName = tableName
     self.dbName = dbName
     self.owner = owner
@@ -3965,6 +4497,7 @@ class Table:
     self.temporary = temporary
     self.rewriteEnabled = rewriteEnabled
     self.creationMetadata = creationMetadata
+    self.catName = catName
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -4014,22 +4547,22 @@ class Table:
       elif fid == 8:
         if ftype == TType.LIST:
           self.partitionKeys = []
-          (_etype171, _size168) = iprot.readListBegin()
-          for _i172 in xrange(_size168):
-            _elem173 = FieldSchema()
-            _elem173.read(iprot)
-            self.partitionKeys.append(_elem173)
+          (_etype178, _size175) = iprot.readListBegin()
+          for _i179 in xrange(_size175):
+            _elem180 = FieldSchema()
+            _elem180.read(iprot)
+            self.partitionKeys.append(_elem180)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 9:
         if ftype == TType.MAP:
           self.parameters = {}
-          (_ktype175, _vtype176, _size174 ) = iprot.readMapBegin()
-          for _i178 in xrange(_size174):
-            _key179 = iprot.readString()
-            _val180 = iprot.readString()
-            self.parameters[_key179] = _val180
+          (_ktype182, _vtype183, _size181 ) = iprot.readMapBegin()
+          for _i185 in xrange(_size181):
+            _key186 = iprot.readString()
+            _val187 = iprot.readString()
+            self.parameters[_key186] = _val187
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -4070,6 +4603,11 @@ class Table:
           self.creationMetadata.read(iprot)
         else:
           iprot.skip(ftype)
+      elif fid == 17:
+        if ftype == TType.STRING:
+          self.catName = iprot.readString()
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -4111,16 +4649,16 @@ class Table:
     if self.partitionKeys is not None:
       oprot.writeFieldBegin('partitionKeys', TType.LIST, 8)
       oprot.writeListBegin(TType.STRUCT, len(self.partitionKeys))
-      for iter181 in self.partitionKeys:
-        iter181.write(oprot)
+      for iter188 in self.partitionKeys:
+        iter188.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.parameters is not None:
       oprot.writeFieldBegin('parameters', TType.MAP, 9)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.parameters))
-      for kiter182,viter183 in self.parameters.items():
-        oprot.writeString(kiter182)
-        oprot.writeString(viter183)
+      for kiter189,viter190 in self.parameters.items():
+        oprot.writeString(kiter189)
+        oprot.writeString(viter190)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.viewOriginalText is not None:
@@ -4151,6 +4689,10 @@ class Table:
       oprot.writeFieldBegin('creationMetadata', TType.STRUCT, 16)
       self.creationMetadata.write(oprot)
       oprot.writeFieldEnd()
+    if self.catName is not None:
+      oprot.writeFieldBegin('catName', TType.STRING, 17)
+      oprot.writeString(self.catName)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -4176,6 +4718,7 @@ class Table:
     value = (value * 31) ^ hash(self.temporary)
     value = (value * 31) ^ hash(self.rewriteEnabled)
     value = (value * 31) ^ hash(self.creationMetadata)
+    value = (value * 31) ^ hash(self.catName)
     return value
 
   def __repr__(self):
@@ -4200,6 +4743,7 @@ class Partition:
    - sd
    - parameters
    - privileges
+   - catName
   """
 
   thrift_spec = (
@@ -4212,9 +4756,10 @@ class Partition:
     (6, TType.STRUCT, 'sd', (StorageDescriptor, StorageDescriptor.thrift_spec), None, ), # 6
     (7, TType.MAP, 'parameters', (TType.STRING,None,TType.STRING,None), None, ), # 7
     (8, TType.STRUCT, 'privileges', (PrincipalPrivilegeSet, PrincipalPrivilegeSet.thrift_spec), None, ), # 8
+    (9, TType.STRING, 'catName', None, None, ), # 9
   )
 
-  def __init__(self, values=None, dbName=None, tableName=None, createTime=None, lastAccessTime=None, sd=None, parameters=None, privileges=None,):
+  def __init__(self, values=None, dbName=None, tableName=None, createTime=None, lastAccessTime=None, sd=None, parameters=None, privileges=None, catName=None,):
     self.values = values
     self.dbName = dbName
     self.tableName = tableName
@@ -4223,6 +4768,7 @@ class Partition:
     self.sd = sd
     self.parameters = parameters
     self.privileges = privileges
+    self.catName = catName
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -4236,10 +4782,10 @@ class Partition:
       if fid == 1:
         if ftype == TType.LIST:
           self.values = []
-          (_etype187, _size184) = iprot.readListBegin()
-          for _i188 in xrange(_size184):
-            _elem189 = iprot.readString()
-            self.values.append(_elem189)
+          (_etype194, _size191) = iprot.readListBegin()
+          for _i195 in xrange(_size191):
+            _elem196 = iprot.readString()
+            self.values.append(_elem196)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -4272,11 +4818,11 @@ class Partition:
       elif fid == 7:
         if ftype == TType.MAP:
           self.parameters = {}
-          (_ktype191, _vtype192, _size190 ) = iprot.readMapBegin()
-          for _i194 in xrange(_size190):
-            _key195 = iprot.readString()
-            _val196 = iprot.readString()
-            self.parameters[_key195] = _val196
+          (_ktype198, _vtype199, _size197 ) = iprot.readMapBegin()
+          for _i201 in xrange(_size197):
+            _key202 = iprot.readString()
+            _val203 = iprot.readString()
+            self.parameters[_key202] = _val203
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -4286,6 +4832,11 @@ class Partition:
           self.privileges.read(iprot)
         else:
           iprot.skip(ftype)
+      elif fid == 9:
+        if ftype == TType.STRING:
+          self.catName = iprot.readString()
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -4299,8 +4850,8 @@ class Partition:
     if self.values is not None:
       oprot.writeFieldBegin('values', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.values))
-      for iter197 in self.values:
-        oprot.writeString(iter197)
+      for iter204 in self.values:
+        oprot.writeString(iter204)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.dbName is not None:
@@ -4326,15 +4877,19 @@ class Partition:
     if self.parameters is not None:
       oprot.writeFieldBegin('parameters', TType.MAP, 7)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.parameters))
-      for kiter198,viter199 in self.parameters.items():
-        oprot.writeString(kiter198)
-        oprot.writeString(viter199)
+      for kiter205,viter206 in self.parameters.items():
+        oprot.writeString(kiter205)
+        oprot.writeString(viter206)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.privileges is not None:
       oprot.writeFieldBegin('privileges', TType.STRUCT, 8)
       self.privileges.write(oprot)
       oprot.writeFieldEnd()
+    if self.catName is not None:
+      oprot.writeFieldBegin('catName', TType.STRING, 9)
+      oprot.writeString(self.catName)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -4352,6 +4907,7 @@ class Partition:
     value = (value * 31) ^ hash(self.sd)
     value = (value * 31) ^ hash(self.parameters)
     value = (value * 31) ^ hash(self.privileges)
+    value = (value * 31) ^ hash(self.catName)
     return value
 
   def __repr__(self):
@@ -4406,10 +4962,10 @@ class PartitionWithoutSD:
       if fid == 1:
         if ftype == TType.LIST:
           self.values = []
-          (_etype203, _size200) = iprot.readListBegin()
-          for _i204 in xrange(_size200):
-            _elem205 = iprot.readString()
-            self.values.append(_elem205)
+          (_etype210, _size207) = iprot.readListBegin()
+          for _i211 in xrange(_size207):
+            _elem212 = iprot.readString()
+            self.values.append(_elem212)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -4431,11 +4987,11 @@ class PartitionWithoutSD:
       elif fid == 5:
         if ftype == TType.MAP:
           self.parameters = {}
-          (_ktype207, _vtype208, _size206 ) = iprot.readMapBegin()
-          for _i210 in xrange(_size206):
-            _key211 = iprot.readString()
-            _val212 = iprot.readString()
-            self.parameters[_key211] = _val212
+          (_ktype214, _vtype215, _size213 ) = iprot.readMapBegin()
+          for _i217 in xrange(_size213):
+            _key218 = iprot.readString()
+            _val219 = iprot.readString()
+            self.parameters[_key218] = _val219
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -4458,8 +5014,8 @@ class PartitionWithoutSD:
     if self.values is not None:
       oprot.writeFieldBegin('values', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.values))
-      for iter213 in self.values:
-        oprot.writeString(iter213)
+      for iter220 in self.values:
+        oprot.writeString(iter220)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.createTime is not None:
@@ -4477,9 +5033,9 @@ class PartitionWithoutSD:
     if self.parameters is not None:
       oprot.writeFieldBegin('parameters', TType.MAP, 5)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.parameters))
-      for kiter214,viter215 in self.parameters.items():
-        oprot.writeString(kiter214)
-        oprot.writeString(viter215)
+      for kiter221,viter222 in self.parameters.items():
+        oprot.writeString(kiter221)
+        oprot.writeString(viter222)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.privileges is not None:
@@ -4543,11 +5099,11 @@ class PartitionSpecWithSharedSD:
       if fid == 1:
         if ftype == TType.LIST:
           self.partitions = []
-          (_etype219, _size216) = iprot.readListBegin()
-          for _i220 in xrange(_size216):
-            _elem221 = PartitionWithoutSD()
-            _elem221.read(iprot)
-            self.partitions.append(_elem221)
+          (_etype226, _size223) = iprot.readListBegin()
+          for _i227 in xrange(_size223):
+            _elem228 = PartitionWithoutSD()
+            _elem228.read(iprot)
+            self.partitions.append(_elem228)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -4570,8 +5126,8 @@ class PartitionSpecWithSharedSD:
     if self.partitions is not None:
       oprot.writeFieldBegin('partitions', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.partitions))
-      for iter222 in self.partitions:
-        iter222.write(oprot)
+      for iter229 in self.partitions:
+        iter229.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.sd is not None:
@@ -4628,11 +5184,11 @@ class PartitionListComposingSpec:
       if fid == 1:
         if ftype == TType.LIST:
           self.partitions = []
-          (_etype226, _size223) = iprot.readListBegin()
-          for _i227 in xrange(_size223):
-            _elem228 = Partition()
-            _elem228.read(iprot)
-            self.partitions.append(_elem228)
+          (_etype233, _size230) = iprot.readListBegin()
+          for _i234 in xrange(_size230):
+            _elem235 = Partition()
+            _elem235.read(iprot)
+            self.partitions.append(_elem235)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -4649,8 +5205,8 @@ class PartitionListComposingSpec:
     if self.partitions is not None:
       oprot.writeFieldBegin('partitions', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.partitions))
-      for iter229 in self.partitions:
-        iter229.write(oprot)
+      for iter236 in self.partitions:
+        iter236.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -4684,6 +5240,7 @@ class PartitionSpec:
    - rootPath
    - sharedSDPartitionSpec
    - partitionList
+   - catName
   """
 
   thrift_spec = (
@@ -4693,14 +5250,16 @@ class PartitionSpec:
     (3, TType.STRING, 'rootPath', None, None, ), # 3
     (4, TType.STRUCT, 'sharedSDPartitionSpec', (PartitionSpecWithSharedSD, PartitionSpecWithSharedSD.thrift_spec), None, ), # 4
     (5, TType.STRUCT, 'partitionList', (PartitionListComposingSpec, PartitionListComposingSpec.thrift_spec), None, ), # 5
+    (6, TType.STRING, 'catName', None, None, ), # 6
   )
 
-  def __init__(self, dbName=None, tableName=None, rootPath=None, sharedSDPartitionSpec=None, partitionList=None,):
+  def __init__(self, dbName=None, tableName=None, rootPath=None, sharedSDPartitionSpec=None, partitionList=None, catName=None,):
     self.dbName = dbName
     self.tableName = tableName
     self.rootPath = rootPath
     self.sharedSDPartitionSpec = sharedSDPartitionSpec
     self.partitionList = partitionList
+    self.catName = catName
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -4738,6 +5297,11 @@ class PartitionSpec:
           self.partitionList.read(iprot)
         else:
           iprot.skip(ftype)
+      elif fid == 6:
+        if ftype == TType.STRING:
+          self.catName = iprot.readString()
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -4768,6 +5332,10 @@ class PartitionSpec:
       oprot.writeFieldBegin('partitionList', TType.STRUCT, 5)
       self.partitionList.write(oprot)
       oprot.writeFieldEnd()
+    if self.catName is not None:
+      oprot.writeFieldBegin('catName', TType.STRING, 6)
+      oprot.writeString(self.catName)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -4782,6 +5350,7 @@ class PartitionSpec:
     value = (value * 31) ^ hash(self.rootPath)
     value = (value * 31) ^ hash(self.sharedSDPartitionSpec)
     value = (value * 31) ^ hash(self.partitionList)
+    value = (value * 31) ^ hash(self.catName)
     return value
 
   def __repr__(self):
@@ -6034,6 +6603,7 @@ class ColumnStatisticsDesc:
    - tableName
    - partName
    - lastAnalyzed
+   - catName
   """
 
   thrift_spec = (
@@ -6043,14 +6613,16 @@ class ColumnStatisticsDesc:
     (3, TType.STRING, 'tableName', None, None, ), # 3
     (4, TType.STRING, 'partName', None, None, ), # 4
     (5, TType.I64, 'lastAnalyzed', None, None, ), # 5
+    (6, TType.STRING, 'catName', None, None, ), # 6
   )
 
-  def __init__(self, isTblLevel=None, dbName=None, tableName=None, partName=None, lastAnalyzed=None,):
+  def __init__(self, isTblLevel=None, dbName=None, tableName=None, partName=None, lastAnalyzed=None, catName=None,):
     self.isTblLevel = isTblLevel
     self.dbName = dbName
     self.tableName = tableName
     self.partName = partName
     self.lastAnalyzed = lastAnalyzed
+    self.catName = catName
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -6086,6 +6658,11 @@ class ColumnStatisticsDesc:
           self.lastAnalyzed = iprot.readI64()
         else:
           iprot.skip(ftype)
+      elif fid == 6:
+        if ftype == TType.STRING:
+          self.catName = iprot.readString()
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -6116,6 +6693,10 @@ class ColumnStatisticsDesc:
       oprot.writeFieldBegin('lastAnalyzed', TType.I64, 5)
       oprot.writeI64(self.lastAnalyzed)
       oprot.writeFieldEnd()
+    if self.catName is not None:
+      oprot.writeFieldBegin('catName', TType.STRING, 6)
+      oprot.writeString(self.catName)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -6136,6 +6717,7 @@ class ColumnStatisticsDesc:
     value = (value * 31) ^ hash(self.tableName)
     value = (value * 31) ^ hash(self.partName)
     value = (value * 31) ^ hash(self.lastAnalyzed)
+    value = (value * 31) ^ hash(self.catName)
     return value
 
   def __repr__(self):
@@ -6184,11 +6766,11 @@ class ColumnStatistics:
       elif fid == 2:
         if ftype == TType.LIST:
           self.statsObj = []
-          (_etype233, _size230) = iprot.readListBegin()
-          for _i234 in xrange(_size230):
-            _elem235 = ColumnStatisticsObj()
-            _elem235.read(iprot)
-            self.statsObj.append(_elem235)
+          (_etype240, _size237) = iprot.readListBegin()
+          for _i241 in xrange(_size237):
+            _elem242 = ColumnStatisticsObj()
+            _elem242.read(iprot)
+            self.statsObj.append(_elem242)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -6209,8 +6791,8 @@ class ColumnStatistics:
     if self.statsObj is not None:
       oprot.writeFieldBegin('statsObj', TType.LIST, 2)
       oprot.writeListBegin(TType.STRUCT, len(self.statsObj))
-      for iter236 in self.statsObj:
-        iter236.write(oprot)
+      for iter243 in self.statsObj:
+        iter243.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -6270,11 +6852,11 @@ class AggrStats:
       if fid == 1:
         if ftype == TType.LIST:
           self.colStats = []
-          (_etype240, _size237) = iprot.readListBegin()
-          for _i241 in xrange(_size237):
-            _elem242 = ColumnStatisticsObj()
-            _elem242.read(iprot)
-            self.colStats.append(_elem242)
+          (_etype247, _size244) = iprot.readListBegin()
+          for _i248 in xrange(_size244):
+            _elem249 = ColumnStatisticsObj()
+            _elem249.read(iprot)
+            self.colStats.append(_elem249)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -6296,8 +6878,8 @@ class AggrStats:
     if self.colStats is not None:
       oprot.writeFieldBegin('colStats', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.colStats))
-      for iter243 in self.colStats:
-        iter243.write(oprot)
+      for iter250 in self.colStats:
+        iter250.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.partsFound is not None:
@@ -6361,11 +6943,11 @@ class SetPartitionsStatsRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.colStats = []
-          (_etype247, _size244) = iprot.readListBegin()
-          for _i248 in xrange(_size244):
-            _elem249 = ColumnStatistics()
-            _elem249.read(iprot)
-            self.colStats.append(_elem249)
+          (_etype254, _size251) = iprot.readListBegin()
+          for _i255 in xrange(_size251):
+            _elem256 = ColumnStatistics()
+            _elem256.read(iprot)
+            self.colStats.append(_elem256)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -6387,8 +6969,8 @@ class SetPartitionsStatsRequest:
     if self.colStats is not None:
       oprot.writeFieldBegin('colStats', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.colStats))
-      for iter250 in self.colStats:
-        iter250.write(oprot)
+      for iter257 in self.colStats:
+        iter257.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.needMerge is not None:
@@ -6450,22 +7032,22 @@ class Schema:
       if fid == 1:
         if ftype == TType.LIST:
           self.fieldSchemas = []
-          (_etype254, _size251) = iprot.readListBegin()
-          for _i255 in xrange(_size251):
-            _elem256 = FieldSchema()
-            _elem256.read(iprot)
-            self.fieldSchemas.append(_elem256)
+          (_etype261, _size258) = iprot.readListBegin()
+          for _i262 in xrange(_size258):
+            _elem263 = FieldSchema()
+            _elem263.read(iprot)
+            self.fieldSchemas.append(_elem263)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.MAP:
           self.properties = {}
-          (_ktype258, _vtype259, _size257 ) = iprot.readMapBegin()
-          for _i261 in xrange(_size257):
-            _key262 = iprot.readString()
-            _val263 = iprot.readString()
-            self.properties[_key262] = _val263
+          (_ktype265, _vtype266, _size264 ) = iprot.readMapBegin()
+          for _i268 in xrange(_size264):
+            _key269 = iprot.readString()
+            _val270 = iprot.readString()
+            self.properties[_key269] = _val270
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -6482,16 +7064,16 @@ class Schema:
     if self.fieldSchemas is not None:
       oprot.writeFieldBegin('fieldSchemas', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.fieldSchemas))
-      for iter264 in self.fieldSchemas:
-        iter264.write(oprot)
+      for iter271 in self.fieldSchemas:
+        iter271.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.properties is not None:
       oprot.writeFieldBegin('properties', TType.MAP, 2)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.properties))
-      for kiter265,viter266 in self.properties.items():
-        oprot.writeString(kiter265)
-        oprot.writeString(viter266)
+      for kiter272,viter273 in self.properties.items():
+        oprot.writeString(kiter272)
+        oprot.writeString(viter273)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -6544,11 +7126,11 @@ class EnvironmentContext:
       if fid == 1:
         if ftype == TType.MAP:
           self.properties = {}
-          (_ktype268, _vtype269, _size267 ) = iprot.readMapBegin()
-          for _i271 in xrange(_size267):
-            _key272 = iprot.readString()
-            _val273 = iprot.readString()
-            self.properties[_key272] = _val273
+          (_ktype275, _vtype276, _size274 ) = iprot.readMapBegin()
+          for _i278 in xrange(_size274):
+            _key279 = iprot.readString()
+            _val280 = iprot.readString()
+            self.properties[_key279] = _val280
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -6565,9 +7147,9 @@ class EnvironmentContext:
     if self.properties is not None:
       oprot.writeFieldBegin('properties', TType.MAP, 1)
       oprot.writeMapBegin(

<TRUNCATED>

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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
index 8e357f6..987a4f3 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
@@ -544,6 +544,11 @@ void SQLPrimaryKey::__set_rely_cstr(const bool val) {
   this->rely_cstr = val;
 }
 
+void SQLPrimaryKey::__set_catName(const std::string& val) {
+  this->catName = val;
+__isset.catName = true;
+}
+
 uint32_t SQLPrimaryKey::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
@@ -629,6 +634,14 @@ uint32_t SQLPrimaryKey::read(::apache::thrift::protocol::TProtocol* iprot) {
           xfer += iprot->skip(ftype);
         }
         break;
+      case 9:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->catName);
+          this->__isset.catName = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -678,6 +691,11 @@ uint32_t SQLPrimaryKey::write(::apache::thrift::protocol::TProtocol* oprot) cons
   xfer += oprot->writeBool(this->rely_cstr);
   xfer += oprot->writeFieldEnd();
 
+  if (this->__isset.catName) {
+    xfer += oprot->writeFieldBegin("catName", ::apache::thrift::protocol::T_STRING, 9);
+    xfer += oprot->writeString(this->catName);
+    xfer += oprot->writeFieldEnd();
+  }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
@@ -693,6 +711,7 @@ void swap(SQLPrimaryKey &a, SQLPrimaryKey &b) {
   swap(a.enable_cstr, b.enable_cstr);
   swap(a.validate_cstr, b.validate_cstr);
   swap(a.rely_cstr, b.rely_cstr);
+  swap(a.catName, b.catName);
   swap(a.__isset, b.__isset);
 }
 
@@ -705,6 +724,7 @@ SQLPrimaryKey::SQLPrimaryKey(const SQLPrimaryKey& other4) {
   enable_cstr = other4.enable_cstr;
   validate_cstr = other4.validate_cstr;
   rely_cstr = other4.rely_cstr;
+  catName = other4.catName;
   __isset = other4.__isset;
 }
 SQLPrimaryKey& SQLPrimaryKey::operator=(const SQLPrimaryKey& other5) {
@@ -716,6 +736,7 @@ SQLPrimaryKey& SQLPrimaryKey::operator=(const SQLPrimaryKey& other5) {
   enable_cstr = other5.enable_cstr;
   validate_cstr = other5.validate_cstr;
   rely_cstr = other5.rely_cstr;
+  catName = other5.catName;
   __isset = other5.__isset;
   return *this;
 }
@@ -730,6 +751,7 @@ void SQLPrimaryKey::printTo(std::ostream& out) const {
   out << ", " << "enable_cstr=" << to_string(enable_cstr);
   out << ", " << "validate_cstr=" << to_string(validate_cstr);
   out << ", " << "rely_cstr=" << to_string(rely_cstr);
+  out << ", " << "catName="; (__isset.catName ? (out << to_string(catName)) : (out << "<null>"));
   out << ")";
 }
 
@@ -794,6 +816,11 @@ void SQLForeignKey::__set_rely_cstr(const bool val) {
   this->rely_cstr = val;
 }
 
+void SQLForeignKey::__set_catName(const std::string& val) {
+  this->catName = val;
+__isset.catName = true;
+}
+
 uint32_t SQLForeignKey::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
@@ -927,6 +954,14 @@ uint32_t SQLForeignKey::read(::apache::thrift::protocol::TProtocol* iprot) {
           xfer += iprot->skip(ftype);
         }
         break;
+      case 15:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->catName);
+          this->__isset.catName = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -1000,6 +1035,11 @@ uint32_t SQLForeignKey::write(::apache::thrift::protocol::TProtocol* oprot) cons
   xfer += oprot->writeBool(this->rely_cstr);
   xfer += oprot->writeFieldEnd();
 
+  if (this->__isset.catName) {
+    xfer += oprot->writeFieldBegin("catName", ::apache::thrift::protocol::T_STRING, 15);
+    xfer += oprot->writeString(this->catName);
+    xfer += oprot->writeFieldEnd();
+  }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
@@ -1021,6 +1061,7 @@ void swap(SQLForeignKey &a, SQLForeignKey &b) {
   swap(a.enable_cstr, b.enable_cstr);
   swap(a.validate_cstr, b.validate_cstr);
   swap(a.rely_cstr, b.rely_cstr);
+  swap(a.catName, b.catName);
   swap(a.__isset, b.__isset);
 }
 
@@ -1039,6 +1080,7 @@ SQLForeignKey::SQLForeignKey(const SQLForeignKey& other6) {
   enable_cstr = other6.enable_cstr;
   validate_cstr = other6.validate_cstr;
   rely_cstr = other6.rely_cstr;
+  catName = other6.catName;
   __isset = other6.__isset;
 }
 SQLForeignKey& SQLForeignKey::operator=(const SQLForeignKey& other7) {
@@ -1056,6 +1098,7 @@ SQLForeignKey& SQLForeignKey::operator=(const SQLForeignKey& other7) {
   enable_cstr = other7.enable_cstr;
   validate_cstr = other7.validate_cstr;
   rely_cstr = other7.rely_cstr;
+  catName = other7.catName;
   __isset = other7.__isset;
   return *this;
 }
@@ -1076,6 +1119,7 @@ void SQLForeignKey::printTo(std::ostream& out) const {
   out << ", " << "enable_cstr=" << to_string(enable_cstr);
   out << ", " << "validate_cstr=" << to_string(validate_cstr);
   out << ", " << "rely_cstr=" << to_string(rely_cstr);
+  out << ", " << "catName="; (__isset.catName ? (out << to_string(catName)) : (out << "<null>"));
   out << ")";
 }
 
@@ -1084,6 +1128,10 @@ SQLUniqueConstraint::~SQLUniqueConstraint() throw() {
 }
 
 
+void SQLUniqueConstraint::__set_catName(const std::string& val) {
+  this->catName = val;
+}
+
 void SQLUniqueConstraint::__set_table_db(const std::string& val) {
   this->table_db = val;
 }
@@ -1139,13 +1187,21 @@ uint32_t SQLUniqueConstraint::read(::apache::thrift::protocol::TProtocol* iprot)
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->catName);
+          this->__isset.catName = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
           xfer += iprot->readString(this->table_db);
           this->__isset.table_db = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
-      case 2:
+      case 3:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
           xfer += iprot->readString(this->table_name);
           this->__isset.table_name = true;
@@ -1153,7 +1209,7 @@ uint32_t SQLUniqueConstraint::read(::apache::thrift::protocol::TProtocol* iprot)
           xfer += iprot->skip(ftype);
         }
         break;
-      case 3:
+      case 4:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
           xfer += iprot->readString(this->column_name);
           this->__isset.column_name = true;
@@ -1161,7 +1217,7 @@ uint32_t SQLUniqueConstraint::read(::apache::thrift::protocol::TProtocol* iprot)
           xfer += iprot->skip(ftype);
         }
         break;
-      case 4:
+      case 5:
         if (ftype == ::apache::thrift::protocol::T_I32) {
           xfer += iprot->readI32(this->key_seq);
           this->__isset.key_seq = true;
@@ -1169,7 +1225,7 @@ uint32_t SQLUniqueConstraint::read(::apache::thrift::protocol::TProtocol* iprot)
           xfer += iprot->skip(ftype);
         }
         break;
-      case 5:
+      case 6:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
           xfer += iprot->readString(this->uk_name);
           this->__isset.uk_name = true;
@@ -1177,7 +1233,7 @@ uint32_t SQLUniqueConstraint::read(::apache::thrift::protocol::TProtocol* iprot)
           xfer += iprot->skip(ftype);
         }
         break;
-      case 6:
+      case 7:
         if (ftype == ::apache::thrift::protocol::T_BOOL) {
           xfer += iprot->readBool(this->enable_cstr);
           this->__isset.enable_cstr = true;
@@ -1185,7 +1241,7 @@ uint32_t SQLUniqueConstraint::read(::apache::thrift::protocol::TProtocol* iprot)
           xfer += iprot->skip(ftype);
         }
         break;
-      case 7:
+      case 8:
         if (ftype == ::apache::thrift::protocol::T_BOOL) {
           xfer += iprot->readBool(this->validate_cstr);
           this->__isset.validate_cstr = true;
@@ -1193,7 +1249,7 @@ uint32_t SQLUniqueConstraint::read(::apache::thrift::protocol::TProtocol* iprot)
           xfer += iprot->skip(ftype);
         }
         break;
-      case 8:
+      case 9:
         if (ftype == ::apache::thrift::protocol::T_BOOL) {
           xfer += iprot->readBool(this->rely_cstr);
           this->__isset.rely_cstr = true;
@@ -1218,35 +1274,39 @@ uint32_t SQLUniqueConstraint::write(::apache::thrift::protocol::TProtocol* oprot
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
   xfer += oprot->writeStructBegin("SQLUniqueConstraint");
 
-  xfer += oprot->writeFieldBegin("table_db", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeFieldBegin("catName", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->catName);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("table_db", ::apache::thrift::protocol::T_STRING, 2);
   xfer += oprot->writeString(this->table_db);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 3);
   xfer += oprot->writeString(this->table_name);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("column_name", ::apache::thrift::protocol::T_STRING, 3);
+  xfer += oprot->writeFieldBegin("column_name", ::apache::thrift::protocol::T_STRING, 4);
   xfer += oprot->writeString(this->column_name);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("key_seq", ::apache::thrift::protocol::T_I32, 4);
+  xfer += oprot->writeFieldBegin("key_seq", ::apache::thrift::protocol::T_I32, 5);
   xfer += oprot->writeI32(this->key_seq);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("uk_name", ::apache::thrift::protocol::T_STRING, 5);
+  xfer += oprot->writeFieldBegin("uk_name", ::apache::thrift::protocol::T_STRING, 6);
   xfer += oprot->writeString(this->uk_name);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("enable_cstr", ::apache::thrift::protocol::T_BOOL, 6);
+  xfer += oprot->writeFieldBegin("enable_cstr", ::apache::thrift::protocol::T_BOOL, 7);
   xfer += oprot->writeBool(this->enable_cstr);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("validate_cstr", ::apache::thrift::protocol::T_BOOL, 7);
+  xfer += oprot->writeFieldBegin("validate_cstr", ::apache::thrift::protocol::T_BOOL, 8);
   xfer += oprot->writeBool(this->validate_cstr);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("rely_cstr", ::apache::thrift::protocol::T_BOOL, 8);
+  xfer += oprot->writeFieldBegin("rely_cstr", ::apache::thrift::protocol::T_BOOL, 9);
   xfer += oprot->writeBool(this->rely_cstr);
   xfer += oprot->writeFieldEnd();
 
@@ -1257,6 +1317,7 @@ uint32_t SQLUniqueConstraint::write(::apache::thrift::protocol::TProtocol* oprot
 
 void swap(SQLUniqueConstraint &a, SQLUniqueConstraint &b) {
   using ::std::swap;
+  swap(a.catName, b.catName);
   swap(a.table_db, b.table_db);
   swap(a.table_name, b.table_name);
   swap(a.column_name, b.column_name);
@@ -1269,6 +1330,7 @@ void swap(SQLUniqueConstraint &a, SQLUniqueConstraint &b) {
 }
 
 SQLUniqueConstraint::SQLUniqueConstraint(const SQLUniqueConstraint& other8) {
+  catName = other8.catName;
   table_db = other8.table_db;
   table_name = other8.table_name;
   column_name = other8.column_name;
@@ -1280,6 +1342,7 @@ SQLUniqueConstraint::SQLUniqueConstraint(const SQLUniqueConstraint& other8) {
   __isset = other8.__isset;
 }
 SQLUniqueConstraint& SQLUniqueConstraint::operator=(const SQLUniqueConstraint& other9) {
+  catName = other9.catName;
   table_db = other9.table_db;
   table_name = other9.table_name;
   column_name = other9.column_name;
@@ -1294,7 +1357,8 @@ SQLUniqueConstraint& SQLUniqueConstraint::operator=(const SQLUniqueConstraint& o
 void SQLUniqueConstraint::printTo(std::ostream& out) const {
   using ::apache::thrift::to_string;
   out << "SQLUniqueConstraint(";
-  out << "table_db=" << to_string(table_db);
+  out << "catName=" << to_string(catName);
+  out << ", " << "table_db=" << to_string(table_db);
   out << ", " << "table_name=" << to_string(table_name);
   out << ", " << "column_name=" << to_string(column_name);
   out << ", " << "key_seq=" << to_string(key_seq);
@@ -1310,6 +1374,10 @@ SQLNotNullConstraint::~SQLNotNullConstraint() throw() {
 }
 
 
+void SQLNotNullConstraint::__set_catName(const std::string& val) {
+  this->catName = val;
+}
+
 void SQLNotNullConstraint::__set_table_db(const std::string& val) {
   this->table_db = val;
 }
@@ -1361,13 +1429,21 @@ uint32_t SQLNotNullConstraint::read(::apache::thrift::protocol::TProtocol* iprot
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->catName);
+          this->__isset.catName = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
           xfer += iprot->readString(this->table_db);
           this->__isset.table_db = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
-      case 2:
+      case 3:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
           xfer += iprot->readString(this->table_name);
           this->__isset.table_name = true;
@@ -1375,7 +1451,7 @@ uint32_t SQLNotNullConstraint::read(::apache::thrift::protocol::TProtocol* iprot
           xfer += iprot->skip(ftype);
         }
         break;
-      case 3:
+      case 4:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
           xfer += iprot->readString(this->column_name);
           this->__isset.column_name = true;
@@ -1383,7 +1459,7 @@ uint32_t SQLNotNullConstraint::read(::apache::thrift::protocol::TProtocol* iprot
           xfer += iprot->skip(ftype);
         }
         break;
-      case 4:
+      case 5:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
           xfer += iprot->readString(this->nn_name);
           this->__isset.nn_name = true;
@@ -1391,7 +1467,7 @@ uint32_t SQLNotNullConstraint::read(::apache::thrift::protocol::TProtocol* iprot
           xfer += iprot->skip(ftype);
         }
         break;
-      case 5:
+      case 6:
         if (ftype == ::apache::thrift::protocol::T_BOOL) {
           xfer += iprot->readBool(this->enable_cstr);
           this->__isset.enable_cstr = true;
@@ -1399,7 +1475,7 @@ uint32_t SQLNotNullConstraint::read(::apache::thrift::protocol::TProtocol* iprot
           xfer += iprot->skip(ftype);
         }
         break;
-      case 6:
+      case 7:
         if (ftype == ::apache::thrift::protocol::T_BOOL) {
           xfer += iprot->readBool(this->validate_cstr);
           this->__isset.validate_cstr = true;
@@ -1407,7 +1483,7 @@ uint32_t SQLNotNullConstraint::read(::apache::thrift::protocol::TProtocol* iprot
           xfer += iprot->skip(ftype);
         }
         break;
-      case 7:
+      case 8:
         if (ftype == ::apache::thrift::protocol::T_BOOL) {
           xfer += iprot->readBool(this->rely_cstr);
           this->__isset.rely_cstr = true;
@@ -1432,31 +1508,35 @@ uint32_t SQLNotNullConstraint::write(::apache::thrift::protocol::TProtocol* opro
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
   xfer += oprot->writeStructBegin("SQLNotNullConstraint");
 
-  xfer += oprot->writeFieldBegin("table_db", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeFieldBegin("catName", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->catName);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("table_db", ::apache::thrift::protocol::T_STRING, 2);
   xfer += oprot->writeString(this->table_db);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 3);
   xfer += oprot->writeString(this->table_name);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("column_name", ::apache::thrift::protocol::T_STRING, 3);
+  xfer += oprot->writeFieldBegin("column_name", ::apache::thrift::protocol::T_STRING, 4);
   xfer += oprot->writeString(this->column_name);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("nn_name", ::apache::thrift::protocol::T_STRING, 4);
+  xfer += oprot->writeFieldBegin("nn_name", ::apache::thrift::protocol::T_STRING, 5);
   xfer += oprot->writeString(this->nn_name);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("enable_cstr", ::apache::thrift::protocol::T_BOOL, 5);
+  xfer += oprot->writeFieldBegin("enable_cstr", ::apache::thrift::protocol::T_BOOL, 6);
   xfer += oprot->writeBool(this->enable_cstr);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("validate_cstr", ::apache::thrift::protocol::T_BOOL, 6);
+  xfer += oprot->writeFieldBegin("validate_cstr", ::apache::thrift::protocol::T_BOOL, 7);
   xfer += oprot->writeBool(this->validate_cstr);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("rely_cstr", ::apache::thrift::protocol::T_BOOL, 7);
+  xfer += oprot->writeFieldBegin("rely_cstr", ::apache::thrift::protocol::T_BOOL, 8);
   xfer += oprot->writeBool(this->rely_cstr);
   xfer += oprot->writeFieldEnd();
 
@@ -1467,6 +1547,7 @@ uint32_t SQLNotNullConstraint::write(::apache::thrift::protocol::TProtocol* opro
 
 void swap(SQLNotNullConstraint &a, SQLNotNullConstraint &b) {
   using ::std::swap;
+  swap(a.catName, b.catName);
   swap(a.table_db, b.table_db);
   swap(a.table_name, b.table_name);
   swap(a.column_name, b.column_name);
@@ -1478,6 +1559,7 @@ void swap(SQLNotNullConstraint &a, SQLNotNullConstraint &b) {
 }
 
 SQLNotNullConstraint::SQLNotNullConstraint(const SQLNotNullConstraint& other10) {
+  catName = other10.catName;
   table_db = other10.table_db;
   table_name = other10.table_name;
   column_name = other10.column_name;
@@ -1488,6 +1570,7 @@ SQLNotNullConstraint::SQLNotNullConstraint(const SQLNotNullConstraint& other10)
   __isset = other10.__isset;
 }
 SQLNotNullConstraint& SQLNotNullConstraint::operator=(const SQLNotNullConstraint& other11) {
+  catName = other11.catName;
   table_db = other11.table_db;
   table_name = other11.table_name;
   column_name = other11.column_name;
@@ -1501,7 +1584,8 @@ SQLNotNullConstraint& SQLNotNullConstraint::operator=(const SQLNotNullConstraint
 void SQLNotNullConstraint::printTo(std::ostream& out) const {
   using ::apache::thrift::to_string;
   out << "SQLNotNullConstraint(";
-  out << "table_db=" << to_string(table_db);
+  out << "catName=" << to_string(catName);
+  out << ", " << "table_db=" << to_string(table_db);
   out << ", " << "table_name=" << to_string(table_name);
   out << ", " << "column_name=" << to_string(column_name);
   out << ", " << "nn_name=" << to_string(nn_name);
@@ -1516,6 +1600,10 @@ SQLDefaultConstraint::~SQLDefaultConstraint() throw() {
 }
 
 
+void SQLDefaultConstraint::__set_catName(const std::string& val) {
+  this->catName = val;
+}
+
 void SQLDefaultConstraint::__set_table_db(const std::string& val) {
   this->table_db = val;
 }
@@ -1571,13 +1659,21 @@ uint32_t SQLDefaultConstraint::read(::apache::thrift::protocol::TProtocol* iprot
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->catName);
+          this->__isset.catName = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
           xfer += iprot->readString(this->table_db);
           this->__isset.table_db = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
-      case 2:
+      case 3:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
           xfer += iprot->readString(this->table_name);
           this->__isset.table_name = true;
@@ -1585,7 +1681,7 @@ uint32_t SQLDefaultConstraint::read(::apache::thrift::protocol::TProtocol* iprot
           xfer += iprot->skip(ftype);
         }
         break;
-      case 3:
+      case 4:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
           xfer += iprot->readString(this->column_name);
           this->__isset.column_name = true;
@@ -1593,7 +1689,7 @@ uint32_t SQLDefaultConstraint::read(::apache::thrift::protocol::TProtocol* iprot
           xfer += iprot->skip(ftype);
         }
         break;
-      case 4:
+      case 5:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
           xfer += iprot->readString(this->default_value);
           this->__isset.default_value = true;
@@ -1601,7 +1697,7 @@ uint32_t SQLDefaultConstraint::read(::apache::thrift::protocol::TProtocol* iprot
           xfer += iprot->skip(ftype);
         }
         break;
-      case 5:
+      case 6:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
           xfer += iprot->readString(this->dc_name);
           this->__isset.dc_name = true;
@@ -1609,7 +1705,7 @@ uint32_t SQLDefaultConstraint::read(::apache::thrift::protocol::TProtocol* iprot
           xfer += iprot->skip(ftype);
         }
         break;
-      case 6:
+      case 7:
         if (ftype == ::apache::thrift::protocol::T_BOOL) {
           xfer += iprot->readBool(this->enable_cstr);
           this->__isset.enable_cstr = true;
@@ -1617,7 +1713,7 @@ uint32_t SQLDefaultConstraint::read(::apache::thrift::protocol::TProtocol* iprot
           xfer += iprot->skip(ftype);
         }
         break;
-      case 7:
+      case 8:
         if (ftype == ::apache::thrift::protocol::T_BOOL) {
           xfer += iprot->readBool(this->validate_cstr);
           this->__isset.validate_cstr = true;
@@ -1625,7 +1721,7 @@ uint32_t SQLDefaultConstraint::read(::apache::thrift::protocol::TProtocol* iprot
           xfer += iprot->skip(ftype);
         }
         break;
-      case 8:
+      case 9:
         if (ftype == ::apache::thrift::protocol::T_BOOL) {
           xfer += iprot->readBool(this->rely_cstr);
           this->__isset.rely_cstr = true;
@@ -1650,35 +1746,39 @@ uint32_t SQLDefaultConstraint::write(::apache::thrift::protocol::TProtocol* opro
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
   xfer += oprot->writeStructBegin("SQLDefaultConstraint");
 
-  xfer += oprot->writeFieldBegin("table_db", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeFieldBegin("catName", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->catName);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("table_db", ::apache::thrift::protocol::T_STRING, 2);
   xfer += oprot->writeString(this->table_db);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 3);
   xfer += oprot->writeString(this->table_name);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("column_name", ::apache::thrift::protocol::T_STRING, 3);
+  xfer += oprot->writeFieldBegin("column_name", ::apache::thrift::protocol::T_STRING, 4);
   xfer += oprot->writeString(this->column_name);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("default_value", ::apache::thrift::protocol::T_STRING, 4);
+  xfer += oprot->writeFieldBegin("default_value", ::apache::thrift::protocol::T_STRING, 5);
   xfer += oprot->writeString(this->default_value);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("dc_name", ::apache::thrift::protocol::T_STRING, 5);
+  xfer += oprot->writeFieldBegin("dc_name", ::apache::thrift::protocol::T_STRING, 6);
   xfer += oprot->writeString(this->dc_name);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("enable_cstr", ::apache::thrift::protocol::T_BOOL, 6);
+  xfer += oprot->writeFieldBegin("enable_cstr", ::apache::thrift::protocol::T_BOOL, 7);
   xfer += oprot->writeBool(this->enable_cstr);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("validate_cstr", ::apache::thrift::protocol::T_BOOL, 7);
+  xfer += oprot->writeFieldBegin("validate_cstr", ::apache::thrift::protocol::T_BOOL, 8);
   xfer += oprot->writeBool(this->validate_cstr);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("rely_cstr", ::apache::thrift::protocol::T_BOOL, 8);
+  xfer += oprot->writeFieldBegin("rely_cstr", ::apache::thrift::protocol::T_BOOL, 9);
   xfer += oprot->writeBool(this->rely_cstr);
   xfer += oprot->writeFieldEnd();
 
@@ -1689,6 +1789,7 @@ uint32_t SQLDefaultConstraint::write(::apache::thrift::protocol::TProtocol* opro
 
 void swap(SQLDefaultConstraint &a, SQLDefaultConstraint &b) {
   using ::std::swap;
+  swap(a.catName, b.catName);
   swap(a.table_db, b.table_db);
   swap(a.table_name, b.table_name);
   swap(a.column_name, b.column_name);
@@ -1701,6 +1802,7 @@ void swap(SQLDefaultConstraint &a, SQLDefaultConstraint &b) {
 }
 
 SQLDefaultConstraint::SQLDefaultConstraint(const SQLDefaultConstraint& other12) {
+  catName = other12.catName;
   table_db = other12.table_db;
   table_name = other12.table_name;
   column_name = other12.column_name;
@@ -1712,6 +1814,7 @@ SQLDefaultConstraint::SQLDefaultConstraint(const SQLDefaultConstraint& other12)
   __isset = other12.__isset;
 }
 SQLDefaultConstraint& SQLDefaultConstraint::operator=(const SQLDefaultConstraint& other13) {
+  catName = other13.catName;
   table_db = other13.table_db;
   table_name = other13.table_name;
   column_name = other13.column_name;
@@ -1726,7 +1829,8 @@ SQLDefaultConstraint& SQLDefaultConstraint::operator=(const SQLDefaultConstraint
 void SQLDefaultConstraint::printTo(std::ostream& out) const {
   using ::apache::thrift::to_string;
   out << "SQLDefaultConstraint(";
-  out << "table_db=" << to_string(table_db);
+  out << "catName=" << to_string(catName);
+  out << ", " << "table_db=" << to_string(table_db);
   out << ", " << "table_name=" << to_string(table_name);
   out << ", " << "column_name=" << to_string(column_name);
   out << ", " << "default_value=" << to_string(default_value);
@@ -1742,6 +1846,10 @@ SQLCheckConstraint::~SQLCheckConstraint() throw() {
 }
 
 
+void SQLCheckConstraint::__set_catName(const std::string& val) {
+  this->catName = val;
+}
+
 void SQLCheckConstraint::__set_table_db(const std::string& val) {
   this->table_db = val;
 }
@@ -1797,13 +1905,21 @@ uint32_t SQLCheckConstraint::read(::apache::thrift::protocol::TProtocol* iprot)
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->catName);
+          this->__isset.catName = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
           xfer += iprot->readString(this->table_db);
           this->__isset.table_db = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
-      case 2:
+      case 3:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
           xfer += iprot->readString(this->table_name);
           this->__isset.table_name = true;
@@ -1811,7 +1927,7 @@ uint32_t SQLCheckConstraint::read(::apache::thrift::protocol::TProtocol* iprot)
           xfer += iprot->skip(ftype);
         }
         break;
-      case 3:
+      case 4:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
           xfer += iprot->readString(this->column_name);
           this->__isset.column_name = true;
@@ -1819,7 +1935,7 @@ uint32_t SQLCheckConstraint::read(::apache::thrift::protocol::TProtocol* iprot)
           xfer += iprot->skip(ftype);
         }
         break;
-      case 4:
+      case 5:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
           xfer += iprot->readString(this->check_expression);
           this->__isset.check_expression = true;
@@ -1827,7 +1943,7 @@ uint32_t SQLCheckConstraint::read(::apache::thrift::protocol::TProtocol* iprot)
           xfer += iprot->skip(ftype);
         }
         break;
-      case 5:
+      case 6:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
           xfer += iprot->readString(this->dc_name);
           this->__isset.dc_name = true;
@@ -1835,7 +1951,7 @@ uint32_t SQLCheckConstraint::read(::apache::thrift::protocol::TProtocol* iprot)
           xfer += iprot->skip(ftype);
         }
         break;
-      case 6:
+      case 7:
         if (ftype == ::apache::thrift::protocol::T_BOOL) {
           xfer += iprot->readBool(this->enable_cstr);
           this->__isset.enable_cstr = true;
@@ -1843,7 +1959,7 @@ uint32_t SQLCheckConstraint::read(::apache::thrift::protocol::TProtocol* iprot)
           xfer += iprot->skip(ftype);
         }
         break;
-      case 7:
+      case 8:
         if (ftype == ::apache::thrift::protocol::T_BOOL) {
           xfer += iprot->readBool(this->validate_cstr);
           this->__isset.validate_cstr = true;
@@ -1851,7 +1967,7 @@ uint32_t SQLCheckConstraint::read(::apache::thrift::protocol::TProtocol* iprot)
           xfer += iprot->skip(ftype);
         }
         break;
-      case 8:
+      case 9:
         if (ftype == ::apache::thrift::protocol::T_BOOL) {
           xfer += iprot->readBool(this->rely_cstr);
           this->__isset.rely_cstr = true;
@@ -1876,35 +1992,39 @@ uint32_t SQLCheckConstraint::write(::apache::thrift::protocol::TProtocol* oprot)
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
   xfer += oprot->writeStructBegin("SQLCheckConstraint");
 
-  xfer += oprot->writeFieldBegin("table_db", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeFieldBegin("catName", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->catName);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("table_db", ::apache::thrift::protocol::T_STRING, 2);
   xfer += oprot->writeString(this->table_db);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 3);
   xfer += oprot->writeString(this->table_name);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("column_name", ::apache::thrift::protocol::T_STRING, 3);
+  xfer += oprot->writeFieldBegin("column_name", ::apache::thrift::protocol::T_STRING, 4);
   xfer += oprot->writeString(this->column_name);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("check_expression", ::apache::thrift::protocol::T_STRING, 4);
+  xfer += oprot->writeFieldBegin("check_expression", ::apache::thrift::protocol::T_STRING, 5);
   xfer += oprot->writeString(this->check_expression);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("dc_name", ::apache::thrift::protocol::T_STRING, 5);
+  xfer += oprot->writeFieldBegin("dc_name", ::apache::thrift::protocol::T_STRING, 6);
   xfer += oprot->writeString(this->dc_name);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("enable_cstr", ::apache::thrift::protocol::T_BOOL, 6);
+  xfer += oprot->writeFieldBegin("enable_cstr", ::apache::thrift::protocol::T_BOOL, 7);
   xfer += oprot->writeBool(this->enable_cstr);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("validate_cstr", ::apache::thrift::protocol::T_BOOL, 7);
+  xfer += oprot->writeFieldBegin("validate_cstr", ::apache::thrift::protocol::T_BOOL, 8);
   xfer += oprot->writeBool(this->validate_cstr);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("rely_cstr", ::apache::thrift::protocol::T_BOOL, 8);
+  xfer += oprot->writeFieldBegin("rely_cstr", ::apache::thrift::protocol::T_BOOL, 9);
   xfer += oprot->writeBool(this->rely_cstr);
   xfer += oprot->writeFieldEnd();
 
@@ -1915,6 +2035,7 @@ uint32_t SQLCheckConstraint::write(::apache::thrift::protocol::TProtocol* oprot)
 
 void swap(SQLCheckConstraint &a, SQLCheckConstraint &b) {
   using ::std::swap;
+  swap(a.catName, b.catName);
   swap(a.table_db, b.table_db);
   swap(a.table_name, b.table_name);
   swap(a.column_name, b.column_name);
@@ -1927,6 +2048,7 @@ void swap(SQLCheckConstraint &a, SQLCheckConstraint &b) {
 }
 
 SQLCheckConstraint::SQLCheckConstraint(const SQLCheckConstraint& other14) {
+  catName = other14.catName;
   table_db = other14.table_db;
   table_name = other14.table_name;
   column_name = other14.column_name;
@@ -1938,6 +2060,7 @@ SQLCheckConstraint::SQLCheckConstraint(const SQLCheckConstraint& other14) {
   __isset = other14.__isset;
 }
 SQLCheckConstraint& SQLCheckConstraint::operator=(const SQLCheckConstraint& other15) {
+  catName = other15.catName;
   table_db = other15.table_db;
   table_name = other15.table_name;
   column_name = other15.column_name;
@@ -1952,7 +2075,8 @@ SQLCheckConstraint& SQLCheckConstraint::operator=(const SQLCheckConstraint& othe
 void SQLCheckConstraint::printTo(std::ostream& out) const {
   using ::apache::thrift::to_string;
   out << "SQLCheckConstraint(";
-  out << "table_db=" << to_string(table_db);
+  out << "catName=" << to_string(catName);
+  out << ", " << "table_db=" << to_string(table_db);
   out << ", " << "table_name=" << to_string(table_name);
   out << ", " << "column_name=" << to_string(column_name);
   out << ", " << "check_expression=" << to_string(check_expression);
@@ -2160,6 +2284,11 @@ void HiveObjectRef::__set_columnName(const std::string& val) {
   this->columnName = val;
 }
 
+void HiveObjectRef::__set_catName(const std::string& val) {
+  this->catName = val;
+__isset.catName = true;
+}
+
 uint32_t HiveObjectRef::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
@@ -2235,6 +2364,14 @@ uint32_t HiveObjectRef::read(::apache::thrift::protocol::TProtocol* iprot) {
           xfer += iprot->skip(ftype);
         }
         break;
+      case 6:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->catName);
+          this->__isset.catName = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -2280,6 +2417,11 @@ uint32_t HiveObjectRef::write(::apache::thrift::protocol::TProtocol* oprot) cons
   xfer += oprot->writeString(this->columnName);
   xfer += oprot->writeFieldEnd();
 
+  if (this->__isset.catName) {
+    xfer += oprot->writeFieldBegin("catName", ::apache::thrift::protocol::T_STRING, 6);
+    xfer += oprot->writeString(this->catName);
+    xfer += oprot->writeFieldEnd();
+  }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
@@ -2292,6 +2434,7 @@ void swap(HiveObjectRef &a, HiveObjectRef &b) {
   swap(a.objectName, b.objectName);
   swap(a.partValues, b.partValues);
   swap(a.columnName, b.columnName);
+  swap(a.catName, b.catName);
   swap(a.__isset, b.__isset);
 }
 
@@ -2301,6 +2444,7 @@ HiveObjectRef::HiveObjectRef(const HiveObjectRef& other31) {
   objectName = other31.objectName;
   partValues = other31.partValues;
   columnName = other31.columnName;
+  catName = other31.catName;
   __isset = other31.__isset;
 }
 HiveObjectRef& HiveObjectRef::operator=(const HiveObjectRef& other32) {
@@ -2309,6 +2453,7 @@ HiveObjectRef& HiveObjectRef::operator=(const HiveObjectRef& other32) {
   objectName = other32.objectName;
   partValues = other32.partValues;
   columnName = other32.columnName;
+  catName = other32.catName;
   __isset = other32.__isset;
   return *this;
 }
@@ -2320,6 +2465,7 @@ void HiveObjectRef::printTo(std::ostream& out) const {
   out << ", " << "objectName=" << to_string(objectName);
   out << ", " << "partValues=" << to_string(partValues);
   out << ", " << "columnName=" << to_string(columnName);
+  out << ", " << "catName="; (__isset.catName ? (out << to_string(catName)) : (out << "<null>"));
   out << ")";
 }
 
@@ -4194,7 +4340,586 @@ void GrantRevokeRoleResponse::__set_success(const bool val) {
 __isset.success = true;
 }
 
-uint32_t GrantRevokeRoleResponse::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t GrantRevokeRoleResponse::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_BOOL) {
+          xfer += iprot->readBool(this->success);
+          this->__isset.success = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+uint32_t GrantRevokeRoleResponse::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("GrantRevokeRoleResponse");
+
+  if (this->__isset.success) {
+    xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 1);
+    xfer += oprot->writeBool(this->success);
+    xfer += oprot->writeFieldEnd();
+  }
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+void swap(GrantRevokeRoleResponse &a, GrantRevokeRoleResponse &b) {
+  using ::std::swap;
+  swap(a.success, b.success);
+  swap(a.__isset, b.__isset);
+}
+
+GrantRevokeRoleResponse::GrantRevokeRoleResponse(const GrantRevokeRoleResponse& other128) {
+  success = other128.success;
+  __isset = other128.__isset;
+}
+GrantRevokeRoleResponse& GrantRevokeRoleResponse::operator=(const GrantRevokeRoleResponse& other129) {
+  success = other129.success;
+  __isset = other129.__isset;
+  return *this;
+}
+void GrantRevokeRoleResponse::printTo(std::ostream& out) const {
+  using ::apache::thrift::to_string;
+  out << "GrantRevokeRoleResponse(";
+  out << "success="; (__isset.success ? (out << to_string(success)) : (out << "<null>"));
+  out << ")";
+}
+
+
+Catalog::~Catalog() throw() {
+}
+
+
+void Catalog::__set_name(const std::string& val) {
+  this->name = val;
+}
+
+void Catalog::__set_description(const std::string& val) {
+  this->description = val;
+__isset.description = true;
+}
+
+void Catalog::__set_locationUri(const std::string& val) {
+  this->locationUri = val;
+}
+
+uint32_t Catalog::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->name);
+          this->__isset.name = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->description);
+          this->__isset.description = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->locationUri);
+          this->__isset.locationUri = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+uint32_t Catalog::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("Catalog");
+
+  xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->name);
+  xfer += oprot->writeFieldEnd();
+
+  if (this->__isset.description) {
+    xfer += oprot->writeFieldBegin("description", ::apache::thrift::protocol::T_STRING, 2);
+    xfer += oprot->writeString(this->description);
+    xfer += oprot->writeFieldEnd();
+  }
+  xfer += oprot->writeFieldBegin("locationUri", ::apache::thrift::protocol::T_STRING, 3);
+  xfer += oprot->writeString(this->locationUri);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+void swap(Catalog &a, Catalog &b) {
+  using ::std::swap;
+  swap(a.name, b.name);
+  swap(a.description, b.description);
+  swap(a.locationUri, b.locationUri);
+  swap(a.__isset, b.__isset);
+}
+
+Catalog::Catalog(const Catalog& other130) {
+  name = other130.name;
+  description = other130.description;
+  locationUri = other130.locationUri;
+  __isset = other130.__isset;
+}
+Catalog& Catalog::operator=(const Catalog& other131) {
+  name = other131.name;
+  description = other131.description;
+  locationUri = other131.locationUri;
+  __isset = other131.__isset;
+  return *this;
+}
+void Catalog::printTo(std::ostream& out) const {
+  using ::apache::thrift::to_string;
+  out << "Catalog(";
+  out << "name=" << to_string(name);
+  out << ", " << "description="; (__isset.description ? (out << to_string(description)) : (out << "<null>"));
+  out << ", " << "locationUri=" << to_string(locationUri);
+  out << ")";
+}
+
+
+CreateCatalogRequest::~CreateCatalogRequest() throw() {
+}
+
+
+void CreateCatalogRequest::__set_catalog(const Catalog& val) {
+  this->catalog = val;
+}
+
+uint32_t CreateCatalogRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->catalog.read(iprot);
+          this->__isset.catalog = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+uint32_t CreateCatalogRequest::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("CreateCatalogRequest");
+
+  xfer += oprot->writeFieldBegin("catalog", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += this->catalog.write(oprot);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+void swap(CreateCatalogRequest &a, CreateCatalogRequest &b) {
+  using ::std::swap;
+  swap(a.catalog, b.catalog);
+  swap(a.__isset, b.__isset);
+}
+
+CreateCatalogRequest::CreateCatalogRequest(const CreateCatalogRequest& other132) {
+  catalog = other132.catalog;
+  __isset = other132.__isset;
+}
+CreateCatalogRequest& CreateCatalogRequest::operator=(const CreateCatalogRequest& other133) {
+  catalog = other133.catalog;
+  __isset = other133.__isset;
+  return *this;
+}
+void CreateCatalogRequest::printTo(std::ostream& out) const {
+  using ::apache::thrift::to_string;
+  out << "CreateCatalogRequest(";
+  out << "catalog=" << to_string(catalog);
+  out << ")";
+}
+
+
+GetCatalogRequest::~GetCatalogRequest() throw() {
+}
+
+
+void GetCatalogRequest::__set_name(const std::string& val) {
+  this->name = val;
+}
+
+uint32_t GetCatalogRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->name);
+          this->__isset.name = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+uint32_t GetCatalogRequest::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("GetCatalogRequest");
+
+  xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->name);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+void swap(GetCatalogRequest &a, GetCatalogRequest &b) {
+  using ::std::swap;
+  swap(a.name, b.name);
+  swap(a.__isset, b.__isset);
+}
+
+GetCatalogRequest::GetCatalogRequest(const GetCatalogRequest& other134) {
+  name = other134.name;
+  __isset = other134.__isset;
+}
+GetCatalogRequest& GetCatalogRequest::operator=(const GetCatalogRequest& other135) {
+  name = other135.name;
+  __isset = other135.__isset;
+  return *this;
+}
+void GetCatalogRequest::printTo(std::ostream& out) const {
+  using ::apache::thrift::to_string;
+  out << "GetCatalogRequest(";
+  out << "name=" << to_string(name);
+  out << ")";
+}
+
+
+GetCatalogResponse::~GetCatalogResponse() throw() {
+}
+
+
+void GetCatalogResponse::__set_catalog(const Catalog& val) {
+  this->catalog = val;
+}
+
+uint32_t GetCatalogResponse::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->catalog.read(iprot);
+          this->__isset.catalog = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+uint32_t GetCatalogResponse::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("GetCatalogResponse");
+
+  xfer += oprot->writeFieldBegin("catalog", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += this->catalog.write(oprot);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+void swap(GetCatalogResponse &a, GetCatalogResponse &b) {
+  using ::std::swap;
+  swap(a.catalog, b.catalog);
+  swap(a.__isset, b.__isset);
+}
+
+GetCatalogResponse::GetCatalogResponse(const GetCatalogResponse& other136) {
+  catalog = other136.catalog;
+  __isset = other136.__isset;
+}
+GetCatalogResponse& GetCatalogResponse::operator=(const GetCatalogResponse& other137) {
+  catalog = other137.catalog;
+  __isset = other137.__isset;
+  return *this;
+}
+void GetCatalogResponse::printTo(std::ostream& out) const {
+  using ::apache::thrift::to_string;
+  out << "GetCatalogResponse(";
+  out << "catalog=" << to_string(catalog);
+  out << ")";
+}
+
+
+GetCatalogsResponse::~GetCatalogsResponse() throw() {
+}
+
+
+void GetCatalogsResponse::__set_names(const std::vector<std::string> & val) {
+  this->names = val;
+}
+
+uint32_t GetCatalogsResponse::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->names.clear();
+            uint32_t _size138;
+            ::apache::thrift::protocol::TType _etype141;
+            xfer += iprot->readListBegin(_etype141, _size138);
+            this->names.resize(_size138);
+            uint32_t _i142;
+            for (_i142 = 0; _i142 < _size138; ++_i142)
+            {
+              xfer += iprot->readString(this->names[_i142]);
+            }
+            xfer += iprot->readListEnd();
+          }
+          this->__isset.names = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+uint32_t GetCatalogsResponse::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("GetCatalogsResponse");
+
+  xfer += oprot->writeFieldBegin("names", ::apache::thrift::protocol::T_LIST, 1);
+  {
+    xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->names.size()));
+    std::vector<std::string> ::const_iterator _iter143;
+    for (_iter143 = this->names.begin(); _iter143 != this->names.end(); ++_iter143)
+    {
+      xfer += oprot->writeString((*_iter143));
+    }
+    xfer += oprot->writeListEnd();
+  }
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+void swap(GetCatalogsResponse &a, GetCatalogsResponse &b) {
+  using ::std::swap;
+  swap(a.names, b.names);
+  swap(a.__isset, b.__isset);
+}
+
+GetCatalogsResponse::GetCatalogsResponse(const GetCatalogsResponse& other144) {
+  names = other144.names;
+  __isset = other144.__isset;
+}
+GetCatalogsResponse& GetCatalogsResponse::operator=(const GetCatalogsResponse& other145) {
+  names = other145.names;
+  __isset = other145.__isset;
+  return *this;
+}
+void GetCatalogsResponse::printTo(std::ostream& out) const {
+  using ::apache::thrift::to_string;
+  out << "GetCatalogsResponse(";
+  out << "names=" << to_string(names);
+  out << ")";
+}
+
+
+DropCatalogRequest::~DropCatalogRequest() throw() {
+}
+
+
+void DropCatalogRequest::__set_name(const std::string& val) {
+  this->name = val;
+}
+
+uint32_t DropCatalogRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -4216,9 +4941,9 @@ uint32_t GrantRevokeRoleResponse::read(::apache::thrift::protocol::TProtocol* ip
     switch (fid)
     {
       case 1:
-        if (ftype == ::apache::thrift::protocol::T_BOOL) {
-          xfer += iprot->readBool(this->success);
-          this->__isset.success = true;
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->name);
+          this->__isset.name = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -4235,40 +4960,39 @@ uint32_t GrantRevokeRoleResponse::read(::apache::thrift::protocol::TProtocol* ip
   return xfer;
 }
 
-uint32_t GrantRevokeRoleResponse::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t DropCatalogRequest::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("GrantRevokeRoleResponse");
+  xfer += oprot->writeStructBegin("DropCatalogRequest");
+
+  xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->name);
+  xfer += oprot->writeFieldEnd();
 
-  if (this->__isset.success) {
-    xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 1);
-    xfer += oprot->writeBool(this->success);
-    xfer += oprot->writeFieldEnd();
-  }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
 }
 
-void swap(GrantRevokeRoleResponse &a, GrantRevokeRoleResponse &b) {
+void swap(DropCatalogRequest &a, DropCatalogRequest &b) {
   using ::std::swap;
-  swap(a.success, b.success);
+  swap(a.name, b.name);
   swap(a.__isset, b.__isset);
 }
 
-GrantRevokeRoleResponse::GrantRevokeRoleResponse(const GrantRevokeRoleResponse& other128) {
-  success = other128.success;
-  __isset = other128.__isset;
+DropCatalogRequest::DropCatalogRequest(const DropCatalogRequest& other146) {
+  name = other146.name;
+  __isset = other146.__isset;
 }
-GrantRevokeRoleResponse& GrantRevokeRoleResponse::operator=(const GrantRevokeRoleResponse& other129) {
-  success = other129.success;
-  __isset = other129.__isset;
+DropCatalogRequest& DropCatalogRequest::operator=(const DropCatalogRequest& other147) {
+  name = other147.name;
+  __isset = other147.__isset;
   return *this;
 }
-void GrantRevokeRoleResponse::printTo(std::ostream& out) const {
+void DropCatalogRequest::printTo(std::ostream& out) const {
   using ::apache::thrift::to_string;
-  out << "GrantRevokeRoleResponse(";
-  out << "success="; (__isset.success ? (out << to_string(success)) : (out << "<null>"));
+  out << "DropCatalogRequest(";
+  out << "name=" << to_string(name);
   out << ")";
 }
 
@@ -4308,6 +5032,11 @@ void Database::__set_ownerType(const PrincipalType::type val) {
 __isset.ownerType = true;
 }
 
+void Database::__set_catalogName(const std::string& val) {
+  this->catalogName = val;
+__isset.catalogName = true;
+}
+
 uint32_t Database::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
@@ -4357,17 +5086,17 @@ uint32_t Database::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->parameters.clear();
-            uint32_t _size130;
-            ::apache::thrift::protocol::TType _ktype131;
-            ::apache::thrift::protocol::TType _vtype132;
-            xfer += iprot->readMapBegin(_ktype131, _vtype132, _size130);
-            uint32_t _i134;
-            for (_i134 = 0; _i134 < _size130; ++_i134)
+            uint32_t _size148;
+            ::apache::thrift::protocol::TType _ktype149;
+            ::apache::thrift::protocol::TType _vtype150;
+            xfer += iprot->readMapBegin(_ktype149, _vtype150, _size148);
+            uint32_t _i152;
+            for (_i152 = 0; _i152 < _size148; ++_i152)
             {
-              std::string _key135;
-              xfer += iprot->readString(_key135);
-              std::string& _val136 = this->parameters[_key135];
-              xfer += iprot->readString(_val136);
+              std::string _key153;
+              xfer += iprot->readString(_key153);
+              std::string& _val154 = this->parameters[_key153];
+              xfer += iprot->readString(_val154);
             }
             xfer += iprot->readMapEnd();
           }
@@ -4394,14 +5123,22 @@ uint32_t Database::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 7:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast137;
-          xfer += iprot->readI32(ecast137);
-          this->ownerType = (PrincipalType::type)ecast137;
+          int32_t ecast155;
+          xfer += iprot->readI32(ecast155);
+          this->ownerType = (PrincipalType::type)ecast155;
           this->__isset.ownerType = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
+      case 8:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->catalogName);
+          this->__isset.catalogName = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -4434,11 +5171,11 @@ uint32_t Database::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("parameters", ::apache::thrift::protocol::T_MAP, 4);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->parameters.size()));
-    std::map<std::string, std::string> ::const_iterator _iter138;
-    for (_iter138 = this->parameters.begin(); _iter138 != this->parameters.end(); ++_iter138)
+    std::map<std::string, std::string> ::const_iterator _iter156;
+    for (_iter156 = this->parameters.begin(); _iter156 != this->parameters.end(); ++_iter156)
     {
-      xfer += oprot->writeString(_iter138->first);
-      xfer += oprot->writeString(_iter138->second);
+      xfer += oprot->writeString(_iter156->first);
+      xfer += oprot->writeString(_iter156->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -4459,6 +5196,11 @@ uint32_t Database::write(::apache::thrift::protocol::TProtocol* oprot) const {
     xfer += oprot->writeI32((int32_t)this->ownerType);
     xfer += oprot->writeFieldEnd();
   }
+  if (this->__isset.catalogName) {
+    xfer += oprot->writeFieldBegin("catalogName", ::apache::thrift::protocol::T_STRING, 8);
+    xfer += oprot->writeString(this->catalogName);
+    xfer += oprot->writeFieldEnd();
+  }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
@@ -4473,28 +5215,31 @@ void swap(Database &a, Database &b) {
   swap(a.privileges, b.privileges);
   swap(a.ownerName, b.ownerName);
   swap(a.ownerType, b.ownerType);
+  swap(a.catalogName, b.catalogName);
   swap(a.__isset, b.__isset);
 }
 
-Database::Database(const Database& other139) {
-  name = other139.name;
-  description = other139.description;
-  locationUri = other139.locationUri;
-  parameters = other139.parameters;
-  privileges = other139.privileges;
-  ownerName = other139.ownerName;
-  ownerType = other139.ownerType;
-  __isset = other139.__isset;
-}
-Database& Database::operator=(const Database& other140) {
-  name = other140.name;
-  description = other140.description;
-  locationUri = other140.locationUri;
-  parameters = other140.parameters;
-  privileges = other140.privileges;
-  ownerName = other140.ownerName;
-  ownerType = other140.ownerType;
-  __isset = other140.__isset;
+Database::Database(const Database& other157) {
+  name = other157.name;
+  description = other157.description;
+  locationUri = other157.locationUri;
+  parameters = other157.parameters;
+  privileges = other157.privileges;
+  ownerName = other157.ownerName;
+  ownerType = other157.ownerType;
+  catalogName = other157.catalogName;
+  __isset = other157.__isset;
+}
+Database& Database::operator=(const Database& other158) {
+  name = other158.name;
+  description = other158.description;
+  locationUri = other158.locationUri;
+  parameters = other158.parameters;
+  privileges = other158.privileges;
+  ownerName = other158.ownerName;
+  ownerType = other158.ownerType;
+  catalogName = other158.catalogName;
+  __isset = other158.__isset;
   return *this;
 }
 void Database::printTo(std::ostream& out) const {
@@ -4507,6 +5252,7 @@ void Database::printTo(std::ostream& out) const {
   out << ", " << "privileges="; (__isset.privileges ? (out << to_string(privileges)) : (out << "<null>"));
   out << ", " << "ownerName="; (__isset.ownerName ? (out << to_string(ownerName)) : (out << "<null>"));
   out << ", " << "ownerType="; (__isset.ownerType ? (out << to_string(ownerType)) : (out << "<null>"));
+  out << ", " << "catalogName="; (__isset.catalogName ? (out << to_string(catalogName)) : (out << "<null>"));
   out << ")";
 }
 
@@ -4588,17 +5334,17 @@ uint32_t SerDeInfo::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->parameters.clear();
-            uint32_t _size141;
-            ::apache::thrift::protocol::TType _ktype142;
-            ::apache::thrift::protocol::TType _vtype143;
-            xfer += iprot->readMapBegin(_ktype142, _vtype143, _size141);
-            uint32_t _i145;
-            for (_i145 = 0; _i145 < _size141; ++_i145)
+            uint32_t _size159;
+            ::apache::thrift::protocol::TType _ktype160;
+            ::apache::thrift::protocol::TType _vtype161;
+            xfer += iprot->readMapBegin(_ktype160, _vtype161, _size159);
+            uint32_t _i163;
+            for (_i163 = 0; _i163 < _size159; ++_i163)
             {
-              std::string _key146;
-              xfer += iprot->readString(_key146);
-              std::string& _val147 = this->parameters[_key146];
-              xfer += iprot->readString(_val147);
+              std::string _key164;
+              xfer += iprot->readString(_key164);
+              std::string& _val165 = this->parameters[_key164];
+              xfer += iprot->readString(_val165);
             }
             xfer += iprot->readMapEnd();
           }
@@ -4633,9 +5379,9 @@ uint32_t SerDeInfo::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 7:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast148;
-          xfer += iprot->readI32(ecast148);
-          this->serdeType = (SerdeType::type)ecast148;
+          int32_t ecast166;
+          xfer += iprot->readI32(ecast166);
+          this->serdeType = (SerdeType::type)ecast166;
           this->__isset.serdeType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -4669,11 +5415,11 @@ uint32_t SerDeInfo::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("parameters", ::apache::thrift::protocol::T_MAP, 3);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->parameters.size()));
-    std::map<std::string, std::string> ::const_iterator _iter149;
-    for (_iter149 = this->parameters.begin(); _iter149 != this->parameters.end(); ++_iter149)
+    std::map<std::string, std::string> ::const_iterator _iter167;
+    for (_iter167 = this->parameters.begin(); _iter167 != this->parameters.end(); ++_iter167)
     {
-      xfer += oprot->writeString(_iter149->first);
-      xfer += oprot->writeString(_iter149->second);
+      xfer += oprot->writeString(_iter167->first);
+      xfer += oprot->writeString(_iter167->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -4716,25 +5462,25 @@ void swap(SerDeInfo &a, SerDeInfo &b) {
   swap(a.__isset, b.__isset);
 }
 
-SerDeInfo::SerDeInfo(const SerDeInfo& other150) {
-  name = other150.name;
-  serializationLib = other150.serializationLib;
-  parameters = other150.parameters;
-  description = other150.description;
-  serializerClass = other150.serializerClass;
-  deserializerClass = other150.deserializerClass;
-  serdeType = other150.serdeType;
-  __isset = other150.__isset;
-}
-SerDeInfo& SerDeInfo::operator=(const SerDeInfo& other151) {
-  name = other151.name;
-  serializationLib = other151.serializationLib;
-  parameters = other151.parameters;
-  description = other151.description;
-  serializerClass = other151.serializerClass;
-  deserializerClass = other151.deserializerClass;
-  serdeType = other151.serdeType;
-  __isset = other151.__isset;
+SerDeInfo::SerDeInfo(const SerDeInfo& other168) {
+  name = other168.name;
+  serializationLib = other168.serializationLib;
+  parameters = other168.parameters;
+  description = other168.description;
+  serializerClass = other168.serializerClass;
+  deserializerClass = other168.deserializerClass;
+  serdeType = other168.serdeType;
+  __isset = other168.__isset;
+}
+SerDeInfo& SerDeInfo::operator=(const SerDeInfo& other169) {
+  name = other169.name;
+  serializationLib = other169.serializationLib;
+  parameters = other169.parameters;
+  description = other169.description;
+  serializerClass = other169.serializerClass;
+  deserializerClass = other169.deserializerClass;
+  serdeType = other169.serdeType;
+  __isset = other169.__isset;
   return *this;
 }
 void SerDeInfo::printTo(std::ostream& out) const {
@@ -4837,15 +5583,15 @@ void swap(Order &a, Order &b) {
   swap(a.__isset, b.__isset);
 }
 
-Order::Order(const Order& other152) {
-  col = other152.col;
-  order = other152.order;
-  __isset = other152.__isset;
+Order::Order(const Order& other170) {
+  col = other170.col;
+  order = other170.order;
+  __isset = other170.__isset;
 }
-Order& Order::operator=(const Order& other153) {
-  col = other153.col;
-  order = other153.order;
-  __isset = other153.__isset;
+Order& Order::operator=(const Order& other171) {
+  col = other171.col;
+  order = other171.order;
+  __isset = other171.__isset;
   return *this;
 }
 void Order::printTo(std::ostream& out) const {
@@ -4898,14 +5644,14 @@ uint32_t SkewedInfo::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->skewedColNames.clear();
-            uint32_t _size154;
-            ::apache::thrift::protocol::TType _etype157;
-            xfer += iprot->readListBegin(_etype157, _size154);
-            this->skewedColNames.resize(_size154);
-            uint32_t _i158;
-            for (_i158 = 0; _i158 < _size154; ++_i158)
+            uint32_t _size172;
+            ::apache::thrift::protocol::TType _etype175;
+            xfer += iprot->readListBegin(_etype175, _size172);
+            this->skewedColNames.resize(_size172);
+            uint32_t _i176;
+            for (_i176 = 0; _i176 < _size172; ++_i176)
             {
-              xfer += iprot->readString(this->skewedColNames[_i158]);
+              xfer += iprot->readString(this->skewedColNames[_i176]);
             }
             xfer += iprot->readListEnd();
           }
@@ -4918,23 +5664,23 @@ uint32_t SkewedInfo::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->skewedColValues.clear();
-            uint32_t _size159;
-            ::apache::thrift::protocol::TType _etype162;
-            xfer += iprot->readListBegin(_etype162, _size159);
-            this->skewedColValues.resize(_size159);
-            uint32_t _i163;
-            for (_i163 = 0; _i163 < _size159; ++_i163)
+            uint32_t _size177;
+            ::apache::thrift::protocol::TType _etype180;
+            xfer += iprot->readListBegin(_etype180, _size177);
+            this->skewedColValues.resize(_size177);
+            uint32_t _i181;
+            for (_i181 = 0; _i181 < _size177; ++_i181)
             {
               {
-                this->skewedColValues[_i163].clear();
-                uint32_t _size164;
-                ::apache::thrift::protocol::TType _etype167;
-                xfer += iprot->readListBegin(_etype167, _size164);
-                this->skewedColValues[_i163].resize(_size164);
-                uint32_t _i168;
-                for (_i168 = 0; _i168 < _size164; ++_i168)
+                this->skewedColValues[_i181].clear();
+                uint32_t _size182;
+                ::apache::thrift::protocol::TType _etype185;
+                xfer += iprot->readListBegin(_etype185, _size182);
+                this->skewedColValues[_i181].resize(_size182);
+                uint32_t _i186;
+                for (_i186 = 0; _i186 < _size182; ++_i186)
                 {
-                  xfer += iprot->readString(this->skewedColValues[_i163][_i168]);
+                  xfer += iprot->readString(this->skewedColValues[_i181][_i186]);
                 }
                 xfer += iprot->readListEnd();
               }
@@ -4950,29 +5696,29 @@ uint32_t SkewedInfo::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->skewedColValueLocationMaps.clear();
-            uint32_t _size169;
-            ::apache::thrift::protocol::TType _ktype170;
-            ::apache::thrift::protocol::TType _vtype171;
-            xfer += iprot->readMapBegin(_ktype170, _vtype171, _size169);
-            uint32_t _i173;
-            for (_i173 = 0; _i173 < _size169; ++_i173)
+            uint32_t _size187;
+            ::apache::thrift::protocol::TType _ktype188;
+            ::apache::thrift::protocol::TType _vtype189;
+            xfer += iprot->readMapBegin(_ktype188, _vtype189, _size187);
+            uint32_t _i191;
+            for (_i191 = 0; _i191 < _size187; ++_i191)
             {
-              std::vector<std::string>  _key174;
+              std::vector<std::string>  _key192;
               {
-                _key174.clear();
-                uint32_t _size176;
-                ::apache::thrift::protocol::TType _etype179;
-                xfer += iprot->readListBegin(_etype179, _size176);
-                _key174.resize(_size176);
-                uint32_t _i180;
-                for (_i180 = 0; _i180 < _size176; ++_i180)
+                _key192.clear();
+                uint32_t _size194;
+                ::apache::thrift::protocol::TType _etype197;
+                xfer += iprot->readListBegin(_etype197, _size194);
+                _key192.resize(_size194);
+                uint32_t _i198;
+                for (_i198 = 0; _i198 < _size194; ++_i198)
                 {
-                  xfer += iprot->readString(_key174[_i180]);
+                  xfer += iprot->readString(_key192[_i198]);
                 }
                 xfer += iprot->readListEnd();
               }
-              std::string& _val175 = this->skewedColValueLocationMaps[_key174];
-              xfer += iprot->readString(_val175);
+              std::string& _val193 = this->skewedColValueLocationMaps[_key192];
+              xfer += iprot->readString(_val193);
             }
             xfer += iprot->readMapEnd();
           }
@@ -5001,10 +5747,10 @@ uint32_t SkewedInfo::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("skewedColNames", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->skewedColNames.size()));
-    std::vector<std::string> ::const_iterator _iter181;
-    for (_iter181 = this->skewedColNames.begin(); _iter181 != this->skewedColNames.end(); ++_iter181)
+    std::vector<std::string> ::const_iterator _iter199;
+    for (_iter199 = this->skewedColNames.begin(); _iter199 != this->skewedColNames.end(); ++_iter199)
     {
-      xfer += oprot->writeString((*_iter181));
+      xfer += oprot->writeString((*_iter199));
     }
     xfer += oprot->writeListEnd();
   }
@@ -5013,15 +5759,15 @@ uint32_t SkewedInfo::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("skewedColValues", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_LIST, static_cast<uint32_t>(this->skewedColValues.size()));
-    std::vector<std::vector<std::string> > ::const_iterator _iter182;
-    for (_iter182 = this->skewedColValues.begin(); _iter182 != this->skewedColValues.end(); ++_iter182)
+    std::vector<std::vector<std::string> > ::const_iterator _iter200;
+    for (_iter200 = this->skewedColValues.begin(); _iter200 != this->skewedColValues.end(); ++_iter200)
     {
       {
-        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*_iter182).size()));
-        std::vector<std::string> ::const_iterator _iter183;
-        for (_iter183 = (*_iter182).begin(); _iter183 != (*_iter182).end(); ++_iter183)
+        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*_iter200).size()));
+        std::vector<std::string> ::const_iterator _iter201;
+        for (_iter201 = (*_iter200).begin(); _iter201 != (*_iter200).end(); ++_iter201)
         {
-          xfer += oprot->writeString((*_iter183));
+          xfer += oprot->writeString((*_iter201));
         }
         xfer += oprot->writeListEnd();
       }
@@ -5033,19 +5779,19 @@ uint32_t SkewedInfo::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("skewedColValueLocationMaps", ::apache::thrift::protocol::T_MAP, 3);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_LIST, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->skewedColValueLocationMaps.size()));
-    std::map<std::vector<std::string> , std::string> ::const_iterator _iter184;
-    for (_iter184 = this->skewedColValueLocationMaps.begin(); _iter184 != this->skewedColValueLocationMaps.end(); ++_iter184)
+    std::map<std::vector<std::string> , std::string> ::const_iterator _iter202;
+    for (_iter202 = this->skewedColValueLocationMaps.begin(); _iter202 != this->skewedColValueLocationMaps.end(); ++_iter202)
     {
       {
-        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(_iter184->first.size()));
-        std::vector<std::string> ::const_iterator _iter185;
-        for (_iter185 = _iter184->first.begin(); _iter185 != _iter184->first.end(); ++_iter185)
+        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(_iter202->first.size()));
+        std::vector<std::string> ::const_iterator _iter203;
+        for (_iter203 = _iter202->first.begin(); _iter203 != _iter202->first.end(); ++_iter203)
         {
-          xfer += oprot->writeString((*_iter185));
+          xfer += oprot->writeString((*_iter203));
         }
         xfer += oprot->writeListEnd();
       }
-      xfer += oprot->writeString(_iter184->second);
+      xfer += oprot->writeString(_iter202->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -5064,17 +5810,17 @@ void swap(SkewedInfo &a, SkewedInfo &b) {
   swap(a.__isset, b.__isset);
 }
 
-SkewedInfo::SkewedInfo(const SkewedInfo& other186) {
-  skewedColNames = other186.skewedColNames;
-  skewedColValues = other186.skewedColValues;
-  skewedColValueLocationMaps = other186.skewedColValueLocationMaps;
-  __isset = other186.__isset;
+SkewedInfo::SkewedInfo(const SkewedInfo& other204) {
+  skewedColNames = other204.skewedColNames;
+  skewedColValues = other204.skewedColValues;
+  skewedColValueLocationMaps = other204.skewedColValueLocationMaps;
+  __isset = other204.__isset;
 }
-SkewedInfo& SkewedInfo::operator=(const SkewedInfo& other187) {
-  skewedColNames = other187.skewedColNames;
-  skewedColValues = other187.skewedColValues;
-  skewedColValueLocationMaps = other187.skewedColValueLocationMaps;
-  __isset = other187.__isset;
+SkewedInfo& SkewedInfo::operator=(const SkewedInfo& other205) {
+  skewedColNames = other205.skewedColNames;
+  skewedColValues = other205.skewedColValues;
+  skewedColValueLocationMaps = other205.skewedColValueLocationMaps;
+  __isset = other205.__isset;
   return *this;
 }
 void SkewedInfo::printTo(std::ostream& out) const {
@@ -5166,14 +5912,14 @@ uint32_t StorageDescriptor::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->cols.clear();
-            uint32_t _size188;
-            ::apache::thrift::protocol::TType _etype191;
-            xfer += iprot->readListBegin(_etype191, _size188);
-            this->cols.resize(_size188);
-            uint32_t _i192;
-            for (_i192 = 0; _i192 < _size188; ++_i192)
+            uint32_t _size206;
+            ::apache::thrift::protocol::TType _etype209;
+            xfer += iprot->readListBegin(_etype209, _size206);
+            this->cols.resize(_size206);
+            uint32_t _i210;
+            for (_i210 = 0; _i210 < _size206; ++_i210)
             {
-              xfer += this->cols[_i192].read(iprot);
+              xfer += this->cols[_i210].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5234,14 +5980,14 @@ uint32_t StorageDescriptor::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->bucketCols.clear();
-            uint32_t _size193;
-            ::apache::thrift::protocol::TType _etype196;
-            xfer += iprot->readListBegin(_etype196, _size193);
-            this->bucketCols.resize(_size193);
-            uint32_t _i197;
-            for (_i197 = 0; _i197 < _size193; ++_i197)
+            uint32_t _size211;
+            ::apache::thrift::protocol::TType _etype214;
+            xfer += iprot->readListBegin(_etype214, _size211);
+            this->bucketCols.resize(_size211);
+            uint32_t _i215;
+            for (_i215 = 0; _i215 < _size211; ++_i215)
             {
-              xfer += iprot->readString(this->bucketCols[_i197]);
+              xfer += iprot->readString(this->bucketCols[_i215]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5254,14 +6000,14 @@ uint32_t StorageDescriptor::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->sortCols.clear();
-            uint32_t _size198;
-            ::apache::thrift::protocol::TType _etype201;
-            xfer += iprot->readListBegin(_etype201, _size198);
-            this->sortCols.resize(_size198);
-            uint32_t _i202;
-            for (_i202 = 0; _i202 < _size198; ++_i202)
+            uint32_t _size216;
+            ::apache::thrift::protocol::TType _etype219;
+            xfer += iprot->readListBegin(_etype219, _size216);
+            this->sortCols.resize(_size216);
+            uint32_t _i220;
+            for (_i220 = 0; _i220 < _size216; ++_i220)
             {
-              xfer += this->sortCols[_i202].read(iprot);
+              xfer += this->sortCols[_i220].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5274,17 +6020,17 @@ uint32_t StorageDescriptor::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->parameters.clear();
-            uint32_t _size203;
-            ::apache::thrift::protocol::TType _ktype204;
-            ::apache::thrift::protocol::TType _vtype205;
-            xfer += iprot->readMapBegin(_ktype204, _vtype205, _size203);
-            uint32_t _i207;
-            for (_i207 = 0; _i207 < _size203; ++_i207)
+            uint32_t _size221;
+            ::apache::thrift::protocol::TType _ktype222;
+            ::apache::thrift::protocol::TType _vtype223;
+            xfer += iprot->readMapBegin(_ktype222, _vtype223, _size221);
+            uint32_t _i225;
+            for (_i225 = 0; _i225 < _size221; ++_i225)
             {
-              std::string _key208;
-              xfer += iprot->readString(_key208);
-              std::string& _val209 = this->parameters[_key208];
-              xfer += iprot->readString(_val209);
+              std::string _key226;
+              xfer += iprot->readString(_key226);
+              std::string& _val227 = this->parameters[_key226];
+              xfer += iprot->readString(_val227);
             }
             xfer += iprot->readMapEnd();
           }
@@ -5329,10 +6075,10 @@ uint32_t StorageDescriptor::write(::apache::thrift::protocol::TProtocol* oprot)
   xfer += oprot->writeFieldBegin("cols", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->cols.size()));
-    std::vector<FieldSchema> ::const_iterator _iter210;
-    for (_iter210 = this->cols.begin(); _iter210 != this->cols.end(); ++_iter210)
+    std::vector<FieldSchema> ::const_iterator _iter228;
+    for (_iter228 = this->cols.begin(); _iter228 != this->cols.end(); ++_iter228)
     {
-      xfer += (*_iter210).write(oprot);
+      xfer += (*_iter228).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5365,10 +6111,10 @@ uint32_t StorageDescriptor::write(::apache::thrift::protocol::TProtocol* oprot)
   xfer += oprot->writeFieldBegin("bucketCols", ::apache::thrift::protocol::T_LIST, 8);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->bucketCols.size()));
-    std::vector<std::string> ::const_iterator _iter211;
-    for (_iter211 = this->bucketCols.begin(); _iter211 != this->bucketCols.end(); ++_iter211)
+    std::vector<std::string> ::const_iterator _iter229;
+    for (_iter229 = this->bucketCols.begin(); _iter229 != this->bucketCols.end(); ++_iter229)
     {
-      xfer += oprot->writeString((*_iter211));
+      xfer += oprot->writeString((*_iter229));
     }
     xfer += oprot->writeListEnd();
   }
@@ -5377,10 +6123,10 @@ uint32_t StorageDescriptor::write(::apache::thrift::protocol::TProtocol* oprot)
   xfer += oprot->writeFieldBegin("sortCols", ::apache::thrift::protocol::T_LIST, 9);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->sortCols.size()));
-    std::vector<Order> ::const_iterator _iter212;
-    for (_iter212 = this->sortCols.begin(); _iter212 != this->sortCols.end(); ++_iter212)
+    std::vector<Order> ::const_iterator _iter230;
+    for (_iter230 = this->sortCols.begin(); _iter230 != this->sortCols.end(); ++_iter230)
     {
-      xfer += (*_iter212).write(oprot);
+      xfer += (*_iter230).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5389,11 +6135,11 @@ uint32_t StorageDescriptor::write(::apache::thrift::protocol::TProtocol* oprot)
   xfer += oprot->writeFieldBegin("parameters", ::apache::thrift::protocol::T_MAP, 10);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->parameters.size()));
-    std::map<std::string, std::string> ::const_iterator _iter213;
-    for (_iter213 = this->parameters.begin(); _iter213 != this->parameters.end(); ++_iter213)
+    std::map<std::string, std::string> ::const_iterator _iter231;
+    for (_iter231 = this->parameters.begin(); _iter231 != this->parameters.end(); ++_iter231)
     {
-      xfer += oprot->writeString(_iter213->first);
-      xfer += oprot->writeString(_iter213->second);
+      xfer += oprot->writeString(_iter231->first);
+      xfer += oprot->writeString(_iter231->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -5431,35 +6177,35 @@ void swap(StorageDescriptor &a, StorageDescriptor &b) {
   swap(a.__isset, b.__isset);
 }
 
-StorageDescriptor::StorageDescriptor(const StorageDescriptor& other214) {
-  cols = other214.cols;
-  location = other214.location;
-  inputFormat = other214.inputFormat;
-  outputFormat = other214.outputFormat;
-  compressed = other214.compressed;
-  numBuckets = other214.numBuckets;
-  serdeInfo = other214.serdeInfo;
-  bucketCols = other214.bucketCols;
-  sortCols = other214.sortCols;
-  parameters = other214.parameters;
-  skewedInfo = other214.skewedInfo;
-  storedAsSubDirectories = other214.storedAsSubDirectories;
-  __isset = other214.__isset;
-}
-StorageDescriptor& StorageDescriptor::operator=(const StorageDescriptor& other215) {
-  cols = other215.cols;
-  location = other215.location;
-  inputFormat = other215.inputFormat;
-  outputFormat = other215.outputFormat;
-  compressed = other215.compressed;
-  numBuckets = other215.numBuckets;
-  serdeInfo = other215.serdeInfo;
-  bucketCols = other215.bucketCols;
-  sortCols = other215.sortCols;
-  parameters = other215.parameters;
-  skewedInfo = other215.skewedInfo;
-  storedAsSubDirectories = other215.storedAsSubDirectories;
-  __isset = other215.__isset;
+StorageDescriptor::StorageDescriptor(const StorageDescriptor& other232) {
+  cols = other232.cols;
+  location = other232.location;
+  inputFormat = other232.inputFormat;
+  outputFormat = other232.outputFormat;
+  compressed = other232.compressed;
+  numBuckets = other232.numBuckets;
+  serdeInfo = other232.serdeInfo;
+  bucketCols = other232.bucketCols;
+  sortCols = other232.sortCols;
+  parameters = other232.parameters;
+  skewedInfo = other232.skewedInfo;
+  storedAsSubDirectories = other232.storedAsSubDirectories;
+  __isset = other232.__isset;
+}
+StorageDescriptor& StorageDescriptor::operator=(const StorageDescriptor& other233) {
+  cols = other233.cols;
+  location = other233.location;
+  inputFormat = other233.inputFormat;
+  outputFormat = other233.outputFormat;
+  compressed = other233.compressed;
+  numBuckets = other233.numBuckets;
+  serdeInfo = other233.serdeInfo;
+  bucketCols = other233.bucketCols;
+  sortCols = other233.sortCols;
+  parameters = other233.parameters;
+  skewedInfo = other233.skewedInfo;
+  storedAsSubDirectories = other233.storedAsSubDirectories;
+  __isset = other233.__isset;
   return *this;
 }
 void StorageDescriptor::printTo(std::ostream& out) const {
@@ -5553,6 +6299,11 @@ void Table::__set_creationMetadata(const CreationMetadata& val) {
 __isset.creationMetadata = true;
 }
 
+void Table::__set_catName(const std::string& val) {
+  this->catName = val;
+__isset.catName = true;
+}
+
 uint32_t Table::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
@@ -5634,14 +6385,14 @@ uint32_t Table::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitionKeys.clear();
-            uint32_t _size216;
-            ::apache::thrift::protocol::TType _etype219;
-            xfer += iprot->readListBegin(_etype219, _size216);
-            this->partitionKeys.resize(_size216);
-            uint32_t _i220;
-            for (_i220 = 0; _i220 < _size216; ++_i220)
+            uint32_t _size234;
+            ::apache::thrift::protocol::TType _etype237;
+            xfer += iprot->readListBegin(_etype237, _size234);
+            this->partitionKeys.resize(_size234);
+            uint32_t _i238;
+            for (_i238 = 0; _i238 < _size234; ++_i238)
             {
-              xfer += this->partitionKeys[_i220].read(iprot);
+              xfer += this->partitionKeys[_i238].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5654,17 +6405,17 @@ uint32_t Table::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->parameters.clear();
-            uint32_t _size221;
-            ::apache::thrift::protocol::TType _ktype222;
-            ::apache::thrift::protocol::TType _vtype223;
-            xfer += iprot->readMapBegin(_ktype222, _vtype223, _size221);
-            uint32_t _i225;
-            for (_i225 = 0; _i225 < _size221; ++_i225)
+            uint32_t _size239;
+            ::apache::thrift::protocol::TType _ktype240;
+     

<TRUNCATED>

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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/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 8a5de09..07fdcd7 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
@@ -20,6 +20,14 @@ package org.apache.hadoop.hive.metastore;
 import static org.apache.commons.lang.StringUtils.join;
 import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_COMMENT;
 import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
+import static org.apache.hadoop.hive.metastore.Warehouse.getCatalogQualifiedTableName;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.parseDbName;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.CAT_NAME;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.DB_NAME;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.prependCatalogToDbName;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.prependNotNullCatToDbName;
 
 import java.io.IOException;
 import java.net.InetAddress;
@@ -85,11 +93,13 @@ import org.apache.hadoop.hive.metastore.events.AlterPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.AlterSchemaVersionEvent;
 import org.apache.hadoop.hive.metastore.events.AlterTableEvent;
 import org.apache.hadoop.hive.metastore.events.ConfigChangeEvent;
+import org.apache.hadoop.hive.metastore.events.CreateCatalogEvent;
 import org.apache.hadoop.hive.metastore.events.CreateDatabaseEvent;
 import org.apache.hadoop.hive.metastore.events.CreateFunctionEvent;
 import org.apache.hadoop.hive.metastore.events.CreateISchemaEvent;
 import org.apache.hadoop.hive.metastore.events.AddSchemaVersionEvent;
 import org.apache.hadoop.hive.metastore.events.CreateTableEvent;
+import org.apache.hadoop.hive.metastore.events.DropCatalogEvent;
 import org.apache.hadoop.hive.metastore.events.DropConstraintEvent;
 import org.apache.hadoop.hive.metastore.events.DropDatabaseEvent;
 import org.apache.hadoop.hive.metastore.events.DropFunctionEvent;
@@ -106,10 +116,12 @@ import org.apache.hadoop.hive.metastore.events.PreAlterPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.PreAlterSchemaVersionEvent;
 import org.apache.hadoop.hive.metastore.events.PreAlterTableEvent;
 import org.apache.hadoop.hive.metastore.events.PreAuthorizationCallEvent;
+import org.apache.hadoop.hive.metastore.events.PreCreateCatalogEvent;
 import org.apache.hadoop.hive.metastore.events.PreCreateDatabaseEvent;
 import org.apache.hadoop.hive.metastore.events.PreCreateISchemaEvent;
 import org.apache.hadoop.hive.metastore.events.PreAddSchemaVersionEvent;
 import org.apache.hadoop.hive.metastore.events.PreCreateTableEvent;
+import org.apache.hadoop.hive.metastore.events.PreDropCatalogEvent;
 import org.apache.hadoop.hive.metastore.events.PreDropDatabaseEvent;
 import org.apache.hadoop.hive.metastore.events.PreDropISchemaEvent;
 import org.apache.hadoop.hive.metastore.events.PreDropPartitionEvent;
@@ -117,6 +129,7 @@ import org.apache.hadoop.hive.metastore.events.PreDropSchemaVersionEvent;
 import org.apache.hadoop.hive.metastore.events.PreDropTableEvent;
 import org.apache.hadoop.hive.metastore.events.PreEventContext;
 import org.apache.hadoop.hive.metastore.events.PreLoadPartitionDoneEvent;
+import org.apache.hadoop.hive.metastore.events.PreReadCatalogEvent;
 import org.apache.hadoop.hive.metastore.events.PreReadDatabaseEvent;
 import org.apache.hadoop.hive.metastore.events.PreReadISchemaEvent;
 import org.apache.hadoop.hive.metastore.events.PreReadTableEvent;
@@ -161,6 +174,7 @@ import org.apache.thrift.transport.TFramedTransport;
 import org.apache.thrift.transport.TServerSocket;
 import org.apache.thrift.transport.TTransport;
 import org.apache.thrift.transport.TTransportFactory;
+import org.iq80.leveldb.DB;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -671,21 +685,43 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       return txn;
     }
 
-    private static RawStore newRawStoreForConf(Configuration conf) throws MetaException {
+    static RawStore newRawStoreForConf(Configuration conf) throws MetaException {
       Configuration newConf = new Configuration(conf);
       String rawStoreClassName = MetastoreConf.getVar(newConf, ConfVars.RAW_STORE_IMPL);
       LOG.info(addPrefix("Opening raw store with implementation class:" + rawStoreClassName));
       return RawStoreProxy.getProxy(newConf, conf, rawStoreClassName, threadLocalId.get());
     }
 
+    @VisibleForTesting
+    public static void createDefaultCatalog(RawStore ms, Warehouse wh) throws MetaException,
+        InvalidOperationException {
+      try {
+        Catalog defaultCat = ms.getCatalog(DEFAULT_CATALOG_NAME);
+        // Null check because in some test cases we get a null from ms.getCatalog.
+        if (defaultCat !=null && defaultCat.getLocationUri().equals("TBD")) {
+          // One time update issue.  When the new 'hive' catalog is created in an upgrade the
+          // script does not know the location of the warehouse.  So we need to update it.
+          LOG.info("Setting location of default catalog, as it hasn't been done after upgrade");
+          defaultCat.setLocationUri(wh.getWhRoot().toString());
+          ms.alterCatalog(defaultCat.getName(), defaultCat);
+        }
+
+      } catch (NoSuchObjectException e) {
+        Catalog cat = new Catalog(DEFAULT_CATALOG_NAME, wh.getWhRoot().toString());
+        cat.setDescription(Warehouse.DEFAULT_CATALOG_COMMENT);
+        ms.createCatalog(cat);
+      }
+    }
+
     private void createDefaultDB_core(RawStore ms) throws MetaException, InvalidObjectException {
       try {
-        ms.getDatabase(DEFAULT_DATABASE_NAME);
+        ms.getDatabase(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME);
       } catch (NoSuchObjectException e) {
         Database db = new Database(DEFAULT_DATABASE_NAME, DEFAULT_DATABASE_COMMENT,
           wh.getDefaultDatabasePath(DEFAULT_DATABASE_NAME).toString(), null);
         db.setOwnerName(PUBLIC);
         db.setOwnerType(PrincipalType.ROLE);
+        db.setCatalogName(DEFAULT_CATALOG_NAME);
         ms.createDatabase(db);
       }
     }
@@ -702,7 +738,9 @@ public class HiveMetaStore extends ThriftHiveMetastore {
      */
     private void createDefaultDB() throws MetaException {
       try {
-        createDefaultDB_core(getMS());
+        RawStore ms = getMS();
+        createDefaultCatalog(ms, wh);
+        createDefaultDB_core(ms);
       } catch (JDOException e) {
         LOG.warn("Retrying creating default database after error: " + e.getMessage(), e);
         try {
@@ -710,7 +748,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         } catch (InvalidObjectException e1) {
           throw new MetaException(e1.getMessage());
         }
-      } catch (InvalidObjectException e) {
+      } catch (InvalidObjectException|InvalidOperationException e) {
         throw new MetaException(e.getMessage());
       }
     }
@@ -853,8 +891,9 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       return startFunction(function, "");
     }
 
-    private void startTableFunction(String function, String db, String tbl) {
-      startFunction(function, " : db=" + db + " tbl=" + tbl);
+    private void startTableFunction(String function, String catName, String db, String tbl) {
+      startFunction(function, " : tbl=" +
+          getCatalogQualifiedTableName(catName, db, tbl));
     }
 
     private void startMultiTableFunction(String function, String db, List<String> tbls) {
@@ -862,14 +901,16 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       startFunction(function, " : db=" + db + " tbls=" + tableNames);
     }
 
-    private void startPartitionFunction(String function, String db, String tbl,
+    private void startPartitionFunction(String function, String cat, String db, String tbl,
                                         List<String> partVals) {
-      startFunction(function, " : db=" + db + " tbl=" + tbl + "[" + join(partVals, ",") + "]");
+      startFunction(function, " : tbl=" +
+          getCatalogQualifiedTableName(cat, db, tbl) + "[" + join(partVals, ",") + "]");
     }
 
-    private void startPartitionFunction(String function, String db, String tbl,
+    private void startPartitionFunction(String function, String catName, String db, String tbl,
                                         Map<String, String> partName) {
-      startFunction(function, " : db=" + db + " tbl=" + tbl + "partition=" + partName);
+      startFunction(function, " : tbl=" +
+          getCatalogQualifiedTableName(catName, db, tbl) + "partition=" + partName);
     }
 
     private void endFunction(String function, boolean successful, Exception e) {
@@ -920,25 +961,234 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       return counters;
     }
 
+    @Override
+    public void create_catalog(CreateCatalogRequest rqst)
+        throws AlreadyExistsException, InvalidObjectException, MetaException {
+      Catalog catalog = rqst.getCatalog();
+      startFunction("create_catalog", ": " + catalog.toString());
+      boolean success = false;
+      Exception ex = null;
+      try {
+        try {
+          getMS().getCatalog(catalog.getName());
+          throw new AlreadyExistsException("Catalog " + catalog.getName() + " already exists");
+        } catch (NoSuchObjectException e) {
+          // expected
+        }
+
+        if (!MetaStoreUtils.validateName(catalog.getName(), null)) {
+          throw new InvalidObjectException(catalog.getName() + " is not a valid catalog name");
+        }
+
+        if (catalog.getLocationUri() == null) {
+          throw new InvalidObjectException("You must specify a path for the catalog");
+        }
+
+        RawStore ms = getMS();
+        Path catPath = new Path(catalog.getLocationUri());
+        boolean madeDir = false;
+        Map<String, String> transactionalListenersResponses = Collections.emptyMap();
+        try {
+          firePreEvent(new PreCreateCatalogEvent(this, catalog));
+          if (!wh.isDir(catPath)) {
+            if (!wh.mkdirs(catPath)) {
+              throw new MetaException("Unable to create catalog path " + catPath +
+                  ", failed to create catalog " + catalog.getName());
+            }
+            madeDir = true;
+          }
+
+          ms.openTransaction();
+          ms.createCatalog(catalog);
+
+          // Create a default database inside the catalog
+          Database db = new Database(DEFAULT_DATABASE_NAME, "Default database for catalog " +
+              catalog.getName(), catalog.getLocationUri(), Collections.emptyMap());
+          db.setCatalogName(catalog.getName());
+          create_database_core(ms, db);
+
+          if (!transactionalListeners.isEmpty()) {
+            transactionalListenersResponses =
+                MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
+                    EventType.CREATE_CATALOG,
+                    new CreateCatalogEvent(true, this, catalog));
+          }
+
+          success = ms.commitTransaction();
+        } finally {
+          if (!success) {
+            ms.rollbackTransaction();
+            if (madeDir) {
+              wh.deleteDir(catPath, true);
+            }
+          }
+
+          if (!listeners.isEmpty()) {
+            MetaStoreListenerNotifier.notifyEvent(listeners,
+                EventType.CREATE_CATALOG,
+                new CreateCatalogEvent(success, this, catalog),
+                null,
+                transactionalListenersResponses, ms);
+          }
+        }
+        success = true;
+      } catch (AlreadyExistsException|InvalidObjectException|MetaException e) {
+        ex = e;
+        throw e;
+      } finally {
+        endFunction("create_catalog", success, ex);
+      }
+    }
+
+    @Override
+    public GetCatalogResponse get_catalog(GetCatalogRequest rqst)
+        throws NoSuchObjectException, TException {
+      String catName = rqst.getName();
+      startFunction("get_catalog", ": " + catName);
+      Catalog cat = null;
+      Exception ex = null;
+      try {
+        cat = getMS().getCatalog(catName);
+        firePreEvent(new PreReadCatalogEvent(this, cat));
+        return new GetCatalogResponse(cat);
+      } catch (MetaException|NoSuchObjectException e) {
+        ex = e;
+        throw e;
+      } finally {
+        endFunction("get_database", cat != null, ex);
+      }
+    }
+
+    @Override
+    public GetCatalogsResponse get_catalogs() throws MetaException {
+      startFunction("get_catalogs");
+
+      List<String> ret = null;
+      Exception ex = null;
+      try {
+        ret = getMS().getCatalogs();
+      } catch (MetaException e) {
+        ex = e;
+        throw e;
+      } finally {
+        endFunction("get_catalog", ret != null, ex);
+      }
+      return new GetCatalogsResponse(ret == null ? Collections.emptyList() : ret);
+
+    }
+
+    @Override
+    public void drop_catalog(DropCatalogRequest rqst)
+        throws NoSuchObjectException, InvalidOperationException, MetaException {
+      String catName = rqst.getName();
+      startFunction("drop_catalog", ": " + catName);
+      if (DEFAULT_CATALOG_NAME.equalsIgnoreCase(catName)) {
+        endFunction("drop_catalog", false, null);
+        throw new MetaException("Can not drop " + DEFAULT_CATALOG_NAME + " catalog");
+      }
+
+      boolean success = false;
+      Exception ex = null;
+      try {
+        dropCatalogCore(catName);
+        success = true;
+      } catch (NoSuchObjectException|InvalidOperationException|MetaException e) {
+        ex = e;
+        throw e;
+      } catch (Exception e) {
+        ex = e;
+        throw newMetaException(e);
+      } finally {
+        endFunction("drop_catalog", success, ex);
+      }
+
+    }
+
+    private void dropCatalogCore(String catName)
+        throws MetaException, NoSuchObjectException, InvalidOperationException {
+      boolean success = false;
+      Catalog cat = null;
+      Map<String, String> transactionalListenerResponses = Collections.emptyMap();
+      RawStore ms = getMS();
+      try {
+        ms.openTransaction();
+        cat = ms.getCatalog(catName);
+
+        firePreEvent(new PreDropCatalogEvent(this, cat));
+
+        List<String> allDbs = get_databases(prependNotNullCatToDbName(catName, null));
+        if (allDbs != null && !allDbs.isEmpty()) {
+          // It might just be the default, in which case we can drop that one if it's empty
+          if (allDbs.size() == 1 && allDbs.get(0).equals(DEFAULT_DATABASE_NAME)) {
+            try {
+              drop_database_core(ms, catName, DEFAULT_DATABASE_NAME, true, false);
+            } catch (InvalidOperationException e) {
+              // This means there are tables of something in the database
+              throw new InvalidOperationException("There are still objects in the default " +
+                  "database for catalog " + catName);
+            } catch (InvalidObjectException|IOException|InvalidInputException e) {
+              MetaException me = new MetaException("Error attempt to drop default database for " +
+                  "catalog " + catName);
+              me.initCause(e);
+              throw me;
+            }
+          } else {
+            throw new InvalidOperationException("There are non-default databases in the catalog " +
+                catName + " so it cannot be dropped.");
+          }
+        }
+
+        ms.dropCatalog(catName) ;
+        if (!transactionalListeners.isEmpty()) {
+          transactionalListenerResponses =
+              MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
+                  EventType.DROP_CATALOG,
+                  new DropCatalogEvent(true, this, cat));
+        }
+
+        success = ms.commitTransaction();
+      } finally {
+        if (success) {
+          wh.deleteDir(wh.getDnsPath(new Path(cat.getLocationUri())), false);
+        } else {
+          ms.rollbackTransaction();
+        }
+
+        if (!listeners.isEmpty()) {
+          MetaStoreListenerNotifier.notifyEvent(listeners,
+              EventType.DROP_CATALOG,
+              new DropCatalogEvent(success, this, cat),
+              null,
+              transactionalListenerResponses, ms);
+        }
+      }
+    }
+
+
+    // Assumes that the catalog has already been set.
     private void create_database_core(RawStore ms, final Database db)
         throws AlreadyExistsException, InvalidObjectException, MetaException {
       if (!MetaStoreUtils.validateName(db.getName(), null)) {
         throw new InvalidObjectException(db.getName() + " is not a valid database name");
       }
 
-      if (null == db.getLocationUri()) {
-        db.setLocationUri(wh.getDefaultDatabasePath(db.getName()).toString());
-      } else {
-        db.setLocationUri(wh.getDnsPath(new Path(db.getLocationUri())).toString());
+      Catalog cat = null;
+      try {
+        cat = getMS().getCatalog(db.getCatalogName());
+      } catch (NoSuchObjectException e) {
+        LOG.error("No such catalog " + db.getCatalogName());
+        throw new InvalidObjectException("No such catalog " + db.getCatalogName());
       }
+      Path dbPath = wh.determineDatabasePath(cat, db);
+      db.setLocationUri(dbPath.toString());
 
-      Path dbPath = new Path(db.getLocationUri());
       boolean success = false;
       boolean madeDir = false;
       Map<String, String> transactionalListenersResponses = Collections.emptyMap();
       try {
         firePreEvent(new PreCreateDatabaseEvent(db, this));
         if (!wh.isDir(dbPath)) {
+          LOG.debug("Creating database path " + dbPath);
           if (!wh.mkdirs(dbPath)) {
             throw new MetaException("Unable to create database path " + dbPath +
                 ", failed to create database " + db.getName());
@@ -981,9 +1231,10 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       startFunction("create_database", ": " + db.toString());
       boolean success = false;
       Exception ex = null;
+      if (!db.isSetCatalogName()) db.setCatalogName(getDefaultCatalog(conf));
       try {
         try {
-          if (null != get_database_core(db.getName())) {
+          if (null != get_database_core(db.getCatalogName(), db.getName())) {
             throw new AlreadyExistsException("Database " + db.getName() + " already exists");
           }
         } catch (NoSuchObjectException e) {
@@ -1022,7 +1273,8 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       Database db = null;
       Exception ex = null;
       try {
-        db = get_database_core(name);
+        String[] parsedDbName = parseDbName(name, conf);
+        db = get_database_core(parsedDbName[CAT_NAME], parsedDbName[DB_NAME]);
         firePreEvent(new PreReadDatabaseEvent(db, this));
       } catch (MetaException|NoSuchObjectException e) {
         ex = e;
@@ -1034,11 +1286,10 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     }
 
     @Override
-    public Database get_database_core(final String name) throws NoSuchObjectException,
-        MetaException {
+    public Database get_database_core(String catName, final String name) throws NoSuchObjectException, MetaException {
       Database db = null;
       try {
-        db = getMS().getDatabase(name);
+        db = getMS().getDatabase(catName, name);
       } catch (MetaException | NoSuchObjectException e) {
         throw e;
       } catch (Exception e) {
@@ -1050,7 +1301,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
 
     @Override
     public void alter_database(final String dbName, final Database newDB) throws TException {
-      startFunction("alter_database" + dbName);
+      startFunction("alter_database " + dbName);
       boolean success = false;
       Exception ex = null;
       RawStore ms = getMS();
@@ -1062,27 +1313,30 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         newDB.setLocationUri(wh.getDnsPath(new Path(newDB.getLocationUri())).toString());
       }
 
+      String[] parsedDbName = parseDbName(dbName, conf);
+
       try {
-        oldDB = get_database_core(dbName);
+        oldDB = get_database_core(parsedDbName[CAT_NAME], parsedDbName[DB_NAME]);
         if (oldDB == null) {
-          throw new MetaException("Could not alter database \"" + dbName + "\". Could not retrieve old definition.");
+          throw new MetaException("Could not alter database \"" + parsedDbName[DB_NAME] +
+              "\". Could not retrieve old definition.");
         }
         firePreEvent(new PreAlterDatabaseEvent(oldDB, newDB, this));
 
         ms.openTransaction();
-        ms.alterDatabase(dbName, newDB);
+        ms.alterDatabase(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], newDB);
 
         if (!transactionalListeners.isEmpty()) {
           transactionalListenersResponses =
-                  MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
-                          EventType.ALTER_DATABASE,
-                          new AlterDatabaseEvent(oldDB, newDB, true, this));
+              MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
+                  EventType.ALTER_DATABASE,
+                  new AlterDatabaseEvent(oldDB, newDB, true, this));
         }
 
         success = ms.commitTransaction();
-      } catch (Exception e) {
+      } catch (MetaException|NoSuchObjectException e) {
         ex = e;
-        rethrowException(e);
+        throw e;
       } finally {
         if (!success) {
           ms.rollbackTransaction();
@@ -1090,16 +1344,16 @@ public class HiveMetaStore extends ThriftHiveMetastore {
 
         if ((null != oldDB) && (!listeners.isEmpty())) {
           MetaStoreListenerNotifier.notifyEvent(listeners,
-                  EventType.ALTER_DATABASE,
-                  new AlterDatabaseEvent(oldDB, newDB, success, this),
-                  null,
-                  transactionalListenersResponses, ms);
+              EventType.ALTER_DATABASE,
+              new AlterDatabaseEvent(oldDB, newDB, success, this),
+              null,
+              transactionalListenersResponses, ms);
         }
         endFunction("alter_database", success, ex);
       }
     }
 
-    private void drop_database_core(RawStore ms,
+    private void drop_database_core(RawStore ms, String catName,
         final String name, final boolean deleteData, final boolean cascade)
         throws NoSuchObjectException, InvalidOperationException, MetaException,
         IOException, InvalidObjectException, InvalidInputException {
@@ -1110,12 +1364,13 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       Map<String, String> transactionalListenerResponses = Collections.emptyMap();
       try {
         ms.openTransaction();
-        db = ms.getDatabase(name);
+        db = ms.getDatabase(catName, name);
 
         firePreEvent(new PreDropDatabaseEvent(db, this));
+        String catPrependedName = MetaStoreUtils.prependCatalogToDbName(catName, name, conf);
 
-        List<String> allTables = get_all_tables(db.getName());
-        List<String> allFunctions = get_functions(db.getName(), "*");
+        List<String> allTables = get_all_tables(catPrependedName);
+        List<String> allFunctions = get_functions(catPrependedName, "*");
 
         if (!cascade) {
           if (!allTables.isEmpty()) {
@@ -1138,7 +1393,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
 
         // drop any functions before dropping db
         for (String funcName : allFunctions) {
-          drop_function(name, funcName);
+          drop_function(catPrependedName, funcName);
         }
 
         // drop tables before dropping db
@@ -1152,7 +1407,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
 
           List<Table> tables;
           try {
-            tables = ms.getTableObjectsByName(name, allTables.subList(startIndex, endIndex));
+            tables = ms.getTableObjectsByName(catName, name, allTables.subList(startIndex, endIndex));
           } catch (UnknownDBException e) {
             throw new MetaException(e.getMessage());
           }
@@ -1178,18 +1433,19 @@ public class HiveMetaStore extends ThriftHiveMetastore {
 
               // For each partition in each table, drop the partitions and get a list of
               // partitions' locations which might need to be deleted
-              partitionPaths = dropPartitionsAndGetLocations(ms, name, table.getTableName(),
+              partitionPaths = dropPartitionsAndGetLocations(ms, catName, name, table.getTableName(),
                   tablePath, table.getPartitionKeys(), deleteData && !isExternal(table));
 
               // Drop the table but not its data
-              drop_table(name, table.getTableName(), false);
+              drop_table(MetaStoreUtils.prependCatalogToDbName(table.getCatName(), table.getDbName(), conf),
+                  table.getTableName(), false);
             }
 
             startIndex = endIndex;
           }
         }
 
-        if (ms.dropDatabase(name)) {
+        if (ms.dropDatabase(catName, name)) {
           if (!transactionalListeners.isEmpty()) {
             transactionalListenerResponses =
                 MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
@@ -1246,45 +1502,46 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     @Override
     public void drop_database(final String dbName, final boolean deleteData, final boolean cascade)
         throws NoSuchObjectException, InvalidOperationException, MetaException {
-
       startFunction("drop_database", ": " + dbName);
-      if (DEFAULT_DATABASE_NAME.equalsIgnoreCase(dbName)) {
+      String[] parsedDbName = parseDbName(dbName, conf);
+      if (DEFAULT_CATALOG_NAME.equalsIgnoreCase(parsedDbName[CAT_NAME]) &&
+          DEFAULT_DATABASE_NAME.equalsIgnoreCase(parsedDbName[DB_NAME])) {
         endFunction("drop_database", false, null);
-        throw new MetaException("Can not drop default database");
+        throw new MetaException("Can not drop " + DEFAULT_DATABASE_NAME + " database in catalog "
+            + DEFAULT_CATALOG_NAME);
       }
 
       boolean success = false;
       Exception ex = null;
       try {
-        drop_database_core(getMS(), dbName, deleteData, cascade);
+        drop_database_core(getMS(), parsedDbName[CAT_NAME], parsedDbName[DB_NAME], deleteData,
+            cascade);
         success = true;
-      } catch (IOException e) {
+      } catch (NoSuchObjectException|InvalidOperationException|MetaException e) {
         ex = e;
-        throw new MetaException(e.getMessage());
+        throw e;
       } catch (Exception e) {
         ex = e;
-        if (e instanceof MetaException) {
-          throw (MetaException) e;
-        } else if (e instanceof InvalidOperationException) {
-          throw (InvalidOperationException) e;
-        } else if (e instanceof NoSuchObjectException) {
-          throw (NoSuchObjectException) e;
-        } else {
-          throw newMetaException(e);
-        }
+        throw newMetaException(e);
       } finally {
         endFunction("drop_database", success, ex);
       }
     }
 
+
     @Override
     public List<String> get_databases(final String pattern) throws MetaException {
       startFunction("get_databases", ": " + pattern);
 
+      String[] parsedDbNamed = parseDbName(pattern, conf);
       List<String> ret = null;
       Exception ex = null;
       try {
-        ret = getMS().getDatabases(pattern);
+        if (parsedDbNamed[DB_NAME] == null) {
+          ret = getMS().getAllDatabases(parsedDbNamed[CAT_NAME]);
+        } else {
+          ret = getMS().getDatabases(parsedDbNamed[CAT_NAME], parsedDbNamed[DB_NAME]);
+        }
       } catch (Exception e) {
         ex = e;
         if (e instanceof MetaException) {
@@ -1300,23 +1557,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
 
     @Override
     public List<String> get_all_databases() throws MetaException {
-      startFunction("get_all_databases");
-
-      List<String> ret = null;
-      Exception ex = null;
-      try {
-        ret = getMS().getAllDatabases();
-      } catch (Exception e) {
-        ex = e;
-        if (e instanceof MetaException) {
-          throw (MetaException) e;
-        } else {
-          throw newMetaException(e);
-        }
-      } finally {
-        endFunction("get_all_databases", ret != null, ex);
-      }
-      return ret;
+      return get_databases(MetaStoreUtils.prependCatalogToDbName(null, null, conf));
     }
 
     private void create_type_core(final RawStore ms, final Type type)
@@ -1463,18 +1704,20 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       Path tblPath = null;
       boolean success = false, madeDir = false;
       try {
+        if (!tbl.isSetCatName()) tbl.setCatName(getDefaultCatalog(conf));
         firePreEvent(new PreCreateTableEvent(tbl, this));
 
         ms.openTransaction();
 
-        Database db = ms.getDatabase(tbl.getDbName());
+        Database db = ms.getDatabase(tbl.getCatName(), tbl.getDbName());
         if (db == null) {
-          throw new NoSuchObjectException("The database " + tbl.getDbName() + " does not exist");
+          throw new NoSuchObjectException("The database " +
+              Warehouse.getCatalogQualifiedDbName(tbl.getCatName(), tbl.getDbName()) + " does not exist");
         }
 
         // get_table checks whether database exists, it should be moved here
-        if (is_table_exists(ms, tbl.getDbName(), tbl.getTableName())) {
-          throw new AlreadyExistsException("Table " + tbl.getTableName()
+        if (is_table_exists(ms, tbl.getCatName(), tbl.getDbName(), tbl.getTableName())) {
+          throw new AlreadyExistsException("Table " + getCatalogQualifiedTableName(tbl)
               + " already exists");
         }
 
@@ -1482,7 +1725,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
           if (tbl.getSd().getLocation() == null
               || tbl.getSd().getLocation().isEmpty()) {
             tblPath = wh.getDefaultTablePath(
-                ms.getDatabase(tbl.getDbName()), tbl.getTableName());
+                ms.getDatabase(tbl.getCatName(), tbl.getDbName()), tbl.getTableName());
           } else {
             if (!isExternal(tbl) && !MetaStoreUtils.isNonNativeTable(tbl)) {
               LOG.warn("Location: " + tbl.getSd().getLocation()
@@ -1550,7 +1793,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
               }
             }
           }
-          int notNullConstraintSize =  notNullConstraints.size();
+          int notNullConstraintSize =  0;
           if (notNullConstraints != null) {
             for (int i = 0; i < notNullConstraints.size(); i++) {
               if (notNullConstraints.get(i).getNn_name() == null) {
@@ -1558,7 +1801,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
               }
             }
           }
-          int defaultConstraintSize =  defaultConstraints.size();
+          int defaultConstraintSize =  0;
           if (defaultConstraints!= null) {
             for (int i = 0; i < defaultConstraints.size(); i++) {
               if (defaultConstraints.get(i).getDc_name() == null) {
@@ -1649,6 +1892,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         create_table_core(getMS(), tbl, envContext);
         success = true;
       } catch (NoSuchObjectException e) {
+        LOG.warn("create_table_with_environment_context got ", e);
         ex = e;
         throw new InvalidObjectException(e.getMessage());
       } catch (Exception e) {
@@ -1704,6 +1948,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     @Override
     public void drop_constraint(DropConstraintRequest req)
         throws MetaException, InvalidObjectException {
+      String catName = req.isSetCatName() ? req.getCatName() : getDefaultCatalog(conf);
       String dbName = req.getDbname();
       String tableName = req.getTablename();
       String constraintName = req.getConstraintname();
@@ -1713,9 +1958,9 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       RawStore ms = getMS();
       try {
         ms.openTransaction();
-        ms.dropConstraint(dbName, tableName, constraintName);
+        ms.dropConstraint(catName, dbName, tableName, constraintName);
         if (transactionalListeners.size() > 0) {
-          DropConstraintEvent dropConstraintEvent = new DropConstraintEvent(dbName,
+          DropConstraintEvent dropConstraintEvent = new DropConstraintEvent(catName, dbName,
               tableName, constraintName, true, this);
           for (MetaStoreEventListener transactionalListener : transactionalListeners) {
             transactionalListener.onDropConstraint(dropConstraintEvent);
@@ -1737,7 +1982,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
           ms.rollbackTransaction();
         } else {
           for (MetaStoreEventListener listener : listeners) {
-            DropConstraintEvent dropConstraintEvent = new DropConstraintEvent(dbName,
+            DropConstraintEvent dropConstraintEvent = new DropConstraintEvent(catName, dbName,
                 tableName, constraintName, true, this);
             listener.onDropConstraint(dropConstraintEvent);
           }
@@ -2059,15 +2304,16 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       }
     }
 
-    private boolean is_table_exists(RawStore ms, String dbname, String name)
+    private boolean is_table_exists(RawStore ms, String catName, String dbname, String name)
         throws MetaException {
-      return (ms.getTable(dbname, name) != null);
+      return (ms.getTable(catName, dbname, name) != null);
     }
 
-    private boolean drop_table_core(final RawStore ms, final String dbname, final String name,
-        final boolean deleteData, final EnvironmentContext envContext,
-        final String indexName) throws NoSuchObjectException,
-        MetaException, IOException, InvalidObjectException, InvalidInputException {
+    private boolean drop_table_core(final RawStore ms, final String catName, final String dbname,
+                                    final String name, final boolean deleteData,
+                                    final EnvironmentContext envContext, final String indexName)
+        throws NoSuchObjectException, MetaException, IOException, InvalidObjectException,
+        InvalidInputException {
       boolean success = false;
       boolean isExternal = false;
       Path tblPath = null;
@@ -2078,7 +2324,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       try {
         ms.openTransaction();
         // drop any partitions
-        tbl = get_table_core(dbname, name);
+        tbl = get_table_core(catName, dbname, name);
         if (tbl == null) {
           throw new NoSuchObjectException(name + " doesn't exist");
         }
@@ -2101,10 +2347,14 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         }
 
         // Drop the partitions and get a list of locations which need to be deleted
-        partPaths = dropPartitionsAndGetLocations(ms, dbname, name, tblPath,
+        partPaths = dropPartitionsAndGetLocations(ms, catName, dbname, name, tblPath,
             tbl.getPartitionKeys(), deleteData && !isExternal);
-        if (!ms.dropTable(dbname, name)) {
-          String tableName = dbname + "." + name;
+
+        // Drop any constraints on the table
+        ms.dropConstraint(catName, dbname, name, null, true);
+
+        if (!ms.dropTable(catName, dbname, name)) {
+          String tableName = getCatalogQualifiedTableName(catName, dbname, name);
           throw new MetaException(indexName == null ? "Unable to drop table " + tableName:
               "Unable to drop index table " + tableName + " for index " + indexName);
         } else {
@@ -2221,7 +2471,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
      * @throws InvalidObjectException
      * @throws NoSuchObjectException
      */
-    private List<Path> dropPartitionsAndGetLocations(RawStore ms, String dbName,
+    private List<Path> dropPartitionsAndGetLocations(RawStore ms, String catName, String dbName,
       String tableName, Path tablePath, List<FieldSchema> partitionKeys, boolean checkLocation)
       throws MetaException, IOException, NoSuchObjectException, InvalidObjectException,
       InvalidInputException {
@@ -2232,12 +2482,12 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         tableDnsPath = wh.getDnsPath(tablePath);
       }
       List<Path> partPaths = new ArrayList<>();
-      Table tbl = ms.getTable(dbName, tableName);
+      Table tbl = ms.getTable(catName, dbName, tableName);
 
       // call dropPartition on each of the table's partitions to follow the
       // procedure for cleanly dropping partitions.
       while (true) {
-        List<Partition> partsToDelete = ms.getPartitions(dbName, tableName, partitionBatchSize);
+        List<Partition> partsToDelete = ms.getPartitions(catName, dbName, tableName, partitionBatchSize);
         if (partsToDelete == null || partsToDelete.isEmpty()) {
           break;
         }
@@ -2269,7 +2519,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
             }
           }
         }
-        ms.dropPartitions(dbName, tableName, partNames);
+        ms.dropPartitions(catName, dbName, tableName, partNames);
       }
 
       return partPaths;
@@ -2285,12 +2535,14 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     public void drop_table_with_environment_context(final String dbname, final String name,
         final boolean deleteData, final EnvironmentContext envContext)
         throws NoSuchObjectException, MetaException {
-      startTableFunction("drop_table", dbname, name);
+      String[] parsedDbName = parseDbName(dbname, conf);
+      startTableFunction("drop_table", parsedDbName[CAT_NAME], parsedDbName[DB_NAME], name);
 
       boolean success = false;
       Exception ex = null;
       try {
-        success = drop_table_core(getMS(), dbname, name, deleteData, envContext, null);
+        success = drop_table_core(getMS(), parsedDbName[CAT_NAME], parsedDbName[DB_NAME], name,
+            deleteData, envContext, null);
       } catch (IOException e) {
         ex = e;
         throw new MetaException(e.getMessage());
@@ -2323,6 +2575,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     }
 
     private void alterPartitionForTruncate(final RawStore ms,
+                                           final String catName,
                                            final String dbName,
                                            final String tableName,
                                            final Table table,
@@ -2342,18 +2595,20 @@ public class HiveMetaStore extends ThriftHiveMetastore {
                 new AlterPartitionEvent(partition, partition, table, true, true, this));
       }
 
-      alterHandler.alterPartition(ms, wh, dbName, tableName, null, partition, environmentContext, this);
+      alterHandler.alterPartition(ms, wh, catName, dbName, tableName, null, partition,
+          environmentContext, this);
     }
 
     private void alterTableStatsForTruncate(final RawStore ms,
+                                            final String catName,
                                             final String dbName,
                                             final String tableName,
                                             final Table table,
                                             final List<String> partNames) throws Exception {
       if (partNames == null) {
         if (0 != table.getPartitionKeysSize()) {
-          for (Partition partition : ms.getPartitions(dbName, tableName, Integer.MAX_VALUE)) {
-            alterPartitionForTruncate(ms, dbName, tableName, table, partition);
+          for (Partition partition : ms.getPartitions(catName, dbName, tableName, Integer.MAX_VALUE)) {
+            alterPartitionForTruncate(ms, catName, dbName, tableName, table, partition);
           }
         } else {
           EnvironmentContext environmentContext = new EnvironmentContext();
@@ -2371,17 +2626,18 @@ public class HiveMetaStore extends ThriftHiveMetastore {
                     new AlterTableEvent(table, table, true, true, this));
           }
 
-          alterHandler.alterTable(ms, wh, dbName, tableName, table, environmentContext, this);
+          alterHandler.alterTable(ms, wh, catName, dbName, tableName, table, environmentContext, this);
         }
       } else {
-        for (Partition partition : ms.getPartitionsByNames(dbName, tableName, partNames)) {
-          alterPartitionForTruncate(ms, dbName, tableName, table, partition);
+        for (Partition partition : ms.getPartitionsByNames(catName, dbName, tableName, partNames)) {
+          alterPartitionForTruncate(ms, catName, dbName, tableName, table, partition);
         }
       }
       return;
     }
 
     private List<Path> getLocationsForTruncate(final RawStore ms,
+                                               final String catName,
                                                final String dbName,
                                                final String tableName,
                                                final Table table,
@@ -2389,14 +2645,14 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       List<Path> locations = new ArrayList<>();
       if (partNames == null) {
         if (0 != table.getPartitionKeysSize()) {
-          for (Partition partition : ms.getPartitions(dbName, tableName, Integer.MAX_VALUE)) {
+          for (Partition partition : ms.getPartitions(catName, dbName, tableName, Integer.MAX_VALUE)) {
             locations.add(new Path(partition.getSd().getLocation()));
           }
         } else {
           locations.add(new Path(table.getSd().getLocation()));
         }
       } else {
-        for (Partition partition : ms.getPartitionsByNames(dbName, tableName, partNames)) {
+        for (Partition partition : ms.getPartitionsByNames(catName, dbName, tableName, partNames)) {
           locations.add(new Path(partition.getSd().getLocation()));
         }
       }
@@ -2413,11 +2669,13 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     public void truncate_table(final String dbName, final String tableName, List<String> partNames)
       throws NoSuchObjectException, MetaException {
       try {
-        Table tbl = get_table_core(dbName, tableName);
+        String[] parsedDbName = parseDbName(dbName, conf);
+        Table tbl = get_table_core(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tableName);
         boolean isAutopurge = (tbl.isSetParameters() && "true".equalsIgnoreCase(tbl.getParameters().get("auto.purge")));
 
         // This is not transactional
-        for (Path location : getLocationsForTruncate(getMS(), dbName, tableName, tbl, partNames)) {
+        for (Path location : getLocationsForTruncate(getMS(), parsedDbName[CAT_NAME],
+            parsedDbName[DB_NAME], tableName, tbl, partNames)) {
           FileSystem fs = location.getFileSystem(getConf());
           if (!org.apache.hadoop.hive.metastore.utils.HdfsUtils.isPathEncrypted(getConf(), fs.getUri(), location) &&
               !FileUtils.pathHasSnapshotSubDir(location, fs)) {
@@ -2439,7 +2697,8 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         }
 
         // Alter the table/partition stats and also notify truncate table event
-        alterTableStatsForTruncate(getMS(), dbName, tableName, tbl, partNames);
+        alterTableStatsForTruncate(getMS(), parsedDbName[CAT_NAME], parsedDbName[DB_NAME],
+            tableName, tbl, partNames);
       } catch (IOException e) {
         throw new MetaException(e.getMessage());
       } catch (Exception e) {
@@ -2468,17 +2727,19 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     @Deprecated
     public Table get_table(final String dbname, final String name) throws MetaException,
         NoSuchObjectException {
-      return getTableInternal(dbname, name, null);
+      String[] parsedDbName = parseDbName(dbname, conf);
+      return getTableInternal(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], name, null);
     }
 
     @Override
     public GetTableResult get_table_req(GetTableRequest req) throws MetaException,
         NoSuchObjectException {
-      return new GetTableResult(getTableInternal(req.getDbName(), req.getTblName(),
+      String catName = req.isSetCatName() ? req.getCatName() : getDefaultCatalog(conf);
+      return new GetTableResult(getTableInternal(catName, req.getDbName(), req.getTblName(),
           req.getCapabilities()));
     }
 
-    private Table getTableInternal(String dbname, String name,
+    private Table getTableInternal(String catName, String dbname, String name,
         ClientCapabilities capabilities) throws MetaException, NoSuchObjectException {
       if (isInTest) {
         assertClientHasCapability(capabilities, ClientCapability.TEST_CAPABILITY,
@@ -2486,10 +2747,10 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       }
 
       Table t = null;
-      startTableFunction("get_table", dbname, name);
+      startTableFunction("get_table", catName, dbname, name);
       Exception ex = null;
       try {
-        t = get_table_core(dbname, name);
+        t = get_table_core(catName, dbname, name);
         if (MetaStoreUtils.isInsertOnlyTableParam(t.getParameters())) {
           assertClientHasCapability(capabilities, ClientCapability.INSERT_ONLY_TABLES,
               "insert-only tables", "get_table_req");
@@ -2509,10 +2770,11 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     public List<TableMeta> get_table_meta(String dbnames, String tblNames, List<String> tblTypes)
         throws MetaException, NoSuchObjectException {
       List<TableMeta> t = null;
-      startTableFunction("get_table_metas", dbnames, tblNames);
+      String[] parsedDbName = parseDbName(dbnames, conf);
+      startTableFunction("get_table_metas", parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tblNames);
       Exception ex = null;
       try {
-        t = getMS().getTableMeta(dbnames, tblNames, tblTypes);
+        t = getMS().getTableMeta(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tblNames, tblTypes);
       } catch (Exception e) {
         ex = e;
         throw newMetaException(e);
@@ -2523,14 +2785,14 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     }
 
     @Override
-    public Table get_table_core(final String dbname, final String name) throws MetaException,
-        NoSuchObjectException {
+    public Table get_table_core(final String catName, final String dbname, final String name)
+        throws MetaException, NoSuchObjectException {
       Table t = null;
       try {
-        t = getMS().getTable(dbname, name);
+        t = getMS().getTable(catName, dbname, name);
         if (t == null) {
-          throw new NoSuchObjectException(dbname + "." + name
-              + " table not found");
+          throw new NoSuchObjectException(getCatalogQualifiedTableName(catName, dbname, name) +
+            " table not found");
         }
       } catch (Exception e) {
         throwMetaException(e);
@@ -2558,17 +2820,20 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     @Deprecated
     public List<Table> get_table_objects_by_name(final String dbName, final List<String> tableNames)
         throws MetaException, InvalidOperationException, UnknownDBException {
-      return getTableObjectsInternal(dbName, tableNames, null);
+      String[] parsedDbName = parseDbName(dbName, conf);
+      return getTableObjectsInternal(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tableNames, null);
     }
 
     @Override
     public GetTablesResult get_table_objects_by_name_req(GetTablesRequest req) throws TException {
-      return new GetTablesResult(getTableObjectsInternal(
+      String catName = req.isSetCatName() ? req.getCatName() : getDefaultCatalog(conf);
+      return new GetTablesResult(getTableObjectsInternal(catName,
           req.getDbName(), req.getTblNames(), req.getCapabilities()));
     }
 
-    private List<Table> getTableObjectsInternal(
-        String dbName, List<String> tableNames, ClientCapabilities capabilities)
+    private List<Table> getTableObjectsInternal(String catName, String dbName,
+                                                List<String> tableNames,
+                                                ClientCapabilities capabilities)
             throws MetaException, InvalidOperationException, UnknownDBException {
       if (isInTest) {
         assertClientHasCapability(capabilities, ClientCapability.TEST_CAPABILITY,
@@ -2606,7 +2871,8 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         // Oracle cannot have over 1000 expressions in a in-list
         while (startIndex < distinctTableNames.size()) {
           int endIndex = Math.min(startIndex + tableBatchSize, distinctTableNames.size());
-          tables.addAll(ms.getTableObjectsByName(dbName, distinctTableNames.subList(startIndex, endIndex)));
+          tables.addAll(ms.getTableObjectsByName(catName, dbName, distinctTableNames.subList(
+              startIndex, endIndex)));
           startIndex = endIndex;
         }
         for (Table t : tables) {
@@ -2638,8 +2904,8 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     }
 
     @Override
-    public void update_creation_metadata(final String dbName, final String tableName, CreationMetadata cm) throws MetaException {
-      getMS().updateCreationMetadata(dbName, tableName, cm);
+    public void update_creation_metadata(String catName, final String dbName, final String tableName, CreationMetadata cm) throws MetaException {
+      getMS().updateCreationMetadata(catName, dbName, tableName, cm);
     }
 
     private void assertClientHasCapability(ClientCapabilities client,
@@ -2668,14 +2934,16 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       List<String> tables = null;
       startFunction("get_table_names_by_filter", ": db = " + dbName + ", filter = " + filter);
       Exception ex = null;
+      String[] parsedDbName = parseDbName(dbName, conf);
       try {
-        if (dbName == null || dbName.isEmpty()) {
+        if (parsedDbName[CAT_NAME] == null || parsedDbName[CAT_NAME].isEmpty() ||
+            parsedDbName[DB_NAME] == null || parsedDbName[DB_NAME].isEmpty()) {
           throw new UnknownDBException("DB name is null or empty");
         }
         if (filter == null) {
           throw new InvalidOperationException(filter + " cannot apply null filter");
         }
-        tables = getMS().listTableNamesByFilter(dbName, filter, maxTables);
+        tables = getMS().listTableNamesByFilter(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], filter, maxTables);
       } catch (Exception e) {
         ex = e;
         if (e instanceof MetaException) {
@@ -2693,9 +2961,10 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       return tables;
     }
 
-    private Partition append_partition_common(RawStore ms, String dbName, String tableName,
-        List<String> part_vals, EnvironmentContext envContext) throws InvalidObjectException,
-        AlreadyExistsException, MetaException {
+    private Partition append_partition_common(RawStore ms, String catName, String dbName,
+                                              String tableName, List<String> part_vals,
+                                              EnvironmentContext envContext)
+        throws InvalidObjectException, AlreadyExistsException, MetaException {
 
       Partition part = new Partition();
       boolean success = false, madeDir = false;
@@ -2704,13 +2973,14 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       Map<String, String> transactionalListenerResponses = Collections.emptyMap();
       try {
         ms.openTransaction();
+        part.setCatName(catName);
         part.setDbName(dbName);
         part.setTableName(tableName);
         part.setValues(part_vals);
 
         MetaStoreUtils.validatePartitionNameCharacters(part_vals, partitionValidationPattern);
 
-        tbl = ms.getTable(part.getDbName(), part.getTableName());
+        tbl = ms.getTable(part.getCatName(), part.getDbName(), part.getTableName());
         if (tbl == null) {
           throw new InvalidObjectException(
               "Unable to add partition because table or database do not exist");
@@ -2729,7 +2999,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
 
         Partition old_part;
         try {
-          old_part = ms.getPartition(part.getDbName(), part
+          old_part = ms.getPartition(part.getCatName(), part.getDbName(), part
               .getTableName(), part.getValues());
         } catch (NoSuchObjectException e) {
           // this means there is no existing partition
@@ -2810,10 +3080,11 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     public Partition append_partition_with_environment_context(final String dbName,
         final String tableName, final List<String> part_vals, final EnvironmentContext envContext)
         throws InvalidObjectException, AlreadyExistsException, MetaException {
-      if (part_vals == null) {
+      if (part_vals == null || part_vals.isEmpty()) {
         throw new MetaException("The partition values must not be null.");
       }
-      startPartitionFunction("append_partition", dbName, tableName, part_vals);
+      String[] parsedDbName = parseDbName(dbName, conf);
+      startPartitionFunction("append_partition", parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tableName, part_vals);
       if (LOG.isDebugEnabled()) {
         for (String part : part_vals) {
           LOG.debug(part);
@@ -2823,7 +3094,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       Partition ret = null;
       Exception ex = null;
       try {
-        ret = append_partition_common(getMS(), dbName, tableName, part_vals, envContext);
+        ret = append_partition_common(getMS(), parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tableName, part_vals, envContext);
       } catch (Exception e) {
         ex = e;
         if (e instanceof MetaException) {
@@ -2930,7 +3201,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       }
     }
 
-    private List<Partition> add_partitions_core(final RawStore ms,
+    private List<Partition> add_partitions_core(final RawStore ms, String catName,
         String dbName, String tblName, List<Partition> parts, final boolean ifNotExists)
         throws TException {
       logInfo("add_partitions");
@@ -2944,10 +3215,11 @@ public class HiveMetaStore extends ThriftHiveMetastore {
 
       try {
         ms.openTransaction();
-        tbl = ms.getTable(dbName, tblName);
+        tbl = ms.getTable(catName, dbName, tblName);
         if (tbl == null) {
           throw new InvalidObjectException("Unable to add partitions because "
-              + "database or table " + dbName + "." + tblName + " does not exist");
+              + getCatalogQualifiedTableName(catName, dbName, tblName) +
+              " does not exist");
         }
 
         if (!parts.isEmpty()) {
@@ -2958,8 +3230,9 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         final Table table = tbl;
         for (final Partition part : parts) {
           if (!part.getTableName().equals(tblName) || !part.getDbName().equals(dbName)) {
-            throw new MetaException("Partition does not belong to target table "
-                + dbName + "." + tblName + ": " + part);
+            throw new MetaException("Partition does not belong to target table " +
+                getCatalogQualifiedTableName(catName, dbName, tblName) + ": " +
+                    part);
           }
 
           boolean shouldAdd = startAddPartition(ms, part, ifNotExists);
@@ -3018,7 +3291,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         }
 
         if (!newParts.isEmpty()) {
-          success = ms.addPartitions(dbName, tblName, newParts);
+          success = ms.addPartitions(catName, dbName, tblName, newParts);
         } else {
           success = true;
         }
@@ -3080,7 +3353,12 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         return result;
       }
       try {
-        List<Partition> parts = add_partitions_core(getMS(), request.getDbName(),
+        if (!request.isSetCatName()) request.setCatName(getDefaultCatalog(conf));
+        // Make sure all of the partitions have the catalog set as well
+        request.getParts().forEach(p -> {
+          if (!p.isSetCatName()) p.setCatName(getDefaultCatalog(conf));
+        });
+        List<Partition> parts = add_partitions_core(getMS(), request.getCatName(), request.getDbName(),
             request.getTblName(), request.getParts(), request.isIfNotExists());
         if (request.isNeedResult()) {
           result.setPartitions(parts);
@@ -3105,7 +3383,11 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       Exception ex = null;
       try {
         // Old API assumed all partitions belong to the same table; keep the same assumption
-        ret = add_partitions_core(getMS(), parts.get(0).getDbName(),
+        if (!parts.get(0).isSetCatName()) {
+          String defaultCat = getDefaultCatalog(conf);
+          for (Partition p : parts) p.setCatName(defaultCat);
+        }
+        ret = add_partitions_core(getMS(), parts.get(0).getCatName(), parts.get(0).getDbName(),
             parts.get(0).getTableName(), parts, false).size();
         assert ret == parts.size();
       } catch (Exception e) {
@@ -3137,12 +3419,21 @@ public class HiveMetaStore extends ThriftHiveMetastore {
 
       String dbName = partSpecs.get(0).getDbName();
       String tableName = partSpecs.get(0).getTableName();
+      // If the catalog name isn't set, we need to go through and set it.
+      String catName;
+      if (!partSpecs.get(0).isSetCatName()) {
+        catName = getDefaultCatalog(conf);
+        partSpecs.forEach(ps -> ps.setCatName(catName));
+      } else {
+        catName = partSpecs.get(0).getCatName();
+      }
 
-      return add_partitions_pspec_core(getMS(), dbName, tableName, partSpecs, false);
+      return add_partitions_pspec_core(getMS(), catName, dbName, tableName, partSpecs, false);
     }
 
-    private int add_partitions_pspec_core(
-        RawStore ms, String dbName, String tblName, List<PartitionSpec> partSpecs, boolean ifNotExists)
+    private int add_partitions_pspec_core(RawStore ms, String catName, String dbName,
+                                          String tblName, List<PartitionSpec> partSpecs,
+                                          boolean ifNotExists)
         throws TException {
       boolean success = false;
       // Ensures that the list doesn't have dups, and keeps track of directories we have created.
@@ -3154,7 +3445,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       Map<String, String> transactionalListenerResponses = Collections.emptyMap();
       try {
         ms.openTransaction();
-        tbl = ms.getTable(dbName, tblName);
+        tbl = ms.getTable(catName, dbName, tblName);
         if (tbl == null) {
           throw new InvalidObjectException("Unable to add partitions because "
               + "database or table " + dbName + "." + tblName + " does not exist");
@@ -3166,7 +3457,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         while(partitionIterator.hasNext()) {
           final Partition part = partitionIterator.getCurrent();
 
-          if (!part.getTableName().equals(tblName) || !part.getDbName().equals(dbName)) {
+          if (!part.getTableName().equalsIgnoreCase(tblName) || !part.getDbName().equalsIgnoreCase(dbName)) {
             throw new MetaException("Partition does not belong to target table "
                 + dbName + "." + tblName + ": " + part);
           }
@@ -3222,7 +3513,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
           throw new MetaException(e.getMessage());
         }
 
-        success = ms.addPartitions(dbName, tblName, partitionSpecProxy, ifNotExists);
+        success = ms.addPartitions(catName, dbName, tblName, partitionSpecProxy, ifNotExists);
         //setting success to false to make sure that if the listener fails, rollback happens.
         success = false;
 
@@ -3260,7 +3551,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         RawStore ms, Partition part, boolean ifNotExists) throws TException {
       MetaStoreUtils.validatePartitionNameCharacters(part.getValues(),
           partitionValidationPattern);
-      boolean doesExist = ms.doesPartitionExist(
+      boolean doesExist = ms.doesPartitionExist(part.getCatName(),
           part.getDbName(), part.getTableName(), part.getValues());
       if (doesExist && !ifNotExists) {
         throw new AlreadyExistsException("Partition already exists: " + part);
@@ -3358,9 +3649,10 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       boolean success = false;
       Table tbl = null;
       Map<String, String> transactionalListenerResponses = Collections.emptyMap();
+      if (!part.isSetCatName()) part.setCatName(getDefaultCatalog(conf));
       try {
         ms.openTransaction();
-        tbl = ms.getTable(part.getDbName(), part.getTableName());
+        tbl = ms.getTable(part.getCatName(), part.getDbName(), part.getTableName());
         if (tbl == null) {
           throw new InvalidObjectException(
               "Unable to add partition because table or database do not exist");
@@ -3424,7 +3716,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         throws InvalidObjectException, AlreadyExistsException,
         MetaException {
       startTableFunction("add_partition",
-          part.getDbName(), part.getTableName());
+          part.getCatName(), part.getDbName(), part.getTableName());
       Partition ret = null;
       Exception ex = null;
       try {
@@ -3451,6 +3743,8 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         String sourceDbName, String sourceTableName, String destDbName,
         String destTableName) throws TException {
       exchange_partitions(partitionSpecs, sourceDbName, sourceTableName, destDbName, destTableName);
+      // Wouldn't it make more sense to return the first element of the list returned by the
+      // previous call?
       return new Partition();
     }
 
@@ -3458,24 +3752,36 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     public List<Partition> exchange_partitions(Map<String, String> partitionSpecs,
         String sourceDbName, String sourceTableName, String destDbName,
         String destTableName) throws TException {
-      if (partitionSpecs == null || sourceDbName == null || sourceTableName == null
-          || destDbName == null || destTableName == null) {
+      String[] parsedDestDbName = parseDbName(destDbName, conf);
+      String[] parsedSourceDbName = parseDbName(sourceDbName, conf);
+      // No need to check catalog for null as parseDbName() will never return null for the catalog.
+      if (partitionSpecs == null || parsedSourceDbName[DB_NAME] == null || sourceTableName == null
+          || parsedDestDbName[DB_NAME] == null || destTableName == null) {
         throw new MetaException("The DB and table name for the source and destination tables,"
             + " and the partition specs must not be null.");
       }
+      if (!parsedDestDbName[CAT_NAME].equals(parsedSourceDbName[CAT_NAME])) {
+        throw new MetaException("You cannot move a partition across catalogs");
+      }
+
       boolean success = false;
       boolean pathCreated = false;
       RawStore ms = getMS();
       ms.openTransaction();
-      Table destinationTable = ms.getTable(destDbName, destTableName);
+
+      Table destinationTable =
+          ms.getTable(parsedDestDbName[CAT_NAME], parsedDestDbName[DB_NAME], destTableName);
       if (destinationTable == null) {
-        throw new MetaException(
-            "The destination table " + destDbName + "." + destTableName + " not found");
+        throw new MetaException( "The destination table " +
+            getCatalogQualifiedTableName(parsedDestDbName[CAT_NAME],
+                parsedDestDbName[DB_NAME], destTableName) + " not found");
       }
-      Table sourceTable = ms.getTable(sourceDbName, sourceTableName);
+      Table sourceTable =
+          ms.getTable(parsedSourceDbName[CAT_NAME], parsedSourceDbName[DB_NAME], sourceTableName);
       if (sourceTable == null) {
-        throw new MetaException(
-            "The source table " + sourceDbName + "." + sourceTableName + " not found");
+        throw new MetaException("The source table " +
+            getCatalogQualifiedTableName(parsedSourceDbName[CAT_NAME],
+                parsedSourceDbName[DB_NAME], sourceTableName) + " not found");
       }
       List<String> partVals = MetaStoreUtils.getPvals(sourceTable.getPartitionKeys(),
           partitionSpecs);
@@ -3490,6 +3796,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         }
         i++;
       }
+      // Passed the unparsed DB name here, as get_partitions_ps expects to parse it
       List<Partition> partitionsToExchange = get_partitions_ps(sourceDbName, sourceTableName,
           partVals, (short)-1);
       if (partitionsToExchange == null || partitionsToExchange.isEmpty()) {
@@ -3515,8 +3822,8 @@ public class HiveMetaStore extends ThriftHiveMetastore {
           Lists.newArrayListWithCapacity(partitionsToExchange.size());
 
       // Check if any of the partitions already exists in destTable.
-      List<String> destPartitionNames =
-          ms.listPartitionNames(destDbName, destTableName, (short) -1);
+      List<String> destPartitionNames = ms.listPartitionNames(parsedDestDbName[CAT_NAME],
+          parsedDestDbName[DB_NAME], destTableName, (short) -1);
       if (destPartitionNames != null && !destPartitionNames.isEmpty()) {
         for (Partition partition : partitionsToExchange) {
           String partToExchangeName =
@@ -3531,14 +3838,14 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       try {
         for (Partition partition: partitionsToExchange) {
           Partition destPartition = new Partition(partition);
-          destPartition.setDbName(destDbName);
+          destPartition.setDbName(parsedDestDbName[DB_NAME]);
           destPartition.setTableName(destinationTable.getTableName());
           Path destPartitionPath = new Path(destinationTable.getSd().getLocation(),
               Warehouse.makePartName(destinationTable.getPartitionKeys(), partition.getValues()));
           destPartition.getSd().setLocation(destPartitionPath.toString());
           ms.addPartition(destPartition);
           destPartitions.add(destPartition);
-          ms.dropPartition(partition.getDbName(), sourceTable.getTableName(),
+          ms.dropPartition(parsedSourceDbName[CAT_NAME], partition.getDbName(), sourceTable.getTableName(),
             partition.getValues());
         }
         Path destParentPath = destPath.getParent();
@@ -3610,9 +3917,10 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       }
     }
 
-    private boolean drop_partition_common(RawStore ms, String db_name, String tbl_name,
-      List<String> part_vals, final boolean deleteData, final EnvironmentContext envContext)
-      throws MetaException, NoSuchObjectException, IOException, InvalidObjectException,
+    private boolean drop_partition_common(RawStore ms, String catName, String db_name,
+                                          String tbl_name, List<String> part_vals,
+                                          final boolean deleteData, final EnvironmentContext envContext)
+        throws MetaException, NoSuchObjectException, IOException, InvalidObjectException,
       InvalidInputException {
       boolean success = false;
       Path partPath = null;
@@ -3636,8 +3944,8 @@ public class HiveMetaStore extends ThriftHiveMetastore {
 
       try {
         ms.openTransaction();
-        part = ms.getPartition(db_name, tbl_name, part_vals);
-        tbl = get_table_core(db_name, tbl_name);
+        part = ms.getPartition(catName, db_name, tbl_name, part_vals);
+        tbl = get_table_core(catName, db_name, tbl_name);
         isExternalTbl = isExternal(tbl);
         firePreEvent(new PreDropPartitionEvent(tbl, part, deleteData, this));
         mustPurge = isMustPurge(envContext, tbl);
@@ -3658,7 +3966,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
           verifyIsWritablePath(partPath);
         }
 
-        if (!ms.dropPartition(db_name, tbl_name, part_vals)) {
+        if (!ms.dropPartition(catName, db_name, tbl_name, part_vals)) {
           throw new MetaException("Unable to drop partition");
         } else {
           if (!transactionalListeners.isEmpty()) {
@@ -3749,6 +4057,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         DropPartitionsRequest request) throws TException {
       RawStore ms = getMS();
       String dbName = request.getDbName(), tblName = request.getTblName();
+      String catName = request.isSetCatName() ? request.getCatName() : getDefaultCatalog(conf);
       boolean ifExists = request.isSetIfExists() && request.isIfExists();
       boolean deleteData = request.isSetDeleteData() && request.isDeleteData();
       boolean ignoreProtection = request.isSetIgnoreProtection() && request.isIgnoreProtection();
@@ -3768,7 +4077,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       try {
         // We need Partition-s for firing events and for result; DN needs MPartition-s to drop.
         // Great... Maybe we could bypass fetching MPartitions by issuing direct SQL deletes.
-        tbl = get_table_core(dbName, tblName);
+        tbl = get_table_core(catName, dbName, tblName);
         isExternal(tbl);
         mustPurge = isMustPurge(envContext, tbl);
         int minCount = 0;
@@ -3781,7 +4090,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
             ++minCount; // At least one partition per expression, if not ifExists
             List<Partition> result = new ArrayList<>();
             boolean hasUnknown = ms.getPartitionsByExpr(
-                dbName, tblName, expr.getExpr(), null, (short)-1, result);
+                catName, dbName, tblName, expr.getExpr(), null, (short)-1, result);
             if (hasUnknown) {
               // Expr is built by DDLSA, it should only contain part cols and simple ops
               throw new MetaException("Unexpected unknown partitions to drop");
@@ -3802,7 +4111,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         } else if (spec.isSetNames()) {
           partNames = spec.getNames();
           minCount = partNames.size();
-          parts = ms.getPartitionsByNames(dbName, tblName, partNames);
+          parts = ms.getPartitionsByNames(catName, dbName, tblName, partNames);
         } else {
           throw new MetaException("Partition spec is not set");
         }
@@ -3843,7 +4152,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
           }
         }
 
-        ms.dropPartitions(dbName, tblName, partNames);
+        ms.dropPartitions(catName, dbName, tblName, partNames);
         if (parts != null && !transactionalListeners.isEmpty()) {
           for (Partition part : parts) {
             transactionalListenerResponses.add(
@@ -3921,13 +4230,16 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         final String tbl_name, final List<String> part_vals, final boolean deleteData,
         final EnvironmentContext envContext)
         throws TException {
-      startPartitionFunction("drop_partition", db_name, tbl_name, part_vals);
+      String[] parsedDbName = parseDbName(db_name, conf);
+      startPartitionFunction("drop_partition", parsedDbName[CAT_NAME], parsedDbName[DB_NAME],
+          tbl_name, part_vals);
       LOG.info("Partition values:" + part_vals);
 
       boolean ret = false;
       Exception ex = null;
       try {
-        ret = drop_partition_common(getMS(), db_name, tbl_name, part_vals, deleteData, envContext);
+        ret = drop_partition_common(getMS(), parsedDbName[CAT_NAME], parsedDbName[DB_NAME],
+            tbl_name, part_vals, deleteData, envContext);
       } catch (IOException e) {
         ex = e;
         throw new MetaException(e.getMessage());
@@ -3944,13 +4256,15 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     @Override
     public Partition get_partition(final String db_name, final String tbl_name,
         final List<String> part_vals) throws MetaException, NoSuchObjectException {
-      startPartitionFunction("get_partition", db_name, tbl_name, part_vals);
+      String[] parsedDbName = parseDbName(db_name, conf);
+      startPartitionFunction("get_partition", parsedDbName[CAT_NAME], parsedDbName[DB_NAME],
+          tbl_name, part_vals);
 
       Partition ret = null;
       Exception ex = null;
       try {
-        fireReadTablePreEvent(db_name, tbl_name);
-        ret = getMS().getPartition(db_name, tbl_name, part_vals);
+        fireReadTablePreEvent(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name);
+        ret = getMS().getPartition(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name, part_vals);
       } catch (Exception e) {
         ex = e;
         throwMetaException(e);
@@ -3963,19 +4277,15 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     /**
      * Fire a pre-event for read table operation, if there are any
      * pre-event listeners registered
-     *
-     * @param dbName
-     * @param tblName
-     * @throws MetaException
-     * @throws NoSuchObjectException
      */
-    private void fireReadTablePreEvent(String dbName, String tblName) throws MetaException, NoSuchObjectException {
+    private void fireReadTablePreEvent(String catName, String dbName, String tblName)
+        throws MetaException, NoSuchObjectException {
       if(preListeners.size() > 0) {
         // do this only if there is a pre event listener registered (avoid unnecessary
         // metastore api call)
-        Table t = getMS().getTable(dbName, tblName);
+        Table t = getMS().getTable(catName, dbName, tblName);
         if (t == null) {
-          throw new NoSuchObjectException(dbName + "." + tblName
+          throw new NoSuchObjectException(getCatalogQualifiedTableName(catName, dbName, tblName)
               + " table not found");
         }
         firePreEvent(new PreReadTableEvent(t, this));
@@ -3987,14 +4297,15 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         final String tbl_name, final List<String> part_vals,
         final String user_name, final List<String> group_names)
         throws TException {
-      startPartitionFunction("get_partition_with_auth", db_name, tbl_name,
-          part_vals);
-      fireReadTablePreEvent(db_name, tbl_name);
+      String[] parsedDbName = parseDbName(db_name, conf);
+      startPartitionFunction("get_partition_with_auth", parsedDbName[CAT_NAME],
+          parsedDbName[DB_NAME], tbl_name, part_vals);
+      fireReadTablePreEvent(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name);
       Partition ret = null;
       Exception ex = null;
       try {
-        ret = getMS().getPartitionWithAuth(db_name, tbl_name, part_vals,
-            user_name, group_names);
+        ret = getMS().getPartitionWithAuth(parsedDbName[CAT_NAME], parsedDbName[DB_NAME],
+            tbl_name, part_vals, user_name, group_names);
       } catch (InvalidObjectException e) {
         ex = e;
         throw new NoSuchObjectException(e.getMessage());
@@ -4010,13 +4321,16 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     @Override
     public List<Partition> get_partitions(final String db_name, final String tbl_name,
         final short max_parts) throws NoSuchObjectException, MetaException {
-      startTableFunction("get_partitions", db_name, tbl_name);
-      fireReadTablePreEvent(db_name, tbl_name);
+      String[] parsedDbName = parseDbName(db_name, conf);
+      startTableFunction("get_partitions", parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name);
+      fireReadTablePreEvent(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name);
       List<Partition> ret = null;
       Exception ex = null;
       try {
-        checkLimitNumberOfPartitionsByFilter(db_name, tbl_name, NO_FILTER_STRING, max_parts);
-        ret = getMS().getPartitions(db_name, tbl_name, max_parts);
+        checkLimitNumberOfPartitionsByFilter(parsedDbName[CAT_NAME], parsedDbName[DB_NAME],
+            tbl_name, NO_FILTER_STRING, max_parts);
+        ret = getMS().getPartitions(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name,
+            max_parts);
       } catch (Exception e) {
         ex = e;
         throwMetaException(e);
@@ -4031,14 +4345,16 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     public List<Partition> get_partitions_with_auth(final String dbName,
         final String tblName, final short maxParts, final String userName,
         final List<String> groupNames) throws TException {
-      startTableFunction("get_partitions_with_auth", dbName, tblName);
+      String[] parsedDbName = parseDbName(dbName, conf);
+      startTableFunction("get_partitions_with_auth", parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tblName);
 
       List<Partition> ret = null;
       Exception ex = null;
       try {
-        checkLimitNumberOfPartitionsByFilter(dbName, tblName, NO_FILTER_STRING, maxParts);
-        ret = getMS().getPartitionsWithAuth(dbName, tblName, maxParts,
-            userName, groupNames);
+        checkLimitNumberOfPartitionsByFilter(parsedDbName[CAT_NAME], parsedDbName[DB_NAME],
+            tblName, NO_FILTER_STRING, maxParts);
+        ret = getMS().getPartitionsWithAuth(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tblName,
+            maxParts, userName, groupNames);
       } catch (InvalidObjectException e) {
         ex = e;
         throw new NoSuchObjectException(e.getMessage());
@@ -4052,15 +4368,21 @@ public class HiveMetaStore extends ThriftHiveMetastore {
 
     }
 
-    private void checkLimitNumberOfPartitionsByFilter(String dbName, String tblName, String filterString, int maxParts) throws TException {
+    private void checkLimitNumberOfPartitionsByFilter(String catName, String dbName,
+                                                      String tblName, String filterString,
+                                                      int maxParts) throws TException {
       if (isPartitionLimitEnabled()) {
-        checkLimitNumberOfPartitions(tblName, get_num_partitions_by_filter(dbName, tblName, filterString), maxParts);
+        checkLimitNumberOfPartitions(tblName, get_num_partitions_by_filter(prependCatalogToDbName(
+            catName, dbName, conf), tblName, filterString), maxParts);
       }
     }
 
-    private void checkLimitNumberOfPartitionsByExpr(String dbName, String tblName, byte[] filterExpr, int maxParts) throws TException {
+    private void checkLimitNumberOfPartitionsByExpr(String catName, String dbName, String tblName,
+                                                    byte[] filterExpr, int maxParts)
+        throws TException {
       if (isPartitionLimitEnabled()) {
-        checkLimitNumberOfPartitions(tblName, get_num_partitions_by_expr(dbName, tblName, filterExpr), maxParts);
+        checkLimitNumberOfPartitions(tblName, get_num_partitions_by_expr(catName, dbName, tblName,
+            filterExpr), maxParts);
       }
     }
 
@@ -4085,15 +4407,16 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     public List<PartitionSpec> get_partitions_pspec(final String db_name, final String tbl_name, final int max_parts)
       throws NoSuchObjectException, MetaException  {
 
-      String dbName = db_name.toLowerCase();
+      String[] parsedDbName = parseDbName(db_name, conf);
       String tableName = tbl_name.toLowerCase();
 
-      startTableFunction("get_partitions_pspec", dbName, tableName);
+      startTableFunction("get_partitions_pspec", parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tableName);
 
       List<PartitionSpec> partitionSpecs = null;
       try {
-        Table table = get_table_core(dbName, tableName);
-        List<Partition> partitions = get_partitions(dbName, tableName, (short) max_parts);
+        Table table = get_table_core(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tableName);
+        // get_partitions will parse out the catalog and db names itself
+        List<Partition> partitions = get_partitions(db_name, tableName, (short) max_parts);
 
         if (is_partition_spec_grouping_enabled(table)) {
           partitionSpecs = get_partitionspecs_grouped_by_storage_descriptor(table, partitions);
@@ -4101,7 +4424,8 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         else {
           PartitionSpec pSpec = new PartitionSpec();
           pSpec.setPartitionList(new PartitionListComposingSpec(partitions));
-          pSpec.setDbName(dbName);
+          pSpec.setCatName(parsedDbName[CAT_NAME]);
+          pSpec.setDbName(parsedDbName[DB_NAME]);
           pSpec.setTableName(tableName);
           pSpec.setRootPath(table.getSd().getLocation());
           partitionSpecs = Arrays.asList(pSpec);
@@ -4241,12 +4565,14 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     @Override
     public List<String> get_partition_names(final String db_name, final String tbl_name,
         final short max_parts) throws NoSuchObjectException, MetaException {
-      startTableFunction("get_partition_names", db_name, tbl_name);
-      fireReadTablePreEvent(db_name, tbl_name);
+      String[] parsedDbName = parseDbName(db_name, conf);
+      startTableFunction("get_partition_names", parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name);
+      fireReadTablePreEvent(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name);
       List<String> ret = null;
       Exception ex = null;
       try {
-        ret = getMS().listPartitionNames(db_name, tbl_name, max_parts);
+        ret = getMS().listPartitionNames(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name,
+            max_parts);
       } catch (Exception e) {
         ex = e;
         if (e instanceof MetaException) {
@@ -4262,11 +4588,14 @@ public class HiveMetaStore extends ThriftHiveMetastore {
 
     @Override
     public PartitionValuesResponse get_partition_values(PartitionValuesRequest request) throws MetaException {
+      String catName = request.isSetCatName() ? request.getCatName() : getDefaultCatalog(conf);
       String dbName = request.getDbName();
       String tblName = request.getTblName();
+      // This is serious black magic, as the following 2 lines do nothing AFAICT but without them
+      // the subsequent call to listPartitionValues fails.
       List<FieldSchema> partCols = new ArrayList<FieldSchema>();
       partCols.add(request.getPartitionKeys().get(0));
-      return getMS().listPartitionValues(dbName, tblName, request.getPartitionKeys(),
+      return getMS().listPartitionValues(catName, dbName, tblName, request.getPartitionKeys(),
           request.isApplyDistinct(), request.getFilter(), request.isAscending(),
           request.getPartitionOrder(), request.getMaxParts());
     }
@@ -4283,8 +4612,9 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         final String tableName, final Partition newPartition,
         final EnvironmentContext envContext)
         throws TException {
-      rename_partition(dbName, tableName, null,
-          newPartition, envContext);
+      String[] parsedDbName = parseDbName(dbName, conf);
+      rename_partition(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tableName, null, newPartition,
+          envContext);
     }
 
     @Override
@@ -4292,14 +4622,16 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         final List<String> part_vals, final Partition new_part)
         throws TException {
       // Call rename_partition without an environment context.
-      rename_partition(db_name, tbl_name, part_vals, new_part, null);
+      String[] parsedDbName = parseDbName(db_name, conf);
+      rename_partition(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name, part_vals, new_part,
+          null);
     }
 
-    private void rename_partition(final String db_name, final String tbl_name,
+    private void rename_partition(final String catName, final String db_name, final String tbl_name,
         final List<String> part_vals, final Partition new_part,
         final EnvironmentContext envContext)
         throws TException {
-      startTableFunction("alter_partition", db_name, tbl_name);
+      startTableFunction("alter_partition", catName, db_name, tbl_name);
 
       if (LOG.isInfoEnabled()) {
         LOG.info("New partition values:" + new_part.getValues());
@@ -4317,6 +4649,9 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         }
       }
 
+      // Make sure the new partition has the catalog value set
+      if (!new_part.isSetCatName()) new_part.setCatName(catName);
+
       Partition oldPart = null;
       Exception ex = null;
       try {
@@ -4326,14 +4661,14 @@ public class HiveMetaStore extends ThriftHiveMetastore {
               partitionValidationPattern);
         }
 
-        oldPart = alterHandler.alterPartition(getMS(), wh, db_name, tbl_name, part_vals, new_part,
-                envContext, this);
+        oldPart = alterHandler.alterPartition(getMS(), wh, catName, db_name, tbl_name,
+            part_vals, new_part, envContext, this);
 
         // Only fetch the table if we actually have a listener
         Table table = null;
         if (!listeners.isEmpty()) {
           if (table == null) {
-            table = getMS().getTable(db_name, tbl_name);
+            table = getMS().getTable(catName, db_name, tbl_name);
           }
 
           MetaStoreListenerNotifier.notifyEvent(listeners,
@@ -4373,7 +4708,8 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         final List<Partition> new_parts, EnvironmentContext environmentContext)
         throws TException {
 
-      startTableFunction("alter_partitions", db_name, tbl_name);
+      String[] parsedDbName = parseDbName(db_name, conf);
+      startTableFunction("alter_partitions", parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name);
 
       if (LOG.isInfoEnabled()) {
         for (Partition tmpPart : new_parts) {
@@ -4386,10 +4722,12 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       Exception ex = null;
       try {
         for (Partition tmpPart : new_parts) {
-          firePreEvent(new PreAlterPartitionEvent(db_name, tbl_name, null, tmpPart, this));
+          // Make sure the catalog name is set in the new partition
+          if (!tmpPart.isSetCatName()) tmpPart.setCatName(getDefaultCatalog(conf));
+          firePreEvent(new PreAlterPartitionEvent(parsedDbName[DB_NAME], tbl_name, null, tmpPart, this));
         }
-        oldParts = alterHandler.alterPartitions(getMS(), wh, db_name, tbl_name, new_parts,
-                environmentContext, this);
+        oldParts = alterHandler.alterPartitions(getMS(), wh, parsedDbName[CAT_NAME],
+            parsedDbName[DB_NAME], tbl_name, new_parts, environmentContext, this);
         Iterator<Partition> olditr = oldParts.iterator();
         // Only fetch the table if we have a listener that needs it.
         Table table = null;
@@ -4403,7 +4741,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
           }
 
           if (table == null) {
-            table = getMS().getTable(db_name, tbl_name);
+            table = getMS().getTable(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name);
           }
 
           if (!listeners.isEmpty()) {
@@ -4443,7 +4781,8 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         final Table newTable)
         throws InvalidOperationException, MetaException {
       // Do not set an environment context.
-      alter_table_core(dbname,name, newTable, null);
+      String[] parsedDbName = parseDbName(dbname, conf);
+      alter_table_core(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], name, newTable, null);
     }
 
     @Override
@@ -4455,7 +4794,8 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         envContext = new EnvironmentContext();
         envContext.putToProperties(StatsSetupConst.CASCADE, StatsSetupConst.TRUE);
       }
-      alter_table_core(dbname, name, newTable, envContext);
+      String[] parsedDbName = parseDbName(dbname, conf);
+      alter_table_core(parsedDbName[CAT_NAME], 

<TRUNCATED>

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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MaterializationsInvalidationCache.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MaterializationsInvalidationCache.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MaterializationsInvalidationCache.java
index 1636d48..80cb1de 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MaterializationsInvalidationCache.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MaterializationsInvalidationCache.java
@@ -130,10 +130,13 @@ public final class MaterializationsInvalidationCache {
     public void run() {
       try {
         RawStore store = handler.getMS();
-        for (String dbName : store.getAllDatabases()) {
-          for (Table mv : store.getTableObjectsByName(dbName, store.getTables(dbName, null, TableType.MATERIALIZED_VIEW))) {
-            addMaterializedView(mv.getDbName(), mv.getTableName(), ImmutableSet.copyOf(mv.getCreationMetadata().getTablesUsed()),
-                mv.getCreationMetadata().getValidTxnList(), OpType.LOAD);
+        for (String catName : store.getCatalogs()) {
+          for (String dbName : store.getAllDatabases(catName)) {
+            for (Table mv : store.getTableObjectsByName(catName, dbName,
+                store.getTables(catName, dbName, null, TableType.MATERIALIZED_VIEW))) {
+              addMaterializedView(mv.getDbName(), mv.getTableName(), ImmutableSet.copyOf(mv.getCreationMetadata().getTablesUsed()),
+                  mv.getCreationMetadata().getValidTxnList(), OpType.LOAD);
+            }
           }
         }
         LOG.info("Initialized materializations invalidation cache");

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
index 0dcf117..997f5fd 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
@@ -19,7 +19,9 @@
 package org.apache.hadoop.hive.metastore;
 
 import static org.apache.commons.lang.StringUtils.join;
+import static org.apache.commons.lang.StringUtils.normalizeSpace;
 import static org.apache.commons.lang.StringUtils.repeat;
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
 
 import java.sql.Blob;
 import java.sql.Clob;
@@ -311,17 +313,19 @@ class MetaStoreDirectSql {
     }
   }
 
-  public Database getDatabase(String dbName) throws MetaException{
+  public Database getDatabase(String catName, String dbName) throws MetaException{
     Query queryDbSelector = null;
     Query queryDbParams = null;
     try {
       dbName = dbName.toLowerCase();
+      catName = catName.toLowerCase();
 
       String queryTextDbSelector= "select "
           + "\"DB_ID\", \"NAME\", \"DB_LOCATION_URI\", \"DESC\", "
-          + "\"OWNER_NAME\", \"OWNER_TYPE\" "
-          + "FROM "+ DBS +" where \"NAME\" = ? ";
-      Object[] params = new Object[] { dbName };
+          + "\"OWNER_NAME\", \"OWNER_TYPE\", \"CTLG_NAME\" "
+          + "FROM "+ DBS
+          + " where \"NAME\" = ? and \"CTLG_NAME\" = ? ";
+      Object[] params = new Object[] { dbName, catName };
       queryDbSelector = pm.newQuery("javax.jdo.query.SQL", queryTextDbSelector);
 
       if (LOG.isTraceEnabled()) {
@@ -370,6 +374,7 @@ class MetaStoreDirectSql {
       String type = extractSqlString(dbline[5]);
       db.setOwnerType(
           (null == type || type.trim().isEmpty()) ? null : PrincipalType.valueOf(type));
+      db.setCatalogName(extractSqlString(dbline[6]));
       db.setParameters(MetaStoreUtils.trimMapNulls(dbParams,convertMapNullsToEmptyStrings));
       if (LOG.isDebugEnabled()){
         LOG.debug("getDatabase: directsql returning db " + db.getName()
@@ -389,20 +394,22 @@ class MetaStoreDirectSql {
 
   /**
    * Get table names by using direct SQL queries.
-   *
+   * @param catName catalog name
    * @param dbName Metastore database namme
    * @param tableType Table type, or null if we want to get all tables
    * @return list of table names
    */
-  public List<String> getTables(String dbName, TableType tableType) throws MetaException {
+  public List<String> getTables(String catName, String dbName, TableType tableType)
+      throws MetaException {
     String queryText = "SELECT " + TBLS + ".\"TBL_NAME\""
       + " FROM " + TBLS + " "
       + " INNER JOIN " + DBS + " ON " + TBLS + ".\"DB_ID\" = " + DBS + ".\"DB_ID\" "
-      + " WHERE " + DBS + ".\"NAME\" = ? "
+      + " WHERE " + DBS + ".\"NAME\" = ? AND " + DBS + ".\"CTLG_NAME\" = ? "
       + (tableType == null ? "" : "AND " + TBLS + ".\"TBL_TYPE\" = ? ") ;
 
-    List<String> pms = new ArrayList<String>();
+    List<String> pms = new ArrayList<>();
     pms.add(dbName);
+    pms.add(catName);
     if (tableType != null) {
       pms.add(tableType.toString());
     }
@@ -436,13 +443,15 @@ class MetaStoreDirectSql {
   /**
    * Gets partitions by using direct SQL queries.
    * Note that batching is not needed for this method - list of names implies the batch size;
+   * @param catName Metastore catalog name.
    * @param dbName Metastore db name.
    * @param tblName Metastore table name.
    * @param partNames Partition names to get.
    * @return List of partitions.
    */
-  public List<Partition> getPartitionsViaSqlFilter(final String dbName, final String tblName,
-      List<String> partNames) throws MetaException {
+  public List<Partition> getPartitionsViaSqlFilter(final String catName, final String dbName,
+                                                   final String tblName, List<String> partNames)
+      throws MetaException {
     if (partNames.isEmpty()) {
       return Collections.emptyList();
     }
@@ -450,7 +459,7 @@ class MetaStoreDirectSql {
       @Override
       public List<Partition> run(List<String> input) throws MetaException {
         String filter = "" + PARTITIONS + ".\"PART_NAME\" in (" + makeParams(input.size()) + ")";
-        return getPartitionsViaSqlFilterInternal(dbName, tblName, null, filter, input,
+        return getPartitionsViaSqlFilterInternal(catName, dbName, tblName, null, filter, input,
             Collections.<String>emptyList(), null);
       }
     });
@@ -465,13 +474,15 @@ class MetaStoreDirectSql {
   public List<Partition> getPartitionsViaSqlFilter(
       SqlFilterForPushdown filter, Integer max) throws MetaException {
     Boolean isViewTable = isViewTable(filter.table);
-    return getPartitionsViaSqlFilterInternal(filter.table.getDbName(), filter.table.getTableName(),
-        isViewTable, filter.filter, filter.params, filter.joins, max);
+    String catName = filter.table.isSetCatName() ? filter.table.getCatName() :
+        DEFAULT_CATALOG_NAME;
+    return getPartitionsViaSqlFilterInternal(catName, filter.table.getDbName(),
+        filter.table.getTableName(), isViewTable, filter.filter, filter.params, filter.joins, max);
   }
 
   public static class SqlFilterForPushdown {
-    private final List<Object> params = new ArrayList<Object>();
-    private final List<String> joins = new ArrayList<String>();
+    private final List<Object> params = new ArrayList<>();
+    private final List<String> joins = new ArrayList<>();
     private String filter;
     private Table table;
   }
@@ -488,14 +499,15 @@ class MetaStoreDirectSql {
 
   /**
    * Gets all partitions of a table by using direct SQL queries.
+   * @param catName Metastore catalog name.
    * @param dbName Metastore db name.
    * @param tblName Metastore table name.
    * @param max The maximum number of partitions to return.
    * @return List of partitions.
    */
-  public List<Partition> getPartitions(
+  public List<Partition> getPartitions(String catName,
       String dbName, String tblName, Integer max) throws MetaException {
-    return getPartitionsViaSqlFilterInternal(dbName, tblName, null,
+    return getPartitionsViaSqlFilterInternal(catName, dbName, tblName, null,
         null, Collections.<String>emptyList(), Collections.<String>emptyList(), max);
   }
 
@@ -504,13 +516,13 @@ class MetaStoreDirectSql {
         t.getTableType().equals(TableType.VIRTUAL_VIEW.toString()) : null;
   }
 
-  private boolean isViewTable(String dbName, String tblName) throws MetaException {
+  private boolean isViewTable(String catName, String dbName, String tblName) throws MetaException {
     Query query = null;
     try {
       String queryText = "select \"TBL_TYPE\" from " + TBLS + "" +
           " inner join " + DBS + " on " + TBLS + ".\"DB_ID\" = " + DBS + ".\"DB_ID\" " +
-          " where " + TBLS + ".\"TBL_NAME\" = ? and " + DBS + ".\"NAME\" = ?";
-      Object[] params = new Object[] { tblName, dbName };
+          " where " + TBLS + ".\"TBL_NAME\" = ? and " + DBS + ".\"NAME\" = ? and " + DBS + ".\"CTLG_NAME\" = ?";
+      Object[] params = new Object[] { tblName, dbName, catName };
       query = pm.newQuery("javax.jdo.query.SQL", queryText);
       query.setUnique(true);
       Object result = executeWithArray(query, params, queryText);
@@ -536,11 +548,13 @@ class MetaStoreDirectSql {
    * @param max The maximum number of partitions to return.
    * @return List of partition objects.
    */
-  private List<Partition> getPartitionsViaSqlFilterInternal(String dbName, String tblName,
-      final Boolean isView, String sqlFilter, List<? extends Object> paramsForFilter,
-      List<String> joinsForFilter, Integer max) throws MetaException {
+  private List<Partition> getPartitionsViaSqlFilterInternal(
+      String catName, String dbName, String tblName, final Boolean isView, String sqlFilter,
+      List<? extends Object> paramsForFilter, List<String> joinsForFilter,Integer max)
+      throws MetaException {
     boolean doTrace = LOG.isDebugEnabled();
     final String dbNameLcase = dbName.toLowerCase(), tblNameLcase = tblName.toLowerCase();
+    final String catNameLcase = normalizeSpace(catName);
     // We have to be mindful of order during filtering if we are not returning all partitions.
     String orderForFilter = (max != null) ? " order by \"PART_NAME\" asc" : "";
 
@@ -559,12 +573,14 @@ class MetaStoreDirectSql {
       + "  inner join " + DBS + " on " + TBLS + ".\"DB_ID\" = " + DBS + ".\"DB_ID\" "
       + "     and " + DBS + ".\"NAME\" = ? "
       + join(joinsForFilter, ' ')
-      + (StringUtils.isBlank(sqlFilter) ? "" : (" where " + sqlFilter)) + orderForFilter;
-    Object[] params = new Object[paramsForFilter.size() + 2];
+      + " where " + DBS + ".\"CTLG_NAME\" = ? "
+      + (StringUtils.isBlank(sqlFilter) ? "" : (" and " + sqlFilter)) + orderForFilter;
+    Object[] params = new Object[paramsForFilter.size() + 3];
     params[0] = tblNameLcase;
     params[1] = dbNameLcase;
+    params[2] = catNameLcase;
     for (int i = 0; i < paramsForFilter.size(); ++i) {
-      params[i + 2] = paramsForFilter.get(i);
+      params[i + 3] = paramsForFilter.get(i);
     }
 
     long start = doTrace ? System.nanoTime() : 0;
@@ -583,7 +599,8 @@ class MetaStoreDirectSql {
     List<Partition> result = runBatched(sqlResult, new Batchable<Object, Partition>() {
       @Override
       public List<Partition> run(List<Object> input) throws MetaException {
-        return getPartitionsFromPartitionIds(dbNameLcase, tblNameLcase, isView, input);
+        return getPartitionsFromPartitionIds(catNameLcase, dbNameLcase, tblNameLcase, isView,
+            input);
       }
     });
 
@@ -592,7 +609,7 @@ class MetaStoreDirectSql {
   }
 
   /** Should be called with the list short enough to not trip up Oracle/etc. */
-  private List<Partition> getPartitionsFromPartitionIds(String dbName, String tblName,
+  private List<Partition> getPartitionsFromPartitionIds(String catName, String dbName, String tblName,
       Boolean isView, List<Object> partIdList) throws MetaException {
     boolean doTrace = LOG.isDebugEnabled();
     int idStringWidth = (int)Math.ceil(Math.log10(partIdList.size())) + 1; // 1 for comma
@@ -635,6 +652,7 @@ class MetaStoreDirectSql {
     StringBuilder colsSb = new StringBuilder(7); // We expect that there's only one field schema.
     tblName = tblName.toLowerCase();
     dbName = dbName.toLowerCase();
+    catName = catName.toLowerCase();
     for (Object[] fields : sqlResult) {
       // Here comes the ugly part...
       long partitionId = extractSqlLong(fields[0]);
@@ -644,7 +662,7 @@ class MetaStoreDirectSql {
       // A partition must have at least sdId and serdeId set, or nothing set if it's a view.
       if (sdId == null || serdeId == null) {
         if (isView == null) {
-          isView = isViewTable(dbName, tblName);
+          isView = isViewTable(catName, dbName, tblName);
         }
         if ((sdId != null || colId != null || serdeId != null) || !isView) {
           throw new MetaException("Unexpected null for one of the IDs, SD " + sdId +
@@ -655,8 +673,9 @@ class MetaStoreDirectSql {
       Partition part = new Partition();
       orderedResult.add(part);
       // Set the collection fields; some code might not check presence before accessing them.
-      part.setParameters(new HashMap<String, String>());
+      part.setParameters(new HashMap<>());
       part.setValues(new ArrayList<String>());
+      part.setCatName(catName);
       part.setDbName(dbName);
       part.setTableName(tblName);
       if (fields[4] != null) part.setCreateTime(extractSqlInt(fields[4]));
@@ -910,6 +929,7 @@ class MetaStoreDirectSql {
 
   public int getNumPartitionsViaSqlFilter(SqlFilterForPushdown filter) throws MetaException {
     boolean doTrace = LOG.isDebugEnabled();
+    String catName = filter.table.getCatName().toLowerCase();
     String dbName = filter.table.getDbName().toLowerCase();
     String tblName = filter.table.getTableName().toLowerCase();
 
@@ -920,13 +940,15 @@ class MetaStoreDirectSql {
       + "  inner join " + DBS + " on " + TBLS + ".\"DB_ID\" = " + DBS + ".\"DB_ID\" "
       + "     and " + DBS + ".\"NAME\" = ? "
       + join(filter.joins, ' ')
-      + (filter.filter == null || filter.filter.trim().isEmpty() ? "" : (" where " + filter.filter));
+      + " where " + DBS + ".\"CTLG_NAME\" = ? "
+      + (filter.filter == null || filter.filter.trim().isEmpty() ? "" : (" and " + filter.filter));
 
-    Object[] params = new Object[filter.params.size() + 2];
+    Object[] params = new Object[filter.params.size() + 3];
     params[0] = tblName;
     params[1] = dbName;
+    params[2] = catName;
     for (int i = 0; i < filter.params.size(); ++i) {
-      params[i + 2] = filter.params.get(i);
+      params[i + 3] = filter.params.get(i);
     }
 
     long start = doTrace ? System.nanoTime() : 0;
@@ -1291,10 +1313,12 @@ class MetaStoreDirectSql {
         if (dbHasJoinCastBug) {
           // This is a workaround for DERBY-6358 and Oracle bug; it is pretty horrible.
           tableValue += (" and " + TBLS + ".\"TBL_NAME\" = ? and " + DBS + ".\"NAME\" = ? and "
+              + DBS + ".\"CTLG_NAME\" = ? and "
               + "\"FILTER" + partColIndex + "\".\"PART_ID\" = " + PARTITIONS + ".\"PART_ID\" and "
                 + "\"FILTER" + partColIndex + "\".\"INTEGER_IDX\" = " + partColIndex);
           params.add(table.getTableName().toLowerCase());
           params.add(table.getDbName().toLowerCase());
+          params.add(table.getCatName().toLowerCase());
         }
         tableValue += " then " + tableValue0 + " else null end)";
       }
@@ -1311,29 +1335,32 @@ class MetaStoreDirectSql {
   /**
    * Retrieve the column statistics for the specified columns of the table. NULL
    * is returned if the columns are not provided.
+   * @param catName     the catalog name of the table
    * @param dbName      the database name of the table
    * @param tableName   the table name
    * @param colNames    the list of the column names
    * @return            the column statistics for the specified columns
    * @throws MetaException
    */
-  public ColumnStatistics getTableStats(final String dbName, final String tableName,
-      List<String> colNames, boolean enableBitVector) throws MetaException {
+  public ColumnStatistics getTableStats(final String catName, final String dbName,
+                                        final String tableName, List<String> colNames,
+                                        boolean enableBitVector) throws MetaException {
     if (colNames == null || colNames.isEmpty()) {
       return null;
     }
     final boolean doTrace = LOG.isDebugEnabled();
-    final String queryText0 = "select " + getStatsList(enableBitVector) + " from " + TAB_COL_STATS + " "
-          + " where \"DB_NAME\" = ? and \"TABLE_NAME\" = ? and \"COLUMN_NAME\" in (";
+    final String queryText0 = "select " + getStatsList(enableBitVector) + " from " + TAB_COL_STATS
+          + " where \"CAT_NAME\" = ? and \"DB_NAME\" = ? and \"TABLE_NAME\" = ? and \"COLUMN_NAME\" in (";
     Batchable<String, Object[]> b = new Batchable<String, Object[]>() {
       @Override
       public List<Object[]> run(List<String> input) throws MetaException {
         String queryText = queryText0 + makeParams(input.size()) + ")";
-        Object[] params = new Object[input.size() + 2];
-        params[0] = dbName;
-        params[1] = tableName;
+        Object[] params = new Object[input.size() + 3];
+        params[0] = catName;
+        params[1] = dbName;
+        params[2] = tableName;
         for (int i = 0; i < input.size(); ++i) {
-          params[i + 2] = input.get(i);
+          params[i + 3] = input.get(i);
         }
         long start = doTrace ? System.nanoTime() : 0;
         Query query = pm.newQuery("javax.jdo.query.SQL", queryText);
@@ -1352,12 +1379,13 @@ class MetaStoreDirectSql {
       return null;
     }
     ColumnStatisticsDesc csd = new ColumnStatisticsDesc(true, dbName, tableName);
+    csd.setCatName(catName);
     ColumnStatistics result = makeColumnStats(list, csd, 0);
     b.closeAllQueries();
     return result;
   }
 
-  public AggrStats aggrColStatsForPartitions(String dbName, String tableName,
+  public AggrStats aggrColStatsForPartitions(String catName, String dbName, String tableName,
       List<String> partNames, List<String> colNames, boolean useDensityFunctionForNDVEstimation,
       double ndvTuner, boolean enableBitVector) throws MetaException {
     if (colNames.isEmpty() || partNames.isEmpty()) {
@@ -1379,33 +1407,33 @@ class MetaStoreDirectSql {
       boolean computePartsFound = true;
       for (String colName : colNames) {
         // Check the cache first
-        colStatsAggrCached = aggrStatsCache.get(dbName, tableName, colName, partNames);
+        colStatsAggrCached = aggrStatsCache.get(catName, dbName, tableName, colName, partNames);
         if (colStatsAggrCached != null) {
           colStatsList.add(colStatsAggrCached.getColStats());
           partsFound = colStatsAggrCached.getNumPartsCached();
         } else {
           if (computePartsFound) {
-            partsFound = partsFoundForPartitions(dbName, tableName, partNames, colNames);
+            partsFound = partsFoundForPartitions(catName, dbName, tableName, partNames, colNames);
             computePartsFound = false;
           }
-          List<String> colNamesForDB = new ArrayList<String>();
+          List<String> colNamesForDB = new ArrayList<>();
           colNamesForDB.add(colName);
           // Read aggregated stats for one column
           colStatsAggrFromDB =
-              columnStatisticsObjForPartitions(dbName, tableName, partNames, colNamesForDB,
+              columnStatisticsObjForPartitions(catName, dbName, tableName, partNames, colNamesForDB,
                   partsFound, useDensityFunctionForNDVEstimation, ndvTuner, enableBitVector);
           if (!colStatsAggrFromDB.isEmpty()) {
             ColumnStatisticsObj colStatsAggr = colStatsAggrFromDB.get(0);
             colStatsList.add(colStatsAggr);
             // Update the cache to add this new aggregate node
-            aggrStatsCache.add(dbName, tableName, colName, partsFound, colStatsAggr, bloomFilter);
+            aggrStatsCache.add(catName, dbName, tableName, colName, partsFound, colStatsAggr, bloomFilter);
           }
         }
       }
     } else {
-      partsFound = partsFoundForPartitions(dbName, tableName, partNames, colNames);
+      partsFound = partsFoundForPartitions(catName, dbName, tableName, partNames, colNames);
       colStatsList =
-          columnStatisticsObjForPartitions(dbName, tableName, partNames, colNames, partsFound,
+          columnStatisticsObjForPartitions(catName, dbName, tableName, partNames, colNames, partsFound,
               useDensityFunctionForNDVEstimation, ndvTuner, enableBitVector);
     }
     LOG.info("useDensityFunctionForNDVEstimation = " + useDensityFunctionForNDVEstimation
@@ -1423,12 +1451,13 @@ class MetaStoreDirectSql {
     return bloomFilter;
   }
 
-  private long partsFoundForPartitions(final String dbName, final String tableName,
+  private long partsFoundForPartitions(
+      final String catName, final String dbName, final String tableName,
       final List<String> partNames, List<String> colNames) throws MetaException {
     assert !colNames.isEmpty() && !partNames.isEmpty();
     final boolean doTrace = LOG.isDebugEnabled();
     final String queryText0  = "select count(\"COLUMN_NAME\") from " + PART_COL_STATS + ""
-        + " where \"DB_NAME\" = ? and \"TABLE_NAME\" = ? "
+        + " where \"CAT_NAME\" = ? and \"DB_NAME\" = ? and \"TABLE_NAME\" = ? "
         + " and \"COLUMN_NAME\" in (%1$s) and \"PARTITION_NAME\" in (%2$s)"
         + " group by \"PARTITION_NAME\"";
     List<Long> allCounts = runBatched(colNames, new Batchable<String, Long>() {
@@ -1444,7 +1473,7 @@ class MetaStoreDirectSql {
             Query query = pm.newQuery("javax.jdo.query.SQL", queryText);
             try {
               Object qResult = executeWithArray(query, prepareParams(
-                  dbName, tableName, inputPartNames, inputColName), queryText);
+                  catName, dbName, tableName, inputPartNames, inputColName), queryText);
               long end = doTrace ? System.nanoTime() : 0;
               timingTrace(doTrace, queryText, start, end);
               ForwardQueryResult<?> fqr = (ForwardQueryResult<?>) qResult;
@@ -1469,7 +1498,8 @@ class MetaStoreDirectSql {
     return partsFound;
   }
 
-  private List<ColumnStatisticsObj> columnStatisticsObjForPartitions(final String dbName,
+  private List<ColumnStatisticsObj> columnStatisticsObjForPartitions(
+      final String catName, final String dbName,
     final String tableName, final List<String> partNames, List<String> colNames, long partsFound,
     final boolean useDensityFunctionForNDVEstimation, final double ndvTuner, final boolean enableBitVector) throws MetaException {
     final boolean areAllPartsFound = (partsFound == partNames.size());
@@ -1479,7 +1509,7 @@ class MetaStoreDirectSql {
         return runBatched(partNames, new Batchable<String, ColumnStatisticsObj>() {
           @Override
           public List<ColumnStatisticsObj> run(List<String> inputPartNames) throws MetaException {
-            return columnStatisticsObjForPartitionsBatch(dbName, tableName, inputPartNames,
+            return columnStatisticsObjForPartitionsBatch(catName, dbName, tableName, inputPartNames,
                 inputColNames, areAllPartsFound, useDensityFunctionForNDVEstimation, ndvTuner, enableBitVector);
           }
         });
@@ -1487,10 +1517,10 @@ class MetaStoreDirectSql {
     });
   }
 
-  public List<ColStatsObjWithSourceInfo> getColStatsForAllTablePartitions(String dbName,
+  public List<ColStatsObjWithSourceInfo> getColStatsForAllTablePartitions(String catName, String dbName,
       boolean enableBitVector) throws MetaException {
     String queryText = "select \"TABLE_NAME\", \"PARTITION_NAME\", " + getStatsList(enableBitVector)
-        + " from " + " " + PART_COL_STATS + " where \"DB_NAME\" = ?";
+        + " from " + " " + PART_COL_STATS + " where \"DB_NAME\" = ? and \"CAT_NAME\" = ?";
     long start = 0;
     long end = 0;
     Query query = null;
@@ -1500,7 +1530,7 @@ class MetaStoreDirectSql {
     List<ColStatsObjWithSourceInfo> colStatsForDB = new ArrayList<ColStatsObjWithSourceInfo>();
     try {
       query = pm.newQuery("javax.jdo.query.SQL", queryText);
-      qResult = executeWithArray(query, new Object[] { dbName }, queryText);
+      qResult = executeWithArray(query, new Object[] { dbName, catName }, queryText);
       if (qResult == null) {
         query.closeAll();
         return colStatsForDB;
@@ -1512,7 +1542,7 @@ class MetaStoreDirectSql {
         String tblName = (String) row[0];
         String partName = (String) row[1];
         ColumnStatisticsObj colStatObj = prepareCSObj(row, 2);
-        colStatsForDB.add(new ColStatsObjWithSourceInfo(colStatObj, dbName, tblName, partName));
+        colStatsForDB.add(new ColStatsObjWithSourceInfo(colStatObj, catName, dbName, tblName, partName));
         Deadline.checkTimeout();
       }
     } finally {
@@ -1522,31 +1552,31 @@ class MetaStoreDirectSql {
   }
 
   /** Should be called with the list short enough to not trip up Oracle/etc. */
-  private List<ColumnStatisticsObj> columnStatisticsObjForPartitionsBatch(String dbName,
+  private List<ColumnStatisticsObj> columnStatisticsObjForPartitionsBatch(String catName, String dbName,
       String tableName, List<String> partNames, List<String> colNames, boolean areAllPartsFound,
       boolean useDensityFunctionForNDVEstimation, double ndvTuner, boolean enableBitVector)
       throws MetaException {
     if (enableBitVector) {
-      return aggrStatsUseJava(dbName, tableName, partNames, colNames, areAllPartsFound,
+      return aggrStatsUseJava(catName, dbName, tableName, partNames, colNames, areAllPartsFound,
           useDensityFunctionForNDVEstimation, ndvTuner);
     } else {
-      return aggrStatsUseDB(dbName, tableName, partNames, colNames, areAllPartsFound,
+      return aggrStatsUseDB(catName, dbName, tableName, partNames, colNames, areAllPartsFound,
           useDensityFunctionForNDVEstimation, ndvTuner);
     }
   }
 
-  private List<ColumnStatisticsObj> aggrStatsUseJava(String dbName, String tableName,
+  private List<ColumnStatisticsObj> aggrStatsUseJava(String catName, String dbName, String tableName,
       List<String> partNames, List<String> colNames, boolean areAllPartsFound,
       boolean useDensityFunctionForNDVEstimation, double ndvTuner) throws MetaException {
     // 1. get all the stats for colNames in partNames;
     List<ColumnStatistics> partStats =
-        getPartitionStats(dbName, tableName, partNames, colNames, true);
+        getPartitionStats(catName, dbName, tableName, partNames, colNames, true);
     // 2. use util function to aggr stats
-    return MetaStoreUtils.aggrPartitionStats(partStats, dbName, tableName, partNames, colNames,
+    return MetaStoreUtils.aggrPartitionStats(partStats, catName, dbName, tableName, partNames, colNames,
         areAllPartsFound, useDensityFunctionForNDVEstimation, ndvTuner);
   }
 
-  private List<ColumnStatisticsObj> aggrStatsUseDB(String dbName,
+  private List<ColumnStatisticsObj> aggrStatsUseDB(String catName, String dbName,
       String tableName, List<String> partNames, List<String> colNames, boolean areAllPartsFound,
       boolean useDensityFunctionForNDVEstimation, double ndvTuner) throws MetaException {
     // TODO: all the extrapolation logic should be moved out of this class,
@@ -1573,7 +1603,7 @@ class MetaStoreDirectSql {
         + "avg((\"DOUBLE_HIGH_VALUE\"-\"DOUBLE_LOW_VALUE\")/\"NUM_DISTINCTS\"),"
         + "avg((cast(\"BIG_DECIMAL_HIGH_VALUE\" as decimal)-cast(\"BIG_DECIMAL_LOW_VALUE\" as decimal))/\"NUM_DISTINCTS\"),"
         + "sum(\"NUM_DISTINCTS\")" + " from " + PART_COL_STATS + ""
-        + " where \"DB_NAME\" = ? and \"TABLE_NAME\" = ? ";
+        + " where \"CAT_NAME\" = ? and \"DB_NAME\" = ? and \"TABLE_NAME\" = ? ";
     String queryText = null;
     long start = 0;
     long end = 0;
@@ -1589,7 +1619,7 @@ class MetaStoreDirectSql {
           + " group by \"COLUMN_NAME\", \"COLUMN_TYPE\"";
       start = doTrace ? System.nanoTime() : 0;
       query = pm.newQuery("javax.jdo.query.SQL", queryText);
-      qResult = executeWithArray(query, prepareParams(dbName, tableName, partNames, colNames),
+      qResult = executeWithArray(query, prepareParams(catName, dbName, tableName, partNames, colNames),
           queryText);
       if (qResult == null) {
         query.closeAll();
@@ -1612,13 +1642,13 @@ class MetaStoreDirectSql {
       List<ColumnStatisticsObj> colStats = new ArrayList<ColumnStatisticsObj>(colNames.size());
       queryText = "select \"COLUMN_NAME\", \"COLUMN_TYPE\", count(\"PARTITION_NAME\") "
           + " from " + PART_COL_STATS
-          + " where \"DB_NAME\" = ? and \"TABLE_NAME\" = ? "
+          + " where \"CAT_NAME\" = ? and \"DB_NAME\" = ? and \"TABLE_NAME\" = ? "
           + " and \"COLUMN_NAME\" in (" + makeParams(colNames.size()) + ")"
           + " and \"PARTITION_NAME\" in (" + makeParams(partNames.size()) + ")"
           + " group by \"COLUMN_NAME\", \"COLUMN_TYPE\"";
       start = doTrace ? System.nanoTime() : 0;
       query = pm.newQuery("javax.jdo.query.SQL", queryText);
-      qResult = executeWithArray(query, prepareParams(dbName, tableName, partNames, colNames),
+      qResult = executeWithArray(query, prepareParams(catName, dbName, tableName, partNames, colNames),
           queryText);
       end = doTrace ? System.nanoTime() : 0;
       timingTrace(doTrace, queryText, start, end);
@@ -1653,7 +1683,7 @@ class MetaStoreDirectSql {
         start = doTrace ? System.nanoTime() : 0;
         query = pm.newQuery("javax.jdo.query.SQL", queryText);
         qResult = executeWithArray(query,
-            prepareParams(dbName, tableName, partNames, noExtraColumnNames), queryText);
+            prepareParams(catName, dbName, tableName, partNames, noExtraColumnNames), queryText);
         if (qResult == null) {
           query.closeAll();
           return Collections.emptyList();
@@ -1677,7 +1707,7 @@ class MetaStoreDirectSql {
         // get sum for all columns to reduce the number of queries
         Map<String, Map<Integer, Object>> sumMap = new HashMap<String, Map<Integer, Object>>();
         queryText = "select \"COLUMN_NAME\", sum(\"NUM_NULLS\"), sum(\"NUM_TRUES\"), sum(\"NUM_FALSES\"), sum(\"NUM_DISTINCTS\")"
-            + " from " + PART_COL_STATS + " where \"DB_NAME\" = ? and \"TABLE_NAME\" = ? "
+            + " from " + PART_COL_STATS + " where \"CAT_NAME\" = ? and \"DB_NAME\" = ? and \"TABLE_NAME\" = ? "
             + " and \"COLUMN_NAME\" in (" + makeParams(extraColumnNameTypeParts.size())
             + ") and \"PARTITION_NAME\" in (" + makeParams(partNames.size())
             + ") group by \"COLUMN_NAME\"";
@@ -1686,7 +1716,7 @@ class MetaStoreDirectSql {
         List<String> extraColumnNames = new ArrayList<String>();
         extraColumnNames.addAll(extraColumnNameTypeParts.keySet());
         qResult = executeWithArray(query,
-            prepareParams(dbName, tableName, partNames, extraColumnNames), queryText);
+            prepareParams(catName, dbName, tableName, partNames, extraColumnNames), queryText);
         if (qResult == null) {
           query.closeAll();
           return Collections.emptyList();
@@ -1750,20 +1780,20 @@ class MetaStoreDirectSql {
               if (!decimal) {
                 queryText = "select \"" + colStatName
                     + "\",\"PARTITION_NAME\" from " + PART_COL_STATS
-                    + " where \"DB_NAME\" = ? and \"TABLE_NAME\" = ?" + " and \"COLUMN_NAME\" = ?"
+                    + " where \"CAT_NAME\" = ? and \"DB_NAME\" = ? and \"TABLE_NAME\" = ?" + " and \"COLUMN_NAME\" = ?"
                     + " and \"PARTITION_NAME\" in (" + makeParams(partNames.size()) + ")"
                     + " order by \"" + colStatName + "\"";
               } else {
                 queryText = "select \"" + colStatName
                     + "\",\"PARTITION_NAME\" from " + PART_COL_STATS
-                    + " where \"DB_NAME\" = ? and \"TABLE_NAME\" = ?" + " and \"COLUMN_NAME\" = ?"
+                    + " where \"CAT_NAME\" = ? and \"DB_NAME\" = ? and \"TABLE_NAME\" = ?" + " and \"COLUMN_NAME\" = ?"
                     + " and \"PARTITION_NAME\" in (" + makeParams(partNames.size()) + ")"
                     + " order by cast(\"" + colStatName + "\" as decimal)";
               }
               start = doTrace ? System.nanoTime() : 0;
               query = pm.newQuery("javax.jdo.query.SQL", queryText);
               qResult = executeWithArray(query,
-                  prepareParams(dbName, tableName, partNames, Arrays.asList(colName)), queryText);
+                  prepareParams(catName, dbName, tableName, partNames, Arrays.asList(colName)), queryText);
               if (qResult == null) {
                 query.closeAll();
                 return Collections.emptyList();
@@ -1786,13 +1816,13 @@ class MetaStoreDirectSql {
                   + "avg((\"LONG_HIGH_VALUE\"-\"LONG_LOW_VALUE\")/cast(\"NUM_DISTINCTS\" as decimal)),"
                   + "avg((\"DOUBLE_HIGH_VALUE\"-\"DOUBLE_LOW_VALUE\")/\"NUM_DISTINCTS\"),"
                   + "avg((cast(\"BIG_DECIMAL_HIGH_VALUE\" as decimal)-cast(\"BIG_DECIMAL_LOW_VALUE\" as decimal))/\"NUM_DISTINCTS\")"
-                  + " from " + PART_COL_STATS + "" + " where \"DB_NAME\" = ? and \"TABLE_NAME\" = ?"
+                  + " from " + PART_COL_STATS + "" + " where \"CAT_NAME\" = ? and \"DB_NAME\" = ? and \"TABLE_NAME\" = ?"
                   + " and \"COLUMN_NAME\" = ?" + " and \"PARTITION_NAME\" in ("
                   + makeParams(partNames.size()) + ")" + " group by \"COLUMN_NAME\"";
               start = doTrace ? System.nanoTime() : 0;
               query = pm.newQuery("javax.jdo.query.SQL", queryText);
               qResult = executeWithArray(query,
-                  prepareParams(dbName, tableName, partNames, Arrays.asList(colName)), queryText);
+                  prepareParams(catName, dbName, tableName, partNames, Arrays.asList(colName)), queryText);
               if (qResult == null) {
                 query.closeAll();
                 return Collections.emptyList();
@@ -1837,11 +1867,12 @@ class MetaStoreDirectSql {
     return cso;
   }
 
-  private Object[] prepareParams(String dbName, String tableName, List<String> partNames,
-    List<String> colNames) throws MetaException {
+  private Object[] prepareParams(String catName, String dbName, String tableName,
+                                 List<String> partNames, List<String> colNames) throws MetaException {
 
-    Object[] params = new Object[colNames.size() + partNames.size() + 2];
+    Object[] params = new Object[colNames.size() + partNames.size() + 3];
     int paramI = 0;
+    params[paramI++] = catName;
     params[paramI++] = dbName;
     params[paramI++] = tableName;
     for (String colName : colNames) {
@@ -1854,14 +1885,16 @@ class MetaStoreDirectSql {
     return params;
   }
 
-  public List<ColumnStatistics> getPartitionStats(final String dbName, final String tableName,
-      final List<String> partNames, List<String> colNames, boolean enableBitVector) throws MetaException {
+  public List<ColumnStatistics> getPartitionStats(
+      final String catName, final String dbName, final String tableName, final List<String> partNames,
+      List<String> colNames, boolean enableBitVector) throws MetaException {
     if (colNames.isEmpty() || partNames.isEmpty()) {
       return Collections.emptyList();
     }
     final boolean doTrace = LOG.isDebugEnabled();
     final String queryText0 = "select \"PARTITION_NAME\", " + getStatsList(enableBitVector) + " from "
-        + " " + PART_COL_STATS + " where \"DB_NAME\" = ? and \"TABLE_NAME\" = ? and \"COLUMN_NAME\""
+        + " " + PART_COL_STATS + " where \"CAT_NAME\" = ? and \"DB_NAME\" = ? and \"TABLE_NAME\" = ? and " +
+        "\"COLUMN_NAME\""
         + "  in (%1$s) AND \"PARTITION_NAME\" in (%2$s) order by \"PARTITION_NAME\"";
     Batchable<String, Object[]> b = new Batchable<String, Object[]>() {
       @Override
@@ -1874,7 +1907,7 @@ class MetaStoreDirectSql {
             long start = doTrace ? System.nanoTime() : 0;
             Query query = pm.newQuery("javax.jdo.query.SQL", queryText);
             Object qResult = executeWithArray(query, prepareParams(
-                dbName, tableName, inputPartNames, inputColNames), queryText);
+                catName, dbName, tableName, inputPartNames, inputColNames), queryText);
             timingTrace(doTrace, queryText0, start, (doTrace ? System.nanoTime() : 0));
             if (qResult == null) {
               query.closeAll();
@@ -1904,6 +1937,7 @@ class MetaStoreDirectSql {
         continue;
       } else if (from != i) {
         ColumnStatisticsDesc csd = new ColumnStatisticsDesc(false, dbName, tableName);
+        csd.setCatName(catName);
         csd.setPartName(lastPartName);
         result.add(makeColumnStats(list.subList(from, i), csd, 1));
       }
@@ -2036,8 +2070,10 @@ class MetaStoreDirectSql {
     return result;
   }
 
-  public List<SQLForeignKey> getForeignKeys(String parent_db_name, String parent_tbl_name, String foreign_db_name, String foreign_tbl_name) throws MetaException {
-    List<SQLForeignKey> ret = new ArrayList<SQLForeignKey>();
+  public List<SQLForeignKey> getForeignKeys(String catName, String parent_db_name,
+                                            String parent_tbl_name, String foreign_db_name,
+                                            String foreign_tbl_name) throws MetaException {
+    List<SQLForeignKey> ret = new ArrayList<>();
     String queryText =
       "SELECT  \"D2\".\"NAME\", \"T2\".\"TBL_NAME\", "
       + "CASE WHEN \"C2\".\"COLUMN_NAME\" IS NOT NULL THEN \"C2\".\"COLUMN_NAME\" "
@@ -2065,6 +2101,7 @@ class MetaStoreDirectSql {
       + " \"P2\".\"INTEGER_IDX\" = " + KEY_CONSTRAINTS + ".\"PARENT_INTEGER_IDX\" "
       + " WHERE " + KEY_CONSTRAINTS + ".\"CONSTRAINT_TYPE\" = " + MConstraint.FOREIGN_KEY_CONSTRAINT
       + " AND \"KEY_CONSTRAINTS2\".\"CONSTRAINT_TYPE\" = " + MConstraint.PRIMARY_KEY_CONSTRAINT + " AND"
+      + " " + DBS + ".\"CTLG_NAME\" = ? AND"
       + (foreign_db_name == null ? "" : " " + DBS + ".\"NAME\" = ? AND")
       + (foreign_tbl_name == null ? "" : " " + TBLS + ".\"TBL_NAME\" = ? AND")
       + (parent_tbl_name == null ? "" : " \"T2\".\"TBL_NAME\" = ? AND")
@@ -2075,6 +2112,7 @@ class MetaStoreDirectSql {
       queryText = queryText.substring(0, queryText.length()-3);
     }
     List<String> pms = new ArrayList<String>();
+    pms.add(catName);
     if (foreign_db_name != null) {
       pms.add(foreign_db_name);
     }
@@ -2114,19 +2152,22 @@ class MetaStoreDirectSql {
           validate,
           rely
           );
-          ret.add(currKey);
+        currKey.setCatName(catName);
+        ret.add(currKey);
       }
     }
     return ret;
   }
 
-  public List<SQLPrimaryKey> getPrimaryKeys(String db_name, String tbl_name) throws MetaException {
-    List<SQLPrimaryKey> ret = new ArrayList<SQLPrimaryKey>();
+  public List<SQLPrimaryKey> getPrimaryKeys(String catName, String db_name, String tbl_name)
+      throws MetaException {
+    List<SQLPrimaryKey> ret = new ArrayList<>();
     String queryText =
       "SELECT " + DBS + ".\"NAME\", " + TBLS + ".\"TBL_NAME\", "
       + "CASE WHEN " + COLUMNS_V2 + ".\"COLUMN_NAME\" IS NOT NULL THEN " + COLUMNS_V2 + ".\"COLUMN_NAME\" "
       + "ELSE " + PARTITION_KEYS + ".\"PKEY_NAME\" END, " + KEY_CONSTRAINTS + ".\"POSITION\", "
-      + "" + KEY_CONSTRAINTS + ".\"CONSTRAINT_NAME\", " + KEY_CONSTRAINTS + ".\"ENABLE_VALIDATE_RELY\" "
+      + KEY_CONSTRAINTS + ".\"CONSTRAINT_NAME\", " + KEY_CONSTRAINTS + ".\"ENABLE_VALIDATE_RELY\", "
+      + DBS + ".\"CTLG_NAME\""
       + " from " + TBLS + " "
       + " INNER JOIN " + KEY_CONSTRAINTS + " ON " + TBLS + ".\"TBL_ID\" = " + KEY_CONSTRAINTS + ".\"PARENT_TBL_ID\" "
       + " INNER JOIN " + DBS + " ON " + TBLS + ".\"DB_ID\" = " + DBS + ".\"DB_ID\" "
@@ -2135,6 +2176,7 @@ class MetaStoreDirectSql {
       + " LEFT OUTER JOIN " + PARTITION_KEYS + " ON " + TBLS + ".\"TBL_ID\" = " + PARTITION_KEYS + ".\"TBL_ID\" AND "
       + " " + PARTITION_KEYS + ".\"INTEGER_IDX\" = " + KEY_CONSTRAINTS + ".\"PARENT_INTEGER_IDX\" "
       + " WHERE " + KEY_CONSTRAINTS + ".\"CONSTRAINT_TYPE\" = "+ MConstraint.PRIMARY_KEY_CONSTRAINT + " AND"
+      + " " + DBS + ".\"CTLG_NAME\" = ? AND"
       + (db_name == null ? "" : " " + DBS + ".\"NAME\" = ? AND")
       + (tbl_name == null ? "" : " " + TBLS + ".\"TBL_NAME\" = ? ") ;
 
@@ -2142,7 +2184,8 @@ class MetaStoreDirectSql {
     if (queryText.endsWith("AND")) {
       queryText = queryText.substring(0, queryText.length()-3);
     }
-    List<String> pms = new ArrayList<String>();
+    List<String> pms = new ArrayList<>();
+    pms.add(catName);
     if (db_name != null) {
       pms.add(db_name);
     }
@@ -2168,13 +2211,14 @@ class MetaStoreDirectSql {
           enable,
           validate,
           rely);
-          ret.add(currKey);
+        currKey.setCatName(extractSqlString(line[6]));
+        ret.add(currKey);
       }
     }
     return ret;
   }
 
-  public List<SQLUniqueConstraint> getUniqueConstraints(String db_name, String tbl_name)
+  public List<SQLUniqueConstraint> getUniqueConstraints(String catName, String db_name, String tbl_name)
           throws MetaException {
     List<SQLUniqueConstraint> ret = new ArrayList<SQLUniqueConstraint>();
     String queryText =
@@ -2190,6 +2234,7 @@ class MetaStoreDirectSql {
       + " LEFT OUTER JOIN " + PARTITION_KEYS + " ON " + TBLS + ".\"TBL_ID\" = " + PARTITION_KEYS + ".\"TBL_ID\" AND "
       + " " + PARTITION_KEYS + ".\"INTEGER_IDX\" = " + KEY_CONSTRAINTS + ".\"PARENT_INTEGER_IDX\" "
       + " WHERE " + KEY_CONSTRAINTS + ".\"CONSTRAINT_TYPE\" = "+ MConstraint.UNIQUE_CONSTRAINT + " AND"
+      + " " + DBS + ".\"CTLG_NAME\" = ? AND"
       + (db_name == null ? "" : " " + DBS + ".\"NAME\" = ? AND")
       + (tbl_name == null ? "" : " " + TBLS + ".\"TBL_NAME\" = ? ") ;
 
@@ -2198,6 +2243,7 @@ class MetaStoreDirectSql {
       queryText = queryText.substring(0, queryText.length()-3);
     }
     List<String> pms = new ArrayList<String>();
+    pms.add(catName);
     if (db_name != null) {
       pms.add(db_name);
     }
@@ -2215,23 +2261,23 @@ class MetaStoreDirectSql {
           boolean enable = (enableValidateRely & 4) != 0;
           boolean validate = (enableValidateRely & 2) != 0;
           boolean rely = (enableValidateRely & 1) != 0;
-        SQLUniqueConstraint currConstraint = new SQLUniqueConstraint(
-          extractSqlString(line[0]),
-          extractSqlString(line[1]),
-          extractSqlString(line[2]),
-          extractSqlInt(line[3]), extractSqlString(line[4]),
-          enable,
-          validate,
-          rely);
-          ret.add(currConstraint);
+        ret.add(new SQLUniqueConstraint(
+            catName,
+            extractSqlString(line[0]),
+            extractSqlString(line[1]),
+            extractSqlString(line[2]),
+            extractSqlInt(line[3]), extractSqlString(line[4]),
+            enable,
+            validate,
+            rely));
       }
     }
     return ret;
   }
 
-  public List<SQLNotNullConstraint> getNotNullConstraints(String db_name, String tbl_name)
+  public List<SQLNotNullConstraint> getNotNullConstraints(String catName, String db_name, String tbl_name)
           throws MetaException {
-    List<SQLNotNullConstraint> ret = new ArrayList<SQLNotNullConstraint>();
+    List<SQLNotNullConstraint> ret = new ArrayList<>();
     String queryText =
       "SELECT " + DBS + ".\"NAME\", " + TBLS + ".\"TBL_NAME\","
       + "CASE WHEN " + COLUMNS_V2 + ".\"COLUMN_NAME\" IS NOT NULL THEN " + COLUMNS_V2 + ".\"COLUMN_NAME\" "
@@ -2245,6 +2291,7 @@ class MetaStoreDirectSql {
       + " LEFT OUTER JOIN " + PARTITION_KEYS + " ON " + TBLS + ".\"TBL_ID\" = " + PARTITION_KEYS + ".\"TBL_ID\" AND "
       + " " + PARTITION_KEYS + ".\"INTEGER_IDX\" = " + KEY_CONSTRAINTS + ".\"PARENT_INTEGER_IDX\" "
       + " WHERE " + KEY_CONSTRAINTS + ".\"CONSTRAINT_TYPE\" = "+ MConstraint.NOT_NULL_CONSTRAINT + " AND"
+      + " " + DBS + ".\"CTLG_NAME\" = ? AND"
       + (db_name == null ? "" : " " + DBS + ".\"NAME\" = ? AND")
       + (tbl_name == null ? "" : " " + TBLS + ".\"TBL_NAME\" = ? ") ;
 
@@ -2252,7 +2299,8 @@ class MetaStoreDirectSql {
     if (queryText.endsWith("AND")) {
       queryText = queryText.substring(0, queryText.length()-3);
     }
-    List<String> pms = new ArrayList<String>();
+    List<String> pms = new ArrayList<>();
+    pms.add(catName);
     if (db_name != null) {
       pms.add(db_name);
     }
@@ -2270,21 +2318,21 @@ class MetaStoreDirectSql {
           boolean enable = (enableValidateRely & 4) != 0;
           boolean validate = (enableValidateRely & 2) != 0;
           boolean rely = (enableValidateRely & 1) != 0;
-        SQLNotNullConstraint currConstraint = new SQLNotNullConstraint(
-          extractSqlString(line[0]),
-          extractSqlString(line[1]),
-          extractSqlString(line[2]),
-          extractSqlString(line[3]),
-          enable,
-          validate,
-          rely);
-          ret.add(currConstraint);
+        ret.add(new SQLNotNullConstraint(
+            catName,
+            extractSqlString(line[0]),
+            extractSqlString(line[1]),
+            extractSqlString(line[2]),
+            extractSqlString(line[3]),
+            enable,
+            validate,
+            rely));
       }
     }
     return ret;
   }
 
-  public List<SQLDefaultConstraint> getDefaultConstraints(String db_name, String tbl_name)
+  public List<SQLDefaultConstraint> getDefaultConstraints(String catName, String db_name, String tbl_name)
       throws MetaException {
     List<SQLDefaultConstraint> ret = new ArrayList<SQLDefaultConstraint>();
     String queryText =
@@ -2301,6 +2349,7 @@ class MetaStoreDirectSql {
             + " LEFT OUTER JOIN " + PARTITION_KEYS + " ON " + TBLS + ".\"TBL_ID\" = " + PARTITION_KEYS + ".\"TBL_ID\" AND "
             + " " + PARTITION_KEYS + ".\"INTEGER_IDX\" = " + KEY_CONSTRAINTS + ".\"PARENT_INTEGER_IDX\" "
             + " WHERE " + KEY_CONSTRAINTS + ".\"CONSTRAINT_TYPE\" = "+ MConstraint.DEFAULT_CONSTRAINT+ " AND"
+            + " " + DBS + ".\"CTLG_NAME\" = ? AND"
             + (db_name == null ? "" : " " + DBS + ".\"NAME\" = ? AND")
             + (tbl_name == null ? "" : " " + TBLS + ".\"TBL_NAME\" = ? ") ;
 
@@ -2311,7 +2360,8 @@ class MetaStoreDirectSql {
     if (LOG.isDebugEnabled()){
       LOG.debug("getDefaultConstraints: directsql : " + queryText);
     }
-    List<String> pms = new ArrayList<String>();
+    List<String> pms = new ArrayList<>();
+    pms.add(catName);
     if (db_name != null) {
       pms.add(db_name);
     }
@@ -2330,6 +2380,7 @@ class MetaStoreDirectSql {
         boolean validate = (enableValidateRely & 2) != 0;
         boolean rely = (enableValidateRely & 1) != 0;
         SQLDefaultConstraint currConstraint = new SQLDefaultConstraint(
+            catName,
             extractSqlString(line[0]),
             extractSqlString(line[1]),
             extractSqlString(line[2]),
@@ -2344,7 +2395,7 @@ class MetaStoreDirectSql {
     return ret;
   }
 
-  public List<SQLCheckConstraint> getCheckConstraints(String db_name, String tbl_name)
+  public List<SQLCheckConstraint> getCheckConstraints(String catName, String db_name, String tbl_name)
       throws MetaException {
     List<SQLCheckConstraint> ret = new ArrayList<SQLCheckConstraint>();
     String queryText =
@@ -2361,6 +2412,7 @@ class MetaStoreDirectSql {
             + " LEFT OUTER JOIN " + PARTITION_KEYS + " ON " + TBLS + ".\"TBL_ID\" = " + PARTITION_KEYS + ".\"TBL_ID\" AND "
             + " " + PARTITION_KEYS + ".\"INTEGER_IDX\" = " + KEY_CONSTRAINTS + ".\"PARENT_INTEGER_IDX\" "
             + " WHERE " + KEY_CONSTRAINTS + ".\"CONSTRAINT_TYPE\" = "+ MConstraint.CHECK_CONSTRAINT+ " AND"
+            + " " + DBS + ".\"CTLG_NAME\" = ? AND"
             + (db_name == null ? "" : " " + DBS + ".\"NAME\" = ? AND")
             + (tbl_name == null ? "" : " " + TBLS + ".\"TBL_NAME\" = ? ") ;
 
@@ -2371,7 +2423,8 @@ class MetaStoreDirectSql {
     if (LOG.isDebugEnabled()){
       LOG.debug("getCheckConstraints: directsql : " + queryText);
     }
-    List<String> pms = new ArrayList<String>();
+    List<String> pms = new ArrayList<>();
+    pms.add(catName);
     if (db_name != null) {
       pms.add(db_name);
     }
@@ -2390,6 +2443,7 @@ class MetaStoreDirectSql {
         boolean validate = (enableValidateRely & 2) != 0;
         boolean rely = (enableValidateRely & 1) != 0;
         SQLCheckConstraint currConstraint = new SQLCheckConstraint(
+            catName,
             extractSqlString(line[0]),
             extractSqlString(line[1]),
             extractSqlString(line[2]),

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java
index 67600e1..569fff0 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java
@@ -35,10 +35,12 @@ import org.apache.hadoop.hive.metastore.events.AlterPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.AlterSchemaVersionEvent;
 import org.apache.hadoop.hive.metastore.events.AlterTableEvent;
 import org.apache.hadoop.hive.metastore.events.ConfigChangeEvent;
+import org.apache.hadoop.hive.metastore.events.CreateCatalogEvent;
 import org.apache.hadoop.hive.metastore.events.CreateDatabaseEvent;
 import org.apache.hadoop.hive.metastore.events.CreateFunctionEvent;
 import org.apache.hadoop.hive.metastore.events.CreateISchemaEvent;
 import org.apache.hadoop.hive.metastore.events.CreateTableEvent;
+import org.apache.hadoop.hive.metastore.events.DropCatalogEvent;
 import org.apache.hadoop.hive.metastore.events.DropConstraintEvent;
 import org.apache.hadoop.hive.metastore.events.DropDatabaseEvent;
 import org.apache.hadoop.hive.metastore.events.DropFunctionEvent;
@@ -222,6 +224,12 @@ public abstract class MetaStoreEventListener implements Configurable {
       throws MetaException {
   }
 
+  public void onCreateCatalog(CreateCatalogEvent createCatalogEvent) throws MetaException {
+  }
+
+  public void onDropCatalog(DropCatalogEvent dropCatalogEvent) throws MetaException {
+  }
+
   @Override
   public Configuration getConf() {
     return this.conf;

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreFilterHook.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreFilterHook.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreFilterHook.java
index 8522afe..f7a0cd0 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreFilterHook.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreFilterHook.java
@@ -22,12 +22,14 @@ import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.api.Catalog;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.PartitionSpec;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TableMeta;
 
 /**
  * Metadata filter hook for metastore client. This will be useful for authorization
@@ -39,11 +41,31 @@ import org.apache.hadoop.hive.metastore.api.Table;
 public interface MetaStoreFilterHook {
 
   /**
+   * Filter a catalog object.  Default implementation returns the passed in catalog.
+   * @param catalog catalog to filter
+   * @return filtered catalog
+   * @throws MetaException something bad happened
+   */
+  default Catalog filterCatalog(Catalog catalog) throws MetaException {
+    return catalog;
+  }
+
+  /**
+   * Filter a list of catalog names.  Default implementation returns the passed in list.
+   * @param catalogs list of catalog names.
+   * @return filtered list of catalog names.
+   * @throws MetaException something bad happened.
+   */
+  default List<String> filterCatalogs(List<String> catalogs) throws MetaException {
+    return catalogs;
+  }
+
+  /**
    * Filter given list of databases
    * @param dbList
    * @return List of filtered Db names
    */
-  public List<String> filterDatabases(List<String> dbList) throws MetaException;
+  List<String> filterDatabases(List<String> dbList) throws MetaException;
 
   /**
    * filter to given database object if applicable
@@ -51,15 +73,27 @@ public interface MetaStoreFilterHook {
    * @return the same database if it's not filtered out
    * @throws NoSuchObjectException
    */
-  public Database filterDatabase(Database dataBase) throws MetaException, NoSuchObjectException;
+  Database filterDatabase(Database dataBase) throws MetaException, NoSuchObjectException;
 
   /**
    * Filter given list of tables
-   * @param dbName
-   * @param tableList
+   * @param catName catalog name
+   * @param dbName database name
+   * @param tableList list of table returned by the metastore
    * @return List of filtered table names
    */
-  public List<String> filterTableNames(String dbName, List<String> tableList) throws MetaException;
+  List<String> filterTableNames(String catName, String dbName, List<String> tableList)
+      throws MetaException;
+
+  // Previously this was handled by filterTableNames.  But it can't be anymore because we can no
+  // longer depend on a 1-1 mapping between table name and entry in the list.
+  /**
+   * Filter a list of TableMeta objects.
+   * @param tableMetas list of TableMetas to filter
+   * @return filtered table metas
+   * @throws MetaException something went wrong
+   */
+  List<TableMeta> filterTableMetas(List<TableMeta> tableMetas) throws MetaException;
 
   /**
    * filter to given table object if applicable
@@ -67,28 +101,28 @@ public interface MetaStoreFilterHook {
    * @return the same table if it's not filtered out
    * @throws NoSuchObjectException
    */
-  public Table filterTable(Table table) throws MetaException, NoSuchObjectException;
+  Table filterTable(Table table) throws MetaException, NoSuchObjectException;
 
   /**
    * Filter given list of tables
    * @param tableList
    * @return List of filtered table names
    */
-  public List<Table> filterTables(List<Table> tableList) throws MetaException;
+  List<Table> filterTables(List<Table> tableList) throws MetaException;
 
   /**
    * Filter given list of partitions
    * @param partitionList
    * @return
    */
-  public List<Partition> filterPartitions(List<Partition> partitionList) throws MetaException;
+  List<Partition> filterPartitions(List<Partition> partitionList) throws MetaException;
 
   /**
    * Filter given list of partition specs
    * @param partitionSpecList
    * @return
    */
-  public List<PartitionSpec> filterPartitionSpecs(List<PartitionSpec> partitionSpecList)
+  List<PartitionSpec> filterPartitionSpecs(List<PartitionSpec> partitionSpecList)
       throws MetaException;
 
   /**
@@ -97,18 +131,17 @@ public interface MetaStoreFilterHook {
    * @return the same partition object if it's not filtered out
    * @throws NoSuchObjectException
    */
-  public Partition filterPartition(Partition partition) throws MetaException, NoSuchObjectException;
+  Partition filterPartition(Partition partition) throws MetaException, NoSuchObjectException;
 
   /**
    * Filter given list of partition names
-   * @param dbName
-   * @param tblName
-   * @param partitionNames
-   * @return
+   * @param catName catalog name.
+   * @param dbName database name.
+   * @param tblName table name.
+   * @param partitionNames list of partition names.
+   * @return list of filtered partition names.
    */
-  public List<String> filterPartitionNames(String dbName, String tblName,
+  List<String> filterPartitionNames(String catName, String dbName, String tblName,
       List<String> partitionNames) throws MetaException;
-
-
 }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java
index f5a91b4..988fca6 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java
@@ -35,10 +35,12 @@ import org.apache.hadoop.hive.metastore.events.AlterISchemaEvent;
 import org.apache.hadoop.hive.metastore.events.AlterPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.AlterSchemaVersionEvent;
 import org.apache.hadoop.hive.metastore.events.AlterTableEvent;
+import org.apache.hadoop.hive.metastore.events.CreateCatalogEvent;
 import org.apache.hadoop.hive.metastore.events.CreateDatabaseEvent;
 import org.apache.hadoop.hive.metastore.events.CreateFunctionEvent;
 import org.apache.hadoop.hive.metastore.events.CreateISchemaEvent;
 import org.apache.hadoop.hive.metastore.events.CreateTableEvent;
+import org.apache.hadoop.hive.metastore.events.DropCatalogEvent;
 import org.apache.hadoop.hive.metastore.events.DropDatabaseEvent;
 import org.apache.hadoop.hive.metastore.events.DropFunctionEvent;
 import org.apache.hadoop.hive.metastore.events.DropISchemaEvent;
@@ -200,6 +202,10 @@ public class MetaStoreListenerNotifier {
               listener.onDropSchemaVersion((DropSchemaVersionEvent) event);
             }
           })
+          .put(EventType.CREATE_CATALOG,
+              (listener, event) -> listener.onCreateCatalog((CreateCatalogEvent)event))
+          .put(EventType.DROP_CATALOG,
+              (listener, event) -> listener.onDropCatalog((DropCatalogEvent)event))
           .build()
   );
 


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IHMSHandler.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IHMSHandler.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IHMSHandler.java
index e6de001..f59f40b 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IHMSHandler.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IHMSHandler.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore;
 import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 
 /**
  * An interface wrapper for HMSHandler.  This interface contains methods that need to be
@@ -66,25 +67,43 @@ public interface IHMSHandler extends ThriftHiveMetastore.Iface, Configurable {
   /**
    * Equivalent to get_database, but does not write to audit logs, or fire pre-event listeners.
    * Meant to be used for internal hive classes that don't use the thrift interface.
+   * @param catName catalog name
    * @param name database name
    * @return database object
    * @throws NoSuchObjectException If the database does not exist.
    * @throws MetaException If another error occurs.
    */
-  Database get_database_core(final String name) throws NoSuchObjectException, MetaException;
+  Database get_database_core(final String catName, final String name)
+      throws NoSuchObjectException, MetaException;
 
   /**
    * Equivalent of get_table, but does not log audits and fire pre-event listener.
    * Meant to be used for calls made by other hive classes, that are not using the
    * thrift interface.
+   * @param catName catalog name
    * @param dbname database name
    * @param name table name
    * @return Table object
    * @throws NoSuchObjectException If the table does not exist.
    * @throws MetaException  If another error occurs.
    */
-  Table get_table_core(final String dbname, final String name) throws MetaException,
-      NoSuchObjectException;
+  Table get_table_core(final String catName, final String dbname, final String name)
+      throws MetaException, NoSuchObjectException;
+
+  /**
+   * Equivalent of get_table, but does not log audits and fire pre-event listener.
+   * Meant to be used for calls made by other hive classes, that are not using the
+   * thrift interface.  Uses the configured catalog.
+   * @param dbName database name
+   * @param name table name
+   * @return Table object
+   * @throws NoSuchObjectException If the table does not exist.
+   * @throws MetaException  If another error occurs.
+   */
+  default Table get_table_core(final String dbName, final String name)
+      throws MetaException, NoSuchObjectException {
+    return get_table_core(MetaStoreUtils.getDefaultCatalog(getConf()), dbName, name);
+  }
 
   /**
    * Get a list of all transactional listeners.


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
index 4f07619..af545d1 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
@@ -36,7 +36,9 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
+import org.apache.hadoop.hive.metastore.api.Catalog;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
+import org.apache.hadoop.hive.metastore.api.CreationMetadata;
 import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
@@ -44,6 +46,7 @@ import org.apache.hadoop.hive.metastore.api.FileMetadataExprType;
 import org.apache.hadoop.hive.metastore.api.Function;
 import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege;
 import org.apache.hadoop.hive.metastore.api.ISchema;
+import org.apache.hadoop.hive.metastore.api.ISchemaName;
 import org.apache.hadoop.hive.metastore.api.InvalidInputException;
 import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
 import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
@@ -61,14 +64,10 @@ import org.apache.hadoop.hive.metastore.api.PartitionValuesResponse;
 import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet;
 import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.hadoop.hive.metastore.api.PrivilegeBag;
-import org.apache.hadoop.hive.metastore.api.WMNullablePool;
-import org.apache.hadoop.hive.metastore.api.WMNullableResourcePlan;
-import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
-import org.apache.hadoop.hive.metastore.api.WMTrigger;
-import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
 import org.apache.hadoop.hive.metastore.api.SQLCheckConstraint;
+import org.apache.hadoop.hive.metastore.api.SchemaVersionDescriptor;
 import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
 import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
@@ -82,8 +81,14 @@ import org.apache.hadoop.hive.metastore.api.Type;
 import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 import org.apache.hadoop.hive.metastore.api.UnknownPartitionException;
 import org.apache.hadoop.hive.metastore.api.UnknownTableException;
+import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMMapping;
+import org.apache.hadoop.hive.metastore.api.WMNullablePool;
+import org.apache.hadoop.hive.metastore.api.WMNullableResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMPool;
+import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
+import org.apache.hadoop.hive.metastore.api.WMTrigger;
+import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
 import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.ColStatsObjWithSourceInfo;
 import org.apache.thrift.TException;
@@ -126,19 +131,107 @@ public interface RawStore extends Configurable {
   @CanNotRetry
   void rollbackTransaction();
 
+  /**
+   * Create a new catalog.
+   * @param cat Catalog to create.
+   * @throws MetaException if something goes wrong, usually in storing it to the database.
+   */
+  void createCatalog(Catalog cat) throws MetaException;
+
+  /**
+   * Alter an existing catalog.  Only description and location can be changed, and the change of
+   * location is for internal use only.
+   * @param catName name of the catalog to alter.
+   * @param cat new version of the catalog.
+   * @throws MetaException something went wrong, usually in the database.
+   * @throws InvalidOperationException attempt to change something about the catalog that is not
+   * changeable, like the name.
+   */
+  void alterCatalog(String catName, Catalog cat) throws MetaException, InvalidOperationException;
+
+  /**
+   * Get a catalog.
+   * @param catalogName name of the catalog.
+   * @return The catalog.
+   * @throws NoSuchObjectException no catalog of this name exists.
+   * @throws MetaException if something goes wrong, usually in reading it from the database.
+   */
+  Catalog getCatalog(String catalogName) throws NoSuchObjectException, MetaException;
+
+  /**
+   * Get all the catalogs.
+   * @return list of names of all catalogs in the system
+   * @throws MetaException if something goes wrong, usually in reading from the database.
+   */
+  List<String> getCatalogs() throws MetaException;
+
+  /**
+   * Drop a catalog.  The catalog must be empty.
+   * @param catalogName name of the catalog to drop.
+   * @throws NoSuchObjectException no catalog of this name exists.
+   * @throws MetaException could mean the catalog isn't empty, could mean general database error.
+   */
+  void dropCatalog(String catalogName) throws NoSuchObjectException, MetaException;
+
+  /**
+   * Create a database.
+   * @param db database to create.
+   * @throws InvalidObjectException not sure it actually ever throws this.
+   * @throws MetaException if something goes wrong, usually in writing it to the database.
+   */
   void createDatabase(Database db)
       throws InvalidObjectException, MetaException;
 
-  Database getDatabase(String name)
+  /**
+   * Get a database.
+   * @param catalogName catalog the database is in.
+   * @param name name of the database.
+   * @return the database.
+   * @throws NoSuchObjectException if no such database exists.
+   */
+  Database getDatabase(String catalogName, String name)
       throws NoSuchObjectException;
 
-  boolean dropDatabase(String dbname) throws NoSuchObjectException, MetaException;
+  /**
+   * Drop a database.
+   * @param catalogName catalog the database is in.
+   * @param dbname name of the database.
+   * @return true if the database was dropped, pretty much always returns this if it returns.
+   * @throws NoSuchObjectException no database in this catalog of this name to drop
+   * @throws MetaException something went wrong, usually with the database.
+   */
+  boolean dropDatabase(String catalogName, String dbname)
+      throws NoSuchObjectException, MetaException;
 
-  boolean alterDatabase(String dbname, Database db) throws NoSuchObjectException, MetaException;
+  /**
+   * Alter a database.
+   * @param catalogName name of the catalog the database is in.
+   * @param dbname name of the database to alter
+   * @param db new version of the database.  This should be complete as it will fully replace the
+   *          existing db object.
+   * @return true if the change succeeds, could fail due to db constraint violations.
+   * @throws NoSuchObjectException no database of this name exists to alter.
+   * @throws MetaException something went wrong, usually with the database.
+   */
+  boolean alterDatabase(String catalogName, String dbname, Database db)
+      throws NoSuchObjectException, MetaException;
 
-  List<String> getDatabases(String pattern) throws MetaException;
+  /**
+   * Get all database in a catalog having names that match a pattern.
+   * @param catalogName name of the catalog to search for databases in
+   * @param pattern pattern names should match
+   * @return list of matching database names.
+   * @throws MetaException something went wrong, usually with the database.
+   */
+  List<String> getDatabases(String catalogName, String pattern) throws MetaException;
 
-  List<String> getAllDatabases() throws MetaException;
+  /**
+   * Get names of all the databases in a catalog.
+   * @param catalogName name of the catalog to search for databases in
+   * @return list of names of all databases in the catalog
+   * @throws MetaException something went wrong, usually with the database.
+   */
+  List<String> getAllDatabases(String catalogName) throws MetaException;
 
   boolean createType(Type type);
 
@@ -149,53 +242,198 @@ public interface RawStore extends Configurable {
   void createTable(Table tbl) throws InvalidObjectException,
       MetaException;
 
-  boolean dropTable(String dbName, String tableName)
+  /**
+   * Drop a table.
+   * @param catalogName catalog the table is in
+   * @param dbName database the table is in
+   * @param tableName table name
+   * @return true if the table was dropped
+   * @throws MetaException something went wrong, usually in the RDBMS or storage
+   * @throws NoSuchObjectException No table of this name
+   * @throws InvalidObjectException Don't think this is ever actually thrown
+   * @throws InvalidInputException Don't think this is ever actually thrown
+   */
+  boolean dropTable(String catalogName, String dbName, String tableName)
       throws MetaException, NoSuchObjectException, InvalidObjectException, InvalidInputException;
 
-  Table getTable(String dbName, String tableName)
-      throws MetaException;
+  /**
+   * Get a table object.
+   * @param catalogName catalog the table is in.
+   * @param dbName database the table is in.
+   * @param tableName table name.
+   * @return table object, or null if no such table exists (wow it would be nice if we either
+   * consistently returned null or consistently threw NoSuchObjectException).
+   * @throws MetaException something went wrong in the RDBMS
+   */
+  Table getTable(String catalogName, String dbName, String tableName) throws MetaException;
 
+  /**
+   * Add a partition.
+   * @param part partition to add
+   * @return true if the partition was successfully added.
+   * @throws InvalidObjectException the provided partition object is not valid.
+   * @throws MetaException error writing to the RDBMS.
+   */
   boolean addPartition(Partition part)
       throws InvalidObjectException, MetaException;
 
-  boolean addPartitions(String dbName, String tblName, List<Partition> parts)
+  /**
+   * Add a list of partitions to a table.
+   * @param catName catalog name.
+   * @param dbName database name.
+   * @param tblName table name.
+   * @param parts list of partitions to be added.
+   * @return true if the operation succeeded.
+   * @throws InvalidObjectException never throws this AFAICT
+   * @throws MetaException the partitions don't belong to the indicated table or error writing to
+   * the RDBMS.
+   */
+  boolean addPartitions(String catName, String dbName, String tblName, List<Partition> parts)
       throws InvalidObjectException, MetaException;
 
-  boolean addPartitions(String dbName, String tblName, PartitionSpecProxy partitionSpec, boolean ifNotExists)
+  /**
+   * Add a list of partitions to a table.
+   * @param catName catalog name.
+   * @param dbName database name.
+   * @param tblName table name.
+   * @param partitionSpec specification for the partition
+   * @param ifNotExists whether it is in an error if the partition already exists.  If true, then
+   *                   it is not an error if the partition exists, if false, it is.
+   * @return whether the partition was created.
+   * @throws InvalidObjectException The passed in partition spec or table specification is invalid.
+   * @throws MetaException error writing to RDBMS.
+   */
+  boolean addPartitions(String catName, String dbName, String tblName,
+                        PartitionSpecProxy partitionSpec, boolean ifNotExists)
       throws InvalidObjectException, MetaException;
 
-  Partition getPartition(String dbName, String tableName,
+  /**
+   * Get a partition.
+   * @param catName catalog name.
+   * @param dbName database name.
+   * @param tableName table name.
+   * @param part_vals partition values for this table.
+   * @return the partition.
+   * @throws MetaException error reading from RDBMS.
+   * @throws NoSuchObjectException no partition matching this specification exists.
+   */
+  Partition getPartition(String catName, String dbName, String tableName,
       List<String> part_vals) throws MetaException, NoSuchObjectException;
 
-  boolean doesPartitionExist(String dbName, String tableName,
+  /**
+   * Check whether a partition exists.
+   * @param catName catalog name.
+   * @param dbName database name.
+   * @param tableName table name.
+   * @param part_vals list of partition values.
+   * @return true if the partition exists, false otherwise.
+   * @throws MetaException failure reading RDBMS
+   * @throws NoSuchObjectException this is never thrown.
+   */
+  boolean doesPartitionExist(String catName, String dbName, String tableName,
       List<String> part_vals) throws MetaException, NoSuchObjectException;
 
-  boolean dropPartition(String dbName, String tableName,
+  /**
+   * Drop a partition.
+   * @param catName catalog name.
+   * @param dbName database name.
+   * @param tableName table name.
+   * @param part_vals list of partition values.
+   * @return true if the partition was dropped.
+   * @throws MetaException Error accessing the RDBMS.
+   * @throws NoSuchObjectException no partition matching this description exists
+   * @throws InvalidObjectException error dropping the statistics for the partition
+   * @throws InvalidInputException error dropping the statistics for the partition
+   */
+  boolean dropPartition(String catName, String dbName, String tableName,
       List<String> part_vals) throws MetaException, NoSuchObjectException, InvalidObjectException,
       InvalidInputException;
 
-  List<Partition> getPartitions(String dbName,
+  /**
+   * Get some or all partitions for a table.
+   * @param catName catalog name.
+   * @param dbName database name.
+   * @param tableName table name
+   * @param max maximum number of partitions, or -1 to get all partitions.
+   * @return list of partitions
+   * @throws MetaException error access the RDBMS.
+   * @throws NoSuchObjectException no such table exists
+   */
+  List<Partition> getPartitions(String catName, String dbName,
       String tableName, int max) throws MetaException, NoSuchObjectException;
 
-  void alterTable(String dbname, String name, Table newTable)
+  /**
+   * Alter a table.
+   * @param catName catalog the table is in.
+   * @param dbname database the table is in.
+   * @param name name of the table.
+   * @param newTable New table object.  Which parts of the table can be altered are
+   *                 implementation specific.
+   * @throws InvalidObjectException The new table object is invalid.
+   * @throws MetaException something went wrong, usually in the RDBMS or storage.
+   */
+  void alterTable(String catName, String dbname, String name, Table newTable)
       throws InvalidObjectException, MetaException;
 
-  void updateCreationMetadata(String dbname, String tablename, CreationMetadata cm)
+  /**
+   * Update creation metadata for a materialized view.
+   * @param catName catalog name.
+   * @param dbname database name.
+   * @param tablename table name.
+   * @param cm new creation metadata
+   * @throws MetaException error accessing the RDBMS.
+   */
+  void updateCreationMetadata(String catName, String dbname, String tablename, CreationMetadata cm)
       throws MetaException;
 
-  List<String> getTables(String dbName, String pattern)
+  /**
+   * Get table names that match a pattern.
+   * @param catName catalog to search in
+   * @param dbName database to search in
+   * @param pattern pattern to match
+   * @return list of table names, if any
+   * @throws MetaException failure in querying the RDBMS
+   */
+  List<String> getTables(String catName, String dbName, String pattern)
       throws MetaException;
 
-  List<String> getTables(String dbName, String pattern, TableType tableType)
+  /**
+   * Get table names that match a pattern.
+   * @param catName catalog to search in
+   * @param dbName database to search in
+   * @param pattern pattern to match
+   * @param tableType type of table to look for
+   * @return list of table names, if any
+   * @throws MetaException failure in querying the RDBMS
+   */
+  List<String> getTables(String catName, String dbName, String pattern, TableType tableType)
       throws MetaException;
 
-  List<String> getMaterializedViewsForRewriting(String dbName)
+  /**
+   * Get list of materialized views in a database.
+   * @param catName catalog name
+   * @param dbName database name
+   * @return names of all materialized views in the database
+   * @throws MetaException error querying the RDBMS
+   * @throws NoSuchObjectException no such database
+   */
+  List<String> getMaterializedViewsForRewriting(String catName, String dbName)
       throws MetaException, NoSuchObjectException;
 
-  List<TableMeta> getTableMeta(
-      String dbNames, String tableNames, List<String> tableTypes) throws MetaException;
+  /**
+
+   * @param catName catalog name to search in. Search must be confined to one catalog.
+   * @param dbNames databases to search in.
+   * @param tableNames names of tables to select.
+   * @param tableTypes types of tables to look for.
+   * @return list of matching table meta information.
+   * @throws MetaException failure in querying the RDBMS.
+   */
+  List<TableMeta> getTableMeta(String catName, String dbNames, String tableNames,
+                               List<String> tableTypes) throws MetaException;
 
   /**
+   * @param catName catalog name
    * @param dbname
    *        The name of the database from which to retrieve the tables
    * @param tableNames
@@ -203,15 +441,23 @@ public interface RawStore extends Configurable {
    * @return A list of the tables retrievable from the database
    *          whose names are in the list tableNames.
    *         If there are duplicate names, only one instance of the table will be returned
-   * @throws MetaException
+   * @throws MetaException failure in querying the RDBMS.
    */
-  List<Table> getTableObjectsByName(String dbname, List<String> tableNames)
+  List<Table> getTableObjectsByName(String catName, String dbname, List<String> tableNames)
       throws MetaException, UnknownDBException;
 
-  List<String> getAllTables(String dbName) throws MetaException;
+  /**
+   * Get all tables in a database.
+   * @param catName catalog name.
+   * @param dbName database name.
+   * @return list of table names
+   * @throws MetaException failure in querying the RDBMS.
+   */
+  List<String> getAllTables(String catName, String dbName) throws MetaException;
 
   /**
    * Gets a list of tables based on a filter string and filter type.
+   * @param catName catalog name
    * @param dbName
    *          The name of the database from which you will retrieve the table names
    * @param filter
@@ -222,46 +468,145 @@ public interface RawStore extends Configurable {
    * @throws MetaException
    * @throws UnknownDBException
    */
-  List<String> listTableNamesByFilter(String dbName,
-      String filter, short max_tables) throws MetaException, UnknownDBException;
+  List<String> listTableNamesByFilter(String catName, String dbName, String filter,
+                                      short max_tables) throws MetaException, UnknownDBException;
 
-  List<String> listPartitionNames(String db_name,
+  /**
+   * Get a partial or complete list of names for partitions of a table.
+   * @param catName catalog name.
+   * @param db_name database name.
+   * @param tbl_name table name.
+   * @param max_parts maximum number of partitions to retrieve, -1 for all.
+   * @return list of partition names.
+   * @throws MetaException there was an error accessing the RDBMS
+   */
+  List<String> listPartitionNames(String catName, String db_name,
       String tbl_name, short max_parts) throws MetaException;
 
-  PartitionValuesResponse listPartitionValues(String db_name, String tbl_name,
+  /**
+   * Get a list of partition values as one big struct.
+   * @param catName catalog name.
+   * @param db_name database name.
+   * @param tbl_name table name.
+   * @param cols partition key columns
+   * @param applyDistinct whether to apply distinct to the list
+   * @param filter filter to apply to the partition names
+   * @param ascending whether to put in ascending order
+   * @param order whether to order
+   * @param maxParts maximum number of parts to return, or -1 for all
+   * @return struct with all of the partition value information
+   * @throws MetaException error access the RDBMS
+   */
+  PartitionValuesResponse listPartitionValues(String catName, String db_name, String tbl_name,
                                               List<FieldSchema> cols, boolean applyDistinct, String filter, boolean ascending,
                                               List<FieldSchema> order, long maxParts) throws MetaException;
 
-  List<String> listPartitionNamesByFilter(String db_name,
-      String tbl_name, String filter, short max_parts) throws MetaException;
-
-  void alterPartition(String db_name, String tbl_name, List<String> part_vals,
+  /**
+   * Alter a partition.
+   * @param catName catalog name.
+   * @param db_name database name.
+   * @param tbl_name table name.
+   * @param part_vals partition values that describe the partition.
+   * @param new_part new partition object.  This should be a complete copy of the old with
+   *                 changes values, not just the parts to update.
+   * @throws InvalidObjectException No such partition.
+   * @throws MetaException error accessing the RDBMS.
+   */
+  void alterPartition(String catName, String db_name, String tbl_name, List<String> part_vals,
       Partition new_part) throws InvalidObjectException, MetaException;
 
-  void alterPartitions(String db_name, String tbl_name,
+  /**
+   * Alter a set of partitions.
+   * @param catName catalog name.
+   * @param db_name database name.
+   * @param tbl_name table name.
+   * @param part_vals_list list of list of partition values.  Each outer list describes one
+   *                       partition (with its list of partition values).
+   * @param new_parts list of new partitions.  The order must match the old partitions described in
+   *                  part_vals_list.  Each of these should be a complete copy of the new
+   *                  partition, not just the pieces to update.
+   * @throws InvalidObjectException One of the indicated partitions does not exist.
+   * @throws MetaException error accessing the RDBMS.
+   */
+  void alterPartitions(String catName, String db_name, String tbl_name,
       List<List<String>> part_vals_list, List<Partition> new_parts)
       throws InvalidObjectException, MetaException;
 
+  /**
+   * Get partitions with a filter.  This is a portion of the SQL where clause.
+   * @param catName catalog name
+   * @param dbName database name
+   * @param tblName table name
+   * @param filter SQL where clause filter
+   * @param maxParts maximum number of partitions to return, or -1 for all.
+   * @return list of partition objects matching the criteria
+   * @throws MetaException Error accessing the RDBMS or processing the filter.
+   * @throws NoSuchObjectException no such table.
+   */
   List<Partition> getPartitionsByFilter(
-      String dbName, String tblName, String filter, short maxParts)
+      String catName, String dbName, String tblName, String filter, short maxParts)
       throws MetaException, NoSuchObjectException;
 
-  boolean getPartitionsByExpr(String dbName, String tblName,
+  /**
+   * Get partitions using an already parsed expression.
+   * @param catName catalog name.
+   * @param dbName database name
+   * @param tblName table name
+   * @param expr an already parsed Hive expression
+   * @param defaultPartitionName default name of a partition
+   * @param maxParts maximum number of partitions to return, or -1 for all
+   * @param result list to place resulting partitions in
+   * @return true if the result contains unknown partitions.
+   * @throws TException error executing the expression
+   */
+  boolean getPartitionsByExpr(String catName, String dbName, String tblName,
       byte[] expr, String defaultPartitionName, short maxParts, List<Partition> result)
       throws TException;
 
-  int getNumPartitionsByFilter(String dbName, String tblName, String filter)
+  /**
+   * Get the number of partitions that match a provided SQL filter.
+   * @param catName catalog name.
+   * @param dbName database name.
+   * @param tblName table name.
+   * @param filter filter from Hive's SQL where clause
+   * @return number of matching partitions.
+   * @throws MetaException error accessing the RDBMS or executing the filter
+   * @throws NoSuchObjectException no such table
+   */
+  int getNumPartitionsByFilter(String catName, String dbName, String tblName, String filter)
     throws MetaException, NoSuchObjectException;
 
-  int getNumPartitionsByExpr(String dbName, String tblName, byte[] expr) throws MetaException, NoSuchObjectException;
+  /**
+   * Get the number of partitions that match an already parsed expression.
+   * @param catName catalog name.
+   * @param dbName database name.
+   * @param tblName table name.
+   * @param expr an already parsed Hive expression
+   * @return number of matching partitions.
+   * @throws MetaException error accessing the RDBMS or working with the expression.
+   * @throws NoSuchObjectException no such table.
+   */
+  int getNumPartitionsByExpr(String catName, String dbName, String tblName, byte[] expr)
+      throws MetaException, NoSuchObjectException;
 
-  List<Partition> getPartitionsByNames(
-      String dbName, String tblName, List<String> partNames)
+  /**
+   * Get partitions by name.
+   * @param catName catalog name.
+   * @param dbName database name.
+   * @param tblName table name.
+   * @param partNames list of partition names.  These are names not values, so they will include
+   *                  both the key and the value.
+   * @return list of matching partitions
+   * @throws MetaException error accessing the RDBMS.
+   * @throws NoSuchObjectException No such table.
+   */
+  List<Partition> getPartitionsByNames(String catName, String dbName, String tblName,
+                                       List<String> partNames)
       throws MetaException, NoSuchObjectException;
 
-  Table markPartitionForEvent(String dbName, String tblName, Map<String,String> partVals, PartitionEventType evtType) throws MetaException, UnknownTableException, InvalidPartitionException, UnknownPartitionException;
+  Table markPartitionForEvent(String catName, String dbName, String tblName, Map<String,String> partVals, PartitionEventType evtType) throws MetaException, UnknownTableException, InvalidPartitionException, UnknownPartitionException;
 
-  boolean isPartitionMarkedForEvent(String dbName, String tblName, Map<String, String> partName, PartitionEventType evtType) throws MetaException, UnknownTableException, InvalidPartitionException, UnknownPartitionException;
+  boolean isPartitionMarkedForEvent(String catName, String dbName, String tblName, Map<String, String> partName, PartitionEventType evtType) throws MetaException, UnknownTableException, InvalidPartitionException, UnknownPartitionException;
 
   boolean addRole(String rowName, String ownerName)
       throws InvalidObjectException, MetaException, NoSuchObjectException;
@@ -278,38 +623,132 @@ public interface RawStore extends Configurable {
   PrincipalPrivilegeSet getUserPrivilegeSet(String userName,
       List<String> groupNames) throws InvalidObjectException, MetaException;
 
-  PrincipalPrivilegeSet getDBPrivilegeSet (String dbName, String userName,
+  /**
+   * Get privileges for a database for a user.
+   * @param catName catalog name
+   * @param dbName database name
+   * @param userName user name
+   * @param groupNames list of groups the user is in
+   * @return privileges for that user on indicated database
+   * @throws InvalidObjectException no such database
+   * @throws MetaException error accessing the RDBMS
+   */
+  PrincipalPrivilegeSet getDBPrivilegeSet (String catName, String dbName, String userName,
       List<String> groupNames)  throws InvalidObjectException, MetaException;
 
-  PrincipalPrivilegeSet getTablePrivilegeSet (String dbName, String tableName,
+  /**
+   * Get privileges for a table for a user.
+   * @param catName catalog name
+   * @param dbName database name
+   * @param tableName table name
+   * @param userName user name
+   * @param groupNames list of groups the user is in
+   * @return privileges for that user on indicated table
+   * @throws InvalidObjectException no such table
+   * @throws MetaException error accessing the RDBMS
+   */
+  PrincipalPrivilegeSet getTablePrivilegeSet (String catName, String dbName, String tableName,
       String userName, List<String> groupNames) throws InvalidObjectException, MetaException;
 
-  PrincipalPrivilegeSet getPartitionPrivilegeSet (String dbName, String tableName,
+  /**
+   * Get privileges for a partition for a user.
+   * @param catName catalog name
+   * @param dbName database name
+   * @param tableName table name
+   * @param partition partition name
+   * @param userName user name
+   * @param groupNames list of groups the user is in
+   * @return privileges for that user on indicated partition
+   * @throws InvalidObjectException no such partition
+   * @throws MetaException error accessing the RDBMS
+   */
+  PrincipalPrivilegeSet getPartitionPrivilegeSet (String catName, String dbName, String tableName,
       String partition, String userName, List<String> groupNames) throws InvalidObjectException, MetaException;
 
-  PrincipalPrivilegeSet getColumnPrivilegeSet (String dbName, String tableName, String partitionName,
+  /**
+   * Get privileges for a column in a table or partition for a user.
+   * @param catName catalog name
+   * @param dbName database name
+   * @param tableName table name
+   * @param partitionName partition name, or null for table level column permissions
+   * @param columnName column name
+   * @param userName user name
+   * @param groupNames list of groups the user is in
+   * @return privileges for that user on indicated column in the table or partition
+   * @throws InvalidObjectException no such table, partition, or column
+   * @throws MetaException error accessing the RDBMS
+   */
+  PrincipalPrivilegeSet getColumnPrivilegeSet (String catName, String dbName, String tableName, String partitionName,
       String columnName, String userName, List<String> groupNames) throws InvalidObjectException, MetaException;
 
   List<HiveObjectPrivilege> listPrincipalGlobalGrants(String principalName,
       PrincipalType principalType);
 
+  /**
+   * For a given principal name and type, list the DB Grants
+   * @param principalName principal name
+   * @param principalType type
+   * @param catName catalog name
+   * @param dbName database name
+   * @return list of privileges for that principal on the specified database.
+   */
   List<HiveObjectPrivilege> listPrincipalDBGrants(String principalName,
-      PrincipalType principalType, String dbName);
+      PrincipalType principalType, String catName, String dbName);
 
+  /**
+   * For a given principal name and type, list the Table Grants
+   * @param principalName principal name
+   * @param principalType type
+   * @param catName catalog name
+   * @param dbName database name
+   * @param tableName table name
+   * @return list of privileges for that principal on the specified database.
+   */
   List<HiveObjectPrivilege> listAllTableGrants(
-      String principalName, PrincipalType principalType, String dbName,
+      String principalName, PrincipalType principalType, String catName, String dbName,
       String tableName);
 
+  /**
+   * For a given principal name and type, list the Table Grants
+   * @param principalName principal name
+   * @param principalType type
+   * @param catName catalog name
+   * @param dbName database name
+   * @param tableName table name
+   * @param partName partition name (not value)
+   * @return list of privileges for that principal on the specified database.
+   */
   List<HiveObjectPrivilege> listPrincipalPartitionGrants(
-      String principalName, PrincipalType principalType, String dbName,
+      String principalName, PrincipalType principalType, String catName, String dbName,
       String tableName, List<String> partValues, String partName);
 
+  /**
+   * For a given principal name and type, list the Table Grants
+   * @param principalName principal name
+   * @param principalType type
+   * @param catName catalog name
+   * @param dbName database name
+   * @param tableName table name
+   * @param columnName column name
+   * @return list of privileges for that principal on the specified database.
+   */
   List<HiveObjectPrivilege> listPrincipalTableColumnGrants(
-      String principalName, PrincipalType principalType, String dbName,
+      String principalName, PrincipalType principalType, String catName, String dbName,
       String tableName, String columnName);
 
+  /**
+   * For a given principal name and type, list the Table Grants
+   * @param principalName principal name
+   * @param principalType type
+   * @param catName catalog name
+   * @param dbName database name
+   * @param tableName table name
+   * @param partName partition name (not value)
+   * @param columnName column name
+   * @return list of privileges for that principal on the specified database.
+   */
   List<HiveObjectPrivilege> listPrincipalPartitionColumnGrants(
-      String principalName, PrincipalType principalType, String dbName,
+      String principalName, PrincipalType principalType, String catName, String dbName,
       String tableName, List<String> partValues, String partName, String columnName);
 
   boolean grantPrivileges (PrivilegeBag privileges)
@@ -338,16 +777,44 @@ public interface RawStore extends Configurable {
   List<RolePrincipalGrant> listRoleMembers(String roleName);
 
 
-  Partition getPartitionWithAuth(String dbName, String tblName,
+  /**
+   * Fetch a partition along with privilege information for a particular user.
+   * @param catName catalog name.
+   * @param dbName database name.
+   * @param tblName table name.
+   * @param partVals partition values
+   * @param user_name user to get privilege information for.
+   * @param group_names groups to get privilege information for.
+   * @return a partition
+   * @throws MetaException error accessing the RDBMS.
+   * @throws NoSuchObjectException no such partition exists
+   * @throws InvalidObjectException error fetching privilege information
+   */
+  Partition getPartitionWithAuth(String catName, String dbName, String tblName,
       List<String> partVals, String user_name, List<String> group_names)
       throws MetaException, NoSuchObjectException, InvalidObjectException;
 
-  List<Partition> getPartitionsWithAuth(String dbName,
+  /**
+   * Fetch some or all partitions for a table, along with privilege information for a particular
+   * user.
+   * @param catName catalog name.
+   * @param dbName database name.
+   * @param tblName table name.
+   * @param maxParts maximum number of partitions to fetch, -1 for all partitions.
+   * @param userName user to get privilege information for.
+   * @param groupNames groups to get privilege information for.
+   * @return list of partitions.
+   * @throws MetaException error access the RDBMS.
+   * @throws NoSuchObjectException no such table exists
+   * @throws InvalidObjectException error fetching privilege information.
+   */
+  List<Partition> getPartitionsWithAuth(String catName, String dbName,
       String tblName, short maxParts, String userName, List<String> groupNames)
       throws MetaException, NoSuchObjectException, InvalidObjectException;
 
   /**
    * Lists partition names that match a given partial specification
+   * @param catName catalog name.
    * @param db_name
    *          The name of the database which has the partitions
    * @param tbl_name
@@ -358,16 +825,17 @@ public interface RawStore extends Configurable {
    * @param max_parts
    *          The maximum number of partitions to return
    * @return A list of partition names that match the partial spec.
-   * @throws MetaException
-   * @throws NoSuchObjectException
+   * @throws MetaException error accessing RDBMS
+   * @throws NoSuchObjectException No such table exists
    */
-  List<String> listPartitionNamesPs(String db_name, String tbl_name,
+  List<String> listPartitionNamesPs(String catName, String db_name, String tbl_name,
       List<String> part_vals, short max_parts)
       throws MetaException, NoSuchObjectException;
 
   /**
    * Lists partitions that match a given partial specification and sets their auth privileges.
    *   If userName and groupNames null, then no auth privileges are set.
+   * @param catName catalog name.
    * @param db_name
    *          The name of the database which has the partitions
    * @param tbl_name
@@ -382,34 +850,33 @@ public interface RawStore extends Configurable {
    * @param groupNames
    *          The groupNames for the partition for authentication privileges
    * @return A list of partitions that match the partial spec.
-   * @throws MetaException
-   * @throws NoSuchObjectException
-   * @throws InvalidObjectException
+   * @throws MetaException error access RDBMS
+   * @throws NoSuchObjectException No such table exists
+   * @throws InvalidObjectException error access privilege information
    */
-  List<Partition> listPartitionsPsWithAuth(String db_name, String tbl_name,
+  List<Partition> listPartitionsPsWithAuth(String catName, String db_name, String tbl_name,
       List<String> part_vals, short max_parts, String userName, List<String> groupNames)
       throws MetaException, InvalidObjectException, NoSuchObjectException;
 
   /** Persists the given column statistics object to the metastore
    * @param colStats object to persist
    * @return Boolean indicating the outcome of the operation
-   * @throws NoSuchObjectException
-   * @throws MetaException
-   * @throws InvalidObjectException
-   * @throws InvalidInputException
+   * @throws NoSuchObjectException No such table.
+   * @throws MetaException error accessing the RDBMS.
+   * @throws InvalidObjectException the stats object is invalid
+   * @throws InvalidInputException unable to record the stats for the table
    */
   boolean updateTableColumnStatistics(ColumnStatistics colStats)
       throws NoSuchObjectException, MetaException, InvalidObjectException, InvalidInputException;
 
   /** Persists the given column statistics object to the metastore
-   * @param partVals
-   *
    * @param statsObj object to persist
+   * @param partVals partition values to persist the stats for
    * @return Boolean indicating the outcome of the operation
-   * @throws NoSuchObjectException
-   * @throws MetaException
-   * @throws InvalidObjectException
-   * @throws InvalidInputException
+   * @throws NoSuchObjectException No such table.
+   * @throws MetaException error accessing the RDBMS.
+   * @throws InvalidObjectException the stats object is invalid
+   * @throws InvalidInputException unable to record the stats for the table
    */
   boolean updatePartitionColumnStatistics(ColumnStatistics statsObj,
      List<String> partVals)
@@ -418,64 +885,67 @@ public interface RawStore extends Configurable {
   /**
    * Returns the relevant column statistics for a given column in a given table in a given database
    * if such statistics exist.
-   *
+   * @param catName catalog name.
    * @param dbName name of the database, defaults to current database
    * @param tableName name of the table
    * @param colName names of the columns for which statistics is requested
    * @return Relevant column statistics for the column for the given table
-   * @throws NoSuchObjectException
-   * @throws MetaException
+   * @throws NoSuchObjectException No such table
+   * @throws MetaException error accessing the RDBMS
    *
    */
-  ColumnStatistics getTableColumnStatistics(String dbName, String tableName,
+  ColumnStatistics getTableColumnStatistics(String catName, String dbName, String tableName,
     List<String> colName) throws MetaException, NoSuchObjectException;
 
   /**
-   * Returns the relevant column statistics for given columns in given partitions in a given
-   * table in a given database if such statistics exist.
+   * Get statistics for a partition for a set of columns.
+   * @param catName catalog name.
+   * @param dbName database name.
+   * @param tblName table name.
+   * @param partNames list of partition names.  These are names so must be key1=val1[/key2=val2...]
+   * @param colNames list of columns to get stats for
+   * @return list of statistics objects
+   * @throws MetaException error accessing the RDBMS
+   * @throws NoSuchObjectException no such partition.
    */
   List<ColumnStatistics> getPartitionColumnStatistics(
-     String dbName, String tblName, List<String> partNames, List<String> colNames)
+     String catName, String dbName, String tblName, List<String> partNames, List<String> colNames)
       throws MetaException, NoSuchObjectException;
 
   /**
    * Deletes column statistics if present associated with a given db, table, partition and col. If
    * null is passed instead of a colName, stats when present for all columns associated
    * with a given db, table and partition are deleted.
-   *
-   * @param dbName
-   * @param tableName
-   * @param partName
-   * @param partVals
-   * @param colName
+   * @param catName catalog name.
+   * @param dbName database name.
+   * @param tableName table name.
+   * @param partName partition name.
+   * @param partVals partition values.
+   * @param colName column name.
    * @return Boolean indicating the outcome of the operation
-   * @throws NoSuchObjectException
-   * @throws MetaException
-   * @throws InvalidObjectException
-   * @throws InvalidInputException
+   * @throws NoSuchObjectException no such partition
+   * @throws MetaException error access the RDBMS
+   * @throws InvalidObjectException error dropping the stats
+   * @throws InvalidInputException bad input, such as null table or database name.
    */
-
-  boolean deletePartitionColumnStatistics(String dbName, String tableName,
+  boolean deletePartitionColumnStatistics(String catName, String dbName, String tableName,
       String partName, List<String> partVals, String colName)
       throws NoSuchObjectException, MetaException, InvalidObjectException, InvalidInputException;
 
   /**
-   * Deletes column statistics if present associated with a given db, table and col. If
-   * null is passed instead of a colName, stats when present for all columns associated
-   * with a given db and table are deleted.
-   *
-   * @param dbName
-   * @param tableName
-   * @param colName
-   * @return Boolean indicating the outcome of the operation
-   * @throws NoSuchObjectException
-   * @throws MetaException
-   * @throws InvalidObjectException
-   * @throws InvalidInputException
+   * Delete statistics for a single column or all columns in a table.
+   * @param catName catalog name
+   * @param dbName database name
+   * @param tableName table name
+   * @param colName column name.  Null to delete stats for all columns in the table.
+   * @return true if the statistics were deleted.
+   * @throws NoSuchObjectException no such table or column.
+   * @throws MetaException error access the RDBMS.
+   * @throws InvalidObjectException error dropping the stats
+   * @throws InvalidInputException bad inputs, such as null table name.
    */
-
-  boolean deleteTableColumnStatistics(String dbName, String tableName,
-    String colName)
+  boolean deleteTableColumnStatistics(String catName, String dbName, String tableName,
+                                      String colName)
     throws NoSuchObjectException, MetaException, InvalidObjectException, InvalidInputException;
 
   long cleanupEvents();
@@ -503,100 +973,203 @@ public interface RawStore extends Configurable {
 
   abstract void setMetaStoreSchemaVersion(String version, String comment) throws MetaException;
 
-  void dropPartitions(String dbName, String tblName, List<String> partNames)
+  /**
+   * Drop a list of partitions.
+   * @param catName catalog name.
+   * @param dbName database name.
+   * @param tblName table name
+   * @param partNames list of partition names.
+   * @throws MetaException error access RDBMS or storage.
+   * @throws NoSuchObjectException One or more of the partitions does not exist.
+   */
+  void dropPartitions(String catName, String dbName, String tblName, List<String> partNames)
       throws MetaException, NoSuchObjectException;
 
+  /**
+   * List all DB grants for a given principal.
+   * @param principalName principal name
+   * @param principalType type
+   * @return all DB grants for this principal
+   */
   List<HiveObjectPrivilege> listPrincipalDBGrantsAll(
       String principalName, PrincipalType principalType);
 
+  /**
+   * List all Table grants for a given principal
+   * @param principalName principal name
+   * @param principalType type
+   * @return all Table grants for this principal
+   */
   List<HiveObjectPrivilege> listPrincipalTableGrantsAll(
       String principalName, PrincipalType principalType);
 
+  /**
+   * List all Partition grants for a given principal
+   * @param principalName principal name
+   * @param principalType type
+   * @return all Partition grants for this principal
+   */
   List<HiveObjectPrivilege> listPrincipalPartitionGrantsAll(
       String principalName, PrincipalType principalType);
 
+  /**
+   * List all Table column grants for a given principal
+   * @param principalName principal name
+   * @param principalType type
+   * @return all Table column grants for this principal
+   */
   List<HiveObjectPrivilege> listPrincipalTableColumnGrantsAll(
       String principalName, PrincipalType principalType);
 
+  /**
+   * List all Partition column grants for a given principal
+   * @param principalName principal name
+   * @param principalType type
+   * @return all Partition column grants for this principal
+   */
   List<HiveObjectPrivilege> listPrincipalPartitionColumnGrantsAll(
       String principalName, PrincipalType principalType);
 
   List<HiveObjectPrivilege> listGlobalGrantsAll();
 
-  List<HiveObjectPrivilege> listDBGrantsAll(String dbName);
+  /**
+   * Find all the privileges for a given database.
+   * @param catName catalog name
+   * @param dbName database name
+   * @return list of all privileges.
+   */
+  List<HiveObjectPrivilege> listDBGrantsAll(String catName, String dbName);
 
+  /**
+   * Find all of the privileges for a given column in a given partition.
+   * @param catName catalog name
+   * @param dbName database name
+   * @param tableName table name
+   * @param partitionName partition name (not value)
+   * @param columnName column name
+   * @return all privileges on this column in this partition
+   */
   List<HiveObjectPrivilege> listPartitionColumnGrantsAll(
-      String dbName, String tableName, String partitionName, String columnName);
+      String catName, String dbName, String tableName, String partitionName, String columnName);
 
-  List<HiveObjectPrivilege> listTableGrantsAll(String dbName, String tableName);
+  /**
+   * Find all of the privileges for a given table
+   * @param catName catalog name
+   * @param dbName database name
+   * @param tableName table name
+   * @return all privileges on this table
+   */
+  List<HiveObjectPrivilege> listTableGrantsAll(String catName, String dbName, String tableName);
 
+  /**
+   * Find all of the privileges for a given partition.
+   * @param catName catalog name
+   * @param dbName database name
+   * @param tableName table name
+   * @param partitionName partition name (not value)
+   * @return all privileges on this partition
+   */
   List<HiveObjectPrivilege> listPartitionGrantsAll(
-      String dbName, String tableName, String partitionName);
+      String catName, String dbName, String tableName, String partitionName);
 
+  /**
+   * Find all of the privileges for a given column in a given table.
+   * @param catName catalog name
+   * @param dbName database name
+   * @param tableName table name
+   * @param columnName column name
+   * @return all privileges on this column in this table
+   */
   List<HiveObjectPrivilege> listTableColumnGrantsAll(
-      String dbName, String tableName, String columnName);
+      String catName, String dbName, String tableName, String columnName);
 
   /**
    * Register a user-defined function based on the function specification passed in.
-   * @param func
-   * @throws InvalidObjectException
-   * @throws MetaException
+   * @param func function to create
+   * @throws InvalidObjectException incorrectly specified function
+   * @throws MetaException error accessing the RDBMS
    */
   void createFunction(Function func)
       throws InvalidObjectException, MetaException;
 
   /**
    * Alter function based on new function specs.
-   * @param dbName
-   * @param funcName
-   * @param newFunction
-   * @throws InvalidObjectException
-   * @throws MetaException
+   * @param dbName database name
+   * @param funcName function name
+   * @param newFunction new function specification
+   * @throws InvalidObjectException no such function, or incorrectly specified new function
+   * @throws MetaException incorrectly specified function
    */
-  void alterFunction(String dbName, String funcName, Function newFunction)
+  void alterFunction(String catName, String dbName, String funcName, Function newFunction)
       throws InvalidObjectException, MetaException;
 
   /**
    * Drop a function definition.
-   * @param dbName
-   * @param funcName
-   * @throws MetaException
-   * @throws NoSuchObjectException
-   * @throws InvalidObjectException
-   * @throws InvalidInputException
+   * @param dbName database name
+   * @param funcName function name
+   * @throws MetaException incorrectly specified function
+   * @throws NoSuchObjectException no such function
+   * @throws InvalidObjectException not sure when this is thrown
+   * @throws InvalidInputException not sure when this is thrown
    */
-  void dropFunction(String dbName, String funcName)
+  void dropFunction(String catName, String dbName, String funcName)
       throws MetaException, NoSuchObjectException, InvalidObjectException, InvalidInputException;
 
   /**
    * Retrieve function by name.
-   * @param dbName
-   * @param funcName
-   * @return
-   * @throws MetaException
+   * @param dbName database name
+   * @param funcName function name
+   * @return the function
+   * @throws MetaException incorrectly specified function
    */
-  Function getFunction(String dbName, String funcName) throws MetaException;
+  Function getFunction(String catName, String dbName, String funcName) throws MetaException;
 
   /**
    * Retrieve all functions.
-   * @return
-   * @throws MetaException
+   * @return all functions in a catalog
+   * @throws MetaException incorrectly specified function
    */
-  List<Function> getAllFunctions() throws MetaException;
+  List<Function> getAllFunctions(String catName) throws MetaException;
 
   /**
    * Retrieve list of function names based on name pattern.
-   * @param dbName
-   * @param pattern
-   * @return
-   * @throws MetaException
+   * @param dbName database name
+   * @param pattern pattern to match
+   * @return functions that match the pattern
+   * @throws MetaException incorrectly specified function
    */
-  List<String> getFunctions(String dbName, String pattern) throws MetaException;
+  List<String> getFunctions(String catName, String dbName, String pattern) throws MetaException;
 
-  AggrStats get_aggr_stats_for(String dbName, String tblName,
+  /**
+   * Get aggregated stats for a table or partition(s).
+   * @param catName catalog name.
+   * @param dbName database name.
+   * @param tblName table name.
+   * @param partNames list of partition names.  These are the names of the partitions, not
+   *                  values.
+   * @param colNames list of column names
+   * @return aggregated stats
+   * @throws MetaException error accessing RDBMS
+   * @throws NoSuchObjectException no such table or partition
+   */
+  AggrStats get_aggr_stats_for(String catName, String dbName, String tblName,
     List<String> partNames, List<String> colNames) throws MetaException, NoSuchObjectException;
 
   /**
+<<<<<<< HEAD
+=======
+   * Get column stats for all partitions of all tables in the database
+   * @param catName catalog name
+   * @param dbName database name
+   * @return List of column stats objects for all partitions of all tables in the database
+   * @throws MetaException error accessing RDBMS
+   * @throws NoSuchObjectException no such database
+   */
+  List<ColStatsObjWithSourceInfo> getPartitionColStatsForDatabase(String catName, String dbName)
+      throws MetaException, NoSuchObjectException;
+
+  /**
+>>>>>>> e6d9605492... HIVE-18755 Modifications to the metastore for catalogs
    * Get the next notification event.
    * @param rqst Request containing information on the last processed notification.
    * @return list of notifications, sorted by eventId
@@ -629,7 +1202,7 @@ public interface RawStore extends Configurable {
    * This is intended for use by the repl commands to track the progress of incremental dump.
    * @return
    */
-  public NotificationEventsCountResponse getNotificationEventsCount(NotificationEventsCountRequest rqst);
+  NotificationEventsCountResponse getNotificationEventsCount(NotificationEventsCountRequest rqst);
 
   /*
    * Flush any catalog objects held by the metastore implementation.  Note that this does not
@@ -695,12 +1268,22 @@ public interface RawStore extends Configurable {
   @InterfaceStability.Evolving
   int getDatabaseCount() throws MetaException;
 
-  List<SQLPrimaryKey> getPrimaryKeys(String db_name,
-    String tbl_name) throws MetaException;
+  /**
+   * Get the primary associated with a table.  Strangely enough each SQLPrimaryKey is actually a
+   * column in they key, not the key itself.  Thus the list.
+   * @param catName catalog name
+   * @param db_name database name
+   * @param tbl_name table name
+   * @return list of primary key columns or an empty list if the table does not have a primary key
+   * @throws MetaException error accessing the RDBMS
+   */
+  List<SQLPrimaryKey> getPrimaryKeys(String catName, String db_name, String tbl_name)
+      throws MetaException;
 
   /**
    * Get the foreign keys for a table.  All foreign keys for a particular table can be fetched by
    * passing null for the last two arguments.
+   * @param catName catalog name.
    * @param parent_db_name Database the table referred to is in.  This can be null to match all
    *                       databases.
    * @param parent_tbl_name Table that is referred to.  This can be null to match all tables.
@@ -708,43 +1291,156 @@ public interface RawStore extends Configurable {
    * @param foreign_tbl_name Table with the foreign key.
    * @return List of all matching foreign key columns.  Note that if more than one foreign key
    * matches the arguments the results here will be all mixed together into a single list.
-   * @throws MetaException if something goes wrong.
+   * @throws MetaException error access the RDBMS.
    */
-  List<SQLForeignKey> getForeignKeys(String parent_db_name,
+  List<SQLForeignKey> getForeignKeys(String catName, String parent_db_name,
     String parent_tbl_name, String foreign_db_name, String foreign_tbl_name)
     throws MetaException;
 
-  List<SQLUniqueConstraint> getUniqueConstraints(String db_name,
+  /**
+   * Get unique constraints associated with a table.
+   * @param catName catalog name.
+   * @param db_name database name.
+   * @param tbl_name table name.
+   * @return list of unique constraints
+   * @throws MetaException error access the RDBMS.
+   */
+  List<SQLUniqueConstraint> getUniqueConstraints(String catName, String db_name,
     String tbl_name) throws MetaException;
 
-  List<SQLNotNullConstraint> getNotNullConstraints(String db_name,
+  /**
+   * Get not null constraints on a table.
+   * @param catName catalog name.
+   * @param db_name database name.
+   * @param tbl_name table name.
+   * @return list of not null constraints
+   * @throws MetaException error accessing the RDBMS.
+   */
+  List<SQLNotNullConstraint> getNotNullConstraints(String catName, String db_name,
     String tbl_name) throws MetaException;
 
-  List<SQLDefaultConstraint> getDefaultConstraints(String db_name,
+  /**
+   * Get default values for columns in a table.
+   * @param catName catalog name
+   * @param db_name database name
+   * @param tbl_name table name
+   * @return list of default values defined on the table.
+   * @throws MetaException error accessing the RDBMS
+   */
+  List<SQLDefaultConstraint> getDefaultConstraints(String catName, String db_name,
                                                    String tbl_name) throws MetaException;
 
-  List<SQLCheckConstraint> getCheckConstraints(String db_name,
+  /**
+   * Get check constraints for columns in a table.
+   * @param catName catalog name.
+   * @param db_name database name
+   * @param tbl_name table name
+   * @return ccheck constraints for this table
+   * @throws MetaException error accessing the RDBMS
+   */
+  List<SQLCheckConstraint> getCheckConstraints(String catName, String db_name,
                                                    String tbl_name) throws MetaException;
 
+  /**
+   * Create a table with constraints
+   * @param tbl table definition
+   * @param primaryKeys primary key definition, or null
+   * @param foreignKeys foreign key definition, or null
+   * @param uniqueConstraints unique constraints definition, or null
+   * @param notNullConstraints not null constraints definition, or null
+   * @param defaultConstraints default values definition, or null
+   * @return list of constraint names
+   * @throws InvalidObjectException one of the provided objects is malformed.
+   * @throws MetaException error accessing the RDBMS
+   */
   List<String> createTableWithConstraints(Table tbl, List<SQLPrimaryKey> primaryKeys,
     List<SQLForeignKey> foreignKeys, List<SQLUniqueConstraint> uniqueConstraints,
     List<SQLNotNullConstraint> notNullConstraints,
     List<SQLDefaultConstraint> defaultConstraints,
     List<SQLCheckConstraint> checkConstraints) throws InvalidObjectException, MetaException;
 
-  void dropConstraint(String dbName, String tableName, String constraintName) throws NoSuchObjectException;
+  /**
+   * Drop a constraint, any constraint.  I have no idea why add and get each have separate
+   * methods for each constraint type but drop has only one.
+   * @param catName catalog name
+   * @param dbName database name
+   * @param tableName table name
+   * @param constraintName name of the constraint
+   * @throws NoSuchObjectException no constraint of this name exists
+   */
+  default void dropConstraint(String catName, String dbName, String tableName,
+                              String constraintName) throws NoSuchObjectException {
+    dropConstraint(catName, dbName, tableName, constraintName, false);
+  }
+
+  /**
+   * Drop a constraint, any constraint.  I have no idea why add and get each have separate
+   * methods for each constraint type but drop has only one.
+   * @param catName catalog name
+   * @param dbName database name
+   * @param tableName table name
+   * @param constraintName name of the constraint
+   * @param missingOk if true, it is not an error if there is no constraint of this name.  If
+   *                  false and there is no constraint of this name an exception will be thrown.
+   * @throws NoSuchObjectException no constraint of this name exists and missingOk = false
+   */
+  void dropConstraint(String catName, String dbName, String tableName, String constraintName,
+                      boolean missingOk) throws NoSuchObjectException;
 
+  /**
+   * Add a primary key to a table.
+   * @param pks Columns in the primary key.
+   * @return the name of the constraint, as a list of strings.
+   * @throws InvalidObjectException The SQLPrimaryKeys list is malformed
+   * @throws MetaException error accessing the RDMBS
+   */
   List<String> addPrimaryKeys(List<SQLPrimaryKey> pks) throws InvalidObjectException, MetaException;
 
+  /**
+   * Add a foreign key to a table.
+   * @param fks foreign key specification
+   * @return foreign key name.
+   * @throws InvalidObjectException the specification is malformed.
+   * @throws MetaException error accessing the RDBMS.
+   */
   List<String> addForeignKeys(List<SQLForeignKey> fks) throws InvalidObjectException, MetaException;
 
+  /**
+   * Add unique constraints to a table.
+   * @param uks unique constraints specification
+   * @return unique constraint names.
+   * @throws InvalidObjectException the specification is malformed.
+   * @throws MetaException error accessing the RDBMS.
+   */
   List<String> addUniqueConstraints(List<SQLUniqueConstraint> uks) throws InvalidObjectException, MetaException;
 
+  /**
+   * Add not null constraints to a table.
+   * @param nns not null constraint specifications
+   * @return constraint names.
+   * @throws InvalidObjectException the specification is malformed.
+   * @throws MetaException error accessing the RDBMS.
+   */
   List<String> addNotNullConstraints(List<SQLNotNullConstraint> nns) throws InvalidObjectException, MetaException;
 
-  List<String> addDefaultConstraints(List<SQLDefaultConstraint> nns) throws InvalidObjectException, MetaException;
+  /**
+   * Add default values to a table definition
+   * @param dv list of default values
+   * @return constraint names
+   * @throws InvalidObjectException the specification is malformed.
+   * @throws MetaException error accessing the RDBMS.
+   */
+  List<String> addDefaultConstraints(List<SQLDefaultConstraint> dv)
+      throws InvalidObjectException, MetaException;
 
-  List<String> addCheckConstraints(List<SQLCheckConstraint> nns) throws InvalidObjectException, MetaException;
+  /**
+   * Add check constraints to a table
+   * @param cc check constraints to add
+   * @return list of constraint names
+   * @throws InvalidObjectException the specification is malformed
+   * @throws MetaException error accessing the RDBMS
+   */
+  List<String> addCheckConstraints(List<SQLCheckConstraint> cc) throws InvalidObjectException, MetaException;
 
   /**
    * Gets the unique id of the backing datastore for the metadata

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/StatObjectConverter.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/StatObjectConverter.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/StatObjectConverter.java
index 08ea67f..0074e01 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/StatObjectConverter.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/StatObjectConverter.java
@@ -48,6 +48,8 @@ import org.apache.hadoop.hive.metastore.model.MPartitionColumnStatistics;
 import org.apache.hadoop.hive.metastore.model.MTable;
 import org.apache.hadoop.hive.metastore.model.MTableColumnStatistics;
 
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
+
 /**
  * This class contains conversion logic that creates Thrift stat objects from
  * JDO stat objects and plain arrays from DirectSQL.
@@ -65,6 +67,7 @@ public class StatObjectConverter {
      MTableColumnStatistics mColStats = new MTableColumnStatistics();
      mColStats.setTable(table);
      mColStats.setDbName(statsDesc.getDbName());
+     mColStats.setCatName(statsDesc.isSetCatName() ? statsDesc.getCatName() : DEFAULT_CATALOG_NAME);
      mColStats.setTableName(statsDesc.getTableName());
      mColStats.setLastAnalyzed(statsDesc.getLastAnalyzed());
      mColStats.setColName(statsObj.getColName());
@@ -311,6 +314,7 @@ public class StatObjectConverter {
       MTableColumnStatistics mStatsObj) {
     ColumnStatisticsDesc statsDesc = new ColumnStatisticsDesc();
     statsDesc.setIsTblLevel(true);
+    statsDesc.setCatName(mStatsObj.getCatName());
     statsDesc.setDbName(mStatsObj.getDbName());
     statsDesc.setTableName(mStatsObj.getTableName());
     statsDesc.setLastAnalyzed(mStatsObj.getLastAnalyzed());
@@ -326,6 +330,7 @@ public class StatObjectConverter {
 
     MPartitionColumnStatistics mColStats = new MPartitionColumnStatistics();
     mColStats.setPartition(partition);
+    mColStats.setCatName(statsDesc.isSetCatName() ? statsDesc.getCatName() : DEFAULT_CATALOG_NAME);
     mColStats.setDbName(statsDesc.getDbName());
     mColStats.setTableName(statsDesc.getTableName());
     mColStats.setPartitionName(statsDesc.getPartName());
@@ -474,6 +479,7 @@ public class StatObjectConverter {
     MPartitionColumnStatistics mStatsObj) {
     ColumnStatisticsDesc statsDesc = new ColumnStatisticsDesc();
     statsDesc.setIsTblLevel(false);
+    statsDesc.setCatName(mStatsObj.getCatName());
     statsDesc.setDbName(mStatsObj.getDbName());
     statsDesc.setTableName(mStatsObj.getTableName());
     statsDesc.setPartName(mStatsObj.getPartitionName());

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/TransactionalValidationListener.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/TransactionalValidationListener.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/TransactionalValidationListener.java
index f977f14..80dfd30 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/TransactionalValidationListener.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/TransactionalValidationListener.java
@@ -423,8 +423,8 @@ public final class TransactionalValidationListener extends MetaStorePreEventList
     try {
       Warehouse wh = hmsHandler.getWh();
       if (table.getSd().getLocation() == null || table.getSd().getLocation().isEmpty()) {
-        tablePath = wh.getDefaultTablePath(hmsHandler.getMS().getDatabase(table.getDbName()),
-          table.getTableName());
+        tablePath = wh.getDefaultTablePath(hmsHandler.getMS().getDatabase(
+            MetaStoreUtils.getDefaultCatalog(getConf()), table.getDbName()), table.getTableName());
       } else {
         tablePath = wh.getDnsPath(new Path(table.getSd().getLocation()));
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/Warehouse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/Warehouse.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/Warehouse.java
index d4a0819..88cbfcd 100755
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/Warehouse.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/Warehouse.java
@@ -31,6 +31,7 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.hive.metastore.api.Catalog;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
 import org.apache.hadoop.hive.metastore.utils.FileUtils;
@@ -58,10 +59,13 @@ import org.apache.hadoop.util.ReflectionUtils;
  * This class represents a warehouse where data of Hive tables is stored
  */
 public class Warehouse {
+  public static final String DEFAULT_CATALOG_NAME = "hive";
+  public static final String DEFAULT_CATALOG_COMMENT = "Default catalog, for Hive";
   public static final String DEFAULT_DATABASE_NAME = "default";
   public static final String DEFAULT_DATABASE_COMMENT = "Default Hive database";
   public static final String DEFAULT_SERIALIZATION_FORMAT = "1";
   public static final String DATABASE_WAREHOUSE_SUFFIX = ".db";
+  private static final String CAT_DB_TABLE_SEPARATOR = ".";
 
   private Path whRoot;
   private final Configuration conf;
@@ -154,14 +158,59 @@ public class Warehouse {
     return whRoot;
   }
 
+  /**
+   * Build the database path based on catalog name and database name.  This should only be used
+   * when a database is being created or altered.  If you just want to find out the path a
+   * database is already using call {@link #getDatabasePath(Database)}.  If the passed in
+   * database already has a path set that will be used.  If not the location will be built using
+   * catalog's path and the database name.
+   * @param cat catalog the database is in
+   * @param db database object
+   * @return Path representing the directory for the database
+   * @throws MetaException when the file path cannot be properly determined from the configured
+   * file system.
+   */
+  public Path determineDatabasePath(Catalog cat, Database db) throws MetaException {
+    if (db.isSetLocationUri()) {
+      return getDnsPath(new Path(db.getLocationUri()));
+    }
+    if (cat == null || cat.getName().equalsIgnoreCase(DEFAULT_CATALOG_NAME)) {
+      if (db.getName().equalsIgnoreCase(DEFAULT_DATABASE_NAME)) {
+        return getWhRoot();
+      } else {
+        return new Path(getWhRoot(), dbDirFromDbName(db));
+      }
+    } else {
+      return new Path(getDnsPath(new Path(cat.getLocationUri())), dbDirFromDbName(db));
+    }
+  }
+
+  private String dbDirFromDbName(Database db) throws MetaException {
+    return db.getName().toLowerCase() + DATABASE_WAREHOUSE_SUFFIX;
+  }
+
+  /**
+   * Get the path specified by the database.  In the case of the default database the root of the
+   * warehouse is returned.
+   * @param db database to get the path of
+   * @return path to the database directory
+   * @throws MetaException when the file path cannot be properly determined from the configured
+   * file system.
+   */
   public Path getDatabasePath(Database db) throws MetaException {
-    if (db.getName().equalsIgnoreCase(DEFAULT_DATABASE_NAME)) {
+    if (db.getCatalogName().equalsIgnoreCase(DEFAULT_CATALOG_NAME) &&
+        db.getName().equalsIgnoreCase(DEFAULT_DATABASE_NAME)) {
       return getWhRoot();
     }
     return new Path(db.getLocationUri());
   }
 
   public Path getDefaultDatabasePath(String dbName) throws MetaException {
+    // TODO CAT - I am fairly certain that most calls to this are in error.  This should only be
+    // used when the database location is unset, which should never happen except when a
+    // new database is being created.  Once I have confirmation of this change calls of this to
+    // getDatabasePath(), since it does the right thing.  Also, merge this with
+    // determineDatabasePath() as it duplicates much of the logic.
     if (dbName.equalsIgnoreCase(DEFAULT_DATABASE_NAME)) {
       return getWhRoot();
     }
@@ -177,7 +226,8 @@ public class Warehouse {
    */
   public Path getDefaultTablePath(Database db, String tableName)
       throws MetaException {
-    return getDnsPath(new Path(getDatabasePath(db), MetaStoreUtils.encodeTableName(tableName.toLowerCase())));
+    return getDnsPath(new Path(getDatabasePath(db),
+        MetaStoreUtils.encodeTableName(tableName.toLowerCase())));
   }
 
   public static String getQualifiedName(Table table) {
@@ -185,13 +235,37 @@ public class Warehouse {
   }
 
   public static String getQualifiedName(String dbName, String tableName) {
-    return dbName + "." + tableName;
+    return dbName + CAT_DB_TABLE_SEPARATOR + tableName;
   }
 
   public static String getQualifiedName(Partition partition) {
     return partition.getDbName() + "." + partition.getTableName() + partition.getValues();
   }
 
+  /**
+   * Get table name in cat.db.table format.
+   * @param table table object
+   * @return fully qualified name.
+   */
+  public static String getCatalogQualifiedTableName(Table table) {
+    return getCatalogQualifiedTableName(table.getCatName(), table.getDbName(), table.getTableName());
+  }
+
+  /**
+   * Get table name in cat.db.table format.
+   * @param catName catalog name
+   * @param dbName database name
+   * @param tableName table name
+   * @return fully qualified name.
+   */
+  public static String getCatalogQualifiedTableName(String catName, String dbName, String tableName) {
+    return catName + CAT_DB_TABLE_SEPARATOR + dbName + CAT_DB_TABLE_SEPARATOR + tableName;
+  }
+
+  public static String getCatalogQualifiedDbName(String catName, String dbName) {
+    return catName + CAT_DB_TABLE_SEPARATOR + dbName;
+  }
+
   public boolean mkdirs(Path f) throws MetaException {
     FileSystem fs;
     try {

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CacheUtils.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CacheUtils.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CacheUtils.java
index 97d8af6..944c813 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CacheUtils.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CacheUtils.java
@@ -32,24 +32,18 @@ import org.apache.hadoop.hive.metastore.utils.StringUtils;
 public class CacheUtils {
   private static final String delimit = "\u0001";
 
-  /**
-   * Builds a key for the table cache which is concatenation of database name and table name
-   * separated by a delimiter
-   *
-   * @param dbName
-   * @param tableName
-   * @return
-   */
-  public static String buildTableCacheKey(String dbName, String tableName) {
-    return dbName + delimit + tableName;
+  public static String buildCatalogKey(String catName) {
+    return catName;
+  }
+
+  public static String buildDbKey(String catName, String dbName) {
+    return buildKey(catName.toLowerCase(), dbName.toLowerCase());
   }
 
   /**
    * Builds a key for the partition cache which is concatenation of partition values, each value
    * separated by a delimiter
    *
-   * @param list of partition values
-   * @return cache key for partitions cache
    */
   public static String buildPartitionCacheKey(List<String> partVals) {
     if (partVals == null || partVals.isEmpty()) {
@@ -58,13 +52,29 @@ public class CacheUtils {
     return String.join(delimit, partVals);
   }
 
+  public static String buildTableKey(String catName, String dbName, String tableName) {
+    return buildKey(catName.toLowerCase(), dbName.toLowerCase(), tableName.toLowerCase());
+  }
+
+  public static String buildTableColKey(String catName, String dbName, String tableName,
+                                        String colName) {
+    return buildKey(catName, dbName, tableName, colName);
+  }
+
+  private static String buildKey(String... elements) {
+    return org.apache.commons.lang.StringUtils.join(elements, delimit);
+  }
+
+  public static String[] splitDbName(String key) {
+    String[] names = key.split(delimit);
+    assert names.length == 2;
+    return names;
+  }
+
   /**
    * Builds a key for the partitions column cache which is concatenation of partition values, each
    * value separated by a delimiter and the column name
    *
-   * @param list of partition values
-   * @param column name
-   * @return cache key for partitions column stats cache
    */
   public static String buildPartitonColStatsCacheKey(List<String> partVals, String colName) {
     return buildPartitionCacheKey(partVals) + delimit + colName;


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
index d79b6ed..51df754 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
@@ -68,6 +68,8 @@ import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.hive.ql.stats.StatsUtils;
 import org.apache.thrift.TException;
 
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
+
 public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements IMetaStoreClient {
 
   SessionHiveMetaStoreClient(Configuration conf, Boolean allowEmbedded) throws MetaException {
@@ -88,6 +90,10 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
     return wh;
   }
 
+  // TODO CAT - a number of these need to be updated.  Don't bother with deprecated methods as
+  // this is just an internal class.  Wait until we're ready to move all the catalog stuff up
+  // into ql.
+
   @Override
   protected void create_table_with_environment_context(
       org.apache.hadoop.hive.metastore.api.Table tbl, EnvironmentContext envContext)
@@ -103,10 +109,13 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
   }
 
   @Override
-  protected void drop_table_with_environment_context(String dbname, String name,
+  protected void drop_table_with_environment_context(String catName, String dbname, String name,
       boolean deleteData, EnvironmentContext envContext) throws MetaException, TException,
       NoSuchObjectException, UnsupportedOperationException {
     // First try temp table
+    // TODO CAT - I think the right thing here is to always put temp tables in the current
+    // catalog.  But we don't yet have a notion of current catalog, so we'll have to hold on
+    // until we do.
     org.apache.hadoop.hive.metastore.api.Table table = getTempTable(dbname, name);
     if (table != null) {
       try {
@@ -120,7 +129,7 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
     }
 
     // Try underlying client
-    super.drop_table_with_environment_context(dbname,  name, deleteData, envContext);
+    super.drop_table_with_environment_context(catName, dbname,  name, deleteData, envContext);
   }
 
   @Override
@@ -143,9 +152,20 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
     if (table != null) {
       return deepCopy(table);  // Original method used deepCopy(), do the same here.
     }
-
     // Try underlying client
-    return super.getTable(dbname, name);
+    return super.getTable(DEFAULT_CATALOG_NAME, dbname, name);
+  }
+
+  // Need to override this one too or dropTable breaks because it doesn't find the table when checks
+  // before the drop.
+  @Override
+  public org.apache.hadoop.hive.metastore.api.Table getTable(String catName, String dbName,
+                                                             String tableName) throws TException {
+    if (!DEFAULT_CATALOG_NAME.equals(catName)) {
+      return super.getTable(catName, dbName, tableName);
+    } else {
+      return getTable(dbName, tableName);
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
index 5eefc15..7470c8a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
@@ -108,6 +108,9 @@ import org.slf4j.LoggerFactory;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableList;
 
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+
 /**
  * BaseSemanticAnalyzer.
  *
@@ -646,16 +649,16 @@ public abstract class BaseSemanticAnalyzer {
   }
 
   protected List<FieldSchema> getColumns(ASTNode ast) throws SemanticException {
-    return getColumns(ast, true);
+    return getColumns(ast, true, conf);
   }
 
   /**
    * Get the list of FieldSchema out of the ASTNode.
    */
-  public static List<FieldSchema> getColumns(ASTNode ast, boolean lowerCase) throws SemanticException {
-    return getColumns(ast, lowerCase, null,new ArrayList<SQLPrimaryKey>(), new ArrayList<SQLForeignKey>(),
-            new ArrayList<SQLUniqueConstraint>(), new ArrayList<SQLNotNullConstraint>(),
-        new ArrayList<SQLDefaultConstraint>(), new ArrayList<SQLCheckConstraint>());
+  public static List<FieldSchema> getColumns(ASTNode ast, boolean lowerCase, Configuration conf)
+      throws SemanticException {
+    return getColumns(ast, lowerCase, null, new ArrayList<>(), new ArrayList<>(), new ArrayList<>(),
+            new ArrayList<>(), new ArrayList<>(), new ArrayList<>(), conf);
   }
 
   private static class ConstraintInfo {
@@ -717,79 +720,83 @@ public abstract class BaseSemanticAnalyzer {
   /**
    * Process the unique constraints from the ast node and populate the SQLUniqueConstraint list.
    */
-  protected static void processUniqueConstraints(String databaseName, String tableName,
+  protected static void processUniqueConstraints(String catName, String databaseName, String tableName,
       ASTNode child, List<SQLUniqueConstraint> uniqueConstraints) throws SemanticException {
     List<ConstraintInfo> uniqueInfos = new ArrayList<ConstraintInfo>();
     generateConstraintInfos(child, uniqueInfos);
-    constraintInfosToUniqueConstraints(databaseName, tableName, uniqueInfos, uniqueConstraints);
+    constraintInfosToUniqueConstraints(catName, databaseName, tableName, uniqueInfos, uniqueConstraints);
   }
 
-  protected static void processUniqueConstraints(String databaseName, String tableName,
+  protected static void processUniqueConstraints(String catName, String databaseName, String tableName,
       ASTNode child, List<String> columnNames, List<SQLUniqueConstraint> uniqueConstraints)
           throws SemanticException {
     List<ConstraintInfo> uniqueInfos = new ArrayList<ConstraintInfo>();
     generateConstraintInfos(child, columnNames, uniqueInfos, null, null);
-    constraintInfosToUniqueConstraints(databaseName, tableName, uniqueInfos, uniqueConstraints);
+    constraintInfosToUniqueConstraints(catName, databaseName, tableName, uniqueInfos, uniqueConstraints);
   }
 
-  private static void constraintInfosToUniqueConstraints(String databaseName, String tableName,
+  private static void constraintInfosToUniqueConstraints(String catName, String databaseName, String tableName,
           List<ConstraintInfo> uniqueInfos, List<SQLUniqueConstraint> uniqueConstraints) {
     int i = 1;
     for (ConstraintInfo uniqueInfo : uniqueInfos) {
-      uniqueConstraints.add(new SQLUniqueConstraint(databaseName, tableName, uniqueInfo.colName,
+      uniqueConstraints.add(new SQLUniqueConstraint(catName, databaseName, tableName, uniqueInfo.colName,
               i++, uniqueInfo.constraintName, uniqueInfo.enable, uniqueInfo.validate, uniqueInfo.rely));
     }
   }
 
-  protected static void processCheckConstraints(String databaseName, String tableName,
+  protected static void processCheckConstraints(String catName, String databaseName, String tableName,
                                                   ASTNode child, List<String> columnNames,
                                                 List<SQLCheckConstraint> checkConstraints, final ASTNode typeChild,
                                                 final TokenRewriteStream tokenRewriteStream)
       throws SemanticException {
     List<ConstraintInfo> checkInfos = new ArrayList<ConstraintInfo>();
     generateConstraintInfos(child, columnNames, checkInfos, typeChild, tokenRewriteStream);
-    constraintInfosToCheckConstraints(databaseName, tableName, checkInfos, checkConstraints);
+    constraintInfosToCheckConstraints(catName, databaseName, tableName, checkInfos, checkConstraints);
   }
 
-  private static void constraintInfosToCheckConstraints(String databaseName, String tableName,
+  private static void constraintInfosToCheckConstraints(String catName, String databaseName, String tableName,
                                                           List<ConstraintInfo> checkInfos,
                                                         List<SQLCheckConstraint> checkConstraints) {
     for (ConstraintInfo checkInfo : checkInfos) {
-      checkConstraints.add(new SQLCheckConstraint(databaseName, tableName, checkInfo.colName,
+      checkConstraints.add(new SQLCheckConstraint(catName, databaseName, tableName, checkInfo.colName,
                                                       checkInfo.defaultValue, checkInfo.constraintName, checkInfo.enable,
                                                       checkInfo.validate, checkInfo.rely));
     }
   }
-  protected static void processDefaultConstraints(String databaseName, String tableName,
+
+  protected static void processDefaultConstraints(String catName, String databaseName, String tableName,
       ASTNode child, List<String> columnNames, List<SQLDefaultConstraint> defaultConstraints, final ASTNode typeChild)
       throws SemanticException {
     List<ConstraintInfo> defaultInfos = new ArrayList<ConstraintInfo>();
     generateConstraintInfos(child, columnNames, defaultInfos, typeChild, null);
-    constraintInfosToDefaultConstraints(databaseName, tableName, defaultInfos, defaultConstraints);
+    constraintInfosToDefaultConstraints(catName, databaseName, tableName, defaultInfos, defaultConstraints);
   }
 
-  private static void constraintInfosToDefaultConstraints(String databaseName, String tableName,
+  private static void constraintInfosToDefaultConstraints(
+      String catName, String databaseName, String tableName,
      List<ConstraintInfo> defaultInfos, List<SQLDefaultConstraint> defaultConstraints) {
     for (ConstraintInfo defaultInfo : defaultInfos) {
-      defaultConstraints.add(new SQLDefaultConstraint(databaseName, tableName, defaultInfo.colName,
-          defaultInfo.defaultValue, defaultInfo.constraintName, defaultInfo.enable,
-          defaultInfo.validate, defaultInfo.rely));
+      defaultConstraints.add(new SQLDefaultConstraint(catName, databaseName, tableName,
+          defaultInfo.colName, defaultInfo.defaultValue, defaultInfo.constraintName,
+          defaultInfo.enable, defaultInfo.validate, defaultInfo.rely));
     }
   }
 
-  protected static void processNotNullConstraints(String databaseName, String tableName,
+  protected static void processNotNullConstraints(String catName, String databaseName, String tableName,
       ASTNode child, List<String> columnNames, List<SQLNotNullConstraint> notNullConstraints)
           throws SemanticException {
     List<ConstraintInfo> notNullInfos = new ArrayList<ConstraintInfo>();
     generateConstraintInfos(child, columnNames, notNullInfos, null, null);
-    constraintInfosToNotNullConstraints(databaseName, tableName, notNullInfos, notNullConstraints);
+    constraintInfosToNotNullConstraints(catName, databaseName, tableName, notNullInfos, notNullConstraints);
   }
 
-  private static void constraintInfosToNotNullConstraints(String databaseName, String tableName,
-          List<ConstraintInfo> notNullInfos, List<SQLNotNullConstraint> notNullConstraints) {
+  private static void constraintInfosToNotNullConstraints(
+      String catName, String databaseName, String tableName, List<ConstraintInfo> notNullInfos,
+      List<SQLNotNullConstraint> notNullConstraints) {
     for (ConstraintInfo notNullInfo : notNullInfos) {
-      notNullConstraints.add(new SQLNotNullConstraint(databaseName, tableName, notNullInfo.colName,
-              notNullInfo.constraintName, notNullInfo.enable, notNullInfo.validate, notNullInfo.rely));
+      notNullConstraints.add(new SQLNotNullConstraint(catName, databaseName, tableName,
+          notNullInfo.colName, notNullInfo.constraintName, notNullInfo.enable, notNullInfo.validate,
+          notNullInfo.rely));
     }
   }
 
@@ -1176,13 +1183,12 @@ public abstract class BaseSemanticAnalyzer {
    * Get the list of FieldSchema out of the ASTNode.
    * Additionally, populate the primaryKeys and foreignKeys if any.
    */
-  public static List<FieldSchema> getColumns(ASTNode ast, boolean lowerCase,
-    TokenRewriteStream tokenRewriteStream,
-    List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys,
-    List<SQLUniqueConstraint> uniqueConstraints, List<SQLNotNullConstraint> notNullConstraints,
-                                           List<SQLDefaultConstraint> defaultConstraints,
-                                             List<SQLCheckConstraint> checkConstraints)
-        throws SemanticException {
+  public static List<FieldSchema> getColumns(
+      ASTNode ast, boolean lowerCase, TokenRewriteStream tokenRewriteStream,
+      List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys,
+      List<SQLUniqueConstraint> uniqueConstraints, List<SQLNotNullConstraint> notNullConstraints,
+      List<SQLDefaultConstraint> defaultConstraints, List<SQLCheckConstraint> checkConstraints,
+      Configuration conf) throws SemanticException {
     List<FieldSchema> colList = new ArrayList<FieldSchema>();
     Tree parent = ast.getParent();
 
@@ -1192,7 +1198,11 @@ public abstract class BaseSemanticAnalyzer {
       switch (child.getToken().getType()) {
         case HiveParser.TOK_UNIQUE: {
             String[] qualifiedTabName = getQualifiedTableName((ASTNode) parent.getChild(0));
-            processUniqueConstraints(qualifiedTabName[0], qualifiedTabName[1], child, uniqueConstraints);
+            // TODO CAT - for now always use the default catalog.  Eventually will want to see if
+            // the user specified a catalog
+            String catName = MetaStoreUtils.getDefaultCatalog(conf);
+            processUniqueConstraints(catName, qualifiedTabName[0], qualifiedTabName[1], child,
+                uniqueConstraints);
           }
           break;
         case HiveParser.TOK_PRIMARY_KEY: {
@@ -1237,23 +1247,26 @@ public abstract class BaseSemanticAnalyzer {
             }
             if (constraintChild != null) {
               String[] qualifiedTabName = getQualifiedTableName((ASTNode) parent.getChild(0));
+              // TODO CAT - for now always use the default catalog.  Eventually will want to see if
+              // the user specified a catalog
+              String catName = MetaStoreUtils.getDefaultCatalog(conf);
               // Process column constraint
               switch (constraintChild.getToken().getType()) {
               case HiveParser.TOK_CHECK_CONSTRAINT:
-                processCheckConstraints(qualifiedTabName[0], qualifiedTabName[1], constraintChild,
+                processCheckConstraints(catName, qualifiedTabName[0], qualifiedTabName[1], constraintChild,
                                           ImmutableList.of(col.getName()), checkConstraints, typeChild,
                                         tokenRewriteStream);
                 break;
               case HiveParser.TOK_DEFAULT_VALUE:
-                processDefaultConstraints(qualifiedTabName[0], qualifiedTabName[1], constraintChild,
+                processDefaultConstraints(catName, qualifiedTabName[0], qualifiedTabName[1], constraintChild,
                     ImmutableList.of(col.getName()), defaultConstraints, typeChild);
                 break;
                 case HiveParser.TOK_NOT_NULL:
-                  processNotNullConstraints(qualifiedTabName[0], qualifiedTabName[1], constraintChild,
+                  processNotNullConstraints(catName, qualifiedTabName[0], qualifiedTabName[1], constraintChild,
                           ImmutableList.of(col.getName()), notNullConstraints);
                   break;
                 case HiveParser.TOK_UNIQUE:
-                  processUniqueConstraints(qualifiedTabName[0], qualifiedTabName[1], constraintChild,
+                  processUniqueConstraints(catName, qualifiedTabName[0], qualifiedTabName[1], constraintChild,
                           ImmutableList.of(col.getName()), uniqueConstraints);
                   break;
                 case HiveParser.TOK_PRIMARY_KEY:

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
index 74c271d..ed1c0ab 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
@@ -285,6 +285,9 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     case HiveParser.TOK_ALTERTABLE: {
       ast = (ASTNode) input.getChild(1);
       String[] qualified = getQualifiedTableName((ASTNode) input.getChild(0));
+      // TODO CAT - for now always use the default catalog.  Eventually will want to see if
+      // the user specified a catalog
+      String catName = MetaStoreUtils.getDefaultCatalog(conf);
       String tableName = getDotName(qualified);
       HashMap<String, String> partSpec = null;
       ASTNode partSpecNode = (ASTNode)input.getChild(2);
@@ -312,7 +315,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
       } else if (ast.getType() == HiveParser.TOK_ALTERTABLE_REPLACECOLS) {
         analyzeAlterTableModifyCols(qualified, ast, partSpec, AlterTableTypes.REPLACECOLS);
       } else if (ast.getType() == HiveParser.TOK_ALTERTABLE_RENAMECOL) {
-        analyzeAlterTableRenameCol(qualified, ast, partSpec);
+        analyzeAlterTableRenameCol(catName, qualified, ast, partSpec);
       } else if (ast.getType() == HiveParser.TOK_ALTERTABLE_ADDPARTS) {
         analyzeAlterTableAddParts(qualified, ast, false);
       } else if (ast.getType() == HiveParser.TOK_ALTERTABLE_DROPPARTS) {
@@ -2152,6 +2155,9 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     throws SemanticException {
     ASTNode parent = (ASTNode) ast.getParent();
     String[] qualifiedTabName = getQualifiedTableName((ASTNode) parent.getChild(0));
+    // TODO CAT - for now always use the default catalog.  Eventually will want to see if
+    // the user specified a catalog
+    String catName = MetaStoreUtils.getDefaultCatalog(conf);
     ASTNode child = (ASTNode) ast.getChild(0);
     List<SQLPrimaryKey> primaryKeys = new ArrayList<>();
     List<SQLForeignKey> foreignKeys = new ArrayList<>();
@@ -2159,7 +2165,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
 
     switch (child.getToken().getType()) {
       case HiveParser.TOK_UNIQUE:
-        BaseSemanticAnalyzer.processUniqueConstraints(qualifiedTabName[0], qualifiedTabName[1],
+        BaseSemanticAnalyzer.processUniqueConstraints(catName, qualifiedTabName[0], qualifiedTabName[1],
                 child, uniqueConstraints);
         break;
       case HiveParser.TOK_PRIMARY_KEY:
@@ -3078,7 +3084,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
         alterTblDesc)));
   }
 
-  private void analyzeAlterTableRenameCol(String[] qualified, ASTNode ast,
+  private void analyzeAlterTableRenameCol(String catName, String[] qualified, ASTNode ast,
       HashMap<String, String> partSpec) throws SemanticException {
     String newComment = null;
     boolean first = false;
@@ -3122,23 +3128,23 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
       switch (constraintChild.getToken().getType()) {
       case HiveParser.TOK_CHECK_CONSTRAINT:
         checkConstraints = new ArrayList<>();
-        processCheckConstraints(qualified[0], qualified[1], constraintChild,
+        processCheckConstraints(catName, qualified[0], qualified[1], constraintChild,
                                   ImmutableList.of(newColName), checkConstraints, (ASTNode)ast.getChild(2),
                                 this.ctx.getTokenRewriteStream());
         break;
       case HiveParser.TOK_DEFAULT_VALUE:
         defaultConstraints = new ArrayList<>();
-        processDefaultConstraints(qualified[0], qualified[1], constraintChild,
+        processDefaultConstraints(catName, qualified[0], qualified[1], constraintChild,
                                   ImmutableList.of(newColName), defaultConstraints, (ASTNode)ast.getChild(2));
         break;
       case HiveParser.TOK_NOT_NULL:
         notNullConstraints = new ArrayList<>();
-        processNotNullConstraints(qualified[0], qualified[1], constraintChild,
+        processNotNullConstraints(catName, qualified[0], qualified[1], constraintChild,
                                   ImmutableList.of(newColName), notNullConstraints);
         break;
       case HiveParser.TOK_UNIQUE:
         uniqueConstraints = new ArrayList<>();
-        processUniqueConstraints(qualified[0], qualified[1], constraintChild,
+        processUniqueConstraints(catName, qualified[0], qualified[1], constraintChild,
                                  ImmutableList.of(newColName), uniqueConstraints);
         break;
       case HiveParser.TOK_PRIMARY_KEY:

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/ql/src/java/org/apache/hadoop/hive/ql/parse/MacroSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/MacroSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/MacroSemanticAnalyzer.java
index 762e438..88b6068 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/MacroSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/MacroSemanticAnalyzer.java
@@ -86,7 +86,7 @@ public class MacroSemanticAnalyzer extends BaseSemanticAnalyzer {
     }
 
     List<FieldSchema> arguments =
-      BaseSemanticAnalyzer.getColumns((ASTNode)ast.getChild(1), true);
+      BaseSemanticAnalyzer.getColumns((ASTNode)ast.getChild(1), true, conf);
     boolean isNoArgumentMacro = arguments.size() == 0;
     RowResolver rowResolver = new RowResolver();
     ArrayList<String> macroColNames = new ArrayList<String>(arguments.size());

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index 26f20f2..53d5a12 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -10957,7 +10957,6 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         if (partitions != null) {
           for (Partition partn : partitions) {
             // inputs.add(new ReadEntity(partn)); // is this needed at all?
-            LOG.info("XXX: adding part: "+partn);
             outputs.add(new WriteEntity(partn, WriteEntity.WriteType.DDL_NO_LOCK));
           }
         }
@@ -12723,14 +12722,14 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         break;
       case HiveParser.TOK_TABCOLLIST:
         cols = getColumns(child, true, ctx.getTokenRewriteStream(), primaryKeys, foreignKeys,
-            uniqueConstraints, notNullConstraints, defaultConstraints, checkConstraints);
+            uniqueConstraints, notNullConstraints, defaultConstraints, checkConstraints, conf);
         break;
       case HiveParser.TOK_TABLECOMMENT:
         comment = unescapeSQLString(child.getChild(0).getText());
         break;
       case HiveParser.TOK_TABLEPARTCOLS:
-        partCols = getColumns(child, false, ctx.getTokenRewriteStream(),primaryKeys, foreignKeys,
-            uniqueConstraints, notNullConstraints, defaultConstraints, checkConstraints);
+        partCols = getColumns(child, false, ctx.getTokenRewriteStream(), primaryKeys, foreignKeys,
+            uniqueConstraints, notNullConstraints, defaultConstraints, checkConstraints, conf);
         if(hasConstraints(partCols, defaultConstraints, notNullConstraints, checkConstraints)) {
           //TODO: these constraints should be supported for partition columns
           throw new SemanticException(

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveAuthorizationProviderBase.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveAuthorizationProviderBase.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveAuthorizationProviderBase.java
index 2c7064b..8a7c06d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveAuthorizationProviderBase.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveAuthorizationProviderBase.java
@@ -21,6 +21,8 @@ package org.apache.hadoop.hive.ql.security.authorization;
 import java.util.List;
 
 import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.Warehouse;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -89,7 +91,7 @@ public abstract class HiveAuthorizationProviderBase implements
         return Hive.getWithFastCheck(conf).getDatabase(dbName);
       } else {
         try {
-          return handler.get_database_core(dbName);
+          return handler.get_database_core(MetaStoreUtils.getDefaultCatalog(conf), dbName);
         } catch (NoSuchObjectException e) {
           throw new HiveException(e);
         } catch (MetaException e) {

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/AuthorizationMetaStoreFilterHook.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/AuthorizationMetaStoreFilterHook.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/AuthorizationMetaStoreFilterHook.java
index 233a48c..ca4b667 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/AuthorizationMetaStoreFilterHook.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/AuthorizationMetaStoreFilterHook.java
@@ -43,7 +43,8 @@ public class AuthorizationMetaStoreFilterHook extends DefaultMetaStoreFilterHook
   }
 
   @Override
-  public List<String> filterTableNames(String dbName, List<String> tableList) throws MetaException {
+  public List<String> filterTableNames(String catName, String dbName, List<String> tableList)
+      throws MetaException {
     List<HivePrivilegeObject> listObjs = getHivePrivObjects(dbName, tableList);
     return getTableNames(getFilteredObjects(listObjs));
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorThread.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorThread.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorThread.java
index ade7726..dd0929f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorThread.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorThread.java
@@ -46,6 +46,8 @@ import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+
 /**
  * Superclass for all threads in the compactor.
  */
@@ -102,7 +104,7 @@ abstract class CompactorThread extends Thread implements MetaStoreThread {
    */
   protected Table resolveTable(CompactionInfo ci) throws MetaException {
     try {
-      return rs.getTable(ci.dbname, ci.tableName);
+      return rs.getTable(getDefaultCatalog(conf), ci.dbname, ci.tableName);
     } catch (MetaException e) {
       LOG.error("Unable to find table " + ci.getFullTableName() + ", " + e.getMessage());
       throw e;
@@ -120,7 +122,7 @@ abstract class CompactorThread extends Thread implements MetaStoreThread {
     if (ci.partName != null) {
       List<Partition> parts;
       try {
-        parts = rs.getPartitionsByNames(ci.dbname, ci.tableName,
+        parts = rs.getPartitionsByNames(getDefaultCatalog(conf), ci.dbname, ci.tableName,
             Collections.singletonList(ci.partName));
         if (parts == null || parts.size() == 0) {
           // The partition got dropped before we went looking for it.

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/ql/src/test/results/clientnegative/alter_table_constraint_duplicate_pk.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/alter_table_constraint_duplicate_pk.q.out b/ql/src/test/results/clientnegative/alter_table_constraint_duplicate_pk.q.out
index 30ad841..acf65f2 100644
--- a/ql/src/test/results/clientnegative/alter_table_constraint_duplicate_pk.q.out
+++ b/ql/src/test/results/clientnegative/alter_table_constraint_duplicate_pk.q.out
@@ -8,4 +8,4 @@ POSTHOOK: Output: database:default
 POSTHOOK: Output: default@table1
 PREHOOK: query: alter table table1 add constraint pk4 primary key (b) disable novalidate rely
 PREHOOK: type: ALTERTABLE_ADDCONSTRAINT
-FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. MetaException(message: Primary key already exists for: default.table1)
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. MetaException(message: Primary key already exists for: hive.default.table1)

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/ql/src/test/results/clientnegative/create_view_failure2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/create_view_failure2.q.out b/ql/src/test/results/clientnegative/create_view_failure2.q.out
index 3342a23..ad5d5fe 100644
--- a/ql/src/test/results/clientnegative/create_view_failure2.q.out
+++ b/ql/src/test/results/clientnegative/create_view_failure2.q.out
@@ -17,4 +17,4 @@ PREHOOK: query: CREATE TABLE xxx4(key int)
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@xxx4
-FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. AlreadyExistsException(message:Table xxx4 already exists)
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. AlreadyExistsException(message:Table hive.default.xxx4 already exists)


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index de94172..3cea4b2 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -19,6 +19,9 @@
 package org.apache.hadoop.hive.metastore;
 
 import static org.apache.commons.lang.StringUtils.join;
+import static org.apache.hadoop.hive.metastore.Warehouse.getCatalogQualifiedDbName;
+import static org.apache.hadoop.hive.metastore.Warehouse.getCatalogQualifiedTableName;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
 import static org.apache.hadoop.hive.metastore.utils.StringUtils.normalizeIdentifier;
 
 import java.io.IOException;
@@ -69,6 +72,7 @@ import javax.sql.DataSource;
 
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.lang.ArrayUtils;
+import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang.exception.ExceptionUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -79,6 +83,8 @@ import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.metastore.MetaStoreDirectSql.SqlFilterForPushdown;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
+import org.apache.hadoop.hive.metastore.api.BasicTxnInfo;
+import org.apache.hadoop.hive.metastore.api.Catalog;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
@@ -156,6 +162,7 @@ import org.apache.hadoop.hive.metastore.datasource.DataSourceProvider;
 import org.apache.hadoop.hive.metastore.datasource.DataSourceProviderFactory;
 import org.apache.hadoop.hive.metastore.metrics.Metrics;
 import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.model.MCatalog;
 import org.apache.hadoop.hive.metastore.model.MColumnDescriptor;
 import org.apache.hadoop.hive.metastore.model.MConstraint;
 import org.apache.hadoop.hive.metastore.model.MCreationMetadata;
@@ -203,6 +210,7 @@ import org.apache.hadoop.hive.metastore.utils.FileUtils;
 import org.apache.hadoop.hive.metastore.utils.JavaUtils;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.utils.ObjectPair;
+import org.apache.thrift.TDeserializer;
 import org.apache.thrift.TException;
 import org.datanucleus.AbstractNucleusContext;
 import org.datanucleus.ClassLoaderResolver;
@@ -792,9 +800,127 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @Override
+  public void createCatalog(Catalog cat) throws MetaException {
+    LOG.debug("Creating catalog " + cat.getName());
+    boolean committed = false;
+    MCatalog mCat = catToMCat(cat);
+    try {
+      openTransaction();
+      pm.makePersistent(mCat);
+      committed = commitTransaction();
+    } finally {
+      if (!committed) rollbackTransaction();
+    }
+  }
+
+  @Override
+  public void alterCatalog(String catName, Catalog cat)
+      throws MetaException, InvalidOperationException {
+    if (!cat.getName().equals(catName)) {
+      throw new InvalidOperationException("You cannot change a catalog's name");
+    }
+    boolean committed = false;
+    try {
+      MCatalog mCat = getMCatalog(catName);
+      if (org.apache.commons.lang.StringUtils.isNotBlank(cat.getLocationUri())) {
+        mCat.setLocationUri(cat.getLocationUri());
+      }
+      if (org.apache.commons.lang.StringUtils.isNotBlank(cat.getDescription())) {
+        mCat.setDescription(cat.getDescription());
+      }
+      openTransaction();
+      pm.makePersistent(mCat);
+      committed = commitTransaction();
+    } finally {
+      if (!committed) rollbackTransaction();
+    }
+  }
+
+  @Override
+  public Catalog getCatalog(String catalogName) throws NoSuchObjectException, MetaException {
+    LOG.debug("Fetching catalog " + catalogName);
+    MCatalog mCat = getMCatalog(catalogName);
+    if (mCat == null) throw new NoSuchObjectException("No catalog " + catalogName);
+    return mCatToCat(mCat);
+  }
+
+  @Override
+  public List<String> getCatalogs() throws MetaException {
+    LOG.debug("Fetching all catalog names");
+    boolean commited = false;
+    List<String> catalogs = null;
+
+    String queryStr = "select name from org.apache.hadoop.hive.metastore.model.MCatalog";
+    Query query = null;
+
+    openTransaction();
+    try {
+      query = pm.newQuery(queryStr);
+      query.setResult("name");
+      catalogs = new ArrayList<>((Collection<String>) query.execute());
+      commited = commitTransaction();
+    } finally {
+      rollbackAndCleanup(commited, query);
+    }
+    Collections.sort(catalogs);
+    return catalogs;
+  }
+
+  @Override
+  public void dropCatalog(String catalogName) throws NoSuchObjectException, MetaException {
+    LOG.debug("Dropping catalog " + catalogName);
+    boolean committed = false;
+    try {
+      openTransaction();
+      MCatalog mCat = getMCatalog(catalogName);
+      pm.retrieve(mCat);
+      if (mCat == null) throw new NoSuchObjectException("No catalog " + catalogName);
+      pm.deletePersistent(mCat);
+      committed = commitTransaction();
+    } finally {
+      if (!committed) rollbackTransaction();
+    }
+  }
+
+  private MCatalog getMCatalog(String catalogName) throws MetaException {
+    boolean committed = false;
+    Query query = null;
+    try {
+      openTransaction();
+      catalogName = normalizeIdentifier(catalogName);
+      query = pm.newQuery(MCatalog.class, "name == catname");
+      query.declareParameters("java.lang.String catname");
+      query.setUnique(true);
+      MCatalog mCat = (MCatalog)query.execute(catalogName);
+      pm.retrieve(mCat);
+      committed = commitTransaction();
+      return mCat;
+    } finally {
+      rollbackAndCleanup(committed, query);
+    }
+  }
+
+  private MCatalog catToMCat(Catalog cat) {
+    MCatalog mCat = new MCatalog();
+    mCat.setName(normalizeIdentifier(cat.getName()));
+    if (cat.isSetDescription()) mCat.setDescription(cat.getDescription());
+    mCat.setLocationUri(cat.getLocationUri());
+    return mCat;
+  }
+
+  private Catalog mCatToCat(MCatalog mCat) {
+    Catalog cat = new Catalog(mCat.getName(), mCat.getLocationUri());
+    if (mCat.getDescription() != null) cat.setDescription(mCat.getDescription());
+    return cat;
+  }
+
+  @Override
   public void createDatabase(Database db) throws InvalidObjectException, MetaException {
     boolean commited = false;
     MDatabase mdb = new MDatabase();
+    assert db.getCatalogName() != null;
+    mdb.setCatalogName(normalizeIdentifier(db.getCatalogName()));
+    assert mdb.getCatalogName() != null;
     mdb.setName(db.getName().toLowerCase());
     mdb.setLocationUri(db.getLocationUri());
     mdb.setDescription(db.getDescription());
@@ -814,34 +940,35 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @SuppressWarnings("nls")
-  private MDatabase getMDatabase(String name) throws NoSuchObjectException {
+  private MDatabase getMDatabase(String catName, String name) throws NoSuchObjectException {
     MDatabase mdb = null;
     boolean commited = false;
     Query query = null;
     try {
       openTransaction();
       name = normalizeIdentifier(name);
-      query = pm.newQuery(MDatabase.class, "name == dbname");
-      query.declareParameters("java.lang.String dbname");
+      catName = normalizeIdentifier(catName);
+      query = pm.newQuery(MDatabase.class, "name == dbname && catalogName == catname");
+      query.declareParameters("java.lang.String dbname, java.lang.String catname");
       query.setUnique(true);
-      mdb = (MDatabase) query.execute(name);
+      mdb = (MDatabase) query.execute(name, catName);
       pm.retrieve(mdb);
       commited = commitTransaction();
     } finally {
       rollbackAndCleanup(commited, query);
     }
     if (mdb == null) {
-      throw new NoSuchObjectException("There is no database named " + name);
+      throw new NoSuchObjectException("There is no database " + catName + "." + name);
     }
     return mdb;
   }
 
   @Override
-  public Database getDatabase(String name) throws NoSuchObjectException {
+  public Database getDatabase(String catalogName, String name) throws NoSuchObjectException {
     MetaException ex = null;
     Database db = null;
     try {
-      db = getDatabaseInternal(name);
+      db = getDatabaseInternal(catalogName, name);
     } catch (MetaException e) {
       // Signature restriction to NSOE, and NSOE being a flat exception prevents us from
       // setting the cause of the NSOE as the MetaException. We should not lose the info
@@ -850,32 +977,34 @@ public class ObjectStore implements RawStore, Configurable {
       ex = e;
     }
     if (db == null) {
-      LOG.warn("Failed to get database {}, returning NoSuchObjectException", name, ex);
+      LOG.warn("Failed to get database {}.{}, returning NoSuchObjectException",
+          catalogName, name, ex);
       throw new NoSuchObjectException(name + (ex == null ? "" : (": " + ex.getMessage())));
     }
     return db;
   }
 
-  public Database getDatabaseInternal(String name) throws MetaException, NoSuchObjectException {
-    return new GetDbHelper(name, true, true) {
+  public Database getDatabaseInternal(String catalogName, String name)
+      throws MetaException, NoSuchObjectException {
+    return new GetDbHelper(catalogName, name, true, true) {
       @Override
       protected Database getSqlResult(GetHelper<Database> ctx) throws MetaException {
-        return directSql.getDatabase(dbName);
+        return directSql.getDatabase(catalogName, dbName);
       }
 
       @Override
       protected Database getJdoResult(GetHelper<Database> ctx) throws MetaException, NoSuchObjectException {
-        return getJDODatabase(dbName);
+        return getJDODatabase(catalogName, dbName);
       }
     }.run(false);
    }
 
-  public Database getJDODatabase(String name) throws NoSuchObjectException {
+  public Database getJDODatabase(String catName, String name) throws NoSuchObjectException {
     MDatabase mdb = null;
     boolean commited = false;
     try {
       openTransaction();
-      mdb = getMDatabase(name);
+      mdb = getMDatabase(catName, name);
       commited = commitTransaction();
     } finally {
       if (!commited) {
@@ -891,6 +1020,7 @@ public class ObjectStore implements RawStore, Configurable {
     String type = org.apache.commons.lang.StringUtils.defaultIfBlank(mdb.getOwnerType(), null);
     PrincipalType principalType = (type == null) ? null : PrincipalType.valueOf(type);
     db.setOwnerType(principalType);
+    db.setCatalogName(catName);
     return db;
   }
 
@@ -903,13 +1033,13 @@ public class ObjectStore implements RawStore, Configurable {
    * @throws NoSuchObjectException
    */
   @Override
-  public boolean alterDatabase(String dbName, Database db)
+  public boolean alterDatabase(String catName, String dbName, Database db)
     throws MetaException, NoSuchObjectException {
 
     MDatabase mdb = null;
     boolean committed = false;
     try {
-      mdb = getMDatabase(dbName);
+      mdb = getMDatabase(catName, dbName);
       mdb.setParameters(db.getParameters());
       mdb.setOwnerName(db.getOwnerName());
       if (db.getOwnerType() != null) {
@@ -934,19 +1064,21 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @Override
-  public boolean dropDatabase(String dbname) throws NoSuchObjectException, MetaException {
+  public boolean dropDatabase(String catName, String dbname)
+      throws NoSuchObjectException, MetaException {
     boolean success = false;
-    LOG.info("Dropping database {} along with all tables", dbname);
+    LOG.info("Dropping database {}.{} along with all tables", catName, dbname);
     dbname = normalizeIdentifier(dbname);
+    catName = normalizeIdentifier(catName);
     QueryWrapper queryWrapper = new QueryWrapper();
     try {
       openTransaction();
 
       // then drop the database
-      MDatabase db = getMDatabase(dbname);
+      MDatabase db = getMDatabase(catName, dbname);
       pm.retrieve(db);
       if (db != null) {
-        List<MDBPrivilege> dbGrants = this.listDatabaseGrants(dbname, queryWrapper);
+        List<MDBPrivilege> dbGrants = this.listDatabaseGrants(catName, dbname, queryWrapper);
         if (CollectionUtils.isNotEmpty(dbGrants)) {
           pm.deletePersistentAll(dbGrants);
         }
@@ -960,9 +1092,9 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @Override
-  public List<String> getDatabases(String pattern) throws MetaException {
+  public List<String> getDatabases(String catName, String pattern) throws MetaException {
     if (pattern == null || pattern.equals("*")) {
-      return getAllDatabases();
+      return getAllDatabases(catName);
     }
     boolean commited = false;
     List<String> databases = null;
@@ -974,6 +1106,7 @@ public class ObjectStore implements RawStore, Configurable {
       String[] subpatterns = pattern.trim().split("\\|");
       StringBuilder filterBuilder = new StringBuilder();
       List<String> parameterVals = new ArrayList<>(subpatterns.length);
+      appendSimpleCondition(filterBuilder, "catalogName", new String[] {catName}, parameterVals);
       appendPatternCondition(filterBuilder, "name", subpatterns, parameterVals);
       query = pm.newQuery(MDatabase.class, filterBuilder.toString());
       query.setResult("name");
@@ -988,18 +1121,20 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @Override
-  public List<String> getAllDatabases() throws MetaException {
+  public List<String> getAllDatabases(String catName) throws MetaException {
     boolean commited = false;
     List<String> databases = null;
 
-    String queryStr = "select name from org.apache.hadoop.hive.metastore.model.MDatabase";
     Query query = null;
+    catName = normalizeIdentifier(catName);
 
     openTransaction();
     try {
-      query = pm.newQuery(queryStr);
+      query = pm.newQuery("select name from org.apache.hadoop.hive.metastore.model.MDatabase " +
+          "where catalogName == catname");
+      query.declareParameters("java.lang.String catname");
       query.setResult("name");
-      databases = new ArrayList<>((Collection<String>) query.execute());
+      databases = new ArrayList<>((Collection<String>) query.execute(catName));
       commited = commitTransaction();
     } finally {
       rollbackAndCleanup(commited, query);
@@ -1112,12 +1247,25 @@ public class ObjectStore implements RawStore, Configurable {
       // Add constraints.
       // We need not do a deep retrieval of the Table Column Descriptor while persisting the
       // constraints since this transaction involving create table is not yet committed.
-      List<String> constraintNames = addForeignKeys(foreignKeys, false, primaryKeys, uniqueConstraints);
-      constraintNames.addAll(addPrimaryKeys(primaryKeys, false));
-      constraintNames.addAll(addUniqueConstraints(uniqueConstraints, false));
-      constraintNames.addAll(addNotNullConstraints(notNullConstraints, false));
-      constraintNames.addAll(addDefaultConstraints(defaultConstraints, false));
-      constraintNames.addAll(addCheckConstraints(checkConstraints, false));
+      List<String> constraintNames = new ArrayList<>();
+      if (foreignKeys != null) {
+        constraintNames.addAll(addForeignKeys(foreignKeys, false, primaryKeys, uniqueConstraints));
+      }
+      if (primaryKeys != null) {
+        constraintNames.addAll(addPrimaryKeys(primaryKeys, false));
+      }
+      if (uniqueConstraints != null) {
+        constraintNames.addAll(addUniqueConstraints(uniqueConstraints, false));
+      }
+      if (notNullConstraints != null) {
+        constraintNames.addAll(addNotNullConstraints(notNullConstraints, false));
+      }
+      if (defaultConstraints != null) {
+        constraintNames.addAll(addDefaultConstraints(defaultConstraints, false));
+      }
+      if (checkConstraints != null) {
+        constraintNames.addAll(addCheckConstraints(checkConstraints, false));
+      }
       success = commitTransaction();
       return constraintNames;
     } finally {
@@ -1205,47 +1353,47 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @Override
-  public boolean dropTable(String dbName, String tableName) throws MetaException,
-    NoSuchObjectException, InvalidObjectException, InvalidInputException {
+  public boolean dropTable(String catName, String dbName, String tableName)
+      throws MetaException, NoSuchObjectException, InvalidObjectException, InvalidInputException {
     boolean materializedView = false;
     boolean success = false;
     try {
       openTransaction();
-      MTable tbl = getMTable(dbName, tableName);
+      MTable tbl = getMTable(catName, dbName, tableName);
       pm.retrieve(tbl);
       if (tbl != null) {
         materializedView = TableType.MATERIALIZED_VIEW.toString().equals(tbl.getTableType());
         // first remove all the grants
-        List<MTablePrivilege> tabGrants = listAllTableGrants(dbName, tableName);
+        List<MTablePrivilege> tabGrants = listAllTableGrants(catName, dbName, tableName);
         if (CollectionUtils.isNotEmpty(tabGrants)) {
           pm.deletePersistentAll(tabGrants);
         }
-        List<MTableColumnPrivilege> tblColGrants = listTableAllColumnGrants(dbName,
+        List<MTableColumnPrivilege> tblColGrants = listTableAllColumnGrants(catName, dbName,
             tableName);
         if (CollectionUtils.isNotEmpty(tblColGrants)) {
           pm.deletePersistentAll(tblColGrants);
         }
 
-        List<MPartitionPrivilege> partGrants = this.listTableAllPartitionGrants(dbName, tableName);
+        List<MPartitionPrivilege> partGrants = this.listTableAllPartitionGrants(catName, dbName, tableName);
         if (CollectionUtils.isNotEmpty(partGrants)) {
           pm.deletePersistentAll(partGrants);
         }
 
-        List<MPartitionColumnPrivilege> partColGrants = listTableAllPartitionColumnGrants(dbName,
+        List<MPartitionColumnPrivilege> partColGrants = listTableAllPartitionColumnGrants(catName, dbName,
             tableName);
         if (CollectionUtils.isNotEmpty(partColGrants)) {
           pm.deletePersistentAll(partColGrants);
         }
         // delete column statistics if present
         try {
-          deleteTableColumnStatistics(dbName, tableName, null);
+          deleteTableColumnStatistics(catName, dbName, tableName, null);
         } catch (NoSuchObjectException e) {
-          LOG.info("Found no table level column statistics associated with db {}" +
-          " table {} record to delete", dbName, tableName);
+          LOG.info("Found no table level column statistics associated with {} to delete",
+              getCatalogQualifiedTableName(catName, dbName, tableName));
         }
 
         List<MConstraint> tabConstraints = listAllTableConstraintsWithOptionalConstraintName(
-                                           dbName, tableName, null);
+                                           catName, dbName, tableName, null);
         if (CollectionUtils.isNotEmpty(tabConstraints)) {
           pm.deletePersistentAll(tabConstraints);
         }
@@ -1253,7 +1401,7 @@ public class ObjectStore implements RawStore, Configurable {
         preDropStorageDescriptor(tbl.getSd());
 
         if (materializedView) {
-          dropCreationMetadata(
+          dropCreationMetadata(tbl.getDatabase().getCatalogName(),
               tbl.getDatabase().getName(), tbl.getTableName());
         }
 
@@ -1273,14 +1421,14 @@ public class ObjectStore implements RawStore, Configurable {
     return success;
   }
 
-  private boolean dropCreationMetadata(String dbName, String tableName) throws MetaException,
+  private boolean dropCreationMetadata(String catName, String dbName, String tableName) throws MetaException,
       NoSuchObjectException, InvalidObjectException, InvalidInputException {
     boolean success = false;
     dbName = normalizeIdentifier(dbName);
     tableName = normalizeIdentifier(tableName);
     try {
       openTransaction();
-      MCreationMetadata mcm = getCreationMetadata(dbName, tableName);
+      MCreationMetadata mcm = getCreationMetadata(catName, dbName, tableName);
       pm.retrieve(mcm);
       if (mcm != null) {
         pm.deletePersistentAll(mcm);
@@ -1294,8 +1442,9 @@ public class ObjectStore implements RawStore, Configurable {
     return success;
   }
 
-  private List<MConstraint> listAllTableConstraintsWithOptionalConstraintName
-    (String dbName, String tableName, String constraintname) {
+  private List<MConstraint> listAllTableConstraintsWithOptionalConstraintName(
+      String catName, String dbName, String tableName, String constraintname) {
+    catName = normalizeIdentifier(catName);
     dbName = normalizeIdentifier(dbName);
     tableName = normalizeIdentifier(tableName);
     constraintname = constraintname!=null?normalizeIdentifier(constraintname):null;
@@ -1305,19 +1454,21 @@ public class ObjectStore implements RawStore, Configurable {
 
     try {
       query = pm.newQuery("select constraintName from org.apache.hadoop.hive.metastore.model.MConstraint  where "
-        + "((parentTable.tableName == ptblname && parentTable.database.name == pdbname) || "
-        + "(childTable != null && childTable.tableName == ctblname && "
-        + "childTable.database.name == cdbname)) " + (constraintname != null ?
-        " && constraintName == constraintname" : ""));
+        + "((parentTable.tableName == ptblname && parentTable.database.name == pdbname && " +
+              "parentTable.database.catalogName == pcatname) || "
+        + "(childTable != null && childTable.tableName == ctblname &&" +
+              "childTable.database.name == cdbname && childTable.database.catalogName == ccatname)) " +
+          (constraintname != null ? " && constraintName == constraintname" : ""));
       query.declareParameters("java.lang.String ptblname, java.lang.String pdbname,"
-      + "java.lang.String ctblname, java.lang.String cdbname" +
+          + "java.lang.String pcatname, java.lang.String ctblname, java.lang.String cdbname," +
+          "java.lang.String ccatname" +
         (constraintname != null ? ", java.lang.String constraintname" : ""));
       Collection<?> constraintNamesColl =
         constraintname != null ?
           ((Collection<?>) query.
-            executeWithArray(tableName, dbName, tableName, dbName, constraintname)):
+            executeWithArray(tableName, dbName, catName, tableName, dbName, catName, constraintname)):
           ((Collection<?>) query.
-            executeWithArray(tableName, dbName, tableName, dbName));
+            executeWithArray(tableName, dbName, catName, tableName, dbName, catName));
       for (Iterator<?> i = constraintNamesColl.iterator(); i.hasNext();) {
         String currName = (String) i.next();
         constraintNames.add(currName);
@@ -1340,16 +1491,16 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @Override
-  public Table getTable(String dbName, String tableName) throws MetaException {
+  public Table getTable(String catName, String dbName, String tableName) throws MetaException {
     boolean commited = false;
     Table tbl = null;
     try {
       openTransaction();
-      tbl = convertToTable(getMTable(dbName, tableName));
+      tbl = convertToTable(getMTable(catName, dbName, tableName));
       // Retrieve creation metadata if needed
       if (tbl != null && TableType.MATERIALIZED_VIEW.toString().equals(tbl.getTableType())) {
         tbl.setCreationMetadata(
-            convertToCreationMetadata(getCreationMetadata(dbName, tableName)));
+            convertToCreationMetadata(getCreationMetadata(catName, dbName, tableName)));
       }
       commited = commitTransaction();
     } finally {
@@ -1361,40 +1512,46 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @Override
-  public List<String> getTables(String dbName, String pattern) throws MetaException {
-    return getTables(dbName, pattern, null);
+  public List<String> getTables(String catName, String dbName, String pattern)
+      throws MetaException {
+    return getTables(catName, dbName, pattern, null);
   }
 
   @Override
-  public List<String> getTables(String dbName, String pattern, TableType tableType) throws MetaException {
+  public List<String> getTables(String catName, String dbName, String pattern, TableType tableType)
+      throws MetaException {
     try {
       // We only support pattern matching via jdo since pattern matching in Java
       // might be different than the one used by the metastore backends
-      return getTablesInternal(dbName, pattern, tableType, (pattern == null || pattern.equals(".*")), true);
+      return getTablesInternal(catName, dbName, pattern, tableType,
+          (pattern == null || pattern.equals(".*")), true);
     } catch (NoSuchObjectException e) {
       throw new MetaException(ExceptionUtils.getStackTrace(e));
     }
   }
 
-  protected List<String> getTablesInternal(String dbName, String pattern, TableType tableType,
-      boolean allowSql, boolean allowJdo) throws MetaException, NoSuchObjectException {
+  protected List<String> getTablesInternal(String catName, String dbName, String pattern,
+                                           TableType tableType, boolean allowSql, boolean allowJdo)
+      throws MetaException, NoSuchObjectException {
     final String db_name = normalizeIdentifier(dbName);
-    return new GetListHelper<String>(dbName, null, allowSql, allowJdo) {
+    final String cat_name = normalizeIdentifier(catName);
+    return new GetListHelper<String>(cat_name, dbName, null, allowSql, allowJdo) {
       @Override
       protected List<String> getSqlResult(GetHelper<List<String>> ctx)
               throws MetaException {
-        return directSql.getTables(db_name, tableType);
+        return directSql.getTables(cat_name, db_name, tableType);
       }
 
       @Override
       protected List<String> getJdoResult(GetHelper<List<String>> ctx)
               throws MetaException, NoSuchObjectException {
-        return getTablesInternalViaJdo(db_name, pattern, tableType);
+        return getTablesInternalViaJdo(cat_name, db_name, pattern, tableType);
       }
     }.run(false);
   }
 
-  private List<String> getTablesInternalViaJdo(String dbName, String pattern, TableType tableType) throws MetaException {
+  private List<String> getTablesInternalViaJdo(String catName, String dbName, String pattern,
+                                               TableType tableType) throws MetaException {
     boolean commited = false;
     Query query = null;
     List<String> tbls = null;
@@ -1407,6 +1564,7 @@ public class ObjectStore implements RawStore, Configurable {
       StringBuilder filterBuilder = new StringBuilder();
       //adds database.name == dbName to the filter
       appendSimpleCondition(filterBuilder, "database.name", new String[] {dbName}, parameterVals);
+      appendSimpleCondition(filterBuilder, "database.catalogName", new String[] {catName}, parameterVals);
       if(pattern != null) {
         appendPatternCondition(filterBuilder, "tableName", pattern, parameterVals);
       }
@@ -1427,21 +1585,23 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @Override
-  public List<String> getMaterializedViewsForRewriting(String dbName)
+  public List<String> getMaterializedViewsForRewriting(String catName, String dbName)
       throws MetaException, NoSuchObjectException {
     final String db_name = normalizeIdentifier(dbName);
+    catName = normalizeIdentifier(catName);
     boolean commited = false;
     Query<?> query = null;
     List<String> tbls = null;
     try {
       openTransaction();
       dbName = normalizeIdentifier(dbName);
-      query = pm.newQuery(MTable.class, "database.name == db && tableType == tt"
-          + " && rewriteEnabled == re");
-      query.declareParameters("java.lang.String db, java.lang.String tt, boolean re");
+      query = pm.newQuery(MTable.class,
+          "database.name == db && database.catalogName == cat && tableType == tt && rewriteEnabled == re");
+      query.declareParameters(
+          "java.lang.String db, java.lang.String cat, java.lang.String tt, boolean re");
       query.setResult("tableName");
-      Collection<String> names = (Collection<String>) query.execute(
-          db_name, TableType.MATERIALIZED_VIEW.toString(), true);
+      Collection<String> names = (Collection<String>) query.executeWithArray(
+          db_name, catName, TableType.MATERIALIZED_VIEW.toString(), true);
       tbls = new ArrayList<>(names);
       commited = commitTransaction();
     } finally {
@@ -1483,8 +1643,8 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @Override
-  public List<TableMeta> getTableMeta(String dbNames, String tableNames, List<String> tableTypes)
-      throws MetaException {
+  public List<TableMeta> getTableMeta(String catName, String dbNames, String tableNames,
+                                      List<String> tableTypes) throws MetaException {
 
     boolean commited = false;
     Query query = null;
@@ -1495,6 +1655,7 @@ public class ObjectStore implements RawStore, Configurable {
       // patterns
       StringBuilder filterBuilder = new StringBuilder();
       List<String> parameterVals = new ArrayList<>();
+      appendSimpleCondition(filterBuilder, "database.catalogName", new String[] {catName}, parameterVals);
       if (dbNames != null && !dbNames.equals("*")) {
         appendPatternCondition(filterBuilder, "database.name", dbNames, parameterVals);
       }
@@ -1505,6 +1666,10 @@ public class ObjectStore implements RawStore, Configurable {
         appendSimpleCondition(filterBuilder, "tableType", tableTypes.toArray(new String[0]), parameterVals);
       }
 
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("getTableMeta with filter " + filterBuilder.toString() + " params: " +
+            StringUtils.join(parameterVals, ", "));
+      }
       query = pm.newQuery(MTable.class, filterBuilder.toString());
       Collection<MTable> tables = (Collection<MTable>) query.executeWithArray(parameterVals.toArray(new String[parameterVals.size()]));
       for (MTable table : tables) {
@@ -1563,8 +1728,8 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @Override
-  public List<String> getAllTables(String dbName) throws MetaException {
-    return getTables(dbName, ".*");
+  public List<String> getAllTables(String catName, String dbName) throws MetaException {
+    return getTables(catName, dbName, ".*");
   }
 
   class AttachedMTableInfo {
@@ -1579,19 +1744,25 @@ public class ObjectStore implements RawStore, Configurable {
     }
   }
 
-  private AttachedMTableInfo getMTable(String db, String table, boolean retrieveCD) {
+  private AttachedMTableInfo getMTable(String catName, String db, String table,
+                                       boolean retrieveCD) {
     AttachedMTableInfo nmtbl = new AttachedMTableInfo();
     MTable mtbl = null;
     boolean commited = false;
     Query query = null;
     try {
       openTransaction();
+      catName = normalizeIdentifier(catName);
       db = normalizeIdentifier(db);
       table = normalizeIdentifier(table);
-      query = pm.newQuery(MTable.class, "tableName == table && database.name == db");
-      query.declareParameters("java.lang.String table, java.lang.String db");
+      query = pm.newQuery(MTable.class,
+          "tableName == table && database.name == db && database.catalogName == catname");
+      query.declareParameters(
+          "java.lang.String table, java.lang.String db, java.lang.String catname");
       query.setUnique(true);
-      mtbl = (MTable) query.execute(table, db);
+      LOG.debug("Executing getMTable for " +
+          getCatalogQualifiedTableName(catName, db, table));
+      mtbl = (MTable) query.execute(table, db, catName);
       pm.retrieve(mtbl);
       // Retrieving CD can be expensive and unnecessary, so do it only when required.
       if (mtbl != null && retrieveCD) {
@@ -1607,17 +1778,17 @@ public class ObjectStore implements RawStore, Configurable {
     return nmtbl;
   }
 
-  private MCreationMetadata getCreationMetadata(String dbName, String tblName) {
+  private MCreationMetadata getCreationMetadata(String catName, String dbName, String tblName) {
     boolean commited = false;
     MCreationMetadata mcm = null;
     Query query = null;
     try {
       openTransaction();
       query = pm.newQuery(
-          MCreationMetadata.class, "tblName == table && dbName == db");
-      query.declareParameters("java.lang.String table, java.lang.String db");
+          MCreationMetadata.class, "tblName == table && dbName == db && catalogName == cat");
+      query.declareParameters("java.lang.String table, java.lang.String db, java.lang.String cat");
       query.setUnique(true);
-      mcm = (MCreationMetadata) query.execute(tblName, dbName);
+      mcm = (MCreationMetadata) query.execute(tblName, dbName, catName);
       pm.retrieve(mcm);
       commited = commitTransaction();
     } finally {
@@ -1626,14 +1797,14 @@ public class ObjectStore implements RawStore, Configurable {
     return mcm;
   }
 
-  private MTable getMTable(String db, String table) {
-    AttachedMTableInfo nmtbl = getMTable(db, table, false);
+  private MTable getMTable(String catName, String db, String table) {
+    AttachedMTableInfo nmtbl = getMTable(catName, db, table, false);
     return nmtbl.mtbl;
   }
 
   @Override
-  public List<Table> getTableObjectsByName(String db, List<String> tbl_names) throws MetaException,
-      UnknownDBException {
+  public List<Table> getTableObjectsByName(String catName, String db, List<String> tbl_names)
+      throws MetaException, UnknownDBException {
     List<Table> tables = new ArrayList<>();
     boolean committed = false;
     Query dbExistsQuery = null;
@@ -1641,25 +1812,31 @@ public class ObjectStore implements RawStore, Configurable {
     try {
       openTransaction();
       db = normalizeIdentifier(db);
-      dbExistsQuery = pm.newQuery(MDatabase.class, "name == db");
-      dbExistsQuery.declareParameters("java.lang.String db");
-      dbExistsQuery.setUnique(true);
-      dbExistsQuery.setResult("name");
-      String dbNameIfExists = (String) dbExistsQuery.execute(db);
-      if (org.apache.commons.lang.StringUtils.isEmpty(dbNameIfExists)) {
-        throw new UnknownDBException("Could not find database " + db);
-      }
+      catName = normalizeIdentifier(catName);
 
       List<String> lowered_tbl_names = new ArrayList<>(tbl_names.size());
       for (String t : tbl_names) {
         lowered_tbl_names.add(normalizeIdentifier(t));
       }
       query = pm.newQuery(MTable.class);
-      query.setFilter("database.name == db && tbl_names.contains(tableName)");
-      query.declareParameters("java.lang.String db, java.util.Collection tbl_names");
-      Collection mtables = (Collection) query.execute(db, lowered_tbl_names);
-      for (Iterator iter = mtables.iterator(); iter.hasNext();) {
-        tables.add(convertToTable((MTable) iter.next()));
+      query.setFilter("database.name == db && database.catalogName == cat && tbl_names.contains(tableName)");
+      query.declareParameters("java.lang.String db, java.lang.String cat, java.util.Collection tbl_names");
+      Collection mtables = (Collection) query.execute(db, catName, lowered_tbl_names);
+      if (mtables == null || mtables.isEmpty()) {
+        // Need to differentiate between an unmatched pattern and a non-existent database
+        dbExistsQuery = pm.newQuery(MDatabase.class, "name == db && catalogName == cat");
+        dbExistsQuery.declareParameters("java.lang.String db, java.lang.String cat");
+        dbExistsQuery.setUnique(true);
+        dbExistsQuery.setResult("name");
+        String dbNameIfExists = (String) dbExistsQuery.execute(db, catName);
+        if (org.apache.commons.lang.StringUtils.isEmpty(dbNameIfExists)) {
+          throw new UnknownDBException("Could not find database " +
+              getCatalogQualifiedDbName(catName, db));
+        }
+      } else {
+        for (Iterator iter = mtables.iterator(); iter.hasNext(); ) {
+          tables.add(convertToTable((MTable) iter.next()));
+        }
       }
       committed = commitTransaction();
     } finally {
@@ -1703,6 +1880,7 @@ public class ObjectStore implements RawStore, Configurable {
         convertToFieldSchemas(mtbl.getPartitionKeys()), convertMap(mtbl.getParameters()),
         mtbl.getViewOriginalText(), mtbl.getViewExpandedText(), tableType);
     t.setRewriteEnabled(mtbl.isRewriteEnabled());
+    t.setCatName(mtbl.getDatabase().getCatalogName());
     return t;
   }
 
@@ -1712,12 +1890,13 @@ public class ObjectStore implements RawStore, Configurable {
       return null;
     }
     MDatabase mdb = null;
+    String catName = tbl.isSetCatName() ? tbl.getCatName() : getDefaultCatalog(conf);
     try {
-      mdb = getMDatabase(tbl.getDbName());
+      mdb = getMDatabase(catName, tbl.getDbName());
     } catch (NoSuchObjectException e) {
       LOG.error("Could not convert to MTable", e);
-      throw new InvalidObjectException("Database " + tbl.getDbName()
-          + " doesn't exist.");
+      throw new InvalidObjectException("Database " +
+          getCatalogQualifiedDbName(catName, tbl.getDbName()) + " doesn't exist.");
     }
 
     // If the table has property EXTERNAL set, update table type
@@ -1965,9 +2144,9 @@ public class ObjectStore implements RawStore, Configurable {
     Set<MTable> tablesUsed = new HashSet<>();
     for (String fullyQualifiedName : m.getTablesUsed()) {
       String[] names =  fullyQualifiedName.split("\\.");
-      tablesUsed.add(getMTable(names[0], names[1], false).mtbl);
+      tablesUsed.add(getMTable(m.getCatName(), names[0], names[1], false).mtbl);
     }
-    return new MCreationMetadata(m.getDbName(), m.getTblName(),
+    return new MCreationMetadata(m.getCatName(), m.getDbName(), m.getTblName(),
         tablesUsed, m.getValidTxnList());
   }
 
@@ -1982,7 +2161,7 @@ public class ObjectStore implements RawStore, Configurable {
           Warehouse.getQualifiedName(
               mtbl.getDatabase().getName(), mtbl.getTableName()));
     }
-    CreationMetadata r = new CreationMetadata(
+    CreationMetadata r = new CreationMetadata(s.getCatalogName(),
         s.getDbName(), s.getTblName(), tablesUsed);
     if (s.getTxnList() != null) {
       r.setValidTxnList(s.getTxnList());
@@ -1991,17 +2170,17 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @Override
-  public boolean addPartitions(String dbName, String tblName, List<Partition> parts)
+  public boolean addPartitions(String catName, String dbName, String tblName, List<Partition> parts)
       throws InvalidObjectException, MetaException {
     boolean success = false;
     openTransaction();
     try {
       List<MTablePrivilege> tabGrants = null;
       List<MTableColumnPrivilege> tabColumnGrants = null;
-      MTable table = this.getMTable(dbName, tblName);
+      MTable table = this.getMTable(catName, dbName, tblName);
       if ("TRUE".equalsIgnoreCase(table.getParameters().get("PARTITION_LEVEL_PRIVILEGE"))) {
-        tabGrants = this.listAllTableGrants(dbName, tblName);
-        tabColumnGrants = this.listTableAllColumnGrants(dbName, tblName);
+        tabGrants = this.listAllTableGrants(catName, dbName, tblName);
+        tabColumnGrants = this.listTableAllColumnGrants(catName, dbName, tblName);
       }
       List<Object> toPersist = new ArrayList<>();
       for (Partition part : parts) {
@@ -2046,7 +2225,7 @@ public class ObjectStore implements RawStore, Configurable {
       Partition part, boolean ifNotExists) throws MetaException {
     MetaStoreUtils.validatePartitionNameCharacters(part.getValues(),
         partitionValidationPattern);
-    boolean doesExist = doesPartitionExist(
+    boolean doesExist = doesPartitionExist(part.getCatName(),
         part.getDbName(), part.getTableName(), part.getValues());
     if (doesExist && !ifNotExists) {
       throw new MetaException("Partition already exists: " + part);
@@ -2055,7 +2234,7 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @Override
-  public boolean addPartitions(String dbName, String tblName,
+  public boolean addPartitions(String catName, String dbName, String tblName,
                                PartitionSpecProxy partitionSpec, boolean ifNotExists)
       throws InvalidObjectException, MetaException {
     boolean success = false;
@@ -2063,10 +2242,10 @@ public class ObjectStore implements RawStore, Configurable {
     try {
       List<MTablePrivilege> tabGrants = null;
       List<MTableColumnPrivilege> tabColumnGrants = null;
-      MTable table = this.getMTable(dbName, tblName);
+      MTable table = this.getMTable(catName, dbName, tblName);
       if ("TRUE".equalsIgnoreCase(table.getParameters().get("PARTITION_LEVEL_PRIVILEGE"))) {
-        tabGrants = this.listAllTableGrants(dbName, tblName);
-        tabColumnGrants = this.listTableAllColumnGrants(dbName, tblName);
+        tabGrants = this.listAllTableGrants(catName, dbName, tblName);
+        tabColumnGrants = this.listTableAllColumnGrants(catName, dbName, tblName);
       }
 
       if (!partitionSpec.getTableName().equals(tblName) || !partitionSpec.getDbName().equals(dbName)) {
@@ -2117,14 +2296,14 @@ public class ObjectStore implements RawStore, Configurable {
     boolean success = false;
     boolean commited = false;
     try {
-      MTable table = this.getMTable(part.getDbName(), part.getTableName());
+      String catName = part.isSetCatName() ? part.getCatName() : getDefaultCatalog(conf);
+      MTable table = this.getMTable(catName, part.getDbName(), part.getTableName());
       List<MTablePrivilege> tabGrants = null;
       List<MTableColumnPrivilege> tabColumnGrants = null;
       if ("TRUE".equalsIgnoreCase(table.getParameters().get("PARTITION_LEVEL_PRIVILEGE"))) {
-        tabGrants = this.listAllTableGrants(part
-            .getDbName(), part.getTableName());
+        tabGrants = this.listAllTableGrants(catName, part.getDbName(), part.getTableName());
         tabColumnGrants = this.listTableAllColumnGrants(
-            part.getDbName(), part.getTableName());
+            catName, part.getDbName(), part.getTableName());
       }
       openTransaction();
       MPartition mpart = convertToMPart(part, true);
@@ -2167,10 +2346,10 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @Override
-  public Partition getPartition(String dbName, String tableName,
+  public Partition getPartition(String catName, String dbName, String tableName,
       List<String> part_vals) throws NoSuchObjectException, MetaException {
     openTransaction();
-    Partition part = convertToPart(getMPartition(dbName, tableName, part_vals));
+    Partition part = convertToPart(getMPartition(catName, dbName, tableName, part_vals));
     commitTransaction();
     if(part == null) {
       throw new NoSuchObjectException("partition values="
@@ -2180,7 +2359,7 @@ public class ObjectStore implements RawStore, Configurable {
     return part;
   }
 
-  private MPartition getMPartition(String dbName, String tableName, List<String> part_vals)
+  private MPartition getMPartition(String catName, String dbName, String tableName, List<String> part_vals)
       throws MetaException {
     List<MPartition> mparts = null;
     MPartition ret = null;
@@ -2188,9 +2367,10 @@ public class ObjectStore implements RawStore, Configurable {
     Query query = null;
     try {
       openTransaction();
+      catName = normalizeIdentifier(catName);
       dbName = normalizeIdentifier(dbName);
       tableName = normalizeIdentifier(tableName);
-      MTable mtbl = getMTable(dbName, tableName);
+      MTable mtbl = getMTable(catName, dbName, tableName);
       if (mtbl == null) {
         commited = commitTransaction();
         return null;
@@ -2201,9 +2381,11 @@ public class ObjectStore implements RawStore, Configurable {
           Warehouse.makePartName(convertToFieldSchemas(mtbl.getPartitionKeys()), part_vals);
       query =
           pm.newQuery(MPartition.class,
-              "table.tableName == t1 && table.database.name == t2 && partitionName == t3");
-      query.declareParameters("java.lang.String t1, java.lang.String t2, java.lang.String t3");
-      mparts = (List<MPartition>) query.execute(tableName, dbName, name);
+              "table.tableName == t1 && table.database.name == t2 && partitionName == t3 " +
+                  " && table.database.catalogName == t4");
+      query.declareParameters("java.lang.String t1, java.lang.String t2, java.lang.String t3, "
+          + "java.lang.String t4");
+      mparts = (List<MPartition>) query.executeWithArray(tableName, dbName, name, catName);
       pm.retrieveAll(mparts);
       commited = commitTransaction();
       // We need to compare partition name with requested name since some DBs
@@ -2246,7 +2428,7 @@ public class ObjectStore implements RawStore, Configurable {
     if (part == null) {
       return null;
     }
-    MTable mt = getMTable(part.getDbName(), part.getTableName());
+    MTable mt = getMTable(part.getCatName(), part.getDbName(), part.getTableName());
     if (mt == null) {
       throw new InvalidObjectException(
           "Partition doesn't have a valid table or database name");
@@ -2277,30 +2459,34 @@ public class ObjectStore implements RawStore, Configurable {
     if (mpart == null) {
       return null;
     }
-    return new Partition(convertList(mpart.getValues()), mpart.getTable().getDatabase()
+    Partition p = new Partition(convertList(mpart.getValues()), mpart.getTable().getDatabase()
         .getName(), mpart.getTable().getTableName(), mpart.getCreateTime(),
         mpart.getLastAccessTime(), convertToStorageDescriptor(mpart.getSd()),
         convertMap(mpart.getParameters()));
+    p.setCatName(mpart.getTable().getDatabase().getCatalogName());
+    return p;
   }
 
-  private Partition convertToPart(String dbName, String tblName, MPartition mpart)
+  private Partition convertToPart(String catName, String dbName, String tblName, MPartition mpart)
       throws MetaException {
     if (mpart == null) {
       return null;
     }
-    return new Partition(convertList(mpart.getValues()), dbName, tblName,
+    Partition p = new Partition(convertList(mpart.getValues()), dbName, tblName,
         mpart.getCreateTime(), mpart.getLastAccessTime(),
         convertToStorageDescriptor(mpart.getSd(), false), convertMap(mpart.getParameters()));
+    p.setCatName(catName);
+    return p;
   }
 
   @Override
-  public boolean dropPartition(String dbName, String tableName,
+  public boolean dropPartition(String catName, String dbName, String tableName,
     List<String> part_vals) throws MetaException, NoSuchObjectException, InvalidObjectException,
     InvalidInputException {
     boolean success = false;
     try {
       openTransaction();
-      MPartition part = getMPartition(dbName, tableName, part_vals);
+      MPartition part = getMPartition(catName, dbName, tableName, part_vals);
       dropPartitionCommon(part);
       success = commitTransaction();
     } finally {
@@ -2312,7 +2498,7 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @Override
-  public void dropPartitions(String dbName, String tblName, List<String> partNames)
+  public void dropPartitions(String catName, String dbName, String tblName, List<String> partNames)
       throws MetaException, NoSuchObjectException {
     if (CollectionUtils.isEmpty(partNames)) {
       return;
@@ -2321,15 +2507,15 @@ public class ObjectStore implements RawStore, Configurable {
     openTransaction();
     try {
       // Delete all things.
-      dropPartitionGrantsNoTxn(dbName, tblName, partNames);
-      dropPartitionAllColumnGrantsNoTxn(dbName, tblName, partNames);
-      dropPartitionColumnStatisticsNoTxn(dbName, tblName, partNames);
+      dropPartitionGrantsNoTxn(catName, dbName, tblName, partNames);
+      dropPartitionAllColumnGrantsNoTxn(catName, dbName, tblName, partNames);
+      dropPartitionColumnStatisticsNoTxn(catName, dbName, tblName, partNames);
 
       // CDs are reused; go thry partition SDs, detach all CDs from SDs, then remove unused CDs.
-      for (MColumnDescriptor mcd : detachCdsFromSdsNoTxn(dbName, tblName, partNames)) {
+      for (MColumnDescriptor mcd : detachCdsFromSdsNoTxn(catName, dbName, tblName, partNames)) {
         removeUnusedColumnDescriptor(mcd);
       }
-      dropPartitionsNoTxn(dbName, tblName, partNames);
+      dropPartitionsNoTxn(catName, dbName, tblName, partNames);
       if (!(success = commitTransaction())) {
         throw new MetaException("Failed to drop partitions"); // Should not happen?
       }
@@ -2343,12 +2529,6 @@ public class ObjectStore implements RawStore, Configurable {
   /**
    * Drop an MPartition and cascade deletes (e.g., delete partition privilege grants,
    *   drop the storage descriptor cleanly, etc.)
-   * @param part - the MPartition to drop
-   * @return whether the transaction committed successfully
-   * @throws InvalidInputException
-   * @throws InvalidObjectException
-   * @throws MetaException
-   * @throws NoSuchObjectException
    */
   private boolean dropPartitionCommon(MPartition part) throws NoSuchObjectException, MetaException,
     InvalidObjectException, InvalidInputException {
@@ -2364,6 +2544,7 @@ public class ObjectStore implements RawStore, Configurable {
         String partName = FileUtils.makePartName(colNames, part.getValues());
 
         List<MPartitionPrivilege> partGrants = listPartitionGrants(
+            part.getTable().getDatabase().getCatalogName(),
             part.getTable().getDatabase().getName(),
             part.getTable().getTableName(),
             Lists.newArrayList(partName));
@@ -2373,6 +2554,7 @@ public class ObjectStore implements RawStore, Configurable {
         }
 
         List<MPartitionColumnPrivilege> partColumnGrants = listPartitionAllColumnGrants(
+            part.getTable().getDatabase().getCatalogName(),
             part.getTable().getDatabase().getName(),
             part.getTable().getTableName(),
             Lists.newArrayList(partName));
@@ -2380,12 +2562,13 @@ public class ObjectStore implements RawStore, Configurable {
           pm.deletePersistentAll(partColumnGrants);
         }
 
+        String catName = part.getTable().getDatabase().getCatalogName();
         String dbName = part.getTable().getDatabase().getName();
         String tableName = part.getTable().getTableName();
 
         // delete partition level column stats if it exists
        try {
-          deletePartitionColumnStatistics(dbName, tableName, partName, part.getValues(), null);
+          deletePartitionColumnStatistics(catName, dbName, tableName, partName, part.getValues(), null);
         } catch (NoSuchObjectException e) {
           LOG.info("No column statistics records found to delete");
         }
@@ -2403,26 +2586,26 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @Override
-  public List<Partition> getPartitions(
-      String dbName, String tableName, int maxParts) throws MetaException, NoSuchObjectException {
-    return getPartitionsInternal(dbName, tableName, maxParts, true, true);
+  public List<Partition> getPartitions(String catName, String dbName, String tableName,
+                                       int maxParts) throws MetaException, NoSuchObjectException {
+    return getPartitionsInternal(catName, dbName, tableName, maxParts, true, true);
   }
 
-  protected List<Partition> getPartitionsInternal(
-      String dbName, String tblName, final int maxParts, boolean allowSql, boolean allowJdo)
+  protected List<Partition> getPartitionsInternal(String catName, String dbName, String tblName,
+                                                  final int maxParts, boolean allowSql, boolean allowJdo)
           throws MetaException, NoSuchObjectException {
-    return new GetListHelper<Partition>(dbName, tblName, allowSql, allowJdo) {
+    return new GetListHelper<Partition>(catName, dbName, tblName, allowSql, allowJdo) {
       @Override
       protected List<Partition> getSqlResult(GetHelper<List<Partition>> ctx) throws MetaException {
         Integer max = (maxParts < 0) ? null : maxParts;
-        return directSql.getPartitions(dbName, tblName, max);
+        return directSql.getPartitions(catName, dbName, tblName, max);
       }
       @Override
       protected List<Partition> getJdoResult(
           GetHelper<List<Partition>> ctx) throws MetaException {
         QueryWrapper queryWrapper = new QueryWrapper();
         try {
-          return convertToParts(listMPartitions(dbName, tblName, maxParts, queryWrapper));
+          return convertToParts(listMPartitions(catName, dbName, tblName, maxParts, queryWrapper));
         } finally {
           queryWrapper.close();
         }
@@ -2431,7 +2614,7 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @Override
-  public List<Partition> getPartitionsWithAuth(String dbName, String tblName,
+  public List<Partition> getPartitionsWithAuth(String catName, String dbName, String tblName,
       short max, String userName, List<String> groupNames)
           throws MetaException, InvalidObjectException {
     boolean success = false;
@@ -2439,7 +2622,7 @@ public class ObjectStore implements RawStore, Configurable {
 
     try {
       openTransaction();
-      List<MPartition> mparts = listMPartitions(dbName, tblName, max, queryWrapper);
+      List<MPartition> mparts = listMPartitions(catName, dbName, tblName, max, queryWrapper);
       List<Partition> parts = new ArrayList<>(mparts.size());
       if (CollectionUtils.isNotEmpty(mparts)) {
         for (MPartition mpart : mparts) {
@@ -2450,7 +2633,7 @@ public class ObjectStore implements RawStore, Configurable {
           if ("TRUE".equalsIgnoreCase(mtbl.getParameters().get("PARTITION_LEVEL_PRIVILEGE"))) {
             String partName = Warehouse.makePartName(this.convertToFieldSchemas(mtbl
                 .getPartitionKeys()), part.getValues());
-            PrincipalPrivilegeSet partAuth = this.getPartitionPrivilegeSet(dbName,
+            PrincipalPrivilegeSet partAuth = this.getPartitionPrivilegeSet(catName, dbName,
                 tblName, partName, userName, groupNames);
             part.setPrivileges(partAuth);
           }
@@ -2464,13 +2647,13 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @Override
-  public Partition getPartitionWithAuth(String dbName, String tblName,
+  public Partition getPartitionWithAuth(String catName, String dbName, String tblName,
       List<String> partVals, String user_name, List<String> group_names)
       throws NoSuchObjectException, MetaException, InvalidObjectException {
     boolean success = false;
     try {
       openTransaction();
-      MPartition mpart = getMPartition(dbName, tblName, partVals);
+      MPartition mpart = getMPartition(catName, dbName, tblName, partVals);
       if (mpart == null) {
         commitTransaction();
         throw new NoSuchObjectException("partition values="
@@ -2482,7 +2665,7 @@ public class ObjectStore implements RawStore, Configurable {
       if ("TRUE".equalsIgnoreCase(mtbl.getParameters().get("PARTITION_LEVEL_PRIVILEGE"))) {
         String partName = Warehouse.makePartName(this.convertToFieldSchemas(mtbl
             .getPartitionKeys()), partVals);
-        PrincipalPrivilegeSet partAuth = this.getPartitionPrivilegeSet(dbName,
+        PrincipalPrivilegeSet partAuth = this.getPartitionPrivilegeSet(catName, dbName,
             tblName, partName, user_name, group_names);
         part.setPrivileges(partAuth);
       }
@@ -2515,11 +2698,11 @@ public class ObjectStore implements RawStore, Configurable {
     return dest;
   }
 
-  private List<Partition> convertToParts(String dbName, String tblName, List<MPartition> mparts)
+  private List<Partition> convertToParts(String catName, String dbName, String tblName, List<MPartition> mparts)
       throws MetaException {
     List<Partition> parts = new ArrayList<>(mparts.size());
     for (MPartition mp : mparts) {
-      parts.add(convertToPart(dbName, tblName, mp));
+      parts.add(convertToPart(catName, dbName, tblName, mp));
       Deadline.checkTimeout();
     }
     return parts;
@@ -2527,14 +2710,14 @@ public class ObjectStore implements RawStore, Configurable {
 
   // TODO:pc implement max
   @Override
-  public List<String> listPartitionNames(String dbName, String tableName,
+  public List<String> listPartitionNames(String catName, String dbName, String tableName,
       short max) throws MetaException {
     List<String> pns = null;
     boolean success = false;
     try {
       openTransaction();
       LOG.debug("Executing getPartitionNames");
-      pns = getPartitionNamesNoTxn(dbName, tableName, max);
+      pns = getPartitionNamesNoTxn(catName, dbName, tableName, max);
       success = commitTransaction();
     } finally {
       if (!success) {
@@ -2584,21 +2767,24 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @Override
-  public PartitionValuesResponse listPartitionValues(String dbName, String tableName, List<FieldSchema> cols,
-                                                     boolean applyDistinct, String filter, boolean ascending,
-                                                     List<FieldSchema> order, long maxParts) throws MetaException {
+  public PartitionValuesResponse listPartitionValues(String catName, String dbName,
+                                                     String tableName, List<FieldSchema> cols,
+                                                     boolean applyDistinct, String filter,
+                                                     boolean ascending, List<FieldSchema> order,
+                                                     long maxParts) throws MetaException {
 
+    catName = normalizeIdentifier(catName);
     dbName = dbName.toLowerCase().trim();
     tableName = tableName.toLowerCase().trim();
     try {
       if (filter == null || filter.isEmpty()) {
-        PartitionValuesResponse response =
-            getDistinctValuesForPartitionsNoTxn(dbName, tableName, cols, applyDistinct, ascending, maxParts);
+        PartitionValuesResponse response = getDistinctValuesForPartitionsNoTxn(catName, dbName,
+            tableName, cols, applyDistinct, maxParts);
         LOG.info("Number of records fetched: {}", response.getPartitionValues().size());
         return response;
       } else {
         PartitionValuesResponse response =
-            extractPartitionNamesByFilter(dbName, tableName, filter, cols, ascending, applyDistinct, maxParts);
+            extractPartitionNamesByFilter(catName, dbName, tableName, filter, cols, ascending, maxParts);
         if (response != null && response.getPartitionValues() != null) {
           LOG.info("Number of records fetched with filter: {}", response.getPartitionValues().size());
         }
@@ -2607,31 +2793,33 @@ public class ObjectStore implements RawStore, Configurable {
     } catch (Exception t) {
       LOG.error("Exception in ORM", t);
       throw new MetaException("Error retrieving partition values: " + t);
-    } finally {
     }
   }
 
-  private PartitionValuesResponse extractPartitionNamesByFilter(String dbName, String tableName, String filter,
-                                                                List<FieldSchema> cols, boolean ascending, boolean applyDistinct, long maxParts)
+  private PartitionValuesResponse extractPartitionNamesByFilter(
+      String catName, String dbName, String tableName, String filter, List<FieldSchema> cols,
+      boolean ascending, long maxParts)
       throws MetaException, NoSuchObjectException {
 
-    LOG.info("Database: {} Table: {} filter: \"{}\" cols: {}", dbName, tableName, filter, cols);
+    LOG.info("Table: {} filter: \"{}\" cols: {}",
+        getCatalogQualifiedTableName(catName, dbName, tableName), filter, cols);
     List<String> partitionNames = null;
     List<Partition> partitions = null;
-    Table tbl = getTable(dbName, tableName);
+    Table tbl = getTable(catName, dbName, tableName);
     try {
       // Get partitions by name - ascending or descending
-      partitionNames = getPartitionNamesByFilter(dbName, tableName, filter, ascending, maxParts);
+      partitionNames = getPartitionNamesByFilter(catName, dbName, tableName, filter, ascending,
+          maxParts);
     } catch (MetaException e) {
       LOG.warn("Querying by partition names failed, trying out with partition objects, filter: {}", filter);
     }
 
     if (partitionNames == null) {
-      partitions = getPartitionsByFilter(dbName, tableName, filter, (short) maxParts);
+      partitions = getPartitionsByFilter(catName, dbName, tableName, filter, (short) maxParts);
     }
 
     if (partitions != null) {
-      partitionNames = new ArrayList<String>(partitions.size());
+      partitionNames = new ArrayList<>(partitions.size());
       for (Partition partition : partitions) {
         // Check for NULL's just to be safe
         if (tbl.getPartitionKeys() != null && partition.getValues() != null) {
@@ -2642,7 +2830,7 @@ public class ObjectStore implements RawStore, Configurable {
 
     if (partitionNames == null && partitions == null) {
       throw new MetaException("Cannot obtain list of partitions by filter:\"" + filter +
-          "\" for " + dbName + ":" + tableName);
+          "\" for " + getCatalogQualifiedTableName(catName, dbName, tableName));
     }
 
     if (!ascending) {
@@ -2651,7 +2839,7 @@ public class ObjectStore implements RawStore, Configurable {
 
     // Return proper response
     PartitionValuesResponse response = new PartitionValuesResponse();
-    response.setPartitionValues(new ArrayList<PartitionValuesRow>(partitionNames.size()));
+    response.setPartitionValues(new ArrayList<>(partitionNames.size()));
     LOG.info("Converting responses to Partition values for items: {}", partitionNames.size());
     for (String partName : partitionNames) {
       ArrayList<String> vals = new ArrayList<String>(Collections.nCopies(tbl.getPartitionKeys().size(), null));
@@ -2665,26 +2853,27 @@ public class ObjectStore implements RawStore, Configurable {
     return response;
   }
 
-  private List<String> getPartitionNamesByFilter(String dbName, String tableName,
+  private List<String> getPartitionNamesByFilter(String catName, String dbName, String tableName,
                                                  String filter, boolean ascending, long maxParts)
       throws MetaException {
 
     boolean success = false;
-    List<String> partNames = new ArrayList<String>();
+    List<String> partNames = new ArrayList<>();
     try {
       openTransaction();
       LOG.debug("Executing getPartitionNamesByFilter");
+      catName = normalizeIdentifier(catName);
       dbName = dbName.toLowerCase();
       tableName = tableName.toLowerCase();
 
-      MTable mtable = getMTable(dbName, tableName);
+      MTable mtable = getMTable(catName, dbName, tableName);
       if( mtable == null ) {
         // To be consistent with the behavior of listPartitionNames, if the
         // table or db does not exist, we return an empty list
         return partNames;
       }
-      Map<String, Object> params = new HashMap<String, Object>();
-      String queryFilterString = makeQueryFilterString(dbName, mtable, filter, params);
+      Map<String, Object> params = new HashMap<>();
+      String queryFilterString = makeQueryFilterString(catName, dbName, mtable, filter, params);
       Query query = pm.newQuery(
           "select partitionName from org.apache.hadoop.hive.metastore.model.MPartition "
               + "where " + queryFilterString);
@@ -2723,15 +2912,16 @@ public class ObjectStore implements RawStore, Configurable {
     return partNames;
   }
 
-  private PartitionValuesResponse getDistinctValuesForPartitionsNoTxn(String dbName, String tableName, List<FieldSchema> cols,
-                                                                      boolean applyDistinct, boolean ascending, long maxParts)
+  private PartitionValuesResponse getDistinctValuesForPartitionsNoTxn(
+      String catName, String dbName, String tableName, List<FieldSchema> cols,
+      boolean applyDistinct, long maxParts)
       throws MetaException {
-
     try {
       openTransaction();
       Query q = pm.newQuery("select partitionName from org.apache.hadoop.hive.metastore.model.MPartition "
-          + "where table.database.name == t1 && table.tableName == t2 ");
-      q.declareParameters("java.lang.String t1, java.lang.String t2");
+          + "where table.database.name == t1 && table.database.catalogName == t2 && " +
+          "table.tableName == t3 ");
+      q.declareParameters("java.lang.String t1, java.lang.String t2, java.lang.String t3");
 
       // TODO: Ordering seems to affect the distinctness, needs checking, disabling.
 /*
@@ -2748,7 +2938,7 @@ public class ObjectStore implements RawStore, Configurable {
       if (applyDistinct) {
         partValuesSelect.append("DISTINCT ");
       }
-      List<FieldSchema> partitionKeys = getTable(dbName, tableName).getPartitionKeys();
+      List<FieldSchema> partitionKeys = getTable(catName, dbName, tableName).getPartitionKeys();
       for (FieldSchema key : cols) {
         partValuesSelect.append(extractPartitionKey(key, partitionKeys)).append(", ");
       }
@@ -2757,9 +2947,9 @@ public class ObjectStore implements RawStore, Configurable {
       q.setResult(partValuesSelect.toString());
 
       PartitionValuesResponse response = new PartitionValuesResponse();
-      response.setPartitionValues(new ArrayList<PartitionValuesRow>());
+      response.setPartitionValues(new ArrayList<>());
       if (cols.size() > 1) {
-        List<Object[]> results = (List<Object[]>) q.execute(dbName, tableName);
+        List<Object[]> results = (List<Object[]>) q.execute(dbName, catName, tableName);
         for (Object[] row : results) {
           PartitionValuesRow rowResponse = new PartitionValuesRow();
           for (Object columnValue : row) {
@@ -2768,7 +2958,7 @@ public class ObjectStore implements RawStore, Configurable {
           response.addToPartitionValues(rowResponse);
         }
       } else {
-        List<Object> results = (List<Object>) q.execute(dbName, tableName);
+        List<Object> results = (List<Object>) q.execute(dbName, catName, tableName);
         for (Object row : results) {
           PartitionValuesRow rowResponse = new PartitionValuesRow();
           rowResponse.addToRow((String) row);
@@ -2782,24 +2972,25 @@ public class ObjectStore implements RawStore, Configurable {
     }
   }
 
-  private List<String> getPartitionNamesNoTxn(String dbName, String tableName, short max) {
+  private List<String> getPartitionNamesNoTxn(String catName, String dbName, String tableName, short max) {
     List<String> pns = new ArrayList<>();
     if (max == 0) {
       return pns;
     }
+    catName = normalizeIdentifier(catName);
     dbName = normalizeIdentifier(dbName);
     tableName = normalizeIdentifier(tableName);
     Query query =
         pm.newQuery("select partitionName from org.apache.hadoop.hive.metastore.model.MPartition "
-            + "where table.database.name == t1 && table.tableName == t2 "
+            + "where table.database.name == t1 && table.tableName == t2 && table.database.catalogName == t3 "
             + "order by partitionName asc");
-    query.declareParameters("java.lang.String t1, java.lang.String t2");
+    query.declareParameters("java.lang.String t1, java.lang.String t2, java.lang.String t3");
     query.setResult("partitionName");
 
     if (max > 0) {
       query.setRange(0, max);
     }
-    Collection<String> names = (Collection<String>) query.execute(dbName, tableName);
+    Collection<String> names = (Collection<String>) query.execute(dbName, tableName, catName);
     pns.addAll(names);
 
     if (query != null) {
@@ -2823,14 +3014,16 @@ public class ObjectStore implements RawStore, Configurable {
    *          you want results for.  E.g., if resultsCol is partitionName, the Collection
    *          has types of String, and if resultsCol is null, the types are MPartition.
    */
-  private Collection getPartitionPsQueryResults(String dbName, String tableName,
+  private Collection getPartitionPsQueryResults(String catName, String dbName, String tableName,
       List<String> part_vals, short max_parts, String resultsCol, QueryWrapper queryWrapper)
       throws MetaException, NoSuchObjectException {
+    catName = normalizeIdentifier(catName);
     dbName = normalizeIdentifier(dbName);
     tableName = normalizeIdentifier(tableName);
-    Table table = getTable(dbName, tableName);
+    Table table = getTable(catName, dbName, tableName);
     if (table == null) {
-      throw new NoSuchObjectException(dbName + "." + tableName + " table not found");
+      throw new NoSuchObjectException(getCatalogQualifiedTableName(catName, dbName, tableName)
+          + " table not found");
     }
     List<FieldSchema> partCols = table.getPartitionKeys();
     int numPartKeys = partCols.size();
@@ -2851,10 +3044,11 @@ public class ObjectStore implements RawStore, Configurable {
     }
     Query query = queryWrapper.query = pm.newQuery(MPartition.class);
     StringBuilder queryFilter = new StringBuilder("table.database.name == dbName");
+    queryFilter.append(" && table.database.catalogName == catName");
     queryFilter.append(" && table.tableName == tableName");
     queryFilter.append(" && partitionName.matches(partialRegex)");
     query.setFilter(queryFilter.toString());
-    query.declareParameters("java.lang.String dbName, "
+    query.declareParameters("java.lang.String dbName, java.lang.String catName, "
         + "java.lang.String tableName, java.lang.String partialRegex");
     if (max_parts >= 0) {
       // User specified a row limit, set it on the Query
@@ -2864,11 +3058,11 @@ public class ObjectStore implements RawStore, Configurable {
       query.setResult(resultsCol);
     }
 
-    return (Collection) query.execute(dbName, tableName, partNameMatcher);
+    return (Collection) query.executeWithArray(dbName, catName, tableName, partNameMatcher);
   }
 
   @Override
-  public List<Partition> listPartitionsPsWithAuth(String db_name, String tbl_name,
+  public List<Partition> listPartitionsPsWithAuth(String catName, String db_name, String tbl_name,
       List<String> part_vals, short max_parts, String userName, List<String> groupNames)
       throws MetaException, InvalidObjectException, NoSuchObjectException {
     List<Partition> partitions = new ArrayList<>();
@@ -2878,9 +3072,9 @@ public class ObjectStore implements RawStore, Configurable {
     try {
       openTransaction();
       LOG.debug("executing listPartitionNamesPsWithAuth");
-      Collection parts = getPartitionPsQueryResults(db_name, tbl_name,
+      Collection parts = getPartitionPsQueryResults(catName, db_name, tbl_name,
           part_vals, max_parts, null, queryWrapper);
-      MTable mtbl = getMTable(db_name, tbl_name);
+      MTable mtbl = getMTable(catName, db_name, tbl_name);
       for (Object o : parts) {
         Partition part = convertToPart((MPartition) o);
         //set auth privileges
@@ -2888,7 +3082,7 @@ public class ObjectStore implements RawStore, Configurable {
             "TRUE".equalsIgnoreCase(mtbl.getParameters().get("PARTITION_LEVEL_PRIVILEGE"))) {
           String partName = Warehouse.makePartName(this.convertToFieldSchemas(mtbl
               .getPartitionKeys()), part.getValues());
-          PrincipalPrivilegeSet partAuth = getPartitionPrivilegeSet(db_name,
+          PrincipalPrivilegeSet partAuth = getPartitionPrivilegeSet(catName, db_name,
               tbl_name, partName, userName, groupNames);
           part.setPrivileges(partAuth);
         }
@@ -2902,7 +3096,7 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @Override
-  public List<String> listPartitionNamesPs(String dbName, String tableName,
+  public List<String> listPartitionNamesPs(String catName, String dbName, String tableName,
       List<String> part_vals, short max_parts) throws MetaException, NoSuchObjectException {
     List<String> partitionNames = new ArrayList<>();
     boolean success = false;
@@ -2911,7 +3105,7 @@ public class ObjectStore implements RawStore, Configurable {
     try {
       openTransaction();
       LOG.debug("Executing listPartitionNamesPs");
-      Collection<String> names = getPartitionPsQueryResults(dbName, tableName,
+      Collection<String> names = getPartitionPsQueryResults(catName, dbName, tableName,
           part_vals, max_parts, "partitionName", queryWrapper);
       partitionNames.addAll(names);
       success = commitTransaction();
@@ -2922,7 +3116,8 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   // TODO:pc implement max
-  private List<MPartition> listMPartitions(String dbName, String tableName, int max, QueryWrapper queryWrapper) {
+  private List<MPartition> listMPartitions(String catName, String dbName, String tableName,
+                                           int max, QueryWrapper queryWrapper) {
     boolean success = false;
     List<MPartition> mparts = null;
     try {
@@ -2930,13 +3125,14 @@ public class ObjectStore implements RawStore, Configurable {
       LOG.debug("Executing listMPartitions");
       dbName = normalizeIdentifier(dbName);
       tableName = normalizeIdentifier(tableName);
-      Query query = queryWrapper.query = pm.newQuery(MPartition.class, "table.tableName == t1 && table.database.name == t2");
-      query.declareParameters("java.lang.String t1, java.lang.String t2");
+      Query query = queryWrapper.query = pm.newQuery(MPartition.class,
+          "table.tableName == t1 && table.database.name == t2 && table.database.catalogName == t3");
+      query.declareParameters("java.lang.String t1, java.lang.String t2, java.lang.String t3");
       query.setOrdering("partitionName ascending");
       if (max > 0) {
         query.setRange(0, max);
       }
-      mparts = (List<MPartition>) query.execute(tableName, dbName);
+      mparts = (List<MPartition>) query.execute(tableName, dbName, catName);
       LOG.debug("Done executing query for listMPartitions");
       pm.retrieveAll(mparts);
       success = commitTransaction();
@@ -2950,41 +3146,43 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @Override
-  public List<Partition> getPartitionsByNames(String dbName, String tblName,
+  public List<Partition> getPartitionsByNames(String catName, String dbName, String tblName,
       List<String> partNames) throws MetaException, NoSuchObjectException {
-    return getPartitionsByNamesInternal(dbName, tblName, partNames, true, true);
+    return getPartitionsByNamesInternal(catName, dbName, tblName, partNames, true, true);
   }
 
-  protected List<Partition> getPartitionsByNamesInternal(String dbName, String tblName,
-      final List<String> partNames, boolean allowSql, boolean allowJdo)
+  protected List<Partition> getPartitionsByNamesInternal(String catName, String dbName,
+                                                         String tblName,
+                                                         final List<String> partNames,
+                                                         boolean allowSql, boolean allowJdo)
           throws MetaException, NoSuchObjectException {
-    return new GetListHelper<Partition>(dbName, tblName, allowSql, allowJdo) {
+    return new GetListHelper<Partition>(catName, dbName, tblName, allowSql, allowJdo) {
       @Override
       protected List<Partition> getSqlResult(GetHelper<List<Partition>> ctx) throws MetaException {
-        return directSql.getPartitionsViaSqlFilter(dbName, tblName, partNames);
+        return directSql.getPartitionsViaSqlFilter(catName, dbName, tblName, partNames);
       }
       @Override
       protected List<Partition> getJdoResult(
           GetHelper<List<Partition>> ctx) throws MetaException, NoSuchObjectException {
-        return getPartitionsViaOrmFilter(dbName, tblName, partNames);
+        return getPartitionsViaOrmFilter(catName, dbName, tblName, partNames);
       }
     }.run(false);
   }
 
   @Override
-  public boolean getPartitionsByExpr(String dbName, String tblName, byte[] expr,
+  public boolean getPartitionsByExpr(String catName, String dbName, String tblName, byte[] expr,
       String defaultPartitionName, short maxParts, List<Partition> result) throws TException {
     return getPartitionsByExprInternal(
-        dbName, tblName, expr, defaultPartitionName, maxParts, result, true, true);
+        catName, dbName, tblName, expr, defaultPartitionName, maxParts, result, true, true);
   }
 
-  protected boolean getPartitionsByExprInternal(String dbName, String tblName, final byte[] expr,
+  protected boolean getPartitionsByExprInternal(String catName, String dbName, String tblName, final byte[] expr,
       final String defaultPartitionName, final  short maxParts, List<Partition> result,
       boolean allowSql, boolean allowJdo) throws TException {
     assert result != null;
     final ExpressionTree exprTree = PartFilterExprUtil.makeExpressionTree(expressionProxy, expr);
     final AtomicBoolean hasUnknownPartitions = new AtomicBoolean(false);
-    result.addAll(new GetListHelper<Partition>(dbName, tblName, allowSql, allowJdo) {
+    result.addAll(new GetListHelper<Partition>(catName, dbName, tblName, allowSql, allowJdo) {
       @Override
       protected List<Partition> getSqlResult(GetHelper<List<Partition>> ctx) throws MetaException {
         // If we have some sort of expression tree, try SQL filter pushdown.
@@ -2999,7 +3197,7 @@ public class ObjectStore implements RawStore, Configurable {
         List<String> partNames = new LinkedList<>();
         hasUnknownPartitions.set(getPartitionNamesPrunedByExprNoTxn(
             ctx.getTable(), expr, defaultPartitionName, maxParts, partNames));
-        return directSql.getPartitionsViaSqlFilter(dbName, tblName, partNames);
+        return directSql.getPartitionsViaSqlFilter(catName, dbName, tblName, partNames);
       }
 
       @Override
@@ -3015,7 +3213,7 @@ public class ObjectStore implements RawStore, Configurable {
           List<String> partNames = new ArrayList<>();
           hasUnknownPartitions.set(getPartitionNamesPrunedByExprNoTxn(
               ctx.getTable(), expr, defaultPartitionName, maxParts, partNames));
-          result = getPartitionsViaOrmFilter(dbName, tblName, partNames);
+          result = getPartitionsViaOrmFilter(catName, dbName, tblName, partNames);
         }
         return result;
       }
@@ -3036,7 +3234,7 @@ public class ObjectStore implements RawStore, Configurable {
    */
   private boolean getPartitionNamesPrunedByExprNoTxn(Table table, byte[] expr,
       String defaultPartName, short maxParts, List<String> result) throws MetaException {
-    result.addAll(getPartitionNamesNoTxn(
+    result.addAll(getPartitionNamesNoTxn(table.getCatName(),
         table.getDbName(), table.getTableName(), maxParts));
     if (defaultPartName == null || defaultPartName.isEmpty()) {
       defaultPartName = MetastoreConf.getVar(getConf(), ConfVars.DEFAULTPARTITIONNAME);
@@ -3058,7 +3256,7 @@ public class ObjectStore implements RawStore, Configurable {
       short maxParts, boolean isValidatedFilter) throws MetaException {
     Map<String, Object> params = new HashMap<>();
     String jdoFilter =
-        makeQueryFilterString(table.getDbName(), table, tree, params, isValidatedFilter);
+        makeQueryFilterString(table.getCatName(), table.getDbName(), table, tree, params, isValidatedFilter);
     if (jdoFilter == null) {
       assert !isValidatedFilter;
       return null;
@@ -3085,7 +3283,8 @@ public class ObjectStore implements RawStore, Configurable {
   private Integer getNumPartitionsViaOrmFilter(Table table, ExpressionTree tree, boolean isValidatedFilter)
     throws MetaException {
     Map<String, Object> params = new HashMap<>();
-    String jdoFilter = makeQueryFilterString(table.getDbName(), table, tree, params, isValidatedFilter);
+    String jdoFilter = makeQueryFilterString(table.getCatName(), table.getDbName(), table, tree,
+        params, isValidatedFilter);
     if (jdoFilter == null) {
       assert !isValidatedFilter;
       return null;
@@ -3110,29 +3309,29 @@ public class ObjectStore implements RawStore, Configurable {
    * @param partNames Partition names to get the objects for.
    * @return Resulting partitions.
    */
-  private List<Partition> getPartitionsViaOrmFilter(
+  private List<Partition> getPartitionsViaOrmFilter(String catName,
       String dbName, String tblName, List<String> partNames) throws MetaException {
     if (partNames.isEmpty()) {
       return new ArrayList<>();
     }
     ObjectPair<Query, Map<String, String>> queryWithParams =
-        getPartQueryWithParams(dbName, tblName, partNames);
+        getPartQueryWithParams(catName, dbName, tblName, partNames);
     Query query = queryWithParams.getFirst();
     query.setResultClass(MPartition.class);
     query.setClass(MPartition.class);
     query.setOrdering("partitionName ascending");
     @SuppressWarnings("unchecked")
     List<MPartition> mparts = (List<MPartition>)query.executeWithMap(queryWithParams.getSecond());
-    List<Partition> partitions = convertToParts(dbName, tblName, mparts);
+    List<Partition> partitions = convertToParts(catName, dbName, tblName, mparts);
     if (query != null) {
       query.closeAll();
     }
     return partitions;
   }
 
-  private void dropPartitionsNoTxn(String dbName, String tblName, List<String> partNames) {
+  private void dropPartitionsNoTxn(String catName, String dbName, String tblName, List<String> partNames) {
     ObjectPair<Query, Map<String, String>> queryWithParams =
-        getPartQueryWithParams(dbName, tblName, partNames);
+        getPartQueryWithParams(catName, dbName, tblName, partNames);
     Query query = queryWithParams.getFirst();
     query.setClass(MPartition.class);
     long deleted = query.deletePersistentAll(queryWithParams.getSecond());
@@ -3147,9 +3346,9 @@ public class ObjectStore implements RawStore, Configurable {
    * they are referenced by other SDs.
    */
   private HashSet<MColumnDescriptor> detachCdsFromSdsNoTxn(
-      String dbName, String tblName, List<String> partNames) {
+      String catName, String dbName, String tblName, List<String> partNames) {
     ObjectPair<Query, Map<String, String>> queryWithParams =
-        getPartQueryWithParams(dbName, tblName, partNames);
+        getPartQueryWithParams(catName, dbName, tblName, partNames);
     Query query = queryWithParams.getFirst();
     query.setClass(MPartition.class);
     query.setResult("sd");
@@ -3169,9 +3368,10 @@ public class ObjectStore implements RawStore, Configurable {
     return candidateCds;
   }
 
-  private ObjectPair<Query, Map<String, String>> getPartQueryWithParams(String dbName,
-      String tblName, List<String> partNames) {
-    StringBuilder sb = new StringBuilder("table.tableName == t1 && table.database.name == t2 && (");
+  private ObjectPair<Query, Map<String, String>> getPartQueryWithParams(
+      String catName, String dbName, String tblName, List<String> partNames) {
+    StringBuilder sb = new StringBuilder("table.tableName == t1 && table.database.name == t2 &&" +
+        " table.database.catalogName == t3 && (");
     int n = 0;
     Map<String, String> params = new HashMap<>();
     for (Iterator<String> itr = partNames.iterator(); itr.hasNext();) {
@@ -3189,14 +3389,15 @@ public class ObjectStore implements RawStore, Configurable {
     LOG.debug(" JDOQL filter is {}", sb);
     params.put("t1", normalizeIdentifier(tblName));
     params.put("t2", normalizeIdentifier(dbName));
+    params.put("t3", normalizeIdentifier(catName));
     query.declareParameters(makeParameterDeclarationString(params));
     return new ObjectPair<>(query, params);
   }
 
   @Override
-  public List<Partition> getPartitionsByFilter(String dbName, String tblName,
+  public List<Partition> getPartitionsByFilter(String catName, String dbName, String tblName,
       String filter, short maxParts) throws MetaException, NoSuchObjectException {
-    return getPartitionsByFilterInternal(dbName, tblName, filter, maxParts, true, true);
+    return getPartitionsByFilterInternal(catName, dbName, tblName, filter, maxParts, true, true);
   }
 
   /** Helper class for getting stuff w/transaction, direct SQL, perf logging, etc. */
@@ -3206,14 +3407,15 @@ public class ObjectStore implements RawStore, Configurable {
     private boolean doUseDirectSql;
     private long start;
     private Table table;
-    protected final String dbName, tblName;
+    protected final String catName, dbName, tblName;
     private boolean success = false;
     protected T results = null;
 
-    public GetHelper(String dbName, String tblName, boolean allowSql, boolean allowJdo)
-        throws MetaException {
+    public GetHelper(String catalogName, String dbName, String tblName,
+                     boolean allowSql, boolean allowJdo) throws MetaException {
       assert allowSql || allowJdo;
       this.allowJdo = allowJdo;
+      this.catName = normalizeIdentifier(catalogName);
       this.dbName = normalizeIdentifier(dbName);
       if (tblName != null){
         this.tblName = normalizeIdentifier(tblName);
@@ -3284,7 +3486,7 @@ public class ObjectStore implements RawStore, Configurable {
       start = doTrace ? System.nanoTime() : 0;
       openTransaction();
       if (initTable && (tblName != null)) {
-        table = ensureGetTable(dbName, tblName);
+        table = ensureGetTable(catName, dbName, tblName);
       }
       doUseDirectSql = doUseDirectSql && canUseDirectSql(this);
     }
@@ -3324,7 +3526,7 @@ public class ObjectStore implements RawStore, Configurable {
         start = doTrace ? System.nanoTime() : 0;
         openTransaction();
         if (table != null) {
-          table = ensureGetTable(dbName, tblName);
+          table = ensureGetTable(catName, dbName, tblName);
         }
       } else {
         start = doTrace ? System.nanoTime() : 0;
@@ -3393,9 +3595,9 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   private abstract class GetListHelper<T> extends GetHelper<List<T>> {
-    public GetListHelper(
-        String dbName, String tblName, boolean allowSql, boolean allowJdo) throws MetaException {
-      super(dbName, tblName, allowSql, allowJdo);
+    public GetListHelper(String catName, String dbName, String tblName, boolean allowSql,
+                         boolean allowJdo) throws MetaException {
+      super(catName, dbName, tblName, allowSql, allowJdo);
     }
 
     @Override
@@ -3413,9 +3615,9 @@ public class ObjectStore implements RawStore, Configurable {
      * @param allowJdo Whether or not we allow ORM to perform this query.
      * @throws MetaException
      */
-    public GetDbHelper(
-        String dbName,boolean allowSql, boolean allowJdo) throws MetaException {
-      super(dbName,null,allowSql,allowJdo);
+    public GetDbHelper(String catalogName, String dbName,boolean allowSql, boolean allowJdo)
+        throws MetaException {
+      super(catalogName, dbName,null,allowSql,allowJdo);
     }
 
     @Override
@@ -3425,9 +3627,9 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   private abstract class GetStatHelper extends GetHelper<ColumnStatistics> {
-    public GetStatHelper(
-        String dbName, String tblName, boolean allowSql, boolean allowJdo) throws MetaException {
-      super(dbName, tblName, allowSql, allowJdo);
+    public GetStatHelper(String catalogName, String dbName, String tblName, boolean allowSql,
+                         boolean allowJdo) throws MetaException {
+      super(catalogName, dbName, tblName, allowSql, allowJdo);
     }
 
     @Override
@@ -3437,12 +3639,12 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @Override
-  public int getNumPartitionsByFilter(String dbName, String tblName,
+  public int getNumPartitionsByFilter(String catName, String dbName, String tblName,
                                       String filter) throws MetaException, NoSuchObjectException {
     final ExpressionTree exprTree = org.apache.commons.lang

<TRUNCATED>

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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventsCountRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventsCountRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventsCountRequest.java
index 4855575..a4a5218 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventsCountRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventsCountRequest.java
@@ -40,6 +40,7 @@ import org.slf4j.LoggerFactory;
 
   private static final org.apache.thrift.protocol.TField FROM_EVENT_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("fromEventId", org.apache.thrift.protocol.TType.I64, (short)1);
   private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbName", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)3);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -49,11 +50,13 @@ import org.slf4j.LoggerFactory;
 
   private long fromEventId; // required
   private String dbName; // required
+  private String catName; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
     FROM_EVENT_ID((short)1, "fromEventId"),
-    DB_NAME((short)2, "dbName");
+    DB_NAME((short)2, "dbName"),
+    CAT_NAME((short)3, "catName");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -72,6 +75,8 @@ import org.slf4j.LoggerFactory;
           return FROM_EVENT_ID;
         case 2: // DB_NAME
           return DB_NAME;
+        case 3: // CAT_NAME
+          return CAT_NAME;
         default:
           return null;
       }
@@ -114,6 +119,7 @@ import org.slf4j.LoggerFactory;
   // isset id assignments
   private static final int __FROMEVENTID_ISSET_ID = 0;
   private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.CAT_NAME};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -121,6 +127,8 @@ import org.slf4j.LoggerFactory;
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
     tmpMap.put(_Fields.DB_NAME, new org.apache.thrift.meta_data.FieldMetaData("dbName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(NotificationEventsCountRequest.class, metaDataMap);
   }
@@ -147,6 +155,9 @@ import org.slf4j.LoggerFactory;
     if (other.isSetDbName()) {
       this.dbName = other.dbName;
     }
+    if (other.isSetCatName()) {
+      this.catName = other.catName;
+    }
   }
 
   public NotificationEventsCountRequest deepCopy() {
@@ -158,6 +169,7 @@ import org.slf4j.LoggerFactory;
     setFromEventIdIsSet(false);
     this.fromEventId = 0;
     this.dbName = null;
+    this.catName = null;
   }
 
   public long getFromEventId() {
@@ -205,6 +217,29 @@ import org.slf4j.LoggerFactory;
     }
   }
 
+  public String getCatName() {
+    return this.catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
+  public void unsetCatName() {
+    this.catName = null;
+  }
+
+  /** Returns true if field catName is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatName() {
+    return this.catName != null;
+  }
+
+  public void setCatNameIsSet(boolean value) {
+    if (!value) {
+      this.catName = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case FROM_EVENT_ID:
@@ -223,6 +258,14 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case CAT_NAME:
+      if (value == null) {
+        unsetCatName();
+      } else {
+        setCatName((String)value);
+      }
+      break;
+
     }
   }
 
@@ -234,6 +277,9 @@ import org.slf4j.LoggerFactory;
     case DB_NAME:
       return getDbName();
 
+    case CAT_NAME:
+      return getCatName();
+
     }
     throw new IllegalStateException();
   }
@@ -249,6 +295,8 @@ import org.slf4j.LoggerFactory;
       return isSetFromEventId();
     case DB_NAME:
       return isSetDbName();
+    case CAT_NAME:
+      return isSetCatName();
     }
     throw new IllegalStateException();
   }
@@ -284,6 +332,15 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_catName = true && this.isSetCatName();
+    boolean that_present_catName = true && that.isSetCatName();
+    if (this_present_catName || that_present_catName) {
+      if (!(this_present_catName && that_present_catName))
+        return false;
+      if (!this.catName.equals(that.catName))
+        return false;
+    }
+
     return true;
   }
 
@@ -301,6 +358,11 @@ import org.slf4j.LoggerFactory;
     if (present_dbName)
       list.add(dbName);
 
+    boolean present_catName = true && (isSetCatName());
+    list.add(present_catName);
+    if (present_catName)
+      list.add(catName);
+
     return list.hashCode();
   }
 
@@ -332,6 +394,16 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetCatName()).compareTo(other.isSetCatName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catName, other.catName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -363,6 +435,16 @@ import org.slf4j.LoggerFactory;
       sb.append(this.dbName);
     }
     first = false;
+    if (isSetCatName()) {
+      if (!first) sb.append(", ");
+      sb.append("catName:");
+      if (this.catName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.catName);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -432,6 +514,14 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 3: // CAT_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.catName = iprot.readString();
+              struct.setCatNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -453,6 +543,13 @@ import org.slf4j.LoggerFactory;
         oprot.writeString(struct.dbName);
         oprot.writeFieldEnd();
       }
+      if (struct.catName != null) {
+        if (struct.isSetCatName()) {
+          oprot.writeFieldBegin(CAT_NAME_FIELD_DESC);
+          oprot.writeString(struct.catName);
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -472,6 +569,14 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       oprot.writeI64(struct.fromEventId);
       oprot.writeString(struct.dbName);
+      BitSet optionals = new BitSet();
+      if (struct.isSetCatName()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.isSetCatName()) {
+        oprot.writeString(struct.catName);
+      }
     }
 
     @Override
@@ -481,6 +586,11 @@ import org.slf4j.LoggerFactory;
       struct.setFromEventIdIsSet(true);
       struct.dbName = iprot.readString();
       struct.setDbNameIsSet(true);
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.catName = iprot.readString();
+        struct.setCatNameIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnsResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnsResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnsResponse.java
index 8f08ed9..7adac3a 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnsResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnsResponse.java
@@ -351,13 +351,13 @@ import org.slf4j.LoggerFactory;
           case 1: // TXN_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list562 = iprot.readListBegin();
-                struct.txn_ids = new ArrayList<Long>(_list562.size);
-                long _elem563;
-                for (int _i564 = 0; _i564 < _list562.size; ++_i564)
+                org.apache.thrift.protocol.TList _list570 = iprot.readListBegin();
+                struct.txn_ids = new ArrayList<Long>(_list570.size);
+                long _elem571;
+                for (int _i572 = 0; _i572 < _list570.size; ++_i572)
                 {
-                  _elem563 = iprot.readI64();
-                  struct.txn_ids.add(_elem563);
+                  _elem571 = iprot.readI64();
+                  struct.txn_ids.add(_elem571);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TXN_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.txn_ids.size()));
-          for (long _iter565 : struct.txn_ids)
+          for (long _iter573 : struct.txn_ids)
           {
-            oprot.writeI64(_iter565);
+            oprot.writeI64(_iter573);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.txn_ids.size());
-        for (long _iter566 : struct.txn_ids)
+        for (long _iter574 : struct.txn_ids)
         {
-          oprot.writeI64(_iter566);
+          oprot.writeI64(_iter574);
         }
       }
     }
@@ -421,13 +421,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, OpenTxnsResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list567 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.txn_ids = new ArrayList<Long>(_list567.size);
-        long _elem568;
-        for (int _i569 = 0; _i569 < _list567.size; ++_i569)
+        org.apache.thrift.protocol.TList _list575 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.txn_ids = new ArrayList<Long>(_list575.size);
+        long _elem576;
+        for (int _i577 = 0; _i577 < _list575.size; ++_i577)
         {
-          _elem568 = iprot.readI64();
-          struct.txn_ids.add(_elem568);
+          _elem576 = iprot.readI64();
+          struct.txn_ids.add(_elem576);
         }
       }
       struct.setTxn_idsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Partition.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Partition.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Partition.java
index 3a13753..c58e1cb 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Partition.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Partition.java
@@ -46,6 +46,7 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField SD_FIELD_DESC = new org.apache.thrift.protocol.TField("sd", org.apache.thrift.protocol.TType.STRUCT, (short)6);
   private static final org.apache.thrift.protocol.TField PARAMETERS_FIELD_DESC = new org.apache.thrift.protocol.TField("parameters", org.apache.thrift.protocol.TType.MAP, (short)7);
   private static final org.apache.thrift.protocol.TField PRIVILEGES_FIELD_DESC = new org.apache.thrift.protocol.TField("privileges", org.apache.thrift.protocol.TType.STRUCT, (short)8);
+  private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)9);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -61,6 +62,7 @@ import org.slf4j.LoggerFactory;
   private StorageDescriptor sd; // required
   private Map<String,String> parameters; // required
   private PrincipalPrivilegeSet privileges; // optional
+  private String catName; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -71,7 +73,8 @@ import org.slf4j.LoggerFactory;
     LAST_ACCESS_TIME((short)5, "lastAccessTime"),
     SD((short)6, "sd"),
     PARAMETERS((short)7, "parameters"),
-    PRIVILEGES((short)8, "privileges");
+    PRIVILEGES((short)8, "privileges"),
+    CAT_NAME((short)9, "catName");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -102,6 +105,8 @@ import org.slf4j.LoggerFactory;
           return PARAMETERS;
         case 8: // PRIVILEGES
           return PRIVILEGES;
+        case 9: // CAT_NAME
+          return CAT_NAME;
         default:
           return null;
       }
@@ -145,7 +150,7 @@ import org.slf4j.LoggerFactory;
   private static final int __CREATETIME_ISSET_ID = 0;
   private static final int __LASTACCESSTIME_ISSET_ID = 1;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.PRIVILEGES};
+  private static final _Fields optionals[] = {_Fields.PRIVILEGES,_Fields.CAT_NAME};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -168,6 +173,8 @@ import org.slf4j.LoggerFactory;
             new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
     tmpMap.put(_Fields.PRIVILEGES, new org.apache.thrift.meta_data.FieldMetaData("privileges", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, PrincipalPrivilegeSet.class)));
+    tmpMap.put(_Fields.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(Partition.class, metaDataMap);
   }
@@ -223,6 +230,9 @@ import org.slf4j.LoggerFactory;
     if (other.isSetPrivileges()) {
       this.privileges = new PrincipalPrivilegeSet(other.privileges);
     }
+    if (other.isSetCatName()) {
+      this.catName = other.catName;
+    }
   }
 
   public Partition deepCopy() {
@@ -241,6 +251,7 @@ import org.slf4j.LoggerFactory;
     this.sd = null;
     this.parameters = null;
     this.privileges = null;
+    this.catName = null;
   }
 
   public int getValuesSize() {
@@ -451,6 +462,29 @@ import org.slf4j.LoggerFactory;
     }
   }
 
+  public String getCatName() {
+    return this.catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
+  public void unsetCatName() {
+    this.catName = null;
+  }
+
+  /** Returns true if field catName is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatName() {
+    return this.catName != null;
+  }
+
+  public void setCatNameIsSet(boolean value) {
+    if (!value) {
+      this.catName = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case VALUES:
@@ -517,6 +551,14 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case CAT_NAME:
+      if (value == null) {
+        unsetCatName();
+      } else {
+        setCatName((String)value);
+      }
+      break;
+
     }
   }
 
@@ -546,6 +588,9 @@ import org.slf4j.LoggerFactory;
     case PRIVILEGES:
       return getPrivileges();
 
+    case CAT_NAME:
+      return getCatName();
+
     }
     throw new IllegalStateException();
   }
@@ -573,6 +618,8 @@ import org.slf4j.LoggerFactory;
       return isSetParameters();
     case PRIVILEGES:
       return isSetPrivileges();
+    case CAT_NAME:
+      return isSetCatName();
     }
     throw new IllegalStateException();
   }
@@ -662,6 +709,15 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_catName = true && this.isSetCatName();
+    boolean that_present_catName = true && that.isSetCatName();
+    if (this_present_catName || that_present_catName) {
+      if (!(this_present_catName && that_present_catName))
+        return false;
+      if (!this.catName.equals(that.catName))
+        return false;
+    }
+
     return true;
   }
 
@@ -709,6 +765,11 @@ import org.slf4j.LoggerFactory;
     if (present_privileges)
       list.add(privileges);
 
+    boolean present_catName = true && (isSetCatName());
+    list.add(present_catName);
+    if (present_catName)
+      list.add(catName);
+
     return list.hashCode();
   }
 
@@ -800,6 +861,16 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetCatName()).compareTo(other.isSetCatName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catName, other.catName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -877,6 +948,16 @@ import org.slf4j.LoggerFactory;
       }
       first = false;
     }
+    if (isSetCatName()) {
+      if (!first) sb.append(", ");
+      sb.append("catName:");
+      if (this.catName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.catName);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -931,13 +1012,13 @@ import org.slf4j.LoggerFactory;
           case 1: // VALUES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list208 = iprot.readListBegin();
-                struct.values = new ArrayList<String>(_list208.size);
-                String _elem209;
-                for (int _i210 = 0; _i210 < _list208.size; ++_i210)
+                org.apache.thrift.protocol.TList _list216 = iprot.readListBegin();
+                struct.values = new ArrayList<String>(_list216.size);
+                String _elem217;
+                for (int _i218 = 0; _i218 < _list216.size; ++_i218)
                 {
-                  _elem209 = iprot.readString();
-                  struct.values.add(_elem209);
+                  _elem217 = iprot.readString();
+                  struct.values.add(_elem217);
                 }
                 iprot.readListEnd();
               }
@@ -990,15 +1071,15 @@ import org.slf4j.LoggerFactory;
           case 7: // PARAMETERS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map211 = iprot.readMapBegin();
-                struct.parameters = new HashMap<String,String>(2*_map211.size);
-                String _key212;
-                String _val213;
-                for (int _i214 = 0; _i214 < _map211.size; ++_i214)
+                org.apache.thrift.protocol.TMap _map219 = iprot.readMapBegin();
+                struct.parameters = new HashMap<String,String>(2*_map219.size);
+                String _key220;
+                String _val221;
+                for (int _i222 = 0; _i222 < _map219.size; ++_i222)
                 {
-                  _key212 = iprot.readString();
-                  _val213 = iprot.readString();
-                  struct.parameters.put(_key212, _val213);
+                  _key220 = iprot.readString();
+                  _val221 = iprot.readString();
+                  struct.parameters.put(_key220, _val221);
                 }
                 iprot.readMapEnd();
               }
@@ -1016,6 +1097,14 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 9: // CAT_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.catName = iprot.readString();
+              struct.setCatNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -1033,9 +1122,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(VALUES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.values.size()));
-          for (String _iter215 : struct.values)
+          for (String _iter223 : struct.values)
           {
-            oprot.writeString(_iter215);
+            oprot.writeString(_iter223);
           }
           oprot.writeListEnd();
         }
@@ -1066,10 +1155,10 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PARAMETERS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.parameters.size()));
-          for (Map.Entry<String, String> _iter216 : struct.parameters.entrySet())
+          for (Map.Entry<String, String> _iter224 : struct.parameters.entrySet())
           {
-            oprot.writeString(_iter216.getKey());
-            oprot.writeString(_iter216.getValue());
+            oprot.writeString(_iter224.getKey());
+            oprot.writeString(_iter224.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -1082,6 +1171,13 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldEnd();
         }
       }
+      if (struct.catName != null) {
+        if (struct.isSetCatName()) {
+          oprot.writeFieldBegin(CAT_NAME_FIELD_DESC);
+          oprot.writeString(struct.catName);
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -1124,13 +1220,16 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetPrivileges()) {
         optionals.set(7);
       }
-      oprot.writeBitSet(optionals, 8);
+      if (struct.isSetCatName()) {
+        optionals.set(8);
+      }
+      oprot.writeBitSet(optionals, 9);
       if (struct.isSetValues()) {
         {
           oprot.writeI32(struct.values.size());
-          for (String _iter217 : struct.values)
+          for (String _iter225 : struct.values)
           {
-            oprot.writeString(_iter217);
+            oprot.writeString(_iter225);
           }
         }
       }
@@ -1152,31 +1251,34 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetParameters()) {
         {
           oprot.writeI32(struct.parameters.size());
-          for (Map.Entry<String, String> _iter218 : struct.parameters.entrySet())
+          for (Map.Entry<String, String> _iter226 : struct.parameters.entrySet())
           {
-            oprot.writeString(_iter218.getKey());
-            oprot.writeString(_iter218.getValue());
+            oprot.writeString(_iter226.getKey());
+            oprot.writeString(_iter226.getValue());
           }
         }
       }
       if (struct.isSetPrivileges()) {
         struct.privileges.write(oprot);
       }
+      if (struct.isSetCatName()) {
+        oprot.writeString(struct.catName);
+      }
     }
 
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, Partition struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
-      BitSet incoming = iprot.readBitSet(8);
+      BitSet incoming = iprot.readBitSet(9);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list219 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.values = new ArrayList<String>(_list219.size);
-          String _elem220;
-          for (int _i221 = 0; _i221 < _list219.size; ++_i221)
+          org.apache.thrift.protocol.TList _list227 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.values = new ArrayList<String>(_list227.size);
+          String _elem228;
+          for (int _i229 = 0; _i229 < _list227.size; ++_i229)
           {
-            _elem220 = iprot.readString();
-            struct.values.add(_elem220);
+            _elem228 = iprot.readString();
+            struct.values.add(_elem228);
           }
         }
         struct.setValuesIsSet(true);
@@ -1204,15 +1306,15 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(6)) {
         {
-          org.apache.thrift.protocol.TMap _map222 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.parameters = new HashMap<String,String>(2*_map222.size);
-          String _key223;
-          String _val224;
-          for (int _i225 = 0; _i225 < _map222.size; ++_i225)
+          org.apache.thrift.protocol.TMap _map230 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.parameters = new HashMap<String,String>(2*_map230.size);
+          String _key231;
+          String _val232;
+          for (int _i233 = 0; _i233 < _map230.size; ++_i233)
           {
-            _key223 = iprot.readString();
-            _val224 = iprot.readString();
-            struct.parameters.put(_key223, _val224);
+            _key231 = iprot.readString();
+            _val232 = iprot.readString();
+            struct.parameters.put(_key231, _val232);
           }
         }
         struct.parameters = org.apache.hadoop.hive.metastore.utils.StringUtils.intern(struct.parameters); struct.setParametersIsSet(true);
@@ -1222,6 +1324,10 @@ import org.slf4j.LoggerFactory;
         struct.privileges.read(iprot);
         struct.setPrivilegesIsSet(true);
       }
+      if (incoming.get(8)) {
+        struct.catName = iprot.readString();
+        struct.setCatNameIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionListComposingSpec.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionListComposingSpec.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionListComposingSpec.java
index 186eb23..1f32e38 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionListComposingSpec.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionListComposingSpec.java
@@ -350,14 +350,14 @@ import org.slf4j.LoggerFactory;
           case 1: // PARTITIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list252 = iprot.readListBegin();
-                struct.partitions = new ArrayList<Partition>(_list252.size);
-                Partition _elem253;
-                for (int _i254 = 0; _i254 < _list252.size; ++_i254)
+                org.apache.thrift.protocol.TList _list260 = iprot.readListBegin();
+                struct.partitions = new ArrayList<Partition>(_list260.size);
+                Partition _elem261;
+                for (int _i262 = 0; _i262 < _list260.size; ++_i262)
                 {
-                  _elem253 = new Partition();
-                  _elem253.read(iprot);
-                  struct.partitions.add(_elem253);
+                  _elem261 = new Partition();
+                  _elem261.read(iprot);
+                  struct.partitions.add(_elem261);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PARTITIONS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitions.size()));
-          for (Partition _iter255 : struct.partitions)
+          for (Partition _iter263 : struct.partitions)
           {
-            _iter255.write(oprot);
+            _iter263.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -416,9 +416,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetPartitions()) {
         {
           oprot.writeI32(struct.partitions.size());
-          for (Partition _iter256 : struct.partitions)
+          for (Partition _iter264 : struct.partitions)
           {
-            _iter256.write(oprot);
+            _iter264.write(oprot);
           }
         }
       }
@@ -430,14 +430,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list257 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.partitions = new ArrayList<Partition>(_list257.size);
-          Partition _elem258;
-          for (int _i259 = 0; _i259 < _list257.size; ++_i259)
+          org.apache.thrift.protocol.TList _list265 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.partitions = new ArrayList<Partition>(_list265.size);
+          Partition _elem266;
+          for (int _i267 = 0; _i267 < _list265.size; ++_i267)
           {
-            _elem258 = new Partition();
-            _elem258.read(iprot);
-            struct.partitions.add(_elem258);
+            _elem266 = new Partition();
+            _elem266.read(iprot);
+            struct.partitions.add(_elem266);
           }
         }
         struct.setPartitionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpec.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpec.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpec.java
index fc91ce3..247fdaa 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpec.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpec.java
@@ -43,6 +43,7 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField ROOT_PATH_FIELD_DESC = new org.apache.thrift.protocol.TField("rootPath", org.apache.thrift.protocol.TType.STRING, (short)3);
   private static final org.apache.thrift.protocol.TField SHARED_SDPARTITION_SPEC_FIELD_DESC = new org.apache.thrift.protocol.TField("sharedSDPartitionSpec", org.apache.thrift.protocol.TType.STRUCT, (short)4);
   private static final org.apache.thrift.protocol.TField PARTITION_LIST_FIELD_DESC = new org.apache.thrift.protocol.TField("partitionList", org.apache.thrift.protocol.TType.STRUCT, (short)5);
+  private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)6);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -55,6 +56,7 @@ import org.slf4j.LoggerFactory;
   private String rootPath; // required
   private PartitionSpecWithSharedSD sharedSDPartitionSpec; // optional
   private PartitionListComposingSpec partitionList; // optional
+  private String catName; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -62,7 +64,8 @@ import org.slf4j.LoggerFactory;
     TABLE_NAME((short)2, "tableName"),
     ROOT_PATH((short)3, "rootPath"),
     SHARED_SDPARTITION_SPEC((short)4, "sharedSDPartitionSpec"),
-    PARTITION_LIST((short)5, "partitionList");
+    PARTITION_LIST((short)5, "partitionList"),
+    CAT_NAME((short)6, "catName");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -87,6 +90,8 @@ import org.slf4j.LoggerFactory;
           return SHARED_SDPARTITION_SPEC;
         case 5: // PARTITION_LIST
           return PARTITION_LIST;
+        case 6: // CAT_NAME
+          return CAT_NAME;
         default:
           return null;
       }
@@ -127,7 +132,7 @@ import org.slf4j.LoggerFactory;
   }
 
   // isset id assignments
-  private static final _Fields optionals[] = {_Fields.SHARED_SDPARTITION_SPEC,_Fields.PARTITION_LIST};
+  private static final _Fields optionals[] = {_Fields.SHARED_SDPARTITION_SPEC,_Fields.PARTITION_LIST,_Fields.CAT_NAME};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -141,6 +146,8 @@ import org.slf4j.LoggerFactory;
         new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, PartitionSpecWithSharedSD.class)));
     tmpMap.put(_Fields.PARTITION_LIST, new org.apache.thrift.meta_data.FieldMetaData("partitionList", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, PartitionListComposingSpec.class)));
+    tmpMap.put(_Fields.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(PartitionSpec.class, metaDataMap);
   }
@@ -178,6 +185,9 @@ import org.slf4j.LoggerFactory;
     if (other.isSetPartitionList()) {
       this.partitionList = new PartitionListComposingSpec(other.partitionList);
     }
+    if (other.isSetCatName()) {
+      this.catName = other.catName;
+    }
   }
 
   public PartitionSpec deepCopy() {
@@ -191,6 +201,7 @@ import org.slf4j.LoggerFactory;
     this.rootPath = null;
     this.sharedSDPartitionSpec = null;
     this.partitionList = null;
+    this.catName = null;
   }
 
   public String getDbName() {
@@ -308,6 +319,29 @@ import org.slf4j.LoggerFactory;
     }
   }
 
+  public String getCatName() {
+    return this.catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
+  public void unsetCatName() {
+    this.catName = null;
+  }
+
+  /** Returns true if field catName is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatName() {
+    return this.catName != null;
+  }
+
+  public void setCatNameIsSet(boolean value) {
+    if (!value) {
+      this.catName = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case DB_NAME:
@@ -350,6 +384,14 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case CAT_NAME:
+      if (value == null) {
+        unsetCatName();
+      } else {
+        setCatName((String)value);
+      }
+      break;
+
     }
   }
 
@@ -370,6 +412,9 @@ import org.slf4j.LoggerFactory;
     case PARTITION_LIST:
       return getPartitionList();
 
+    case CAT_NAME:
+      return getCatName();
+
     }
     throw new IllegalStateException();
   }
@@ -391,6 +436,8 @@ import org.slf4j.LoggerFactory;
       return isSetSharedSDPartitionSpec();
     case PARTITION_LIST:
       return isSetPartitionList();
+    case CAT_NAME:
+      return isSetCatName();
     }
     throw new IllegalStateException();
   }
@@ -453,6 +500,15 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_catName = true && this.isSetCatName();
+    boolean that_present_catName = true && that.isSetCatName();
+    if (this_present_catName || that_present_catName) {
+      if (!(this_present_catName && that_present_catName))
+        return false;
+      if (!this.catName.equals(that.catName))
+        return false;
+    }
+
     return true;
   }
 
@@ -485,6 +541,11 @@ import org.slf4j.LoggerFactory;
     if (present_partitionList)
       list.add(partitionList);
 
+    boolean present_catName = true && (isSetCatName());
+    list.add(present_catName);
+    if (present_catName)
+      list.add(catName);
+
     return list.hashCode();
   }
 
@@ -546,6 +607,16 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetCatName()).compareTo(other.isSetCatName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catName, other.catName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -609,6 +680,16 @@ import org.slf4j.LoggerFactory;
       }
       first = false;
     }
+    if (isSetCatName()) {
+      if (!first) sb.append(", ");
+      sb.append("catName:");
+      if (this.catName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.catName);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -700,6 +781,14 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 6: // CAT_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.catName = iprot.readString();
+              struct.setCatNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -742,6 +831,13 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldEnd();
         }
       }
+      if (struct.catName != null) {
+        if (struct.isSetCatName()) {
+          oprot.writeFieldBegin(CAT_NAME_FIELD_DESC);
+          oprot.writeString(struct.catName);
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -775,7 +871,10 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetPartitionList()) {
         optionals.set(4);
       }
-      oprot.writeBitSet(optionals, 5);
+      if (struct.isSetCatName()) {
+        optionals.set(5);
+      }
+      oprot.writeBitSet(optionals, 6);
       if (struct.isSetDbName()) {
         oprot.writeString(struct.dbName);
       }
@@ -791,12 +890,15 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetPartitionList()) {
         struct.partitionList.write(oprot);
       }
+      if (struct.isSetCatName()) {
+        oprot.writeString(struct.catName);
+      }
     }
 
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, PartitionSpec struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
-      BitSet incoming = iprot.readBitSet(5);
+      BitSet incoming = iprot.readBitSet(6);
       if (incoming.get(0)) {
         struct.dbName = iprot.readString();
         struct.setDbNameIsSet(true);
@@ -819,6 +921,10 @@ import org.slf4j.LoggerFactory;
         struct.partitionList.read(iprot);
         struct.setPartitionListIsSet(true);
       }
+      if (incoming.get(5)) {
+        struct.catName = iprot.readString();
+        struct.setCatNameIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpecWithSharedSD.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpecWithSharedSD.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpecWithSharedSD.java
index e7ab52a..a450cd4 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpecWithSharedSD.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpecWithSharedSD.java
@@ -434,14 +434,14 @@ import org.slf4j.LoggerFactory;
           case 1: // PARTITIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list244 = iprot.readListBegin();
-                struct.partitions = new ArrayList<PartitionWithoutSD>(_list244.size);
-                PartitionWithoutSD _elem245;
-                for (int _i246 = 0; _i246 < _list244.size; ++_i246)
+                org.apache.thrift.protocol.TList _list252 = iprot.readListBegin();
+                struct.partitions = new ArrayList<PartitionWithoutSD>(_list252.size);
+                PartitionWithoutSD _elem253;
+                for (int _i254 = 0; _i254 < _list252.size; ++_i254)
                 {
-                  _elem245 = new PartitionWithoutSD();
-                  _elem245.read(iprot);
-                  struct.partitions.add(_elem245);
+                  _elem253 = new PartitionWithoutSD();
+                  _elem253.read(iprot);
+                  struct.partitions.add(_elem253);
                 }
                 iprot.readListEnd();
               }
@@ -476,9 +476,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PARTITIONS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitions.size()));
-          for (PartitionWithoutSD _iter247 : struct.partitions)
+          for (PartitionWithoutSD _iter255 : struct.partitions)
           {
-            _iter247.write(oprot);
+            _iter255.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -517,9 +517,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetPartitions()) {
         {
           oprot.writeI32(struct.partitions.size());
-          for (PartitionWithoutSD _iter248 : struct.partitions)
+          for (PartitionWithoutSD _iter256 : struct.partitions)
           {
-            _iter248.write(oprot);
+            _iter256.write(oprot);
           }
         }
       }
@@ -534,14 +534,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(2);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list249 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.partitions = new ArrayList<PartitionWithoutSD>(_list249.size);
-          PartitionWithoutSD _elem250;
-          for (int _i251 = 0; _i251 < _list249.size; ++_i251)
+          org.apache.thrift.protocol.TList _list257 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.partitions = new ArrayList<PartitionWithoutSD>(_list257.size);
+          PartitionWithoutSD _elem258;
+          for (int _i259 = 0; _i259 < _list257.size; ++_i259)
           {
-            _elem250 = new PartitionWithoutSD();
-            _elem250.read(iprot);
-            struct.partitions.add(_elem250);
+            _elem258 = new PartitionWithoutSD();
+            _elem258.read(iprot);
+            struct.partitions.add(_elem258);
           }
         }
         struct.setPartitionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesRequest.java
index 9cac668..c8707ca 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesRequest.java
@@ -46,6 +46,7 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField PARTITION_ORDER_FIELD_DESC = new org.apache.thrift.protocol.TField("partitionOrder", org.apache.thrift.protocol.TType.LIST, (short)6);
   private static final org.apache.thrift.protocol.TField ASCENDING_FIELD_DESC = new org.apache.thrift.protocol.TField("ascending", org.apache.thrift.protocol.TType.BOOL, (short)7);
   private static final org.apache.thrift.protocol.TField MAX_PARTS_FIELD_DESC = new org.apache.thrift.protocol.TField("maxParts", org.apache.thrift.protocol.TType.I64, (short)8);
+  private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)9);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -61,6 +62,7 @@ import org.slf4j.LoggerFactory;
   private List<FieldSchema> partitionOrder; // optional
   private boolean ascending; // optional
   private long maxParts; // optional
+  private String catName; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -71,7 +73,8 @@ import org.slf4j.LoggerFactory;
     FILTER((short)5, "filter"),
     PARTITION_ORDER((short)6, "partitionOrder"),
     ASCENDING((short)7, "ascending"),
-    MAX_PARTS((short)8, "maxParts");
+    MAX_PARTS((short)8, "maxParts"),
+    CAT_NAME((short)9, "catName");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -102,6 +105,8 @@ import org.slf4j.LoggerFactory;
           return ASCENDING;
         case 8: // MAX_PARTS
           return MAX_PARTS;
+        case 9: // CAT_NAME
+          return CAT_NAME;
         default:
           return null;
       }
@@ -146,7 +151,7 @@ import org.slf4j.LoggerFactory;
   private static final int __ASCENDING_ISSET_ID = 1;
   private static final int __MAXPARTS_ISSET_ID = 2;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.APPLY_DISTINCT,_Fields.FILTER,_Fields.PARTITION_ORDER,_Fields.ASCENDING,_Fields.MAX_PARTS};
+  private static final _Fields optionals[] = {_Fields.APPLY_DISTINCT,_Fields.FILTER,_Fields.PARTITION_ORDER,_Fields.ASCENDING,_Fields.MAX_PARTS,_Fields.CAT_NAME};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -168,6 +173,8 @@ import org.slf4j.LoggerFactory;
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
     tmpMap.put(_Fields.MAX_PARTS, new org.apache.thrift.meta_data.FieldMetaData("maxParts", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(PartitionValuesRequest.class, metaDataMap);
   }
@@ -223,6 +230,9 @@ import org.slf4j.LoggerFactory;
     }
     this.ascending = other.ascending;
     this.maxParts = other.maxParts;
+    if (other.isSetCatName()) {
+      this.catName = other.catName;
+    }
   }
 
   public PartitionValuesRequest deepCopy() {
@@ -242,6 +252,7 @@ import org.slf4j.LoggerFactory;
 
     this.maxParts = -1L;
 
+    this.catName = null;
   }
 
   public String getDbName() {
@@ -455,6 +466,29 @@ import org.slf4j.LoggerFactory;
     __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MAXPARTS_ISSET_ID, value);
   }
 
+  public String getCatName() {
+    return this.catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
+  public void unsetCatName() {
+    this.catName = null;
+  }
+
+  /** Returns true if field catName is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatName() {
+    return this.catName != null;
+  }
+
+  public void setCatNameIsSet(boolean value) {
+    if (!value) {
+      this.catName = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case DB_NAME:
@@ -521,6 +555,14 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case CAT_NAME:
+      if (value == null) {
+        unsetCatName();
+      } else {
+        setCatName((String)value);
+      }
+      break;
+
     }
   }
 
@@ -550,6 +592,9 @@ import org.slf4j.LoggerFactory;
     case MAX_PARTS:
       return getMaxParts();
 
+    case CAT_NAME:
+      return getCatName();
+
     }
     throw new IllegalStateException();
   }
@@ -577,6 +622,8 @@ import org.slf4j.LoggerFactory;
       return isSetAscending();
     case MAX_PARTS:
       return isSetMaxParts();
+    case CAT_NAME:
+      return isSetCatName();
     }
     throw new IllegalStateException();
   }
@@ -666,6 +713,15 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_catName = true && this.isSetCatName();
+    boolean that_present_catName = true && that.isSetCatName();
+    if (this_present_catName || that_present_catName) {
+      if (!(this_present_catName && that_present_catName))
+        return false;
+      if (!this.catName.equals(that.catName))
+        return false;
+    }
+
     return true;
   }
 
@@ -713,6 +769,11 @@ import org.slf4j.LoggerFactory;
     if (present_maxParts)
       list.add(maxParts);
 
+    boolean present_catName = true && (isSetCatName());
+    list.add(present_catName);
+    if (present_catName)
+      list.add(catName);
+
     return list.hashCode();
   }
 
@@ -804,6 +865,16 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetCatName()).compareTo(other.isSetCatName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catName, other.catName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -885,6 +956,16 @@ import org.slf4j.LoggerFactory;
       sb.append(this.maxParts);
       first = false;
     }
+    if (isSetCatName()) {
+      if (!first) sb.append(", ");
+      sb.append("catName:");
+      if (this.catName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.catName);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -961,14 +1042,14 @@ import org.slf4j.LoggerFactory;
           case 3: // PARTITION_KEYS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list506 = iprot.readListBegin();
-                struct.partitionKeys = new ArrayList<FieldSchema>(_list506.size);
-                FieldSchema _elem507;
-                for (int _i508 = 0; _i508 < _list506.size; ++_i508)
+                org.apache.thrift.protocol.TList _list514 = iprot.readListBegin();
+                struct.partitionKeys = new ArrayList<FieldSchema>(_list514.size);
+                FieldSchema _elem515;
+                for (int _i516 = 0; _i516 < _list514.size; ++_i516)
                 {
-                  _elem507 = new FieldSchema();
-                  _elem507.read(iprot);
-                  struct.partitionKeys.add(_elem507);
+                  _elem515 = new FieldSchema();
+                  _elem515.read(iprot);
+                  struct.partitionKeys.add(_elem515);
                 }
                 iprot.readListEnd();
               }
@@ -996,14 +1077,14 @@ import org.slf4j.LoggerFactory;
           case 6: // PARTITION_ORDER
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list509 = iprot.readListBegin();
-                struct.partitionOrder = new ArrayList<FieldSchema>(_list509.size);
-                FieldSchema _elem510;
-                for (int _i511 = 0; _i511 < _list509.size; ++_i511)
+                org.apache.thrift.protocol.TList _list517 = iprot.readListBegin();
+                struct.partitionOrder = new ArrayList<FieldSchema>(_list517.size);
+                FieldSchema _elem518;
+                for (int _i519 = 0; _i519 < _list517.size; ++_i519)
                 {
-                  _elem510 = new FieldSchema();
-                  _elem510.read(iprot);
-                  struct.partitionOrder.add(_elem510);
+                  _elem518 = new FieldSchema();
+                  _elem518.read(iprot);
+                  struct.partitionOrder.add(_elem518);
                 }
                 iprot.readListEnd();
               }
@@ -1028,6 +1109,14 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 9: // CAT_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.catName = iprot.readString();
+              struct.setCatNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -1055,9 +1144,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PARTITION_KEYS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitionKeys.size()));
-          for (FieldSchema _iter512 : struct.partitionKeys)
+          for (FieldSchema _iter520 : struct.partitionKeys)
           {
-            _iter512.write(oprot);
+            _iter520.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -1080,9 +1169,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PARTITION_ORDER_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitionOrder.size()));
-            for (FieldSchema _iter513 : struct.partitionOrder)
+            for (FieldSchema _iter521 : struct.partitionOrder)
             {
-              _iter513.write(oprot);
+              _iter521.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -1099,6 +1188,13 @@ import org.slf4j.LoggerFactory;
         oprot.writeI64(struct.maxParts);
         oprot.writeFieldEnd();
       }
+      if (struct.catName != null) {
+        if (struct.isSetCatName()) {
+          oprot.writeFieldBegin(CAT_NAME_FIELD_DESC);
+          oprot.writeString(struct.catName);
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -1120,9 +1216,9 @@ import org.slf4j.LoggerFactory;
       oprot.writeString(struct.tblName);
       {
         oprot.writeI32(struct.partitionKeys.size());
-        for (FieldSchema _iter514 : struct.partitionKeys)
+        for (FieldSchema _iter522 : struct.partitionKeys)
         {
-          _iter514.write(oprot);
+          _iter522.write(oprot);
         }
       }
       BitSet optionals = new BitSet();
@@ -1141,7 +1237,10 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetMaxParts()) {
         optionals.set(4);
       }
-      oprot.writeBitSet(optionals, 5);
+      if (struct.isSetCatName()) {
+        optionals.set(5);
+      }
+      oprot.writeBitSet(optionals, 6);
       if (struct.isSetApplyDistinct()) {
         oprot.writeBool(struct.applyDistinct);
       }
@@ -1151,9 +1250,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetPartitionOrder()) {
         {
           oprot.writeI32(struct.partitionOrder.size());
-          for (FieldSchema _iter515 : struct.partitionOrder)
+          for (FieldSchema _iter523 : struct.partitionOrder)
           {
-            _iter515.write(oprot);
+            _iter523.write(oprot);
           }
         }
       }
@@ -1163,6 +1262,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetMaxParts()) {
         oprot.writeI64(struct.maxParts);
       }
+      if (struct.isSetCatName()) {
+        oprot.writeString(struct.catName);
+      }
     }
 
     @Override
@@ -1173,18 +1275,18 @@ import org.slf4j.LoggerFactory;
       struct.tblName = iprot.readString();
       struct.setTblNameIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list516 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.partitionKeys = new ArrayList<FieldSchema>(_list516.size);
-        FieldSchema _elem517;
-        for (int _i518 = 0; _i518 < _list516.size; ++_i518)
+        org.apache.thrift.protocol.TList _list524 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.partitionKeys = new ArrayList<FieldSchema>(_list524.size);
+        FieldSchema _elem525;
+        for (int _i526 = 0; _i526 < _list524.size; ++_i526)
         {
-          _elem517 = new FieldSchema();
-          _elem517.read(iprot);
-          struct.partitionKeys.add(_elem517);
+          _elem525 = new FieldSchema();
+          _elem525.read(iprot);
+          struct.partitionKeys.add(_elem525);
         }
       }
       struct.setPartitionKeysIsSet(true);
-      BitSet incoming = iprot.readBitSet(5);
+      BitSet incoming = iprot.readBitSet(6);
       if (incoming.get(0)) {
         struct.applyDistinct = iprot.readBool();
         struct.setApplyDistinctIsSet(true);
@@ -1195,14 +1297,14 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TList _list519 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.partitionOrder = new ArrayList<FieldSchema>(_list519.size);
-          FieldSchema _elem520;
-          for (int _i521 = 0; _i521 < _list519.size; ++_i521)
+          org.apache.thrift.protocol.TList _list527 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.partitionOrder = new ArrayList<FieldSchema>(_list527.size);
+          FieldSchema _elem528;
+          for (int _i529 = 0; _i529 < _list527.size; ++_i529)
           {
-            _elem520 = new FieldSchema();
-            _elem520.read(iprot);
-            struct.partitionOrder.add(_elem520);
+            _elem528 = new FieldSchema();
+            _elem528.read(iprot);
+            struct.partitionOrder.add(_elem528);
           }
         }
         struct.setPartitionOrderIsSet(true);
@@ -1215,6 +1317,10 @@ import org.slf4j.LoggerFactory;
         struct.maxParts = iprot.readI64();
         struct.setMaxPartsIsSet(true);
       }
+      if (incoming.get(5)) {
+        struct.catName = iprot.readString();
+        struct.setCatNameIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesResponse.java
index 635b57e..e336aa1 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesResponse.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // PARTITION_VALUES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list530 = iprot.readListBegin();
-                struct.partitionValues = new ArrayList<PartitionValuesRow>(_list530.size);
-                PartitionValuesRow _elem531;
-                for (int _i532 = 0; _i532 < _list530.size; ++_i532)
+                org.apache.thrift.protocol.TList _list538 = iprot.readListBegin();
+                struct.partitionValues = new ArrayList<PartitionValuesRow>(_list538.size);
+                PartitionValuesRow _elem539;
+                for (int _i540 = 0; _i540 < _list538.size; ++_i540)
                 {
-                  _elem531 = new PartitionValuesRow();
-                  _elem531.read(iprot);
-                  struct.partitionValues.add(_elem531);
+                  _elem539 = new PartitionValuesRow();
+                  _elem539.read(iprot);
+                  struct.partitionValues.add(_elem539);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PARTITION_VALUES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitionValues.size()));
-          for (PartitionValuesRow _iter533 : struct.partitionValues)
+          for (PartitionValuesRow _iter541 : struct.partitionValues)
           {
-            _iter533.write(oprot);
+            _iter541.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.partitionValues.size());
-        for (PartitionValuesRow _iter534 : struct.partitionValues)
+        for (PartitionValuesRow _iter542 : struct.partitionValues)
         {
-          _iter534.write(oprot);
+          _iter542.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, PartitionValuesResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list535 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.partitionValues = new ArrayList<PartitionValuesRow>(_list535.size);
-        PartitionValuesRow _elem536;
-        for (int _i537 = 0; _i537 < _list535.size; ++_i537)
+        org.apache.thrift.protocol.TList _list543 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.partitionValues = new ArrayList<PartitionValuesRow>(_list543.size);
+        PartitionValuesRow _elem544;
+        for (int _i545 = 0; _i545 < _list543.size; ++_i545)
         {
-          _elem536 = new PartitionValuesRow();
-          _elem536.read(iprot);
-          struct.partitionValues.add(_elem536);
+          _elem544 = new PartitionValuesRow();
+          _elem544.read(iprot);
+          struct.partitionValues.add(_elem544);
         }
       }
       struct.setPartitionValuesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesRow.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesRow.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesRow.java
index 83e9e06..082c6c2 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesRow.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionValuesRow.java
@@ -351,13 +351,13 @@ import org.slf4j.LoggerFactory;
           case 1: // ROW
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list522 = iprot.readListBegin();
-                struct.row = new ArrayList<String>(_list522.size);
-                String _elem523;
-                for (int _i524 = 0; _i524 < _list522.size; ++_i524)
+                org.apache.thrift.protocol.TList _list530 = iprot.readListBegin();
+                struct.row = new ArrayList<String>(_list530.size);
+                String _elem531;
+                for (int _i532 = 0; _i532 < _list530.size; ++_i532)
                 {
-                  _elem523 = iprot.readString();
-                  struct.row.add(_elem523);
+                  _elem531 = iprot.readString();
+                  struct.row.add(_elem531);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(ROW_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.row.size()));
-          for (String _iter525 : struct.row)
+          for (String _iter533 : struct.row)
           {
-            oprot.writeString(_iter525);
+            oprot.writeString(_iter533);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.row.size());
-        for (String _iter526 : struct.row)
+        for (String _iter534 : struct.row)
         {
-          oprot.writeString(_iter526);
+          oprot.writeString(_iter534);
         }
       }
     }
@@ -421,13 +421,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, PartitionValuesRow struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list527 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.row = new ArrayList<String>(_list527.size);
-        String _elem528;
-        for (int _i529 = 0; _i529 < _list527.size; ++_i529)
+        org.apache.thrift.protocol.TList _list535 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.row = new ArrayList<String>(_list535.size);
+        String _elem536;
+        for (int _i537 = 0; _i537 < _list535.size; ++_i537)
         {
-          _elem528 = iprot.readString();
-          struct.row.add(_elem528);
+          _elem536 = iprot.readString();
+          struct.row.add(_elem536);
         }
       }
       struct.setRowIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionWithoutSD.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionWithoutSD.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionWithoutSD.java
index ba8a7ca..5807618 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionWithoutSD.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionWithoutSD.java
@@ -766,13 +766,13 @@ import org.slf4j.LoggerFactory;
           case 1: // VALUES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list226 = iprot.readListBegin();
-                struct.values = new ArrayList<String>(_list226.size);
-                String _elem227;
-                for (int _i228 = 0; _i228 < _list226.size; ++_i228)
+                org.apache.thrift.protocol.TList _list234 = iprot.readListBegin();
+                struct.values = new ArrayList<String>(_list234.size);
+                String _elem235;
+                for (int _i236 = 0; _i236 < _list234.size; ++_i236)
                 {
-                  _elem227 = iprot.readString();
-                  struct.values.add(_elem227);
+                  _elem235 = iprot.readString();
+                  struct.values.add(_elem235);
                 }
                 iprot.readListEnd();
               }
@@ -808,15 +808,15 @@ import org.slf4j.LoggerFactory;
           case 5: // PARAMETERS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map229 = iprot.readMapBegin();
-                struct.parameters = new HashMap<String,String>(2*_map229.size);
-                String _key230;
-                String _val231;
-                for (int _i232 = 0; _i232 < _map229.size; ++_i232)
+                org.apache.thrift.protocol.TMap _map237 = iprot.readMapBegin();
+                struct.parameters = new HashMap<String,String>(2*_map237.size);
+                String _key238;
+                String _val239;
+                for (int _i240 = 0; _i240 < _map237.size; ++_i240)
                 {
-                  _key230 = iprot.readString();
-                  _val231 = iprot.readString();
-                  struct.parameters.put(_key230, _val231);
+                  _key238 = iprot.readString();
+                  _val239 = iprot.readString();
+                  struct.parameters.put(_key238, _val239);
                 }
                 iprot.readMapEnd();
               }
@@ -851,9 +851,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(VALUES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.values.size()));
-          for (String _iter233 : struct.values)
+          for (String _iter241 : struct.values)
           {
-            oprot.writeString(_iter233);
+            oprot.writeString(_iter241);
           }
           oprot.writeListEnd();
         }
@@ -874,10 +874,10 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PARAMETERS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.parameters.size()));
-          for (Map.Entry<String, String> _iter234 : struct.parameters.entrySet())
+          for (Map.Entry<String, String> _iter242 : struct.parameters.entrySet())
           {
-            oprot.writeString(_iter234.getKey());
-            oprot.writeString(_iter234.getValue());
+            oprot.writeString(_iter242.getKey());
+            oprot.writeString(_iter242.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -930,9 +930,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetValues()) {
         {
           oprot.writeI32(struct.values.size());
-          for (String _iter235 : struct.values)
+          for (String _iter243 : struct.values)
           {
-            oprot.writeString(_iter235);
+            oprot.writeString(_iter243);
           }
         }
       }
@@ -948,10 +948,10 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetParameters()) {
         {
           oprot.writeI32(struct.parameters.size());
-          for (Map.Entry<String, String> _iter236 : struct.parameters.entrySet())
+          for (Map.Entry<String, String> _iter244 : struct.parameters.entrySet())
           {
-            oprot.writeString(_iter236.getKey());
-            oprot.writeString(_iter236.getValue());
+            oprot.writeString(_iter244.getKey());
+            oprot.writeString(_iter244.getValue());
           }
         }
       }
@@ -966,13 +966,13 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(6);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list237 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.values = new ArrayList<String>(_list237.size);
-          String _elem238;
-          for (int _i239 = 0; _i239 < _list237.size; ++_i239)
+          org.apache.thrift.protocol.TList _list245 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.values = new ArrayList<String>(_list245.size);
+          String _elem246;
+          for (int _i247 = 0; _i247 < _list245.size; ++_i247)
           {
-            _elem238 = iprot.readString();
-            struct.values.add(_elem238);
+            _elem246 = iprot.readString();
+            struct.values.add(_elem246);
           }
         }
         struct.setValuesIsSet(true);
@@ -991,15 +991,15 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(4)) {
         {
-          org.apache.thrift.protocol.TMap _map240 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.parameters = new HashMap<String,String>(2*_map240.size);
-          String _key241;
-          String _val242;
-          for (int _i243 = 0; _i243 < _map240.size; ++_i243)
+          org.apache.thrift.protocol.TMap _map248 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.parameters = new HashMap<String,String>(2*_map248.size);
+          String _key249;
+          String _val250;
+          for (int _i251 = 0; _i251 < _map248.size; ++_i251)
           {
-            _key241 = iprot.readString();
-            _val242 = iprot.readString();
-            struct.parameters.put(_key241, _val242);
+            _key249 = iprot.readString();
+            _val250 = iprot.readString();
+            struct.parameters.put(_key249, _val250);
           }
         }
         struct.setParametersIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprRequest.java
index 13a5d6a..0e72625 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprRequest.java
@@ -43,6 +43,7 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField EXPR_FIELD_DESC = new org.apache.thrift.protocol.TField("expr", org.apache.thrift.protocol.TType.STRING, (short)3);
   private static final org.apache.thrift.protocol.TField DEFAULT_PARTITION_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("defaultPartitionName", org.apache.thrift.protocol.TType.STRING, (short)4);
   private static final org.apache.thrift.protocol.TField MAX_PARTS_FIELD_DESC = new org.apache.thrift.protocol.TField("maxParts", org.apache.thrift.protocol.TType.I16, (short)5);
+  private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)6);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -55,6 +56,7 @@ import org.slf4j.LoggerFactory;
   private ByteBuffer expr; // required
   private String defaultPartitionName; // optional
   private short maxParts; // optional
+  private String catName; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -62,7 +64,8 @@ import org.slf4j.LoggerFactory;
     TBL_NAME((short)2, "tblName"),
     EXPR((short)3, "expr"),
     DEFAULT_PARTITION_NAME((short)4, "defaultPartitionName"),
-    MAX_PARTS((short)5, "maxParts");
+    MAX_PARTS((short)5, "maxParts"),
+    CAT_NAME((short)6, "catName");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -87,6 +90,8 @@ import org.slf4j.LoggerFactory;
           return DEFAULT_PARTITION_NAME;
         case 5: // MAX_PARTS
           return MAX_PARTS;
+        case 6: // CAT_NAME
+          return CAT_NAME;
         default:
           return null;
       }
@@ -129,7 +134,7 @@ import org.slf4j.LoggerFactory;
   // isset id assignments
   private static final int __MAXPARTS_ISSET_ID = 0;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.DEFAULT_PARTITION_NAME,_Fields.MAX_PARTS};
+  private static final _Fields optionals[] = {_Fields.DEFAULT_PARTITION_NAME,_Fields.MAX_PARTS,_Fields.CAT_NAME};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -143,6 +148,8 @@ import org.slf4j.LoggerFactory;
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.MAX_PARTS, new org.apache.thrift.meta_data.FieldMetaData("maxParts", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I16)));
+    tmpMap.put(_Fields.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(PartitionsByExprRequest.class, metaDataMap);
   }
@@ -181,6 +188,9 @@ import org.slf4j.LoggerFactory;
       this.defaultPartitionName = other.defaultPartitionName;
     }
     this.maxParts = other.maxParts;
+    if (other.isSetCatName()) {
+      this.catName = other.catName;
+    }
   }
 
   public PartitionsByExprRequest deepCopy() {
@@ -195,6 +205,7 @@ import org.slf4j.LoggerFactory;
     this.defaultPartitionName = null;
     this.maxParts = (short)-1;
 
+    this.catName = null;
   }
 
   public String getDbName() {
@@ -320,6 +331,29 @@ import org.slf4j.LoggerFactory;
     __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MAXPARTS_ISSET_ID, value);
   }
 
+  public String getCatName() {
+    return this.catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
+  public void unsetCatName() {
+    this.catName = null;
+  }
+
+  /** Returns true if field catName is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatName() {
+    return this.catName != null;
+  }
+
+  public void setCatNameIsSet(boolean value) {
+    if (!value) {
+      this.catName = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case DB_NAME:
@@ -362,6 +396,14 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case CAT_NAME:
+      if (value == null) {
+        unsetCatName();
+      } else {
+        setCatName((String)value);
+      }
+      break;
+
     }
   }
 
@@ -382,6 +424,9 @@ import org.slf4j.LoggerFactory;
     case MAX_PARTS:
       return getMaxParts();
 
+    case CAT_NAME:
+      return getCatName();
+
     }
     throw new IllegalStateException();
   }
@@ -403,6 +448,8 @@ import org.slf4j.LoggerFactory;
       return isSetDefaultPartitionName();
     case MAX_PARTS:
       return isSetMaxParts();
+    case CAT_NAME:
+      return isSetCatName();
     }
     throw new IllegalStateException();
   }
@@ -465,6 +512,15 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_catName = true && this.isSetCatName();
+    boolean that_present_catName = true && that.isSetCatName();
+    if (this_present_catName || that_present_catName) {
+      if (!(this_present_catName && that_present_catName))
+        return false;
+      if (!this.catName.equals(that.catName))
+        return false;
+    }
+
     return true;
   }
 
@@ -497,6 +553,11 @@ import org.slf4j.LoggerFactory;
     if (present_maxParts)
       list.add(maxParts);
 
+    boolean present_catName = true && (isSetCatName());
+    list.add(present_catName);
+    if (present_catName)
+      list.add(catName);
+
     return list.hashCode();
   }
 
@@ -558,6 +619,16 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetCatName()).compareTo(other.isSetCatName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catName, other.catName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -617,6 +688,16 @@ import org.slf4j.LoggerFactory;
       sb.append(this.maxParts);
       first = false;
     }
+    if (isSetCatName()) {
+      if (!first) sb.append(", ");
+      sb.append("catName:");
+      if (this.catName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.catName);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -714,6 +795,14 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 6: // CAT_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.catName = iprot.readString();
+              struct.setCatNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -754,6 +843,13 @@ import org.slf4j.LoggerFactory;
         oprot.writeI16(struct.maxParts);
         oprot.writeFieldEnd();
       }
+      if (struct.catName != null) {
+        if (struct.isSetCatName()) {
+          oprot.writeFieldBegin(CAT_NAME_FIELD_DESC);
+          oprot.writeString(struct.catName);
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -781,13 +877,19 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetMaxParts()) {
         optionals.set(1);
       }
-      oprot.writeBitSet(optionals, 2);
+      if (struct.isSetCatName()) {
+        optionals.set(2);
+      }
+      oprot.writeBitSet(optionals, 3);
       if (struct.isSetDefaultPartitionName()) {
         oprot.writeString(struct.defaultPartitionName);
       }
       if (struct.isSetMaxParts()) {
         oprot.writeI16(struct.maxParts);
       }
+      if (struct.isSetCatName()) {
+        oprot.writeString(struct.catName);
+      }
     }
 
     @Override
@@ -799,7 +901,7 @@ import org.slf4j.LoggerFactory;
       struct.setTblNameIsSet(true);
       struct.expr = iprot.readBinary();
       struct.setExprIsSet(true);
-      BitSet incoming = iprot.readBitSet(2);
+      BitSet incoming = iprot.readBitSet(3);
       if (incoming.get(0)) {
         struct.defaultPartitionName = iprot.readString();
         struct.setDefaultPartitionNameIsSet(true);
@@ -808,6 +910,10 @@ import org.slf4j.LoggerFactory;
         struct.maxParts = iprot.readI16();
         struct.setMaxPartsIsSet(true);
       }
+      if (incoming.get(2)) {
+        struct.catName = iprot.readString();
+        struct.setCatNameIsSet(true);
+      }
     }
   }
 


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
index 0f49d93..5897fae 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
@@ -419,11 +419,11 @@ uint32_t ThriftHiveMetastore_setMetaConf_presult::read(::apache::thrift::protoco
 }
 
 
-ThriftHiveMetastore_create_database_args::~ThriftHiveMetastore_create_database_args() throw() {
+ThriftHiveMetastore_create_catalog_args::~ThriftHiveMetastore_create_catalog_args() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_create_database_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_create_catalog_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -446,8 +446,8 @@ uint32_t ThriftHiveMetastore_create_database_args::read(::apache::thrift::protoc
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->database.read(iprot);
-          this->__isset.database = true;
+          xfer += this->catalog.read(iprot);
+          this->__isset.catalog = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -464,13 +464,13 @@ uint32_t ThriftHiveMetastore_create_database_args::read(::apache::thrift::protoc
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_create_database_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_create_catalog_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_database_args");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_catalog_args");
 
-  xfer += oprot->writeFieldBegin("database", ::apache::thrift::protocol::T_STRUCT, 1);
-  xfer += this->database.write(oprot);
+  xfer += oprot->writeFieldBegin("catalog", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += this->catalog.write(oprot);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -479,17 +479,17 @@ uint32_t ThriftHiveMetastore_create_database_args::write(::apache::thrift::proto
 }
 
 
-ThriftHiveMetastore_create_database_pargs::~ThriftHiveMetastore_create_database_pargs() throw() {
+ThriftHiveMetastore_create_catalog_pargs::~ThriftHiveMetastore_create_catalog_pargs() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_create_database_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_create_catalog_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_database_pargs");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_catalog_pargs");
 
-  xfer += oprot->writeFieldBegin("database", ::apache::thrift::protocol::T_STRUCT, 1);
-  xfer += (*(this->database)).write(oprot);
+  xfer += oprot->writeFieldBegin("catalog", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += (*(this->catalog)).write(oprot);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -498,11 +498,11 @@ uint32_t ThriftHiveMetastore_create_database_pargs::write(::apache::thrift::prot
 }
 
 
-ThriftHiveMetastore_create_database_result::~ThriftHiveMetastore_create_database_result() throw() {
+ThriftHiveMetastore_create_catalog_result::~ThriftHiveMetastore_create_catalog_result() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_create_database_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_create_catalog_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -559,11 +559,11 @@ uint32_t ThriftHiveMetastore_create_database_result::read(::apache::thrift::prot
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_create_database_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_create_catalog_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_database_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_catalog_result");
 
   if (this->__isset.o1) {
     xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1);
@@ -584,11 +584,11 @@ uint32_t ThriftHiveMetastore_create_database_result::write(::apache::thrift::pro
 }
 
 
-ThriftHiveMetastore_create_database_presult::~ThriftHiveMetastore_create_database_presult() throw() {
+ThriftHiveMetastore_create_catalog_presult::~ThriftHiveMetastore_create_catalog_presult() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_create_database_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_create_catalog_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -646,11 +646,11 @@ uint32_t ThriftHiveMetastore_create_database_presult::read(::apache::thrift::pro
 }
 
 
-ThriftHiveMetastore_get_database_args::~ThriftHiveMetastore_get_database_args() throw() {
+ThriftHiveMetastore_get_catalog_args::~ThriftHiveMetastore_get_catalog_args() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_get_database_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_catalog_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -672,9 +672,9 @@ uint32_t ThriftHiveMetastore_get_database_args::read(::apache::thrift::protocol:
     switch (fid)
     {
       case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->name);
-          this->__isset.name = true;
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->catName.read(iprot);
+          this->__isset.catName = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -691,13 +691,13 @@ uint32_t ThriftHiveMetastore_get_database_args::read(::apache::thrift::protocol:
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_database_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_catalog_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_database_args");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_catalog_args");
 
-  xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString(this->name);
+  xfer += oprot->writeFieldBegin("catName", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += this->catName.write(oprot);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -706,17 +706,17 @@ uint32_t ThriftHiveMetastore_get_database_args::write(::apache::thrift::protocol
 }
 
 
-ThriftHiveMetastore_get_database_pargs::~ThriftHiveMetastore_get_database_pargs() throw() {
+ThriftHiveMetastore_get_catalog_pargs::~ThriftHiveMetastore_get_catalog_pargs() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_get_database_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_catalog_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_database_pargs");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_catalog_pargs");
 
-  xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString((*(this->name)));
+  xfer += oprot->writeFieldBegin("catName", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += (*(this->catName)).write(oprot);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -725,11 +725,11 @@ uint32_t ThriftHiveMetastore_get_database_pargs::write(::apache::thrift::protoco
 }
 
 
-ThriftHiveMetastore_get_database_result::~ThriftHiveMetastore_get_database_result() throw() {
+ThriftHiveMetastore_get_catalog_result::~ThriftHiveMetastore_get_catalog_result() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_get_database_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_catalog_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -786,11 +786,11 @@ uint32_t ThriftHiveMetastore_get_database_result::read(::apache::thrift::protoco
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_database_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_catalog_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_database_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_catalog_result");
 
   if (this->__isset.success) {
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0);
@@ -811,11 +811,11 @@ uint32_t ThriftHiveMetastore_get_database_result::write(::apache::thrift::protoc
 }
 
 
-ThriftHiveMetastore_get_database_presult::~ThriftHiveMetastore_get_database_presult() throw() {
+ThriftHiveMetastore_get_catalog_presult::~ThriftHiveMetastore_get_catalog_presult() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_get_database_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_catalog_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -873,11 +873,11 @@ uint32_t ThriftHiveMetastore_get_database_presult::read(::apache::thrift::protoc
 }
 
 
-ThriftHiveMetastore_drop_database_args::~ThriftHiveMetastore_drop_database_args() throw() {
+ThriftHiveMetastore_get_catalogs_args::~ThriftHiveMetastore_get_catalogs_args() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_drop_database_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_catalogs_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -896,36 +896,7 @@ uint32_t ThriftHiveMetastore_drop_database_args::read(::apache::thrift::protocol
     if (ftype == ::apache::thrift::protocol::T_STOP) {
       break;
     }
-    switch (fid)
-    {
-      case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->name);
-          this->__isset.name = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 2:
-        if (ftype == ::apache::thrift::protocol::T_BOOL) {
-          xfer += iprot->readBool(this->deleteData);
-          this->__isset.deleteData = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 3:
-        if (ftype == ::apache::thrift::protocol::T_BOOL) {
-          xfer += iprot->readBool(this->cascade);
-          this->__isset.cascade = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
+    xfer += iprot->skip(ftype);
     xfer += iprot->readFieldEnd();
   }
 
@@ -934,22 +905,10 @@ uint32_t ThriftHiveMetastore_drop_database_args::read(::apache::thrift::protocol
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_drop_database_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_catalogs_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_database_args");
-
-  xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString(this->name);
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 2);
-  xfer += oprot->writeBool(this->deleteData);
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldBegin("cascade", ::apache::thrift::protocol::T_BOOL, 3);
-  xfer += oprot->writeBool(this->cascade);
-  xfer += oprot->writeFieldEnd();
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_catalogs_args");
 
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
@@ -957,26 +916,14 @@ uint32_t ThriftHiveMetastore_drop_database_args::write(::apache::thrift::protoco
 }
 
 
-ThriftHiveMetastore_drop_database_pargs::~ThriftHiveMetastore_drop_database_pargs() throw() {
+ThriftHiveMetastore_get_catalogs_pargs::~ThriftHiveMetastore_get_catalogs_pargs() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_drop_database_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_catalogs_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_database_pargs");
-
-  xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString((*(this->name)));
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 2);
-  xfer += oprot->writeBool((*(this->deleteData)));
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldBegin("cascade", ::apache::thrift::protocol::T_BOOL, 3);
-  xfer += oprot->writeBool((*(this->cascade)));
-  xfer += oprot->writeFieldEnd();
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_catalogs_pargs");
 
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
@@ -984,11 +931,11 @@ uint32_t ThriftHiveMetastore_drop_database_pargs::write(::apache::thrift::protoc
 }
 
 
-ThriftHiveMetastore_drop_database_result::~ThriftHiveMetastore_drop_database_result() throw() {
+ThriftHiveMetastore_get_catalogs_result::~ThriftHiveMetastore_get_catalogs_result() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_drop_database_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_catalogs_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -1009,92 +956,14 @@ uint32_t ThriftHiveMetastore_drop_database_result::read(::apache::thrift::protoc
     }
     switch (fid)
     {
-      case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o1.read(iprot);
-          this->__isset.o1 = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 2:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o2.read(iprot);
-          this->__isset.o2 = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 3:
+      case 0:
         if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o3.read(iprot);
-          this->__isset.o3 = true;
+          xfer += this->success.read(iprot);
+          this->__isset.success = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-uint32_t ThriftHiveMetastore_drop_database_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
-
-  uint32_t xfer = 0;
-
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_database_result");
-
-  if (this->__isset.o1) {
-    xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1);
-    xfer += this->o1.write(oprot);
-    xfer += oprot->writeFieldEnd();
-  } else if (this->__isset.o2) {
-    xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2);
-    xfer += this->o2.write(oprot);
-    xfer += oprot->writeFieldEnd();
-  } else if (this->__isset.o3) {
-    xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3);
-    xfer += this->o3.write(oprot);
-    xfer += oprot->writeFieldEnd();
-  }
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHiveMetastore_drop_database_presult::~ThriftHiveMetastore_drop_database_presult() throw() {
-}
-
-
-uint32_t ThriftHiveMetastore_drop_database_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
-
-  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
-  uint32_t xfer = 0;
-  std::string fname;
-  ::apache::thrift::protocol::TType ftype;
-  int16_t fid;
-
-  xfer += iprot->readStructBegin(fname);
-
-  using ::apache::thrift::protocol::TProtocolException;
-
-
-  while (true)
-  {
-    xfer += iprot->readFieldBegin(fname, ftype, fid);
-    if (ftype == ::apache::thrift::protocol::T_STOP) {
-      break;
-    }
-    switch (fid)
-    {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRUCT) {
           xfer += this->o1.read(iprot);
@@ -1103,22 +972,6 @@ uint32_t ThriftHiveMetastore_drop_database_presult::read(::apache::thrift::proto
           xfer += iprot->skip(ftype);
         }
         break;
-      case 2:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o2.read(iprot);
-          this->__isset.o2 = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 3:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o3.read(iprot);
-          this->__isset.o3 = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -1131,91 +984,32 @@ uint32_t ThriftHiveMetastore_drop_database_presult::read(::apache::thrift::proto
   return xfer;
 }
 
+uint32_t ThriftHiveMetastore_get_catalogs_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
-ThriftHiveMetastore_get_databases_args::~ThriftHiveMetastore_get_databases_args() throw() {
-}
-
-
-uint32_t ThriftHiveMetastore_get_databases_args::read(::apache::thrift::protocol::TProtocol* iprot) {
-
-  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
-  std::string fname;
-  ::apache::thrift::protocol::TType ftype;
-  int16_t fid;
 
-  xfer += iprot->readStructBegin(fname);
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_catalogs_result");
 
-  using ::apache::thrift::protocol::TProtocolException;
-
-
-  while (true)
-  {
-    xfer += iprot->readFieldBegin(fname, ftype, fid);
-    if (ftype == ::apache::thrift::protocol::T_STOP) {
-      break;
-    }
-    switch (fid)
-    {
-      case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->pattern);
-          this->__isset.pattern = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
-    xfer += iprot->readFieldEnd();
+  if (this->__isset.success) {
+    xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0);
+    xfer += this->success.write(oprot);
+    xfer += oprot->writeFieldEnd();
+  } else if (this->__isset.o1) {
+    xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1);
+    xfer += this->o1.write(oprot);
+    xfer += oprot->writeFieldEnd();
   }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-uint32_t ThriftHiveMetastore_get_databases_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
-  uint32_t xfer = 0;
-  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_databases_args");
-
-  xfer += oprot->writeFieldBegin("pattern", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString(this->pattern);
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHiveMetastore_get_databases_pargs::~ThriftHiveMetastore_get_databases_pargs() throw() {
-}
-
-
-uint32_t ThriftHiveMetastore_get_databases_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
-  uint32_t xfer = 0;
-  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_databases_pargs");
-
-  xfer += oprot->writeFieldBegin("pattern", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString((*(this->pattern)));
-  xfer += oprot->writeFieldEnd();
-
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
 }
 
 
-ThriftHiveMetastore_get_databases_result::~ThriftHiveMetastore_get_databases_result() throw() {
+ThriftHiveMetastore_get_catalogs_presult::~ThriftHiveMetastore_get_catalogs_presult() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_get_databases_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_catalogs_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -1237,20 +1031,8 @@ uint32_t ThriftHiveMetastore_get_databases_result::read(::apache::thrift::protoc
     switch (fid)
     {
       case 0:
-        if (ftype == ::apache::thrift::protocol::T_LIST) {
-          {
-            this->success.clear();
-            uint32_t _size1157;
-            ::apache::thrift::protocol::TType _etype1160;
-            xfer += iprot->readListBegin(_etype1160, _size1157);
-            this->success.resize(_size1157);
-            uint32_t _i1161;
-            for (_i1161 = 0; _i1161 < _size1157; ++_i1161)
-            {
-              xfer += iprot->readString(this->success[_i1161]);
-            }
-            xfer += iprot->readListEnd();
-          }
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += (*(this->success)).read(iprot);
           this->__isset.success = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -1276,40 +1058,12 @@ uint32_t ThriftHiveMetastore_get_databases_result::read(::apache::thrift::protoc
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_databases_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
-
-  uint32_t xfer = 0;
-
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_databases_result");
 
-  if (this->__isset.success) {
-    xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
-    {
-      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1162;
-      for (_iter1162 = this->success.begin(); _iter1162 != this->success.end(); ++_iter1162)
-      {
-        xfer += oprot->writeString((*_iter1162));
-      }
-      xfer += oprot->writeListEnd();
-    }
-    xfer += oprot->writeFieldEnd();
-  } else if (this->__isset.o1) {
-    xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1);
-    xfer += this->o1.write(oprot);
-    xfer += oprot->writeFieldEnd();
-  }
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHiveMetastore_get_databases_presult::~ThriftHiveMetastore_get_databases_presult() throw() {
+ThriftHiveMetastore_drop_catalog_args::~ThriftHiveMetastore_drop_catalog_args() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_get_databases_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_drop_catalog_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -1330,30 +1084,10 @@ uint32_t ThriftHiveMetastore_get_databases_presult::read(::apache::thrift::proto
     }
     switch (fid)
     {
-      case 0:
-        if (ftype == ::apache::thrift::protocol::T_LIST) {
-          {
-            (*(this->success)).clear();
-            uint32_t _size1163;
-            ::apache::thrift::protocol::TType _etype1166;
-            xfer += iprot->readListBegin(_etype1166, _size1163);
-            (*(this->success)).resize(_size1163);
-            uint32_t _i1167;
-            for (_i1167 = 0; _i1167 < _size1163; ++_i1167)
-            {
-              xfer += iprot->readString((*(this->success))[_i1167]);
-            }
-            xfer += iprot->readListEnd();
-          }
-          this->__isset.success = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o1.read(iprot);
-          this->__isset.o1 = true;
+          xfer += this->catName.read(iprot);
+          this->__isset.catName = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -1370,43 +1104,14 @@ uint32_t ThriftHiveMetastore_get_databases_presult::read(::apache::thrift::proto
   return xfer;
 }
 
-
-ThriftHiveMetastore_get_all_databases_args::~ThriftHiveMetastore_get_all_databases_args() throw() {
-}
-
-
-uint32_t ThriftHiveMetastore_get_all_databases_args::read(::apache::thrift::protocol::TProtocol* iprot) {
-
-  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
-  uint32_t xfer = 0;
-  std::string fname;
-  ::apache::thrift::protocol::TType ftype;
-  int16_t fid;
-
-  xfer += iprot->readStructBegin(fname);
-
-  using ::apache::thrift::protocol::TProtocolException;
-
-
-  while (true)
-  {
-    xfer += iprot->readFieldBegin(fname, ftype, fid);
-    if (ftype == ::apache::thrift::protocol::T_STOP) {
-      break;
-    }
-    xfer += iprot->skip(ftype);
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-uint32_t ThriftHiveMetastore_get_all_databases_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_drop_catalog_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_databases_args");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_catalog_args");
+
+  xfer += oprot->writeFieldBegin("catName", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += this->catName.write(oprot);
+  xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
@@ -1414,14 +1119,18 @@ uint32_t ThriftHiveMetastore_get_all_databases_args::write(::apache::thrift::pro
 }
 
 
-ThriftHiveMetastore_get_all_databases_pargs::~ThriftHiveMetastore_get_all_databases_pargs() throw() {
+ThriftHiveMetastore_drop_catalog_pargs::~ThriftHiveMetastore_drop_catalog_pargs() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_get_all_databases_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_drop_catalog_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_databases_pargs");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_catalog_pargs");
+
+  xfer += oprot->writeFieldBegin("catName", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += (*(this->catName)).write(oprot);
+  xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
@@ -1429,11 +1138,11 @@ uint32_t ThriftHiveMetastore_get_all_databases_pargs::write(::apache::thrift::pr
 }
 
 
-ThriftHiveMetastore_get_all_databases_result::~ThriftHiveMetastore_get_all_databases_result() throw() {
+ThriftHiveMetastore_drop_catalog_result::~ThriftHiveMetastore_drop_catalog_result() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_get_all_databases_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_drop_catalog_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -1454,30 +1163,26 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::read(::apache::thrift::pr
     }
     switch (fid)
     {
-      case 0:
-        if (ftype == ::apache::thrift::protocol::T_LIST) {
-          {
-            this->success.clear();
-            uint32_t _size1168;
-            ::apache::thrift::protocol::TType _etype1171;
-            xfer += iprot->readListBegin(_etype1171, _size1168);
-            this->success.resize(_size1168);
-            uint32_t _i1172;
-            for (_i1172 = 0; _i1172 < _size1168; ++_i1172)
-            {
-              xfer += iprot->readString(this->success[_i1172]);
-            }
-            xfer += iprot->readListEnd();
-          }
-          this->__isset.success = true;
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->o1.read(iprot);
+          this->__isset.o1 = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
-      case 1:
+      case 2:
         if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o1.read(iprot);
-          this->__isset.o1 = true;
+          xfer += this->o2.read(iprot);
+          this->__isset.o2 = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->o3.read(iprot);
+          this->__isset.o3 = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -1494,28 +1199,24 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::read(::apache::thrift::pr
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_all_databases_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_drop_catalog_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_databases_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_catalog_result");
 
-  if (this->__isset.success) {
-    xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
-    {
-      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1173;
-      for (_iter1173 = this->success.begin(); _iter1173 != this->success.end(); ++_iter1173)
-      {
-        xfer += oprot->writeString((*_iter1173));
-      }
-      xfer += oprot->writeListEnd();
-    }
-    xfer += oprot->writeFieldEnd();
-  } else if (this->__isset.o1) {
+  if (this->__isset.o1) {
     xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1);
     xfer += this->o1.write(oprot);
     xfer += oprot->writeFieldEnd();
+  } else if (this->__isset.o2) {
+    xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2);
+    xfer += this->o2.write(oprot);
+    xfer += oprot->writeFieldEnd();
+  } else if (this->__isset.o3) {
+    xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3);
+    xfer += this->o3.write(oprot);
+    xfer += oprot->writeFieldEnd();
   }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
@@ -1523,11 +1224,11 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::write(::apache::thrift::p
 }
 
 
-ThriftHiveMetastore_get_all_databases_presult::~ThriftHiveMetastore_get_all_databases_presult() throw() {
+ThriftHiveMetastore_drop_catalog_presult::~ThriftHiveMetastore_drop_catalog_presult() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_get_all_databases_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_drop_catalog_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -1548,30 +1249,26 @@ uint32_t ThriftHiveMetastore_get_all_databases_presult::read(::apache::thrift::p
     }
     switch (fid)
     {
-      case 0:
-        if (ftype == ::apache::thrift::protocol::T_LIST) {
-          {
-            (*(this->success)).clear();
-            uint32_t _size1174;
-            ::apache::thrift::protocol::TType _etype1177;
-            xfer += iprot->readListBegin(_etype1177, _size1174);
-            (*(this->success)).resize(_size1174);
-            uint32_t _i1178;
-            for (_i1178 = 0; _i1178 < _size1174; ++_i1178)
-            {
-              xfer += iprot->readString((*(this->success))[_i1178]);
-            }
-            xfer += iprot->readListEnd();
-          }
-          this->__isset.success = true;
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->o1.read(iprot);
+          this->__isset.o1 = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
-      case 1:
+      case 2:
         if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o1.read(iprot);
-          this->__isset.o1 = true;
+          xfer += this->o2.read(iprot);
+          this->__isset.o2 = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->o3.read(iprot);
+          this->__isset.o3 = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -1589,11 +1286,11 @@ uint32_t ThriftHiveMetastore_get_all_databases_presult::read(::apache::thrift::p
 }
 
 
-ThriftHiveMetastore_alter_database_args::~ThriftHiveMetastore_alter_database_args() throw() {
+ThriftHiveMetastore_create_database_args::~ThriftHiveMetastore_create_database_args() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_alter_database_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_create_database_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -1615,17 +1312,9 @@ uint32_t ThriftHiveMetastore_alter_database_args::read(::apache::thrift::protoco
     switch (fid)
     {
       case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->dbname);
-          this->__isset.dbname = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 2:
         if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->db.read(iprot);
-          this->__isset.db = true;
+          xfer += this->database.read(iprot);
+          this->__isset.database = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -1642,17 +1331,13 @@ uint32_t ThriftHiveMetastore_alter_database_args::read(::apache::thrift::protoco
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_alter_database_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_create_database_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_database_args");
-
-  xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString(this->dbname);
-  xfer += oprot->writeFieldEnd();
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_database_args");
 
-  xfer += oprot->writeFieldBegin("db", ::apache::thrift::protocol::T_STRUCT, 2);
-  xfer += this->db.write(oprot);
+  xfer += oprot->writeFieldBegin("database", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += this->database.write(oprot);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -1661,21 +1346,17 @@ uint32_t ThriftHiveMetastore_alter_database_args::write(::apache::thrift::protoc
 }
 
 
-ThriftHiveMetastore_alter_database_pargs::~ThriftHiveMetastore_alter_database_pargs() throw() {
+ThriftHiveMetastore_create_database_pargs::~ThriftHiveMetastore_create_database_pargs() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_alter_database_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_create_database_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_database_pargs");
-
-  xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString((*(this->dbname)));
-  xfer += oprot->writeFieldEnd();
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_database_pargs");
 
-  xfer += oprot->writeFieldBegin("db", ::apache::thrift::protocol::T_STRUCT, 2);
-  xfer += (*(this->db)).write(oprot);
+  xfer += oprot->writeFieldBegin("database", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += (*(this->database)).write(oprot);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -1684,11 +1365,11 @@ uint32_t ThriftHiveMetastore_alter_database_pargs::write(::apache::thrift::proto
 }
 
 
-ThriftHiveMetastore_alter_database_result::~ThriftHiveMetastore_alter_database_result() throw() {
+ThriftHiveMetastore_create_database_result::~ThriftHiveMetastore_create_database_result() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_alter_database_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_create_database_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -1725,6 +1406,14 @@ uint32_t ThriftHiveMetastore_alter_database_result::read(::apache::thrift::proto
           xfer += iprot->skip(ftype);
         }
         break;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->o3.read(iprot);
+          this->__isset.o3 = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -1737,11 +1426,11 @@ uint32_t ThriftHiveMetastore_alter_database_result::read(::apache::thrift::proto
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_alter_database_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_create_database_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_database_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_database_result");
 
   if (this->__isset.o1) {
     xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1);
@@ -1751,6 +1440,10 @@ uint32_t ThriftHiveMetastore_alter_database_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2);
     xfer += this->o2.write(oprot);
     xfer += oprot->writeFieldEnd();
+  } else if (this->__isset.o3) {
+    xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3);
+    xfer += this->o3.write(oprot);
+    xfer += oprot->writeFieldEnd();
   }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
@@ -1758,11 +1451,11 @@ uint32_t ThriftHiveMetastore_alter_database_result::write(::apache::thrift::prot
 }
 
 
-ThriftHiveMetastore_alter_database_presult::~ThriftHiveMetastore_alter_database_presult() throw() {
+ThriftHiveMetastore_create_database_presult::~ThriftHiveMetastore_create_database_presult() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_alter_database_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_create_database_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -1799,6 +1492,14 @@ uint32_t ThriftHiveMetastore_alter_database_presult::read(::apache::thrift::prot
           xfer += iprot->skip(ftype);
         }
         break;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->o3.read(iprot);
+          this->__isset.o3 = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -1812,11 +1513,11 @@ uint32_t ThriftHiveMetastore_alter_database_presult::read(::apache::thrift::prot
 }
 
 
-ThriftHiveMetastore_get_type_args::~ThriftHiveMetastore_get_type_args() throw() {
+ThriftHiveMetastore_get_database_args::~ThriftHiveMetastore_get_database_args() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_get_type_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_database_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -1857,10 +1558,10 @@ uint32_t ThriftHiveMetastore_get_type_args::read(::apache::thrift::protocol::TPr
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_type_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_database_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_type_args");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_database_args");
 
   xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1);
   xfer += oprot->writeString(this->name);
@@ -1872,14 +1573,14 @@ uint32_t ThriftHiveMetastore_get_type_args::write(::apache::thrift::protocol::TP
 }
 
 
-ThriftHiveMetastore_get_type_pargs::~ThriftHiveMetastore_get_type_pargs() throw() {
+ThriftHiveMetastore_get_database_pargs::~ThriftHiveMetastore_get_database_pargs() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_get_type_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_database_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_type_pargs");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_database_pargs");
 
   xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1);
   xfer += oprot->writeString((*(this->name)));
@@ -1891,11 +1592,11 @@ uint32_t ThriftHiveMetastore_get_type_pargs::write(::apache::thrift::protocol::T
 }
 
 
-ThriftHiveMetastore_get_type_result::~ThriftHiveMetastore_get_type_result() throw() {
+ThriftHiveMetastore_get_database_result::~ThriftHiveMetastore_get_database_result() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_get_type_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_database_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -1952,11 +1653,11 @@ uint32_t ThriftHiveMetastore_get_type_result::read(::apache::thrift::protocol::T
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_type_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_database_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_type_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_database_result");
 
   if (this->__isset.success) {
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_STRUCT, 0);
@@ -1977,11 +1678,11 @@ uint32_t ThriftHiveMetastore_get_type_result::write(::apache::thrift::protocol::
 }
 
 
-ThriftHiveMetastore_get_type_presult::~ThriftHiveMetastore_get_type_presult() throw() {
+ThriftHiveMetastore_get_database_presult::~ThriftHiveMetastore_get_database_presult() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_get_type_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_database_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -2039,11 +1740,11 @@ uint32_t ThriftHiveMetastore_get_type_presult::read(::apache::thrift::protocol::
 }
 
 
-ThriftHiveMetastore_create_type_args::~ThriftHiveMetastore_create_type_args() throw() {
+ThriftHiveMetastore_drop_database_args::~ThriftHiveMetastore_drop_database_args() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_create_type_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_drop_database_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -2065,9 +1766,25 @@ uint32_t ThriftHiveMetastore_create_type_args::read(::apache::thrift::protocol::
     switch (fid)
     {
       case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->type.read(iprot);
-          this->__isset.type = true;
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->name);
+          this->__isset.name = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_BOOL) {
+          xfer += iprot->readBool(this->deleteData);
+          this->__isset.deleteData = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_BOOL) {
+          xfer += iprot->readBool(this->cascade);
+          this->__isset.cascade = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -2084,13 +1801,21 @@ uint32_t ThriftHiveMetastore_create_type_args::read(::apache::thrift::protocol::
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_create_type_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_drop_database_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_type_args");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_database_args");
 
-  xfer += oprot->writeFieldBegin("type", ::apache::thrift::protocol::T_STRUCT, 1);
-  xfer += this->type.write(oprot);
+  xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->name);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 2);
+  xfer += oprot->writeBool(this->deleteData);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("cascade", ::apache::thrift::protocol::T_BOOL, 3);
+  xfer += oprot->writeBool(this->cascade);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -2099,17 +1824,25 @@ uint32_t ThriftHiveMetastore_create_type_args::write(::apache::thrift::protocol:
 }
 
 
-ThriftHiveMetastore_create_type_pargs::~ThriftHiveMetastore_create_type_pargs() throw() {
+ThriftHiveMetastore_drop_database_pargs::~ThriftHiveMetastore_drop_database_pargs() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_create_type_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_drop_database_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_type_pargs");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_database_pargs");
 
-  xfer += oprot->writeFieldBegin("type", ::apache::thrift::protocol::T_STRUCT, 1);
-  xfer += (*(this->type)).write(oprot);
+  xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString((*(this->name)));
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("deleteData", ::apache::thrift::protocol::T_BOOL, 2);
+  xfer += oprot->writeBool((*(this->deleteData)));
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("cascade", ::apache::thrift::protocol::T_BOOL, 3);
+  xfer += oprot->writeBool((*(this->cascade)));
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -2118,11 +1851,11 @@ uint32_t ThriftHiveMetastore_create_type_pargs::write(::apache::thrift::protocol
 }
 
 
-ThriftHiveMetastore_create_type_result::~ThriftHiveMetastore_create_type_result() throw() {
+ThriftHiveMetastore_drop_database_result::~ThriftHiveMetastore_drop_database_result() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_create_type_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_drop_database_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -2143,14 +1876,6 @@ uint32_t ThriftHiveMetastore_create_type_result::read(::apache::thrift::protocol
     }
     switch (fid)
     {
-      case 0:
-        if (ftype == ::apache::thrift::protocol::T_BOOL) {
-          xfer += iprot->readBool(this->success);
-          this->__isset.success = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRUCT) {
           xfer += this->o1.read(iprot);
@@ -2187,17 +1912,13 @@ uint32_t ThriftHiveMetastore_create_type_result::read(::apache::thrift::protocol
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_create_type_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_drop_database_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_create_type_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_database_result");
 
-  if (this->__isset.success) {
-    xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0);
-    xfer += oprot->writeBool(this->success);
-    xfer += oprot->writeFieldEnd();
-  } else if (this->__isset.o1) {
+  if (this->__isset.o1) {
     xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1);
     xfer += this->o1.write(oprot);
     xfer += oprot->writeFieldEnd();
@@ -2216,11 +1937,11 @@ uint32_t ThriftHiveMetastore_create_type_result::write(::apache::thrift::protoco
 }
 
 
-ThriftHiveMetastore_create_type_presult::~ThriftHiveMetastore_create_type_presult() throw() {
+ThriftHiveMetastore_drop_database_presult::~ThriftHiveMetastore_drop_database_presult() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_create_type_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_drop_database_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -2241,14 +1962,6 @@ uint32_t ThriftHiveMetastore_create_type_presult::read(::apache::thrift::protoco
     }
     switch (fid)
     {
-      case 0:
-        if (ftype == ::apache::thrift::protocol::T_BOOL) {
-          xfer += iprot->readBool((*(this->success)));
-          this->__isset.success = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRUCT) {
           xfer += this->o1.read(iprot);
@@ -2286,11 +1999,11 @@ uint32_t ThriftHiveMetastore_create_type_presult::read(::apache::thrift::protoco
 }
 
 
-ThriftHiveMetastore_drop_type_args::~ThriftHiveMetastore_drop_type_args() throw() {
+ThriftHiveMetastore_get_databases_args::~ThriftHiveMetastore_get_databases_args() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_drop_type_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_databases_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -2313,8 +2026,8 @@ uint32_t ThriftHiveMetastore_drop_type_args::read(::apache::thrift::protocol::TP
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->type);
-          this->__isset.type = true;
+          xfer += iprot->readString(this->pattern);
+          this->__isset.pattern = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -2331,13 +2044,13 @@ uint32_t ThriftHiveMetastore_drop_type_args::read(::apache::thrift::protocol::TP
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_drop_type_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_databases_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_type_args");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_databases_args");
 
-  xfer += oprot->writeFieldBegin("type", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString(this->type);
+  xfer += oprot->writeFieldBegin("pattern", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->pattern);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -2346,17 +2059,17 @@ uint32_t ThriftHiveMetastore_drop_type_args::write(::apache::thrift::protocol::T
 }
 
 
-ThriftHiveMetastore_drop_type_pargs::~ThriftHiveMetastore_drop_type_pargs() throw() {
+ThriftHiveMetastore_get_databases_pargs::~ThriftHiveMetastore_get_databases_pargs() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_drop_type_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_databases_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_type_pargs");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_databases_pargs");
 
-  xfer += oprot->writeFieldBegin("type", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString((*(this->type)));
+  xfer += oprot->writeFieldBegin("pattern", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString((*(this->pattern)));
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -2365,11 +2078,11 @@ uint32_t ThriftHiveMetastore_drop_type_pargs::write(::apache::thrift::protocol::
 }
 
 
-ThriftHiveMetastore_drop_type_result::~ThriftHiveMetastore_drop_type_result() throw() {
+ThriftHiveMetastore_get_databases_result::~ThriftHiveMetastore_get_databases_result() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_drop_type_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_databases_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -2391,8 +2104,20 @@ uint32_t ThriftHiveMetastore_drop_type_result::read(::apache::thrift::protocol::
     switch (fid)
     {
       case 0:
-        if (ftype == ::apache::thrift::protocol::T_BOOL) {
-          xfer += iprot->readBool(this->success);
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->success.clear();
+            uint32_t _size1175;
+            ::apache::thrift::protocol::TType _etype1178;
+            xfer += iprot->readListBegin(_etype1178, _size1175);
+            this->success.resize(_size1175);
+            uint32_t _i1179;
+            for (_i1179 = 0; _i1179 < _size1175; ++_i1179)
+            {
+              xfer += iprot->readString(this->success[_i1179]);
+            }
+            xfer += iprot->readListEnd();
+          }
           this->__isset.success = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -2406,14 +2131,6 @@ uint32_t ThriftHiveMetastore_drop_type_result::read(::apache::thrift::protocol::
           xfer += iprot->skip(ftype);
         }
         break;
-      case 2:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o2.read(iprot);
-          this->__isset.o2 = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -2426,24 +2143,28 @@ uint32_t ThriftHiveMetastore_drop_type_result::read(::apache::thrift::protocol::
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_drop_type_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_databases_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_drop_type_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_databases_result");
 
   if (this->__isset.success) {
-    xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_BOOL, 0);
-    xfer += oprot->writeBool(this->success);
+    xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
+    {
+      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
+      std::vector<std::string> ::const_iterator _iter1180;
+      for (_iter1180 = this->success.begin(); _iter1180 != this->success.end(); ++_iter1180)
+      {
+        xfer += oprot->writeString((*_iter1180));
+      }
+      xfer += oprot->writeListEnd();
+    }
     xfer += oprot->writeFieldEnd();
   } else if (this->__isset.o1) {
     xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1);
     xfer += this->o1.write(oprot);
     xfer += oprot->writeFieldEnd();
-  } else if (this->__isset.o2) {
-    xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2);
-    xfer += this->o2.write(oprot);
-    xfer += oprot->writeFieldEnd();
   }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
@@ -2451,11 +2172,11 @@ uint32_t ThriftHiveMetastore_drop_type_result::write(::apache::thrift::protocol:
 }
 
 
-ThriftHiveMetastore_drop_type_presult::~ThriftHiveMetastore_drop_type_presult() throw() {
+ThriftHiveMetastore_get_databases_presult::~ThriftHiveMetastore_get_databases_presult() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_drop_type_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_databases_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -2477,8 +2198,20 @@ uint32_t ThriftHiveMetastore_drop_type_presult::read(::apache::thrift::protocol:
     switch (fid)
     {
       case 0:
-        if (ftype == ::apache::thrift::protocol::T_BOOL) {
-          xfer += iprot->readBool((*(this->success)));
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            (*(this->success)).clear();
+            uint32_t _size1181;
+            ::apache::thrift::protocol::TType _etype1184;
+            xfer += iprot->readListBegin(_etype1184, _size1181);
+            (*(this->success)).resize(_size1181);
+            uint32_t _i1185;
+            for (_i1185 = 0; _i1185 < _size1181; ++_i1185)
+            {
+              xfer += iprot->readString((*(this->success))[_i1185]);
+            }
+            xfer += iprot->readListEnd();
+          }
           this->__isset.success = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -2492,14 +2225,6 @@ uint32_t ThriftHiveMetastore_drop_type_presult::read(::apache::thrift::protocol:
           xfer += iprot->skip(ftype);
         }
         break;
-      case 2:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o2.read(iprot);
-          this->__isset.o2 = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -2513,11 +2238,11 @@ uint32_t ThriftHiveMetastore_drop_type_presult::read(::apache::thrift::protocol:
 }
 
 
-ThriftHiveMetastore_get_type_all_args::~ThriftHiveMetastore_get_type_all_args() throw() {
+ThriftHiveMetastore_get_all_databases_args::~ThriftHiveMetastore_get_all_databases_args() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_get_type_all_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_all_databases_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -2536,20 +2261,7 @@ uint32_t ThriftHiveMetastore_get_type_all_args::read(::apache::thrift::protocol:
     if (ftype == ::apache::thrift::protocol::T_STOP) {
       break;
     }
-    switch (fid)
-    {
-      case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->name);
-          this->__isset.name = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
+    xfer += iprot->skip(ftype);
     xfer += iprot->readFieldEnd();
   }
 
@@ -2558,14 +2270,10 @@ uint32_t ThriftHiveMetastore_get_type_all_args::read(::apache::thrift::protocol:
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_type_all_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_all_databases_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_type_all_args");
-
-  xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString(this->name);
-  xfer += oprot->writeFieldEnd();
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_databases_args");
 
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
@@ -2573,18 +2281,14 @@ uint32_t ThriftHiveMetastore_get_type_all_args::write(::apache::thrift::protocol
 }
 
 
-ThriftHiveMetastore_get_type_all_pargs::~ThriftHiveMetastore_get_type_all_pargs() throw() {
+ThriftHiveMetastore_get_all_databases_pargs::~ThriftHiveMetastore_get_all_databases_pargs() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_get_type_all_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_all_databases_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_type_all_pargs");
-
-  xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString((*(this->name)));
-  xfer += oprot->writeFieldEnd();
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_databases_pargs");
 
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
@@ -2592,11 +2296,11 @@ uint32_t ThriftHiveMetastore_get_type_all_pargs::write(::apache::thrift::protoco
 }
 
 
-ThriftHiveMetastore_get_type_all_result::~ThriftHiveMetastore_get_type_all_result() throw() {
+ThriftHiveMetastore_get_all_databases_result::~ThriftHiveMetastore_get_all_databases_result() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_get_type_all_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_all_databases_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -2618,22 +2322,19 @@ uint32_t ThriftHiveMetastore_get_type_all_result::read(::apache::thrift::protoco
     switch (fid)
     {
       case 0:
-        if (ftype == ::apache::thrift::protocol::T_MAP) {
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1179;
-            ::apache::thrift::protocol::TType _ktype1180;
-            ::apache::thrift::protocol::TType _vtype1181;
-            xfer += iprot->readMapBegin(_ktype1180, _vtype1181, _size1179);
-            uint32_t _i1183;
-            for (_i1183 = 0; _i1183 < _size1179; ++_i1183)
+            uint32_t _size1186;
+            ::apache::thrift::protocol::TType _etype1189;
+            xfer += iprot->readListBegin(_etype1189, _size1186);
+            this->success.resize(_size1186);
+            uint32_t _i1190;
+            for (_i1190 = 0; _i1190 < _size1186; ++_i1190)
             {
-              std::string _key1184;
-              xfer += iprot->readString(_key1184);
-              Type& _val1185 = this->success[_key1184];
-              xfer += _val1185.read(iprot);
+              xfer += iprot->readString(this->success[_i1190]);
             }
-            xfer += iprot->readMapEnd();
+            xfer += iprot->readListEnd();
           }
           this->__isset.success = true;
         } else {
@@ -2642,8 +2343,8 @@ uint32_t ThriftHiveMetastore_get_type_all_result::read(::apache::thrift::protoco
         break;
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o2.read(iprot);
-          this->__isset.o2 = true;
+          xfer += this->o1.read(iprot);
+          this->__isset.o1 = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -2660,28 +2361,27 @@ uint32_t ThriftHiveMetastore_get_type_all_result::read(::apache::thrift::protoco
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_type_all_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_all_databases_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_type_all_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_all_databases_result");
 
   if (this->__isset.success) {
-    xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0);
+    xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
-      xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::map<std::string, Type> ::const_iterator _iter1186;
-      for (_iter1186 = this->success.begin(); _iter1186 != this->success.end(); ++_iter1186)
+      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
+      std::vector<std::string> ::const_iterator _iter1191;
+      for (_iter1191 = this->success.begin(); _iter1191 != this->success.end(); ++_iter1191)
       {
-        xfer += oprot->writeString(_iter1186->first);
-        xfer += _iter1186->second.write(oprot);
+        xfer += oprot->writeString((*_iter1191));
       }
-      xfer += oprot->writeMapEnd();
+      xfer += oprot->writeListEnd();
     }
     xfer += oprot->writeFieldEnd();
-  } else if (this->__isset.o2) {
-    xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 1);
-    xfer += this->o2.write(oprot);
+  } else if (this->__isset.o1) {
+    xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1);
+    xfer += this->o1.write(oprot);
     xfer += oprot->writeFieldEnd();
   }
   xfer += oprot->writeFieldStop();
@@ -2690,11 +2390,11 @@ uint32_t ThriftHiveMetastore_get_type_all_result::write(::apache::thrift::protoc
 }
 
 
-ThriftHiveMetastore_get_type_all_presult::~ThriftHiveMetastore_get_type_all_presult() throw() {
+ThriftHiveMetastore_get_all_databases_presult::~ThriftHiveMetastore_get_all_databases_presult() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_get_type_all_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_all_databases_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -2716,22 +2416,19 @@ uint32_t ThriftHiveMetastore_get_type_all_presult::read(::apache::thrift::protoc
     switch (fid)
     {
       case 0:
-        if (ftype == ::apache::thrift::protocol::T_MAP) {
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1187;
-            ::apache::thrift::protocol::TType _ktype1188;
-            ::apache::thrift::protocol::TType _vtype1189;
-            xfer += iprot->readMapBegin(_ktype1188, _vtype1189, _size1187);
-            uint32_t _i1191;
-            for (_i1191 = 0; _i1191 < _size1187; ++_i1191)
+            uint32_t _size1192;
+            ::apache::thrift::protocol::TType _etype1195;
+            xfer += iprot->readListBegin(_etype1195, _size1192);
+            (*(this->success)).resize(_size1192);
+            uint32_t _i1196;
+            for (_i1196 = 0; _i1196 < _size1192; ++_i1196)
             {
-              std::string _key1192;
-              xfer += iprot->readString(_key1192);
-              Type& _val1193 = (*(this->success))[_key1192];
-              xfer += _val1193.read(iprot);
+              xfer += iprot->readString((*(this->success))[_i1196]);
             }
-            xfer += iprot->readMapEnd();
+            xfer += iprot->readListEnd();
           }
           this->__isset.success = true;
         } else {
@@ -2740,8 +2437,8 @@ uint32_t ThriftHiveMetastore_get_type_all_presult::read(::apache::thrift::protoc
         break;
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o2.read(iprot);
-          this->__isset.o2 = true;
+          xfer += this->o1.read(iprot);
+          this->__isset.o1 = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -2759,11 +2456,11 @@ uint32_t ThriftHiveMetastore_get_type_all_presult::read(::apache::thrift::protoc
 }
 
 
-ThriftHiveMetastore_get_fields_args::~ThriftHiveMetastore_get_fields_args() throw() {
+ThriftHiveMetastore_alter_database_args::~ThriftHiveMetastore_alter_database_args() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_get_fields_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_alter_database_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -2786,16 +2483,16 @@ uint32_t ThriftHiveMetastore_get_fields_args::read(::apache::thrift::protocol::T
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->db_name);
-          this->__isset.db_name = true;
+          xfer += iprot->readString(this->dbname);
+          this->__isset.dbname = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
       case 2:
-        if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->table_name);
-          this->__isset.table_name = true;
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->db.read(iprot);
+          this->__isset.db = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -2812,17 +2509,17 @@ uint32_t ThriftHiveMetastore_get_fields_args::read(::apache::thrift::protocol::T
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_fields_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_alter_database_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_fields_args");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_database_args");
 
-  xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString(this->db_name);
+  xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->dbname);
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2);
-  xfer += oprot->writeString(this->table_name);
+  xfer += oprot->writeFieldBegin("db", ::apache::thrift::protocol::T_STRUCT, 2);
+  xfer += this->db.write(oprot);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -2831,21 +2528,21 @@ uint32_t ThriftHiveMetastore_get_fields_args::write(::apache::thrift::protocol::
 }
 
 
-ThriftHiveMetastore_get_fields_pargs::~ThriftHiveMetastore_get_fields_pargs() throw() {
+ThriftHiveMetastore_alter_database_pargs::~ThriftHiveMetastore_alter_database_pargs() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_get_fields_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_alter_database_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_fields_pargs");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_database_pargs");
 
-  xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString((*(this->db_name)));
+  xfer += oprot->writeFieldBegin("dbname", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString((*(this->dbname)));
   xfer += oprot->writeFieldEnd();
 
-  xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2);
-  xfer += oprot->writeString((*(this->table_name)));
+  xfer += oprot->writeFieldBegin("db", ::apache::thrift::protocol::T_STRUCT, 2);
+  xfer += (*(this->db)).write(oprot);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -2854,11 +2551,11 @@ uint32_t ThriftHiveMetastore_get_fields_pargs::write(::apache::thrift::protocol:
 }
 
 
-ThriftHiveMetastore_get_fields_result::~ThriftHiveMetastore_get_fields_result() throw() {
+ThriftHiveMetastore_alter_database_result::~ThriftHiveMetastore_alter_database_result() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_get_fields_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_alter_database_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -2879,26 +2576,6 @@ uint32_t ThriftHiveMetastore_get_fields_result::read(::apache::thrift::protocol:
     }
     switch (fid)
     {
-      case 0:
-        if (ftype == ::apache::thrift::protocol::T_LIST) {
-          {
-            this->success.clear();
-            uint32_t _size1194;
-            ::apache::thrift::protocol::TType _etype1197;
-            xfer += iprot->readListBegin(_etype1197, _size1194);
-            this->success.resize(_size1194);
-            uint32_t _i1198;
-            for (_i1198 = 0; _i1198 < _size1194; ++_i1198)
-            {
-              xfer += this->success[_i1198].read(iprot);
-            }
-            xfer += iprot->readListEnd();
-          }
-          this->__isset.success = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRUCT) {
           xfer += this->o1.read(iprot);
@@ -2915,14 +2592,6 @@ uint32_t ThriftHiveMetastore_get_fields_result::read(::apache::thrift::protocol:
           xfer += iprot->skip(ftype);
         }
         break;
-      case 3:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o3.read(iprot);
-          this->__isset.o3 = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -2935,25 +2604,13 @@ uint32_t ThriftHiveMetastore_get_fields_result::read(::apache::thrift::protocol:
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_fields_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_alter_database_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_fields_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_alter_database_result");
 
-  if (this->__isset.success) {
-    xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
-    {
-      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter1199;
-      for (_iter1199 = this->success.begin(); _iter1199 != this->success.end(); ++_iter1199)
-      {
-        xfer += (*_iter1199).write(oprot);
-      }
-      xfer += oprot->writeListEnd();
-    }
-    xfer += oprot->writeFieldEnd();
-  } else if (this->__isset.o1) {
+  if (this->__isset.o1) {
     xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1);
     xfer += this->o1.write(oprot);
     xfer += oprot->writeFieldEnd();
@@ -2961,10 +2618,6 @@ uint32_t ThriftHiveMetastore_get_fields_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("o2", ::apache::thrift::protocol::T_STRUCT, 2);
     xfer += this->o2.write(oprot);
     xfer += oprot->writeFieldEnd();
-  } else if (this->__isset.o3) {
-    xfer += oprot->writeFieldBegin("o3", ::apache::thrift::protocol::T_STRUCT, 3);
-    xfer += this->o3.write(oprot);
-    xfer += oprot->writeFieldEnd();
   }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
@@ -2972,11 +2625,11 @@ uint32_t ThriftHiveMetastore_get_fields_result::write(::apache::thrift::protocol
 }
 
 
-ThriftHiveMetastore_get_fields_presult::~ThriftHiveMetastore_get_fields_presult() throw() {
+ThriftHiveMetastore_alter_database_presult::~ThriftHiveMetastore_alter_database_presult() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_get_fields_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_alter_database_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -2997,26 +2650,6 @@ uint32_t ThriftHiveMetastore_get_fields_presult::read(::apache::thrift::protocol
     }
     switch (fid)
     {
-      case 0:
-        if (ftype == ::apache::thrift::protocol::T_LIST) {
-          {
-            (*(this->success)).clear();
-            uint32_t _size1200;
-            ::apache::thrift::protocol::TType _etype1203;
-            xfer += iprot->readListBegin(_etype1203, _size1200);
-            (*(this->success)).resize(_size1200);
-            uint32_t _i1204;
-            for (_i1204 = 0; _i1204 < _size1200; ++_i1204)
-            {
-              xfer += (*(this->success))[_i1204].read(iprot);
-            }
-            xfer += iprot->readListEnd();
-          }
-          this->__isset.success = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRUCT) {
           xfer += this->o1.read(iprot);
@@ -3033,14 +2666,6 @@ uint32_t ThriftHiveMetastore_get_fields_presult::read(::apache::thrift::protocol
           xfer += iprot->skip(ftype);
         }
         break;
-      case 3:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o3.read(iprot);
-          this->__isset.o3 = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -3054,11 +2679,11 @@ uint32_t ThriftHiveMetastore_get_fields_presult::read(::apache::thrift::protocol
 }
 
 
-ThriftHiveMetastore_get_fields_with_environment_context_args::~ThriftHiveMetastore_get_fields_with_environment_context_args() throw() {
+ThriftHiveMetastore_get_type_args::~ThriftHiveMetastore_get_type_args() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_get_fields_with_environment_context_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_type_args::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -3081,24 +2706,8 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_args::read(::ap
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->db_name);
-          this->__isset.db_name = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 2:
-        if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->table_name);
-          this->__isset.table_name = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 3:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->environment_context.read(iprot);
-          this->__isset.environment_context = true;
+          xfer += iprot->readString(this->name);
+          this->__isset.name = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -3115,21 +2724,13 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_args::read(::ap
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_fields_with_environment_context_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_type_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_fields_with_environment_context_args");
-
-  xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString(this->db_name);
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2);
-  xfer += oprot->writeString(this->table_name);
-  xfer += oprot->writeFieldEnd();
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_type_args");
 
-  xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 3);
-  xfer += this->environment_context.write(oprot);
+  xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->name);
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -3138,25 +2739,17 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_args::write(::a
 }
 
 
-ThriftHiveMetastore_get_fields_with_environment_context_pargs::~ThriftHiveMetastore_get_fields_with_environment_context_pargs() throw() {
+ThriftHiveMetastore_get_type_pargs::~ThriftHiveMetastore_get_type_pargs() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_get_fields_with_environment_context_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_type_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
   uint32_t xfer = 0;
   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_fields_with_environment_context_pargs");
-
-  xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString((*(this->db_name)));
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldBegin("table_name", ::apache::thrift::protocol::T_STRING, 2);
-  xfer += oprot->writeString((*(this->table_name)));
-  xfer += oprot->writeFieldEnd();
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_type_pargs");
 
-  xfer += oprot->writeFieldBegin("environment_context", ::apache::thrift::protocol::T_STRUCT, 3);
-  xfer += (*(this->environment_context)).write(oprot);
+  xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString((*(this->name)));
   xfer += oprot->writeFieldEnd();
 
   xfer += oprot->writeFieldStop();
@@ -3165,11 +2758,11 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_pargs::write(::
 }
 
 
-ThriftHiveMetastore_get_fields_with_environment_context_result::~ThriftHiveMetastore_get_fields_with_environment_context_result() throw() {
+ThriftHiveMetastore_get_type_result::~ThriftHiveMetastore_get_type_result() throw() {
 }
 
 
-uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+uint32_t ThriftHiveMetastore_get_type_result::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
   uint32_t xfer = 0;
@@ -3191,20 +2784,8 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::read(::
     switch (fid)
     {
       case 0:
-        if (ftype == ::apache::thrift::protocol::T_LIST) {
-          {
-            this->success.clear();
-            uint32_t _size1205;
-            ::apache::thrift::protocol::TType _etype1208;
-            xfer += iprot->readListBegin(_etype1208, _size1205);
-            this->success.resize(_size1205);
-            uint32_t _i1209;
-            for (_i1209 = 0; _i1209 < _size1205; ++_i1209)
-            {
-              xfer += this->success[_i1209].read(iprot);
-            }
-            xfer += iprot->readListEnd();
-          }
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->success.read(iprot);
           this->__isset.success = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -3226,14 +2807,6 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::read(::
           xfer += iprot->skip(ftype);
         }
         break;
-      case 3:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o3.read(iprot);
-          this->__isset.o3 = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -3246,23 +2819,15 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::read(::
   return xfer;
 }
 
-uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+uint32_t ThriftHiveMetastore_get_type_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
 
   uint32_t xfer = 0;
 
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_fields_with_environment_context_result");
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_type_result");
 
   if (this->__isset.success) {
-    xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIS

<TRUNCATED>

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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropConstraintRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropConstraintRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropConstraintRequest.java
index 98f1531..a9c5892 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropConstraintRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropConstraintRequest.java
@@ -41,6 +41,7 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField DBNAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbname", org.apache.thrift.protocol.TType.STRING, (short)1);
   private static final org.apache.thrift.protocol.TField TABLENAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tablename", org.apache.thrift.protocol.TType.STRING, (short)2);
   private static final org.apache.thrift.protocol.TField CONSTRAINTNAME_FIELD_DESC = new org.apache.thrift.protocol.TField("constraintname", org.apache.thrift.protocol.TType.STRING, (short)3);
+  private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)4);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -51,12 +52,14 @@ import org.slf4j.LoggerFactory;
   private String dbname; // required
   private String tablename; // required
   private String constraintname; // required
+  private String catName; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
     DBNAME((short)1, "dbname"),
     TABLENAME((short)2, "tablename"),
-    CONSTRAINTNAME((short)3, "constraintname");
+    CONSTRAINTNAME((short)3, "constraintname"),
+    CAT_NAME((short)4, "catName");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -77,6 +80,8 @@ import org.slf4j.LoggerFactory;
           return TABLENAME;
         case 3: // CONSTRAINTNAME
           return CONSTRAINTNAME;
+        case 4: // CAT_NAME
+          return CAT_NAME;
         default:
           return null;
       }
@@ -117,6 +122,7 @@ import org.slf4j.LoggerFactory;
   }
 
   // isset id assignments
+  private static final _Fields optionals[] = {_Fields.CAT_NAME};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -126,6 +132,8 @@ import org.slf4j.LoggerFactory;
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.CONSTRAINTNAME, new org.apache.thrift.meta_data.FieldMetaData("constraintname", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(DropConstraintRequest.class, metaDataMap);
   }
@@ -157,6 +165,9 @@ import org.slf4j.LoggerFactory;
     if (other.isSetConstraintname()) {
       this.constraintname = other.constraintname;
     }
+    if (other.isSetCatName()) {
+      this.catName = other.catName;
+    }
   }
 
   public DropConstraintRequest deepCopy() {
@@ -168,6 +179,7 @@ import org.slf4j.LoggerFactory;
     this.dbname = null;
     this.tablename = null;
     this.constraintname = null;
+    this.catName = null;
   }
 
   public String getDbname() {
@@ -239,6 +251,29 @@ import org.slf4j.LoggerFactory;
     }
   }
 
+  public String getCatName() {
+    return this.catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
+  public void unsetCatName() {
+    this.catName = null;
+  }
+
+  /** Returns true if field catName is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatName() {
+    return this.catName != null;
+  }
+
+  public void setCatNameIsSet(boolean value) {
+    if (!value) {
+      this.catName = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case DBNAME:
@@ -265,6 +300,14 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case CAT_NAME:
+      if (value == null) {
+        unsetCatName();
+      } else {
+        setCatName((String)value);
+      }
+      break;
+
     }
   }
 
@@ -279,6 +322,9 @@ import org.slf4j.LoggerFactory;
     case CONSTRAINTNAME:
       return getConstraintname();
 
+    case CAT_NAME:
+      return getCatName();
+
     }
     throw new IllegalStateException();
   }
@@ -296,6 +342,8 @@ import org.slf4j.LoggerFactory;
       return isSetTablename();
     case CONSTRAINTNAME:
       return isSetConstraintname();
+    case CAT_NAME:
+      return isSetCatName();
     }
     throw new IllegalStateException();
   }
@@ -340,6 +388,15 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_catName = true && this.isSetCatName();
+    boolean that_present_catName = true && that.isSetCatName();
+    if (this_present_catName || that_present_catName) {
+      if (!(this_present_catName && that_present_catName))
+        return false;
+      if (!this.catName.equals(that.catName))
+        return false;
+    }
+
     return true;
   }
 
@@ -362,6 +419,11 @@ import org.slf4j.LoggerFactory;
     if (present_constraintname)
       list.add(constraintname);
 
+    boolean present_catName = true && (isSetCatName());
+    list.add(present_catName);
+    if (present_catName)
+      list.add(catName);
+
     return list.hashCode();
   }
 
@@ -403,6 +465,16 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetCatName()).compareTo(other.isSetCatName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catName, other.catName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -446,6 +518,16 @@ import org.slf4j.LoggerFactory;
       sb.append(this.constraintname);
     }
     first = false;
+    if (isSetCatName()) {
+      if (!first) sb.append(", ");
+      sb.append("catName:");
+      if (this.catName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.catName);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -525,6 +607,14 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 4: // CAT_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.catName = iprot.readString();
+              struct.setCatNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -553,6 +643,13 @@ import org.slf4j.LoggerFactory;
         oprot.writeString(struct.constraintname);
         oprot.writeFieldEnd();
       }
+      if (struct.catName != null) {
+        if (struct.isSetCatName()) {
+          oprot.writeFieldBegin(CAT_NAME_FIELD_DESC);
+          oprot.writeString(struct.catName);
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -573,6 +670,14 @@ import org.slf4j.LoggerFactory;
       oprot.writeString(struct.dbname);
       oprot.writeString(struct.tablename);
       oprot.writeString(struct.constraintname);
+      BitSet optionals = new BitSet();
+      if (struct.isSetCatName()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.isSetCatName()) {
+        oprot.writeString(struct.catName);
+      }
     }
 
     @Override
@@ -584,6 +689,11 @@ import org.slf4j.LoggerFactory;
       struct.setTablenameIsSet(true);
       struct.constraintname = iprot.readString();
       struct.setConstraintnameIsSet(true);
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.catName = iprot.readString();
+        struct.setCatNameIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsRequest.java
index 6927c78..443f08e 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsRequest.java
@@ -46,6 +46,7 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField IGNORE_PROTECTION_FIELD_DESC = new org.apache.thrift.protocol.TField("ignoreProtection", org.apache.thrift.protocol.TType.BOOL, (short)6);
   private static final org.apache.thrift.protocol.TField ENVIRONMENT_CONTEXT_FIELD_DESC = new org.apache.thrift.protocol.TField("environmentContext", org.apache.thrift.protocol.TType.STRUCT, (short)7);
   private static final org.apache.thrift.protocol.TField NEED_RESULT_FIELD_DESC = new org.apache.thrift.protocol.TField("needResult", org.apache.thrift.protocol.TType.BOOL, (short)8);
+  private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)9);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -61,6 +62,7 @@ import org.slf4j.LoggerFactory;
   private boolean ignoreProtection; // optional
   private EnvironmentContext environmentContext; // optional
   private boolean needResult; // optional
+  private String catName; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -71,7 +73,8 @@ import org.slf4j.LoggerFactory;
     IF_EXISTS((short)5, "ifExists"),
     IGNORE_PROTECTION((short)6, "ignoreProtection"),
     ENVIRONMENT_CONTEXT((short)7, "environmentContext"),
-    NEED_RESULT((short)8, "needResult");
+    NEED_RESULT((short)8, "needResult"),
+    CAT_NAME((short)9, "catName");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -102,6 +105,8 @@ import org.slf4j.LoggerFactory;
           return ENVIRONMENT_CONTEXT;
         case 8: // NEED_RESULT
           return NEED_RESULT;
+        case 9: // CAT_NAME
+          return CAT_NAME;
         default:
           return null;
       }
@@ -147,7 +152,7 @@ import org.slf4j.LoggerFactory;
   private static final int __IGNOREPROTECTION_ISSET_ID = 2;
   private static final int __NEEDRESULT_ISSET_ID = 3;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.DELETE_DATA,_Fields.IF_EXISTS,_Fields.IGNORE_PROTECTION,_Fields.ENVIRONMENT_CONTEXT,_Fields.NEED_RESULT};
+  private static final _Fields optionals[] = {_Fields.DELETE_DATA,_Fields.IF_EXISTS,_Fields.IGNORE_PROTECTION,_Fields.ENVIRONMENT_CONTEXT,_Fields.NEED_RESULT,_Fields.CAT_NAME};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -167,6 +172,8 @@ import org.slf4j.LoggerFactory;
         new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, EnvironmentContext.class)));
     tmpMap.put(_Fields.NEED_RESULT, new org.apache.thrift.meta_data.FieldMetaData("needResult", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(DropPartitionsRequest.class, metaDataMap);
   }
@@ -210,6 +217,9 @@ import org.slf4j.LoggerFactory;
       this.environmentContext = new EnvironmentContext(other.environmentContext);
     }
     this.needResult = other.needResult;
+    if (other.isSetCatName()) {
+      this.catName = other.catName;
+    }
   }
 
   public DropPartitionsRequest deepCopy() {
@@ -230,6 +240,7 @@ import org.slf4j.LoggerFactory;
     this.environmentContext = null;
     this.needResult = true;
 
+    this.catName = null;
   }
 
   public String getDbName() {
@@ -412,6 +423,29 @@ import org.slf4j.LoggerFactory;
     __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NEEDRESULT_ISSET_ID, value);
   }
 
+  public String getCatName() {
+    return this.catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
+  public void unsetCatName() {
+    this.catName = null;
+  }
+
+  /** Returns true if field catName is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatName() {
+    return this.catName != null;
+  }
+
+  public void setCatNameIsSet(boolean value) {
+    if (!value) {
+      this.catName = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case DB_NAME:
@@ -478,6 +512,14 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case CAT_NAME:
+      if (value == null) {
+        unsetCatName();
+      } else {
+        setCatName((String)value);
+      }
+      break;
+
     }
   }
 
@@ -507,6 +549,9 @@ import org.slf4j.LoggerFactory;
     case NEED_RESULT:
       return isNeedResult();
 
+    case CAT_NAME:
+      return getCatName();
+
     }
     throw new IllegalStateException();
   }
@@ -534,6 +579,8 @@ import org.slf4j.LoggerFactory;
       return isSetEnvironmentContext();
     case NEED_RESULT:
       return isSetNeedResult();
+    case CAT_NAME:
+      return isSetCatName();
     }
     throw new IllegalStateException();
   }
@@ -623,6 +670,15 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_catName = true && this.isSetCatName();
+    boolean that_present_catName = true && that.isSetCatName();
+    if (this_present_catName || that_present_catName) {
+      if (!(this_present_catName && that_present_catName))
+        return false;
+      if (!this.catName.equals(that.catName))
+        return false;
+    }
+
     return true;
   }
 
@@ -670,6 +726,11 @@ import org.slf4j.LoggerFactory;
     if (present_needResult)
       list.add(needResult);
 
+    boolean present_catName = true && (isSetCatName());
+    list.add(present_catName);
+    if (present_catName)
+      list.add(catName);
+
     return list.hashCode();
   }
 
@@ -761,6 +822,16 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetCatName()).compareTo(other.isSetCatName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catName, other.catName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -838,6 +909,16 @@ import org.slf4j.LoggerFactory;
       sb.append(this.needResult);
       first = false;
     }
+    if (isSetCatName()) {
+      if (!first) sb.append(", ");
+      sb.append("catName:");
+      if (this.catName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.catName);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -964,6 +1045,14 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 9: // CAT_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.catName = iprot.readString();
+              struct.setCatNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -1019,6 +1108,13 @@ import org.slf4j.LoggerFactory;
         oprot.writeBool(struct.needResult);
         oprot.writeFieldEnd();
       }
+      if (struct.catName != null) {
+        if (struct.isSetCatName()) {
+          oprot.writeFieldBegin(CAT_NAME_FIELD_DESC);
+          oprot.writeString(struct.catName);
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -1055,7 +1151,10 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetNeedResult()) {
         optionals.set(4);
       }
-      oprot.writeBitSet(optionals, 5);
+      if (struct.isSetCatName()) {
+        optionals.set(5);
+      }
+      oprot.writeBitSet(optionals, 6);
       if (struct.isSetDeleteData()) {
         oprot.writeBool(struct.deleteData);
       }
@@ -1071,6 +1170,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetNeedResult()) {
         oprot.writeBool(struct.needResult);
       }
+      if (struct.isSetCatName()) {
+        oprot.writeString(struct.catName);
+      }
     }
 
     @Override
@@ -1083,7 +1185,7 @@ import org.slf4j.LoggerFactory;
       struct.parts = new RequestPartsSpec();
       struct.parts.read(iprot);
       struct.setPartsIsSet(true);
-      BitSet incoming = iprot.readBitSet(5);
+      BitSet incoming = iprot.readBitSet(6);
       if (incoming.get(0)) {
         struct.deleteData = iprot.readBool();
         struct.setDeleteDataIsSet(true);
@@ -1105,6 +1207,10 @@ import org.slf4j.LoggerFactory;
         struct.needResult = iprot.readBool();
         struct.setNeedResultIsSet(true);
       }
+      if (incoming.get(5)) {
+        struct.catName = iprot.readString();
+        struct.setCatNameIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsResult.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsResult.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsResult.java
index e3f9161..0f22168 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsResult.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropPartitionsResult.java
@@ -346,14 +346,14 @@ import org.slf4j.LoggerFactory;
           case 1: // PARTITIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list482 = iprot.readListBegin();
-                struct.partitions = new ArrayList<Partition>(_list482.size);
-                Partition _elem483;
-                for (int _i484 = 0; _i484 < _list482.size; ++_i484)
+                org.apache.thrift.protocol.TList _list490 = iprot.readListBegin();
+                struct.partitions = new ArrayList<Partition>(_list490.size);
+                Partition _elem491;
+                for (int _i492 = 0; _i492 < _list490.size; ++_i492)
                 {
-                  _elem483 = new Partition();
-                  _elem483.read(iprot);
-                  struct.partitions.add(_elem483);
+                  _elem491 = new Partition();
+                  _elem491.read(iprot);
+                  struct.partitions.add(_elem491);
                 }
                 iprot.readListEnd();
               }
@@ -380,9 +380,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PARTITIONS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitions.size()));
-            for (Partition _iter485 : struct.partitions)
+            for (Partition _iter493 : struct.partitions)
             {
-              _iter485.write(oprot);
+              _iter493.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetPartitions()) {
         {
           oprot.writeI32(struct.partitions.size());
-          for (Partition _iter486 : struct.partitions)
+          for (Partition _iter494 : struct.partitions)
           {
-            _iter486.write(oprot);
+            _iter494.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list487 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.partitions = new ArrayList<Partition>(_list487.size);
-          Partition _elem488;
-          for (int _i489 = 0; _i489 < _list487.size; ++_i489)
+          org.apache.thrift.protocol.TList _list495 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.partitions = new ArrayList<Partition>(_list495.size);
+          Partition _elem496;
+          for (int _i497 = 0; _i497 < _list495.size; ++_i497)
           {
-            _elem488 = new Partition();
-            _elem488.read(iprot);
-            struct.partitions.add(_elem488);
+            _elem496 = new Partition();
+            _elem496.read(iprot);
+            struct.partitions.add(_elem496);
           }
         }
         struct.setPartitionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/EnvironmentContext.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/EnvironmentContext.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/EnvironmentContext.java
index e420b9e..52fae26 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/EnvironmentContext.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/EnvironmentContext.java
@@ -344,15 +344,15 @@ import org.slf4j.LoggerFactory;
           case 1: // PROPERTIES
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map302 = iprot.readMapBegin();
-                struct.properties = new HashMap<String,String>(2*_map302.size);
-                String _key303;
-                String _val304;
-                for (int _i305 = 0; _i305 < _map302.size; ++_i305)
+                org.apache.thrift.protocol.TMap _map310 = iprot.readMapBegin();
+                struct.properties = new HashMap<String,String>(2*_map310.size);
+                String _key311;
+                String _val312;
+                for (int _i313 = 0; _i313 < _map310.size; ++_i313)
                 {
-                  _key303 = iprot.readString();
-                  _val304 = iprot.readString();
-                  struct.properties.put(_key303, _val304);
+                  _key311 = iprot.readString();
+                  _val312 = iprot.readString();
+                  struct.properties.put(_key311, _val312);
                 }
                 iprot.readMapEnd();
               }
@@ -378,10 +378,10 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PROPERTIES_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.properties.size()));
-          for (Map.Entry<String, String> _iter306 : struct.properties.entrySet())
+          for (Map.Entry<String, String> _iter314 : struct.properties.entrySet())
           {
-            oprot.writeString(_iter306.getKey());
-            oprot.writeString(_iter306.getValue());
+            oprot.writeString(_iter314.getKey());
+            oprot.writeString(_iter314.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -412,10 +412,10 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetProperties()) {
         {
           oprot.writeI32(struct.properties.size());
-          for (Map.Entry<String, String> _iter307 : struct.properties.entrySet())
+          for (Map.Entry<String, String> _iter315 : struct.properties.entrySet())
           {
-            oprot.writeString(_iter307.getKey());
-            oprot.writeString(_iter307.getValue());
+            oprot.writeString(_iter315.getKey());
+            oprot.writeString(_iter315.getValue());
           }
         }
       }
@@ -427,15 +427,15 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TMap _map308 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.properties = new HashMap<String,String>(2*_map308.size);
-          String _key309;
-          String _val310;
-          for (int _i311 = 0; _i311 < _map308.size; ++_i311)
+          org.apache.thrift.protocol.TMap _map316 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.properties = new HashMap<String,String>(2*_map316.size);
+          String _key317;
+          String _val318;
+          for (int _i319 = 0; _i319 < _map316.size; ++_i319)
           {
-            _key309 = iprot.readString();
-            _val310 = iprot.readString();
-            struct.properties.put(_key309, _val310);
+            _key317 = iprot.readString();
+            _val318 = iprot.readString();
+            struct.properties.put(_key317, _val318);
           }
         }
         struct.setPropertiesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java
index 807f826..b95efc7 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java
@@ -350,14 +350,14 @@ import org.slf4j.LoggerFactory;
           case 1: // SCHEMA_VERSIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list888 = iprot.readListBegin();
-                struct.schemaVersions = new ArrayList<SchemaVersionDescriptor>(_list888.size);
-                SchemaVersionDescriptor _elem889;
-                for (int _i890 = 0; _i890 < _list888.size; ++_i890)
+                org.apache.thrift.protocol.TList _list896 = iprot.readListBegin();
+                struct.schemaVersions = new ArrayList<SchemaVersionDescriptor>(_list896.size);
+                SchemaVersionDescriptor _elem897;
+                for (int _i898 = 0; _i898 < _list896.size; ++_i898)
                 {
-                  _elem889 = new SchemaVersionDescriptor();
-                  _elem889.read(iprot);
-                  struct.schemaVersions.add(_elem889);
+                  _elem897 = new SchemaVersionDescriptor();
+                  _elem897.read(iprot);
+                  struct.schemaVersions.add(_elem897);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(SCHEMA_VERSIONS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.schemaVersions.size()));
-          for (SchemaVersionDescriptor _iter891 : struct.schemaVersions)
+          for (SchemaVersionDescriptor _iter899 : struct.schemaVersions)
           {
-            _iter891.write(oprot);
+            _iter899.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -416,9 +416,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetSchemaVersions()) {
         {
           oprot.writeI32(struct.schemaVersions.size());
-          for (SchemaVersionDescriptor _iter892 : struct.schemaVersions)
+          for (SchemaVersionDescriptor _iter900 : struct.schemaVersions)
           {
-            _iter892.write(oprot);
+            _iter900.write(oprot);
           }
         }
       }
@@ -430,14 +430,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list893 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.schemaVersions = new ArrayList<SchemaVersionDescriptor>(_list893.size);
-          SchemaVersionDescriptor _elem894;
-          for (int _i895 = 0; _i895 < _list893.size; ++_i895)
+          org.apache.thrift.protocol.TList _list901 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.schemaVersions = new ArrayList<SchemaVersionDescriptor>(_list901.size);
+          SchemaVersionDescriptor _elem902;
+          for (int _i903 = 0; _i903 < _list901.size; ++_i903)
           {
-            _elem894 = new SchemaVersionDescriptor();
-            _elem894.read(iprot);
-            struct.schemaVersions.add(_elem894);
+            _elem902 = new SchemaVersionDescriptor();
+            _elem902.read(iprot);
+            struct.schemaVersions.add(_elem902);
           }
         }
         struct.setSchemaVersionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
index 58b1d7c..ddc0a6a 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
@@ -43,6 +43,7 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbName", org.apache.thrift.protocol.TType.STRING, (short)3);
   private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)4);
   private static final org.apache.thrift.protocol.TField PARTITION_VALS_FIELD_DESC = new org.apache.thrift.protocol.TField("partitionVals", org.apache.thrift.protocol.TType.LIST, (short)5);
+  private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)6);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -55,6 +56,7 @@ import org.slf4j.LoggerFactory;
   private String dbName; // optional
   private String tableName; // optional
   private List<String> partitionVals; // optional
+  private String catName; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -62,7 +64,8 @@ import org.slf4j.LoggerFactory;
     DATA((short)2, "data"),
     DB_NAME((short)3, "dbName"),
     TABLE_NAME((short)4, "tableName"),
-    PARTITION_VALS((short)5, "partitionVals");
+    PARTITION_VALS((short)5, "partitionVals"),
+    CAT_NAME((short)6, "catName");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -87,6 +90,8 @@ import org.slf4j.LoggerFactory;
           return TABLE_NAME;
         case 5: // PARTITION_VALS
           return PARTITION_VALS;
+        case 6: // CAT_NAME
+          return CAT_NAME;
         default:
           return null;
       }
@@ -129,7 +134,7 @@ import org.slf4j.LoggerFactory;
   // isset id assignments
   private static final int __SUCCESSFUL_ISSET_ID = 0;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.DB_NAME,_Fields.TABLE_NAME,_Fields.PARTITION_VALS};
+  private static final _Fields optionals[] = {_Fields.DB_NAME,_Fields.TABLE_NAME,_Fields.PARTITION_VALS,_Fields.CAT_NAME};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -144,6 +149,8 @@ import org.slf4j.LoggerFactory;
     tmpMap.put(_Fields.PARTITION_VALS, new org.apache.thrift.meta_data.FieldMetaData("partitionVals", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
             new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    tmpMap.put(_Fields.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(FireEventRequest.class, metaDataMap);
   }
@@ -180,6 +187,9 @@ import org.slf4j.LoggerFactory;
       List<String> __this__partitionVals = new ArrayList<String>(other.partitionVals);
       this.partitionVals = __this__partitionVals;
     }
+    if (other.isSetCatName()) {
+      this.catName = other.catName;
+    }
   }
 
   public FireEventRequest deepCopy() {
@@ -194,6 +204,7 @@ import org.slf4j.LoggerFactory;
     this.dbName = null;
     this.tableName = null;
     this.partitionVals = null;
+    this.catName = null;
   }
 
   public boolean isSuccessful() {
@@ -325,6 +336,29 @@ import org.slf4j.LoggerFactory;
     }
   }
 
+  public String getCatName() {
+    return this.catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
+  public void unsetCatName() {
+    this.catName = null;
+  }
+
+  /** Returns true if field catName is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatName() {
+    return this.catName != null;
+  }
+
+  public void setCatNameIsSet(boolean value) {
+    if (!value) {
+      this.catName = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case SUCCESSFUL:
@@ -367,6 +401,14 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case CAT_NAME:
+      if (value == null) {
+        unsetCatName();
+      } else {
+        setCatName((String)value);
+      }
+      break;
+
     }
   }
 
@@ -387,6 +429,9 @@ import org.slf4j.LoggerFactory;
     case PARTITION_VALS:
       return getPartitionVals();
 
+    case CAT_NAME:
+      return getCatName();
+
     }
     throw new IllegalStateException();
   }
@@ -408,6 +453,8 @@ import org.slf4j.LoggerFactory;
       return isSetTableName();
     case PARTITION_VALS:
       return isSetPartitionVals();
+    case CAT_NAME:
+      return isSetCatName();
     }
     throw new IllegalStateException();
   }
@@ -470,6 +517,15 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_catName = true && this.isSetCatName();
+    boolean that_present_catName = true && that.isSetCatName();
+    if (this_present_catName || that_present_catName) {
+      if (!(this_present_catName && that_present_catName))
+        return false;
+      if (!this.catName.equals(that.catName))
+        return false;
+    }
+
     return true;
   }
 
@@ -502,6 +558,11 @@ import org.slf4j.LoggerFactory;
     if (present_partitionVals)
       list.add(partitionVals);
 
+    boolean present_catName = true && (isSetCatName());
+    list.add(present_catName);
+    if (present_catName)
+      list.add(catName);
+
     return list.hashCode();
   }
 
@@ -563,6 +624,16 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetCatName()).compareTo(other.isSetCatName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catName, other.catName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -624,6 +695,16 @@ import org.slf4j.LoggerFactory;
       }
       first = false;
     }
+    if (isSetCatName()) {
+      if (!first) sb.append(", ");
+      sb.append("catName:");
+      if (this.catName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.catName);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -713,13 +794,13 @@ import org.slf4j.LoggerFactory;
           case 5: // PARTITION_VALS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list708 = iprot.readListBegin();
-                struct.partitionVals = new ArrayList<String>(_list708.size);
-                String _elem709;
-                for (int _i710 = 0; _i710 < _list708.size; ++_i710)
+                org.apache.thrift.protocol.TList _list716 = iprot.readListBegin();
+                struct.partitionVals = new ArrayList<String>(_list716.size);
+                String _elem717;
+                for (int _i718 = 0; _i718 < _list716.size; ++_i718)
                 {
-                  _elem709 = iprot.readString();
-                  struct.partitionVals.add(_elem709);
+                  _elem717 = iprot.readString();
+                  struct.partitionVals.add(_elem717);
                 }
                 iprot.readListEnd();
               }
@@ -728,6 +809,14 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 6: // CAT_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.catName = iprot.readString();
+              struct.setCatNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -768,15 +857,22 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PARTITION_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partitionVals.size()));
-            for (String _iter711 : struct.partitionVals)
+            for (String _iter719 : struct.partitionVals)
             {
-              oprot.writeString(_iter711);
+              oprot.writeString(_iter719);
             }
             oprot.writeListEnd();
           }
           oprot.writeFieldEnd();
         }
       }
+      if (struct.catName != null) {
+        if (struct.isSetCatName()) {
+          oprot.writeFieldBegin(CAT_NAME_FIELD_DESC);
+          oprot.writeString(struct.catName);
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -806,7 +902,10 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetPartitionVals()) {
         optionals.set(2);
       }
-      oprot.writeBitSet(optionals, 3);
+      if (struct.isSetCatName()) {
+        optionals.set(3);
+      }
+      oprot.writeBitSet(optionals, 4);
       if (struct.isSetDbName()) {
         oprot.writeString(struct.dbName);
       }
@@ -816,12 +915,15 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetPartitionVals()) {
         {
           oprot.writeI32(struct.partitionVals.size());
-          for (String _iter712 : struct.partitionVals)
+          for (String _iter720 : struct.partitionVals)
           {
-            oprot.writeString(_iter712);
+            oprot.writeString(_iter720);
           }
         }
       }
+      if (struct.isSetCatName()) {
+        oprot.writeString(struct.catName);
+      }
     }
 
     @Override
@@ -832,7 +934,7 @@ import org.slf4j.LoggerFactory;
       struct.data = new FireEventRequestData();
       struct.data.read(iprot);
       struct.setDataIsSet(true);
-      BitSet incoming = iprot.readBitSet(3);
+      BitSet incoming = iprot.readBitSet(4);
       if (incoming.get(0)) {
         struct.dbName = iprot.readString();
         struct.setDbNameIsSet(true);
@@ -843,17 +945,21 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TList _list713 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.partitionVals = new ArrayList<String>(_list713.size);
-          String _elem714;
-          for (int _i715 = 0; _i715 < _list713.size; ++_i715)
+          org.apache.thrift.protocol.TList _list721 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.partitionVals = new ArrayList<String>(_list721.size);
+          String _elem722;
+          for (int _i723 = 0; _i723 < _list721.size; ++_i723)
           {
-            _elem714 = iprot.readString();
-            struct.partitionVals.add(_elem714);
+            _elem722 = iprot.readString();
+            struct.partitionVals.add(_elem722);
           }
         }
         struct.setPartitionValsIsSet(true);
       }
+      if (incoming.get(3)) {
+        struct.catName = iprot.readString();
+        struct.setCatNameIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ForeignKeysRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ForeignKeysRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ForeignKeysRequest.java
index e4882c7..2f2fcfa 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ForeignKeysRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ForeignKeysRequest.java
@@ -42,6 +42,7 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField PARENT_TBL_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("parent_tbl_name", org.apache.thrift.protocol.TType.STRING, (short)2);
   private static final org.apache.thrift.protocol.TField FOREIGN_DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("foreign_db_name", org.apache.thrift.protocol.TType.STRING, (short)3);
   private static final org.apache.thrift.protocol.TField FOREIGN_TBL_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("foreign_tbl_name", org.apache.thrift.protocol.TType.STRING, (short)4);
+  private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)5);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -53,13 +54,15 @@ import org.slf4j.LoggerFactory;
   private String parent_tbl_name; // required
   private String foreign_db_name; // required
   private String foreign_tbl_name; // required
+  private String catName; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
     PARENT_DB_NAME((short)1, "parent_db_name"),
     PARENT_TBL_NAME((short)2, "parent_tbl_name"),
     FOREIGN_DB_NAME((short)3, "foreign_db_name"),
-    FOREIGN_TBL_NAME((short)4, "foreign_tbl_name");
+    FOREIGN_TBL_NAME((short)4, "foreign_tbl_name"),
+    CAT_NAME((short)5, "catName");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -82,6 +85,8 @@ import org.slf4j.LoggerFactory;
           return FOREIGN_DB_NAME;
         case 4: // FOREIGN_TBL_NAME
           return FOREIGN_TBL_NAME;
+        case 5: // CAT_NAME
+          return CAT_NAME;
         default:
           return null;
       }
@@ -122,6 +127,7 @@ import org.slf4j.LoggerFactory;
   }
 
   // isset id assignments
+  private static final _Fields optionals[] = {_Fields.CAT_NAME};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -133,6 +139,8 @@ import org.slf4j.LoggerFactory;
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.FOREIGN_TBL_NAME, new org.apache.thrift.meta_data.FieldMetaData("foreign_tbl_name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ForeignKeysRequest.class, metaDataMap);
   }
@@ -169,6 +177,9 @@ import org.slf4j.LoggerFactory;
     if (other.isSetForeign_tbl_name()) {
       this.foreign_tbl_name = other.foreign_tbl_name;
     }
+    if (other.isSetCatName()) {
+      this.catName = other.catName;
+    }
   }
 
   public ForeignKeysRequest deepCopy() {
@@ -181,6 +192,7 @@ import org.slf4j.LoggerFactory;
     this.parent_tbl_name = null;
     this.foreign_db_name = null;
     this.foreign_tbl_name = null;
+    this.catName = null;
   }
 
   public String getParent_db_name() {
@@ -275,6 +287,29 @@ import org.slf4j.LoggerFactory;
     }
   }
 
+  public String getCatName() {
+    return this.catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
+  public void unsetCatName() {
+    this.catName = null;
+  }
+
+  /** Returns true if field catName is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatName() {
+    return this.catName != null;
+  }
+
+  public void setCatNameIsSet(boolean value) {
+    if (!value) {
+      this.catName = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case PARENT_DB_NAME:
@@ -309,6 +344,14 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case CAT_NAME:
+      if (value == null) {
+        unsetCatName();
+      } else {
+        setCatName((String)value);
+      }
+      break;
+
     }
   }
 
@@ -326,6 +369,9 @@ import org.slf4j.LoggerFactory;
     case FOREIGN_TBL_NAME:
       return getForeign_tbl_name();
 
+    case CAT_NAME:
+      return getCatName();
+
     }
     throw new IllegalStateException();
   }
@@ -345,6 +391,8 @@ import org.slf4j.LoggerFactory;
       return isSetForeign_db_name();
     case FOREIGN_TBL_NAME:
       return isSetForeign_tbl_name();
+    case CAT_NAME:
+      return isSetCatName();
     }
     throw new IllegalStateException();
   }
@@ -398,6 +446,15 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_catName = true && this.isSetCatName();
+    boolean that_present_catName = true && that.isSetCatName();
+    if (this_present_catName || that_present_catName) {
+      if (!(this_present_catName && that_present_catName))
+        return false;
+      if (!this.catName.equals(that.catName))
+        return false;
+    }
+
     return true;
   }
 
@@ -425,6 +482,11 @@ import org.slf4j.LoggerFactory;
     if (present_foreign_tbl_name)
       list.add(foreign_tbl_name);
 
+    boolean present_catName = true && (isSetCatName());
+    list.add(present_catName);
+    if (present_catName)
+      list.add(catName);
+
     return list.hashCode();
   }
 
@@ -476,6 +538,16 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetCatName()).compareTo(other.isSetCatName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catName, other.catName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -527,6 +599,16 @@ import org.slf4j.LoggerFactory;
       sb.append(this.foreign_tbl_name);
     }
     first = false;
+    if (isSetCatName()) {
+      if (!first) sb.append(", ");
+      sb.append("catName:");
+      if (this.catName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.catName);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -602,6 +684,14 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 5: // CAT_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.catName = iprot.readString();
+              struct.setCatNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -635,6 +725,13 @@ import org.slf4j.LoggerFactory;
         oprot.writeString(struct.foreign_tbl_name);
         oprot.writeFieldEnd();
       }
+      if (struct.catName != null) {
+        if (struct.isSetCatName()) {
+          oprot.writeFieldBegin(CAT_NAME_FIELD_DESC);
+          oprot.writeString(struct.catName);
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -665,7 +762,10 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetForeign_tbl_name()) {
         optionals.set(3);
       }
-      oprot.writeBitSet(optionals, 4);
+      if (struct.isSetCatName()) {
+        optionals.set(4);
+      }
+      oprot.writeBitSet(optionals, 5);
       if (struct.isSetParent_db_name()) {
         oprot.writeString(struct.parent_db_name);
       }
@@ -678,12 +778,15 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetForeign_tbl_name()) {
         oprot.writeString(struct.foreign_tbl_name);
       }
+      if (struct.isSetCatName()) {
+        oprot.writeString(struct.catName);
+      }
     }
 
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, ForeignKeysRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
-      BitSet incoming = iprot.readBitSet(4);
+      BitSet incoming = iprot.readBitSet(5);
       if (incoming.get(0)) {
         struct.parent_db_name = iprot.readString();
         struct.setParent_db_nameIsSet(true);
@@ -700,6 +803,10 @@ import org.slf4j.LoggerFactory;
         struct.foreign_tbl_name = iprot.readString();
         struct.setForeign_tbl_nameIsSet(true);
       }
+      if (incoming.get(4)) {
+        struct.catName = iprot.readString();
+        struct.setCatNameIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ForeignKeysResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ForeignKeysResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ForeignKeysResponse.java
index 081adeb..2890506 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ForeignKeysResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ForeignKeysResponse.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // FOREIGN_KEYS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list320 = iprot.readListBegin();
-                struct.foreignKeys = new ArrayList<SQLForeignKey>(_list320.size);
-                SQLForeignKey _elem321;
-                for (int _i322 = 0; _i322 < _list320.size; ++_i322)
+                org.apache.thrift.protocol.TList _list328 = iprot.readListBegin();
+                struct.foreignKeys = new ArrayList<SQLForeignKey>(_list328.size);
+                SQLForeignKey _elem329;
+                for (int _i330 = 0; _i330 < _list328.size; ++_i330)
                 {
-                  _elem321 = new SQLForeignKey();
-                  _elem321.read(iprot);
-                  struct.foreignKeys.add(_elem321);
+                  _elem329 = new SQLForeignKey();
+                  _elem329.read(iprot);
+                  struct.foreignKeys.add(_elem329);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FOREIGN_KEYS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.foreignKeys.size()));
-          for (SQLForeignKey _iter323 : struct.foreignKeys)
+          for (SQLForeignKey _iter331 : struct.foreignKeys)
           {
-            _iter323.write(oprot);
+            _iter331.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.foreignKeys.size());
-        for (SQLForeignKey _iter324 : struct.foreignKeys)
+        for (SQLForeignKey _iter332 : struct.foreignKeys)
         {
-          _iter324.write(oprot);
+          _iter332.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, ForeignKeysResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list325 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.foreignKeys = new ArrayList<SQLForeignKey>(_list325.size);
-        SQLForeignKey _elem326;
-        for (int _i327 = 0; _i327 < _list325.size; ++_i327)
+        org.apache.thrift.protocol.TList _list333 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.foreignKeys = new ArrayList<SQLForeignKey>(_list333.size);
+        SQLForeignKey _elem334;
+        for (int _i335 = 0; _i335 < _list333.size; ++_i335)
         {
-          _elem326 = new SQLForeignKey();
-          _elem326.read(iprot);
-          struct.foreignKeys.add(_elem326);
+          _elem334 = new SQLForeignKey();
+          _elem334.read(iprot);
+          struct.foreignKeys.add(_elem334);
         }
       }
       struct.setForeignKeysIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java
index 2a6c28d..a1c0de9 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java
@@ -46,6 +46,7 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField CREATE_TIME_FIELD_DESC = new org.apache.thrift.protocol.TField("createTime", org.apache.thrift.protocol.TType.I32, (short)6);
   private static final org.apache.thrift.protocol.TField FUNCTION_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("functionType", org.apache.thrift.protocol.TType.I32, (short)7);
   private static final org.apache.thrift.protocol.TField RESOURCE_URIS_FIELD_DESC = new org.apache.thrift.protocol.TField("resourceUris", org.apache.thrift.protocol.TType.LIST, (short)8);
+  private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)9);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -61,6 +62,7 @@ import org.slf4j.LoggerFactory;
   private int createTime; // required
   private FunctionType functionType; // required
   private List<ResourceUri> resourceUris; // required
+  private String catName; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -79,7 +81,8 @@ import org.slf4j.LoggerFactory;
      * @see FunctionType
      */
     FUNCTION_TYPE((short)7, "functionType"),
-    RESOURCE_URIS((short)8, "resourceUris");
+    RESOURCE_URIS((short)8, "resourceUris"),
+    CAT_NAME((short)9, "catName");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -110,6 +113,8 @@ import org.slf4j.LoggerFactory;
           return FUNCTION_TYPE;
         case 8: // RESOURCE_URIS
           return RESOURCE_URIS;
+        case 9: // CAT_NAME
+          return CAT_NAME;
         default:
           return null;
       }
@@ -152,6 +157,7 @@ import org.slf4j.LoggerFactory;
   // isset id assignments
   private static final int __CREATETIME_ISSET_ID = 0;
   private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.CAT_NAME};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -172,6 +178,8 @@ import org.slf4j.LoggerFactory;
     tmpMap.put(_Fields.RESOURCE_URIS, new org.apache.thrift.meta_data.FieldMetaData("resourceUris", org.apache.thrift.TFieldRequirementType.DEFAULT, 
         new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
             new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ResourceUri.class))));
+    tmpMap.put(_Fields.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(Function.class, metaDataMap);
   }
@@ -232,6 +240,9 @@ import org.slf4j.LoggerFactory;
       }
       this.resourceUris = __this__resourceUris;
     }
+    if (other.isSetCatName()) {
+      this.catName = other.catName;
+    }
   }
 
   public Function deepCopy() {
@@ -249,6 +260,7 @@ import org.slf4j.LoggerFactory;
     this.createTime = 0;
     this.functionType = null;
     this.resourceUris = null;
+    this.catName = null;
   }
 
   public String getFunctionName() {
@@ -465,6 +477,29 @@ import org.slf4j.LoggerFactory;
     }
   }
 
+  public String getCatName() {
+    return this.catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
+  public void unsetCatName() {
+    this.catName = null;
+  }
+
+  /** Returns true if field catName is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatName() {
+    return this.catName != null;
+  }
+
+  public void setCatNameIsSet(boolean value) {
+    if (!value) {
+      this.catName = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case FUNCTION_NAME:
@@ -531,6 +566,14 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case CAT_NAME:
+      if (value == null) {
+        unsetCatName();
+      } else {
+        setCatName((String)value);
+      }
+      break;
+
     }
   }
 
@@ -560,6 +603,9 @@ import org.slf4j.LoggerFactory;
     case RESOURCE_URIS:
       return getResourceUris();
 
+    case CAT_NAME:
+      return getCatName();
+
     }
     throw new IllegalStateException();
   }
@@ -587,6 +633,8 @@ import org.slf4j.LoggerFactory;
       return isSetFunctionType();
     case RESOURCE_URIS:
       return isSetResourceUris();
+    case CAT_NAME:
+      return isSetCatName();
     }
     throw new IllegalStateException();
   }
@@ -676,6 +724,15 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_catName = true && this.isSetCatName();
+    boolean that_present_catName = true && that.isSetCatName();
+    if (this_present_catName || that_present_catName) {
+      if (!(this_present_catName && that_present_catName))
+        return false;
+      if (!this.catName.equals(that.catName))
+        return false;
+    }
+
     return true;
   }
 
@@ -723,6 +780,11 @@ import org.slf4j.LoggerFactory;
     if (present_resourceUris)
       list.add(resourceUris);
 
+    boolean present_catName = true && (isSetCatName());
+    list.add(present_catName);
+    if (present_catName)
+      list.add(catName);
+
     return list.hashCode();
   }
 
@@ -814,6 +876,16 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetCatName()).compareTo(other.isSetCatName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catName, other.catName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -893,6 +965,16 @@ import org.slf4j.LoggerFactory;
       sb.append(this.resourceUris);
     }
     first = false;
+    if (isSetCatName()) {
+      if (!first) sb.append(", ");
+      sb.append("catName:");
+      if (this.catName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.catName);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -997,14 +1079,14 @@ import org.slf4j.LoggerFactory;
           case 8: // RESOURCE_URIS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list538 = iprot.readListBegin();
-                struct.resourceUris = new ArrayList<ResourceUri>(_list538.size);
-                ResourceUri _elem539;
-                for (int _i540 = 0; _i540 < _list538.size; ++_i540)
+                org.apache.thrift.protocol.TList _list546 = iprot.readListBegin();
+                struct.resourceUris = new ArrayList<ResourceUri>(_list546.size);
+                ResourceUri _elem547;
+                for (int _i548 = 0; _i548 < _list546.size; ++_i548)
                 {
-                  _elem539 = new ResourceUri();
-                  _elem539.read(iprot);
-                  struct.resourceUris.add(_elem539);
+                  _elem547 = new ResourceUri();
+                  _elem547.read(iprot);
+                  struct.resourceUris.add(_elem547);
                 }
                 iprot.readListEnd();
               }
@@ -1013,6 +1095,14 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 9: // CAT_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.catName = iprot.readString();
+              struct.setCatNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -1063,14 +1153,21 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(RESOURCE_URIS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.resourceUris.size()));
-          for (ResourceUri _iter541 : struct.resourceUris)
+          for (ResourceUri _iter549 : struct.resourceUris)
           {
-            _iter541.write(oprot);
+            _iter549.write(oprot);
           }
           oprot.writeListEnd();
         }
         oprot.writeFieldEnd();
       }
+      if (struct.catName != null) {
+        if (struct.isSetCatName()) {
+          oprot.writeFieldBegin(CAT_NAME_FIELD_DESC);
+          oprot.writeString(struct.catName);
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -1113,7 +1210,10 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetResourceUris()) {
         optionals.set(7);
       }
-      oprot.writeBitSet(optionals, 8);
+      if (struct.isSetCatName()) {
+        optionals.set(8);
+      }
+      oprot.writeBitSet(optionals, 9);
       if (struct.isSetFunctionName()) {
         oprot.writeString(struct.functionName);
       }
@@ -1138,18 +1238,21 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetResourceUris()) {
         {
           oprot.writeI32(struct.resourceUris.size());
-          for (ResourceUri _iter542 : struct.resourceUris)
+          for (ResourceUri _iter550 : struct.resourceUris)
           {
-            _iter542.write(oprot);
+            _iter550.write(oprot);
           }
         }
       }
+      if (struct.isSetCatName()) {
+        oprot.writeString(struct.catName);
+      }
     }
 
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, Function struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
-      BitSet incoming = iprot.readBitSet(8);
+      BitSet incoming = iprot.readBitSet(9);
       if (incoming.get(0)) {
         struct.functionName = iprot.readString();
         struct.setFunctionNameIsSet(true);
@@ -1180,18 +1283,22 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(7)) {
         {
-          org.apache.thrift.protocol.TList _list543 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.resourceUris = new ArrayList<ResourceUri>(_list543.size);
-          ResourceUri _elem544;
-          for (int _i545 = 0; _i545 < _list543.size; ++_i545)
+          org.apache.thrift.protocol.TList _list551 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.resourceUris = new ArrayList<ResourceUri>(_list551.size);
+          ResourceUri _elem552;
+          for (int _i553 = 0; _i553 < _list551.size; ++_i553)
           {
-            _elem544 = new ResourceUri();
-            _elem544.read(iprot);
-            struct.resourceUris.add(_elem544);
+            _elem552 = new ResourceUri();
+            _elem552.read(iprot);
+            struct.resourceUris.add(_elem552);
           }
         }
         struct.setResourceUrisIsSet(true);
       }
+      if (incoming.get(8)) {
+        struct.catName = iprot.readString();
+        struct.setCatNameIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
index 522fb92..0c5f62b 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
@@ -346,14 +346,14 @@ import org.slf4j.LoggerFactory;
           case 1: // FUNCTIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list776 = iprot.readListBegin();
-                struct.functions = new ArrayList<Function>(_list776.size);
-                Function _elem777;
-                for (int _i778 = 0; _i778 < _list776.size; ++_i778)
+                org.apache.thrift.protocol.TList _list784 = iprot.readListBegin();
+                struct.functions = new ArrayList<Function>(_list784.size);
+                Function _elem785;
+                for (int _i786 = 0; _i786 < _list784.size; ++_i786)
                 {
-                  _elem777 = new Function();
-                  _elem777.read(iprot);
-                  struct.functions.add(_elem777);
+                  _elem785 = new Function();
+                  _elem785.read(iprot);
+                  struct.functions.add(_elem785);
                 }
                 iprot.readListEnd();
               }
@@ -380,9 +380,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(FUNCTIONS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.functions.size()));
-            for (Function _iter779 : struct.functions)
+            for (Function _iter787 : struct.functions)
             {
-              _iter779.write(oprot);
+              _iter787.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetFunctions()) {
         {
           oprot.writeI32(struct.functions.size());
-          for (Function _iter780 : struct.functions)
+          for (Function _iter788 : struct.functions)
           {
-            _iter780.write(oprot);
+            _iter788.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list781 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.functions = new ArrayList<Function>(_list781.size);
-          Function _elem782;
-          for (int _i783 = 0; _i783 < _list781.size; ++_i783)
+          org.apache.thrift.protocol.TList _list789 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.functions = new ArrayList<Function>(_list789.size);
+          Function _elem790;
+          for (int _i791 = 0; _i791 < _list789.size; ++_i791)
           {
-            _elem782 = new Function();
-            _elem782.read(iprot);
-            struct.functions.add(_elem782);
+            _elem790 = new Function();
+            _elem790.read(iprot);
+            struct.functions.add(_elem790);
           }
         }
         struct.setFunctionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetCatalogRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetCatalogRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetCatalogRequest.java
new file mode 100644
index 0000000..c0e6240
--- /dev/null
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetCatalogRequest.java
@@ -0,0 +1,395 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class GetCatalogRequest implements org.apache.thrift.TBase<GetCatalogRequest, GetCatalogRequest._Fields>, java.io.Serializable, Cloneable, Comparable<GetCatalogRequest> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetCatalogRequest");
+
+  private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new GetCatalogRequestStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new GetCatalogRequestTupleSchemeFactory());
+  }
+
+  private String name; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    NAME((short)1, "name");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // NAME
+          return NAME;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetCatalogRequest.class, metaDataMap);
+  }
+
+  public GetCatalogRequest() {
+  }
+
+  public GetCatalogRequest(
+    String name)
+  {
+    this();
+    this.name = name;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public GetCatalogRequest(GetCatalogRequest other) {
+    if (other.isSetName()) {
+      this.name = other.name;
+    }
+  }
+
+  public GetCatalogRequest deepCopy() {
+    return new GetCatalogRequest(this);
+  }
+
+  @Override
+  public void clear() {
+    this.name = null;
+  }
+
+  public String getName() {
+    return this.name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  public void unsetName() {
+    this.name = null;
+  }
+
+  /** Returns true if field name is set (has been assigned a value) and false otherwise */
+  public boolean isSetName() {
+    return this.name != null;
+  }
+
+  public void setNameIsSet(boolean value) {
+    if (!value) {
+      this.name = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case NAME:
+      if (value == null) {
+        unsetName();
+      } else {
+        setName((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case NAME:
+      return getName();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case NAME:
+      return isSetName();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof GetCatalogRequest)
+      return this.equals((GetCatalogRequest)that);
+    return false;
+  }
+
+  public boolean equals(GetCatalogRequest that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_name = true && this.isSetName();
+    boolean that_present_name = true && that.isSetName();
+    if (this_present_name || that_present_name) {
+      if (!(this_present_name && that_present_name))
+        return false;
+      if (!this.name.equals(that.name))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_name = true && (isSetName());
+    list.add(present_name);
+    if (present_name)
+      list.add(name);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(GetCatalogRequest other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetName()).compareTo(other.isSetName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("GetCatalogRequest(");
+    boolean first = true;
+
+    sb.append("name:");
+    if (this.name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.name);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class GetCatalogRequestStandardSchemeFactory implements SchemeFactory {
+    public GetCatalogRequestStandardScheme getScheme() {
+      return new GetCatalogRequestStandardScheme();
+    }
+  }
+
+  private static class GetCatalogRequestStandardScheme extends StandardScheme<GetCatalogRequest> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, GetCatalogRequest struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.name = iprot.readString();
+              struct.setNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, GetCatalogRequest struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.name != null) {
+        oprot.writeFieldBegin(NAME_FIELD_DESC);
+        oprot.writeString(struct.name);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class GetCatalogRequestTupleSchemeFactory implements SchemeFactory {
+    public GetCatalogRequestTupleScheme getScheme() {
+      return new GetCatalogRequestTupleScheme();
+    }
+  }
+
+  private static class GetCatalogRequestTupleScheme extends TupleScheme<GetCatalogRequest> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, GetCatalogRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.isSetName()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.isSetName()) {
+        oprot.writeString(struct.name);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, GetCatalogRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.name = iprot.readString();
+        struct.setNameIsSet(true);
+      }
+    }
+  }
+
+}
+


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
index cf36654..bded16a 100644
--- a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
+++ b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
@@ -38,6 +38,30 @@ class Iface(fb303.FacebookService.Iface):
     """
     pass
 
+  def create_catalog(self, catalog):
+    """
+    Parameters:
+     - catalog
+    """
+    pass
+
+  def get_catalog(self, catName):
+    """
+    Parameters:
+     - catName
+    """
+    pass
+
+  def get_catalogs(self):
+    pass
+
+  def drop_catalog(self, catName):
+    """
+    Parameters:
+     - catName
+    """
+    pass
+
   def create_database(self, database):
     """
     Parameters:
@@ -324,9 +348,10 @@ class Iface(fb303.FacebookService.Iface):
     """
     pass
 
-  def update_creation_metadata(self, dbname, tbl_name, creation_metadata):
+  def update_creation_metadata(self, catName, dbname, tbl_name, creation_metadata):
     """
     Parameters:
+     - catName
      - dbname
      - tbl_name
      - creation_metadata
@@ -1597,6 +1622,139 @@ class Client(fb303.FacebookService.Client, Iface):
       raise result.o1
     return
 
+  def create_catalog(self, catalog):
+    """
+    Parameters:
+     - catalog
+    """
+    self.send_create_catalog(catalog)
+    self.recv_create_catalog()
+
+  def send_create_catalog(self, catalog):
+    self._oprot.writeMessageBegin('create_catalog', TMessageType.CALL, self._seqid)
+    args = create_catalog_args()
+    args.catalog = catalog
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_create_catalog(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = create_catalog_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.o1 is not None:
+      raise result.o1
+    if result.o2 is not None:
+      raise result.o2
+    if result.o3 is not None:
+      raise result.o3
+    return
+
+  def get_catalog(self, catName):
+    """
+    Parameters:
+     - catName
+    """
+    self.send_get_catalog(catName)
+    return self.recv_get_catalog()
+
+  def send_get_catalog(self, catName):
+    self._oprot.writeMessageBegin('get_catalog', TMessageType.CALL, self._seqid)
+    args = get_catalog_args()
+    args.catName = catName
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_get_catalog(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = get_catalog_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
+    if result.o1 is not None:
+      raise result.o1
+    if result.o2 is not None:
+      raise result.o2
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "get_catalog failed: unknown result")
+
+  def get_catalogs(self):
+    self.send_get_catalogs()
+    return self.recv_get_catalogs()
+
+  def send_get_catalogs(self):
+    self._oprot.writeMessageBegin('get_catalogs', TMessageType.CALL, self._seqid)
+    args = get_catalogs_args()
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_get_catalogs(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = get_catalogs_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
+    if result.o1 is not None:
+      raise result.o1
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "get_catalogs failed: unknown result")
+
+  def drop_catalog(self, catName):
+    """
+    Parameters:
+     - catName
+    """
+    self.send_drop_catalog(catName)
+    self.recv_drop_catalog()
+
+  def send_drop_catalog(self, catName):
+    self._oprot.writeMessageBegin('drop_catalog', TMessageType.CALL, self._seqid)
+    args = drop_catalog_args()
+    args.catName = catName
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_drop_catalog(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = drop_catalog_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.o1 is not None:
+      raise result.o1
+    if result.o2 is not None:
+      raise result.o2
+    if result.o3 is not None:
+      raise result.o3
+    return
+
   def create_database(self, database):
     """
     Parameters:
@@ -2925,19 +3083,21 @@ class Client(fb303.FacebookService.Client, Iface):
       raise result.o3
     raise TApplicationException(TApplicationException.MISSING_RESULT, "get_materialization_invalidation_info failed: unknown result")
 
-  def update_creation_metadata(self, dbname, tbl_name, creation_metadata):
+  def update_creation_metadata(self, catName, dbname, tbl_name, creation_metadata):
     """
     Parameters:
+     - catName
      - dbname
      - tbl_name
      - creation_metadata
     """
-    self.send_update_creation_metadata(dbname, tbl_name, creation_metadata)
+    self.send_update_creation_metadata(catName, dbname, tbl_name, creation_metadata)
     self.recv_update_creation_metadata()
 
-  def send_update_creation_metadata(self, dbname, tbl_name, creation_metadata):
+  def send_update_creation_metadata(self, catName, dbname, tbl_name, creation_metadata):
     self._oprot.writeMessageBegin('update_creation_metadata', TMessageType.CALL, self._seqid)
     args = update_creation_metadata_args()
+    args.catName = catName
     args.dbname = dbname
     args.tbl_name = tbl_name
     args.creation_metadata = creation_metadata
@@ -8557,6 +8717,10 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     fb303.FacebookService.Processor.__init__(self, handler)
     self._processMap["getMetaConf"] = Processor.process_getMetaConf
     self._processMap["setMetaConf"] = Processor.process_setMetaConf
+    self._processMap["create_catalog"] = Processor.process_create_catalog
+    self._processMap["get_catalog"] = Processor.process_get_catalog
+    self._processMap["get_catalogs"] = Processor.process_get_catalogs
+    self._processMap["drop_catalog"] = Processor.process_drop_catalog
     self._processMap["create_database"] = Processor.process_create_database
     self._processMap["get_database"] = Processor.process_get_database
     self._processMap["drop_database"] = Processor.process_drop_database
@@ -8811,6 +8975,109 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     oprot.writeMessageEnd()
     oprot.trans.flush()
 
+  def process_create_catalog(self, seqid, iprot, oprot):
+    args = create_catalog_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = create_catalog_result()
+    try:
+      self._handler.create_catalog(args.catalog)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except AlreadyExistsException as o1:
+      msg_type = TMessageType.REPLY
+      result.o1 = o1
+    except InvalidObjectException as o2:
+      msg_type = TMessageType.REPLY
+      result.o2 = o2
+    except MetaException as o3:
+      msg_type = TMessageType.REPLY
+      result.o3 = o3
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("create_catalog", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_get_catalog(self, seqid, iprot, oprot):
+    args = get_catalog_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = get_catalog_result()
+    try:
+      result.success = self._handler.get_catalog(args.catName)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except NoSuchObjectException as o1:
+      msg_type = TMessageType.REPLY
+      result.o1 = o1
+    except MetaException as o2:
+      msg_type = TMessageType.REPLY
+      result.o2 = o2
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("get_catalog", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_get_catalogs(self, seqid, iprot, oprot):
+    args = get_catalogs_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = get_catalogs_result()
+    try:
+      result.success = self._handler.get_catalogs()
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except MetaException as o1:
+      msg_type = TMessageType.REPLY
+      result.o1 = o1
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("get_catalogs", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_drop_catalog(self, seqid, iprot, oprot):
+    args = drop_catalog_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = drop_catalog_result()
+    try:
+      self._handler.drop_catalog(args.catName)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except NoSuchObjectException as o1:
+      msg_type = TMessageType.REPLY
+      result.o1 = o1
+    except InvalidOperationException as o2:
+      msg_type = TMessageType.REPLY
+      result.o2 = o2
+    except MetaException as o3:
+      msg_type = TMessageType.REPLY
+      result.o3 = o3
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("drop_catalog", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
   def process_create_database(self, seqid, iprot, oprot):
     args = create_database_args()
     args.read(iprot)
@@ -9754,7 +10021,7 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     iprot.readMessageEnd()
     result = update_creation_metadata_result()
     try:
-      self._handler.update_creation_metadata(args.dbname, args.tbl_name, args.creation_metadata)
+      self._handler.update_creation_metadata(args.catName, args.dbname, args.tbl_name, args.creation_metadata)
       msg_type = TMessageType.REPLY
     except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
       raise
@@ -13912,19 +14179,19 @@ class setMetaConf_result:
   def __ne__(self, other):
     return not (self == other)
 
-class create_database_args:
+class create_catalog_args:
   """
   Attributes:
-   - database
+   - catalog
   """
 
   thrift_spec = (
     None, # 0
-    (1, TType.STRUCT, 'database', (Database, Database.thrift_spec), None, ), # 1
+    (1, TType.STRUCT, 'catalog', (CreateCatalogRequest, CreateCatalogRequest.thrift_spec), None, ), # 1
   )
 
-  def __init__(self, database=None,):
-    self.database = database
+  def __init__(self, catalog=None,):
+    self.catalog = catalog
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -13937,8 +14204,8 @@ class create_database_args:
         break
       if fid == 1:
         if ftype == TType.STRUCT:
-          self.database = Database()
-          self.database.read(iprot)
+          self.catalog = CreateCatalogRequest()
+          self.catalog.read(iprot)
         else:
           iprot.skip(ftype)
       else:
@@ -13950,10 +14217,10 @@ class create_database_args:
     if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
       oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
       return
-    oprot.writeStructBegin('create_database_args')
-    if self.database is not None:
-      oprot.writeFieldBegin('database', TType.STRUCT, 1)
-      self.database.write(oprot)
+    oprot.writeStructBegin('create_catalog_args')
+    if self.catalog is not None:
+      oprot.writeFieldBegin('catalog', TType.STRUCT, 1)
+      self.catalog.write(oprot)
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
@@ -13964,7 +14231,7 @@ class create_database_args:
 
   def __hash__(self):
     value = 17
-    value = (value * 31) ^ hash(self.database)
+    value = (value * 31) ^ hash(self.catalog)
     return value
 
   def __repr__(self):
@@ -13978,7 +14245,7 @@ class create_database_args:
   def __ne__(self, other):
     return not (self == other)
 
-class create_database_result:
+class create_catalog_result:
   """
   Attributes:
    - o1
@@ -14034,7 +14301,451 @@ class create_database_result:
     if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
       oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
       return
-    oprot.writeStructBegin('create_database_result')
+    oprot.writeStructBegin('create_catalog_result')
+    if self.o1 is not None:
+      oprot.writeFieldBegin('o1', TType.STRUCT, 1)
+      self.o1.write(oprot)
+      oprot.writeFieldEnd()
+    if self.o2 is not None:
+      oprot.writeFieldBegin('o2', TType.STRUCT, 2)
+      self.o2.write(oprot)
+      oprot.writeFieldEnd()
+    if self.o3 is not None:
+      oprot.writeFieldBegin('o3', TType.STRUCT, 3)
+      self.o3.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.o1)
+    value = (value * 31) ^ hash(self.o2)
+    value = (value * 31) ^ hash(self.o3)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class get_catalog_args:
+  """
+  Attributes:
+   - catName
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'catName', (GetCatalogRequest, GetCatalogRequest.thrift_spec), None, ), # 1
+  )
+
+  def __init__(self, catName=None,):
+    self.catName = catName
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.catName = GetCatalogRequest()
+          self.catName.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('get_catalog_args')
+    if self.catName is not None:
+      oprot.writeFieldBegin('catName', TType.STRUCT, 1)
+      self.catName.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.catName)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class get_catalog_result:
+  """
+  Attributes:
+   - success
+   - o1
+   - o2
+  """
+
+  thrift_spec = (
+    (0, TType.STRUCT, 'success', (GetCatalogResponse, GetCatalogResponse.thrift_spec), None, ), # 0
+    (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1
+    (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2
+  )
+
+  def __init__(self, success=None, o1=None, o2=None,):
+    self.success = success
+    self.o1 = o1
+    self.o2 = o2
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 0:
+        if ftype == TType.STRUCT:
+          self.success = GetCatalogResponse()
+          self.success.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 1:
+        if ftype == TType.STRUCT:
+          self.o1 = NoSuchObjectException()
+          self.o1.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.o2 = MetaException()
+          self.o2.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('get_catalog_result')
+    if self.success is not None:
+      oprot.writeFieldBegin('success', TType.STRUCT, 0)
+      self.success.write(oprot)
+      oprot.writeFieldEnd()
+    if self.o1 is not None:
+      oprot.writeFieldBegin('o1', TType.STRUCT, 1)
+      self.o1.write(oprot)
+      oprot.writeFieldEnd()
+    if self.o2 is not None:
+      oprot.writeFieldBegin('o2', TType.STRUCT, 2)
+      self.o2.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.success)
+    value = (value * 31) ^ hash(self.o1)
+    value = (value * 31) ^ hash(self.o2)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class get_catalogs_args:
+
+  thrift_spec = (
+  )
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('get_catalogs_args')
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class get_catalogs_result:
+  """
+  Attributes:
+   - success
+   - o1
+  """
+
+  thrift_spec = (
+    (0, TType.STRUCT, 'success', (GetCatalogsResponse, GetCatalogsResponse.thrift_spec), None, ), # 0
+    (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1
+  )
+
+  def __init__(self, success=None, o1=None,):
+    self.success = success
+    self.o1 = o1
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 0:
+        if ftype == TType.STRUCT:
+          self.success = GetCatalogsResponse()
+          self.success.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 1:
+        if ftype == TType.STRUCT:
+          self.o1 = MetaException()
+          self.o1.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('get_catalogs_result')
+    if self.success is not None:
+      oprot.writeFieldBegin('success', TType.STRUCT, 0)
+      self.success.write(oprot)
+      oprot.writeFieldEnd()
+    if self.o1 is not None:
+      oprot.writeFieldBegin('o1', TType.STRUCT, 1)
+      self.o1.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.success)
+    value = (value * 31) ^ hash(self.o1)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class drop_catalog_args:
+  """
+  Attributes:
+   - catName
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'catName', (DropCatalogRequest, DropCatalogRequest.thrift_spec), None, ), # 1
+  )
+
+  def __init__(self, catName=None,):
+    self.catName = catName
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.catName = DropCatalogRequest()
+          self.catName.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('drop_catalog_args')
+    if self.catName is not None:
+      oprot.writeFieldBegin('catName', TType.STRUCT, 1)
+      self.catName.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.catName)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class drop_catalog_result:
+  """
+  Attributes:
+   - o1
+   - o2
+   - o3
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1
+    (2, TType.STRUCT, 'o2', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 2
+    (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3
+  )
+
+  def __init__(self, o1=None, o2=None, o3=None,):
+    self.o1 = o1
+    self.o2 = o2
+    self.o3 = o3
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.o1 = NoSuchObjectException()
+          self.o1.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.o2 = InvalidOperationException()
+          self.o2.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRUCT:
+          self.o3 = MetaException()
+          self.o3.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('drop_catalog_result')
     if self.o1 is not None:
       oprot.writeFieldBegin('o1', TType.STRUCT, 1)
       self.o1.write(oprot)
@@ -14072,19 +14783,19 @@ class create_database_result:
   def __ne__(self, other):
     return not (self == other)
 
-class get_database_args:
+class create_database_args:
   """
   Attributes:
-   - name
+   - database
   """
 
   thrift_spec = (
     None, # 0
-    (1, TType.STRING, 'name', None, None, ), # 1
+    (1, TType.STRUCT, 'database', (Database, Database.thrift_spec), None, ), # 1
   )
 
-  def __init__(self, name=None,):
-    self.name = name
+  def __init__(self, database=None,):
+    self.database = database
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -14096,8 +14807,9 @@ class get_database_args:
       if ftype == TType.STOP:
         break
       if fid == 1:
-        if ftype == TType.STRING:
-          self.name = iprot.readString()
+        if ftype == TType.STRUCT:
+          self.database = Database()
+          self.database.read(iprot)
         else:
           iprot.skip(ftype)
       else:
@@ -14109,10 +14821,10 @@ class get_database_args:
     if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
       oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
       return
-    oprot.writeStructBegin('get_database_args')
-    if self.name is not None:
-      oprot.writeFieldBegin('name', TType.STRING, 1)
-      oprot.writeString(self.name)
+    oprot.writeStructBegin('create_database_args')
+    if self.database is not None:
+      oprot.writeFieldBegin('database', TType.STRUCT, 1)
+      self.database.write(oprot)
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
@@ -14123,7 +14835,7 @@ class get_database_args:
 
   def __hash__(self):
     value = 17
-    value = (value * 31) ^ hash(self.name)
+    value = (value * 31) ^ hash(self.database)
     return value
 
   def __repr__(self):
@@ -14137,24 +14849,25 @@ class get_database_args:
   def __ne__(self, other):
     return not (self == other)
 
-class get_database_result:
+class create_database_result:
   """
   Attributes:
-   - success
    - o1
    - o2
+   - o3
   """
 
   thrift_spec = (
-    (0, TType.STRUCT, 'success', (Database, Database.thrift_spec), None, ), # 0
-    (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1
-    (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2
+    None, # 0
+    (1, TType.STRUCT, 'o1', (AlreadyExistsException, AlreadyExistsException.thrift_spec), None, ), # 1
+    (2, TType.STRUCT, 'o2', (InvalidObjectException, InvalidObjectException.thrift_spec), None, ), # 2
+    (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3
   )
 
-  def __init__(self, success=None, o1=None, o2=None,):
-    self.success = success
+  def __init__(self, o1=None, o2=None, o3=None,):
     self.o1 = o1
     self.o2 = o2
+    self.o3 = o3
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -14165,24 +14878,24 @@ class get_database_result:
       (fname, ftype, fid) = iprot.readFieldBegin()
       if ftype == TType.STOP:
         break
-      if fid == 0:
-        if ftype == TType.STRUCT:
-          self.success = Database()
-          self.success.read(iprot)
-        else:
-          iprot.skip(ftype)
-      elif fid == 1:
+      if fid == 1:
         if ftype == TType.STRUCT:
-          self.o1 = NoSuchObjectException()
+          self.o1 = AlreadyExistsException()
           self.o1.read(iprot)
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.STRUCT:
-          self.o2 = MetaException()
+          self.o2 = InvalidObjectException()
           self.o2.read(iprot)
         else:
           iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRUCT:
+          self.o3 = MetaException()
+          self.o3.read(iprot)
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -14192,11 +14905,7 @@ class get_database_result:
     if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
       oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
       return
-    oprot.writeStructBegin('get_database_result')
-    if self.success is not None:
-      oprot.writeFieldBegin('success', TType.STRUCT, 0)
-      self.success.write(oprot)
-      oprot.writeFieldEnd()
+    oprot.writeStructBegin('create_database_result')
     if self.o1 is not None:
       oprot.writeFieldBegin('o1', TType.STRUCT, 1)
       self.o1.write(oprot)
@@ -14205,6 +14914,10 @@ class get_database_result:
       oprot.writeFieldBegin('o2', TType.STRUCT, 2)
       self.o2.write(oprot)
       oprot.writeFieldEnd()
+    if self.o3 is not None:
+      oprot.writeFieldBegin('o3', TType.STRUCT, 3)
+      self.o3.write(oprot)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -14214,9 +14927,9 @@ class get_database_result:
 
   def __hash__(self):
     value = 17
-    value = (value * 31) ^ hash(self.success)
     value = (value * 31) ^ hash(self.o1)
     value = (value * 31) ^ hash(self.o2)
+    value = (value * 31) ^ hash(self.o3)
     return value
 
   def __repr__(self):
@@ -14230,25 +14943,19 @@ class get_database_result:
   def __ne__(self, other):
     return not (self == other)
 
-class drop_database_args:
+class get_database_args:
   """
   Attributes:
    - name
-   - deleteData
-   - cascade
   """
 
   thrift_spec = (
     None, # 0
     (1, TType.STRING, 'name', None, None, ), # 1
-    (2, TType.BOOL, 'deleteData', None, None, ), # 2
-    (3, TType.BOOL, 'cascade', None, None, ), # 3
   )
 
-  def __init__(self, name=None, deleteData=None, cascade=None,):
+  def __init__(self, name=None,):
     self.name = name
-    self.deleteData = deleteData
-    self.cascade = cascade
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -14264,16 +14971,6 @@ class drop_database_args:
           self.name = iprot.readString()
         else:
           iprot.skip(ftype)
-      elif fid == 2:
-        if ftype == TType.BOOL:
-          self.deleteData = iprot.readBool()
-        else:
-          iprot.skip(ftype)
-      elif fid == 3:
-        if ftype == TType.BOOL:
-          self.cascade = iprot.readBool()
-        else:
-          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -14283,19 +14980,11 @@ class drop_database_args:
     if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
       oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
       return
-    oprot.writeStructBegin('drop_database_args')
+    oprot.writeStructBegin('get_database_args')
     if self.name is not None:
       oprot.writeFieldBegin('name', TType.STRING, 1)
       oprot.writeString(self.name)
       oprot.writeFieldEnd()
-    if self.deleteData is not None:
-      oprot.writeFieldBegin('deleteData', TType.BOOL, 2)
-      oprot.writeBool(self.deleteData)
-      oprot.writeFieldEnd()
-    if self.cascade is not None:
-      oprot.writeFieldBegin('cascade', TType.BOOL, 3)
-      oprot.writeBool(self.cascade)
-      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -14306,8 +14995,6 @@ class drop_database_args:
   def __hash__(self):
     value = 17
     value = (value * 31) ^ hash(self.name)
-    value = (value * 31) ^ hash(self.deleteData)
-    value = (value * 31) ^ hash(self.cascade)
     return value
 
   def __repr__(self):
@@ -14321,25 +15008,24 @@ class drop_database_args:
   def __ne__(self, other):
     return not (self == other)
 
-class drop_database_result:
+class get_database_result:
   """
   Attributes:
+   - success
    - o1
    - o2
-   - o3
   """
 
   thrift_spec = (
-    None, # 0
+    (0, TType.STRUCT, 'success', (Database, Database.thrift_spec), None, ), # 0
     (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1
-    (2, TType.STRUCT, 'o2', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 2
-    (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3
+    (2, TType.STRUCT, 'o2', (MetaException, MetaException.thrift_spec), None, ), # 2
   )
 
-  def __init__(self, o1=None, o2=None, o3=None,):
+  def __init__(self, success=None, o1=None, o2=None,):
+    self.success = success
     self.o1 = o1
     self.o2 = o2
-    self.o3 = o3
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -14350,7 +15036,13 @@ class drop_database_result:
       (fname, ftype, fid) = iprot.readFieldBegin()
       if ftype == TType.STOP:
         break
-      if fid == 1:
+      if fid == 0:
+        if ftype == TType.STRUCT:
+          self.success = Database()
+          self.success.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 1:
         if ftype == TType.STRUCT:
           self.o1 = NoSuchObjectException()
           self.o1.read(iprot)
@@ -14358,16 +15050,10 @@ class drop_database_result:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.STRUCT:
-          self.o2 = InvalidOperationException()
+          self.o2 = MetaException()
           self.o2.read(iprot)
         else:
           iprot.skip(ftype)
-      elif fid == 3:
-        if ftype == TType.STRUCT:
-          self.o3 = MetaException()
-          self.o3.read(iprot)
-        else:
-          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -14377,7 +15063,11 @@ class drop_database_result:
     if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
       oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
       return
-    oprot.writeStructBegin('drop_database_result')
+    oprot.writeStructBegin('get_database_result')
+    if self.success is not None:
+      oprot.writeFieldBegin('success', TType.STRUCT, 0)
+      self.success.write(oprot)
+      oprot.writeFieldEnd()
     if self.o1 is not None:
       oprot.writeFieldBegin('o1', TType.STRUCT, 1)
       self.o1.write(oprot)
@@ -14386,10 +15076,6 @@ class drop_database_result:
       oprot.writeFieldBegin('o2', TType.STRUCT, 2)
       self.o2.write(oprot)
       oprot.writeFieldEnd()
-    if self.o3 is not None:
-      oprot.writeFieldBegin('o3', TType.STRUCT, 3)
-      self.o3.write(oprot)
-      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -14399,9 +15085,9 @@ class drop_database_result:
 
   def __hash__(self):
     value = 17
+    value = (value * 31) ^ hash(self.success)
     value = (value * 31) ^ hash(self.o1)
     value = (value * 31) ^ hash(self.o2)
-    value = (value * 31) ^ hash(self.o3)
     return value
 
   def __repr__(self):
@@ -14415,19 +15101,25 @@ class drop_database_result:
   def __ne__(self, other):
     return not (self == other)
 
-class get_databases_args:
+class drop_database_args:
   """
   Attributes:
-   - pattern
+   - name
+   - deleteData
+   - cascade
   """
 
   thrift_spec = (
     None, # 0
-    (1, TType.STRING, 'pattern', None, None, ), # 1
+    (1, TType.STRING, 'name', None, None, ), # 1
+    (2, TType.BOOL, 'deleteData', None, None, ), # 2
+    (3, TType.BOOL, 'cascade', None, None, ), # 3
   )
 
-  def __init__(self, pattern=None,):
-    self.pattern = pattern
+  def __init__(self, name=None, deleteData=None, cascade=None,):
+    self.name = name
+    self.deleteData = deleteData
+    self.cascade = cascade
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -14440,7 +15132,17 @@ class get_databases_args:
         break
       if fid == 1:
         if ftype == TType.STRING:
-          self.pattern = iprot.readString()
+          self.name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.BOOL:
+          self.deleteData = iprot.readBool()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.BOOL:
+          self.cascade = iprot.readBool()
         else:
           iprot.skip(ftype)
       else:
@@ -14452,10 +15154,18 @@ class get_databases_args:
     if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
       oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
       return
-    oprot.writeStructBegin('get_databases_args')
-    if self.pattern is not None:
-      oprot.writeFieldBegin('pattern', TType.STRING, 1)
-      oprot.writeString(self.pattern)
+    oprot.writeStructBegin('drop_database_args')
+    if self.name is not None:
+      oprot.writeFieldBegin('name', TType.STRING, 1)
+      oprot.writeString(self.name)
+      oprot.writeFieldEnd()
+    if self.deleteData is not None:
+      oprot.writeFieldBegin('deleteData', TType.BOOL, 2)
+      oprot.writeBool(self.deleteData)
+      oprot.writeFieldEnd()
+    if self.cascade is not None:
+      oprot.writeFieldBegin('cascade', TType.BOOL, 3)
+      oprot.writeBool(self.cascade)
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
@@ -14466,7 +15176,9 @@ class get_databases_args:
 
   def __hash__(self):
     value = 17
-    value = (value * 31) ^ hash(self.pattern)
+    value = (value * 31) ^ hash(self.name)
+    value = (value * 31) ^ hash(self.deleteData)
+    value = (value * 31) ^ hash(self.cascade)
     return value
 
   def __repr__(self):
@@ -14480,21 +15192,25 @@ class get_databases_args:
   def __ne__(self, other):
     return not (self == other)
 
-class get_databases_result:
+class drop_database_result:
   """
   Attributes:
-   - success
    - o1
+   - o2
+   - o3
   """
 
   thrift_spec = (
-    (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0
-    (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1
+    None, # 0
+    (1, TType.STRUCT, 'o1', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 1
+    (2, TType.STRUCT, 'o2', (InvalidOperationException, InvalidOperationException.thrift_spec), None, ), # 2
+    (3, TType.STRUCT, 'o3', (MetaException, MetaException.thrift_spec), None, ), # 3
   )
 
-  def __init__(self, success=None, o1=None,):
-    self.success = success
+  def __init__(self, o1=None, o2=None, o3=None,):
     self.o1 = o1
+    self.o2 = o2
+    self.o3 = o3
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -14505,20 +15221,22 @@ class get_databases_result:
       (fname, ftype, fid) = iprot.readFieldBegin()
       if ftype == TType.STOP:
         break
-      if fid == 0:
-        if ftype == TType.LIST:
-          self.success = []
-          (_etype791, _size788) = iprot.readListBegin()
-          for _i792 in xrange(_size788):
-            _elem793 = iprot.readString()
-            self.success.append(_elem793)
-          iprot.readListEnd()
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.o1 = NoSuchObjectException()
+          self.o1.read(iprot)
         else:
           iprot.skip(ftype)
-      elif fid == 1:
+      elif fid == 2:
         if ftype == TType.STRUCT:
-          self.o1 = MetaException()
-          self.o1.read(iprot)
+          self.o2 = InvalidOperationException()
+          self.o2.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRUCT:
+          self.o3 = MetaException()
+          self.o3.read(iprot)
         else:
           iprot.skip(ftype)
       else:
@@ -14530,18 +15248,19 @@ class get_databases_result:
     if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
       oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
       return
-    oprot.writeStructBegin('get_databases_result')
-    if self.success is not None:
-      oprot.writeFieldBegin('success', TType.LIST, 0)
-      oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter794 in self.success:
-        oprot.writeString(iter794)
-      oprot.writeListEnd()
-      oprot.writeFieldEnd()
+    oprot.writeStructBegin('drop_database_result')
     if self.o1 is not None:
       oprot.writeFieldBegin('o1', TType.STRUCT, 1)
       self.o1.write(oprot)
       oprot.writeFieldEnd()
+    if self.o2 is not None:
+      oprot.writeFieldBegin('o2', TType.STRUCT, 2)
+      self.o2.write(oprot)
+      oprot.writeFieldEnd()
+    if self.o3 is not None:
+      oprot.writeFieldBegin('o3', TType.STRUCT, 3)
+      self.o3.write(oprot)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -14551,8 +15270,9 @@ class get_databases_result:
 
   def __hash__(self):
     value = 17
-    value = (value * 31) ^ hash(self.success)
     value = (value * 31) ^ hash(self.o1)
+    value = (value * 31) ^ hash(self.o2)
+    value = (value * 31) ^ hash(self.o3)
     return value
 
   def __repr__(self):
@@ -14566,11 +15286,20 @@ class get_databases_result:
   def __ne__(self, other):
     return not (self == other)
 
-class get_all_databases_args:
+class get_databases_args:
+  """
+  Attributes:
+   - pattern
+  """
 
   thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'pattern', None, None, ), # 1
   )
 
+  def __init__(self, pattern=None,):
+    self.pattern = pattern
+
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
       fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
@@ -14580,6 +15309,11 @@ class get_all_databases_args:
       (fname, ftype, fid) = iprot.readFieldBegin()
       if ftype == TType.STOP:
         break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.pattern = iprot.readString()
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -14589,7 +15323,11 @@ class get_all_databases_args:
     if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
       oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
       return
-    oprot.writeStructBegin('get_all_databases_args')
+    oprot.writeStructBegin('get_databases_args')
+    if self.pattern is not None:
+      oprot.writeFieldBegin('pattern', TType.STRING, 1)
+      oprot.writeString(self.pattern)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -14599,6 +15337,7 @@ class get_all_databases_args:
 
   def __hash__(self):
     value = 17
+    value = (value * 31) ^ hash(self.pattern)
     return value
 
   def __repr__(self):
@@ -14612,7 +15351,7 @@ class get_all_databases_args:
   def __ne__(self, other):
     return not (self == other)
 
-class get_all_databases_result:
+class get_databases_result:
   """
   Attributes:
    - success
@@ -14662,7 +15401,7 @@ class get_all_databases_result:
     if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
       oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
       return
-    oprot.writeStructBegin('get_all_databases_result')
+    oprot.writeStructBegin('get_databases_result')
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
@@ -14698,6 +15437,138 @@ class get_all_databases_result:
   def __ne__(self, other):
     return not (self == other)
 
+class get_all_databases_args:
+
+  thrift_spec = (
+  )
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('get_all_databases_args')
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class get_all_databases_result:
+  """
+  Attributes:
+   - success
+   - o1
+  """
+
+  thrift_spec = (
+    (0, TType.LIST, 'success', (TType.STRING,None), None, ), # 0
+    (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1
+  )
+
+  def __init__(self, success=None, o1=None,):
+    self.success = success
+    self.o1 = o1
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 0:
+        if ftype == TType.LIST:
+          self.success = []
+          (_etype805, _size802) = iprot.readListBegin()
+          for _i806 in xrange(_size802):
+            _elem807 = iprot.readString()
+            self.success.append(_elem807)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 1:
+        if ftype == TType.STRUCT:
+          self.o1 = MetaException()
+          self.o1.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('get_all_databases_result')
+    if self.success is not None:
+      oprot.writeFieldBegin('success', TType.LIST, 0)
+      oprot.writeListBegin(TType.STRING, len(self.success))
+      for iter808 in self.success:
+        oprot.writeString(iter808)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.o1 is not None:
+      oprot.writeFieldBegin('o1', TType.STRUCT, 1)
+      self.o1.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.success)
+    value = (value * 31) ^ hash(self.o1)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
 class alter_database_args:
   """
   Attributes:
@@ -15437,12 +16308,12 @@ class get_type_all_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype803, _vtype804, _size802 ) = iprot.readMapBegin()
-          for _i806 in xrange(_size802):
-            _key807 = iprot.readString()
-            _val808 = Type()
-            _val808.read(iprot)
-            self.success[_key807] = _val808
+          (_ktype810, _vtype811, _size809 ) = iprot.readMapBegin()
+          for _i813 in xrange(_size809):
+            _key814 = iprot.readString()
+            _val815 = Type()
+            _val815.read(iprot)
+            self.success[_key814] = _val815
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -15465,9 +16336,9 @@ class get_type_all_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.MAP, 0)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.success))
-      for kiter809,viter810 in self.success.items():
-        oprot.writeString(kiter809)
-        viter810.write(oprot)
+      for kiter816,viter817 in self.success.items():
+        oprot.writeString(kiter816)
+        viter817.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -15610,11 +16481,11 @@ class get_fields_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype814, _size811) = iprot.readListBegin()
-          for _i815 in xrange(_size811):
-            _elem816 = FieldSchema()
-            _elem816.read(iprot)
-            self.success.append(_elem816)
+          (_etype821, _size818) = iprot.readListBegin()
+          for _i822 in xrange(_size818):
+            _elem823 = FieldSchema()
+            _elem823.read(iprot)
+            self.success.append(_elem823)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15649,8 +16520,8 @@ class get_fields_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter817 in self.success:
-        iter817.write(oprot)
+      for iter824 in self.success:
+        iter824.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -15817,11 +16688,11 @@ class get_fields_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype821, _size818) = iprot.readListBegin()
-          for _i822 in xrange(_size818):
-            _elem823 = FieldSchema()
-            _elem823.read(iprot)
-            self.success.append(_elem823)
+          (_etype828, _size825) = iprot.readListBegin()
+          for _i829 in xrange(_size825):
+            _elem830 = FieldSchema()
+            _elem830.read(iprot)
+            self.success.append(_elem830)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15856,8 +16727,8 @@ class get_fields_with_environment_context_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter824 in self.success:
-        iter824.write(oprot)
+      for iter831 in self.success:
+        iter831.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -16010,11 +16881,11 @@ class get_schema_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype828, _size825) = iprot.readListBegin()
-          for _i829 in xrange(_size825):
-            _elem830 = FieldSchema()
-            _elem830.read(iprot)
-            self.success.append(_elem830)
+          (_etype835, _size832) = iprot.readListBegin()
+          for _i836 in xrange(_size832):
+            _elem837 = FieldSchema()
+            _elem837.read(iprot)
+            self.success.append(_elem837)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16049,8 +16920,8 @@ class get_schema_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter831 in self.success:
-        iter831.write(oprot)
+      for iter838 in self.success:
+        iter838.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -16217,11 +17088,11 @@ class get_schema_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype835, _size832) = iprot.readListBegin()
-          for _i836 in xrange(_size832):
-            _elem837 = FieldSchema()
-            _elem837.read(iprot)
-            self.success.append(_elem837)
+          (_etype842, _size839) = iprot.readListBegin()
+          for _i843 in xrange(_size839):
+            _elem844 = FieldSchema()
+            _elem844.read(iprot)
+            self.success.append(_elem844)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16256,8 +17127,8 @@ class get_schema_with_environment_context_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter838 in self.success:
-        iter838.write(oprot)
+      for iter845 in self.success:
+        iter845.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -16710,66 +17581,66 @@ class create_table_with_constraints_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.primaryKeys = []
-          (_etype842, _size839) = iprot.readListBegin()
-          for _i843 in xrange(_size839):
-            _elem844 = SQLPrimaryKey()
-            _elem844.read(iprot)
-            self.primaryKeys.append(_elem844)
+          (_etype849, _size846) = iprot.readListBegin()
+          for _i850 in xrange(_size846):
+            _elem851 = SQLPrimaryKey()
+            _elem851.read(iprot)
+            self.primaryKeys.append(_elem851)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.LIST:
           self.foreignKeys = []
-          (_etype848, _size845) = iprot.readListBegin()
-          for _i849 in xrange(_size845):
-            _elem850 = SQLForeignKey()
-            _elem850.read(iprot)
-            self.foreignKeys.append(_elem850)
+          (_etype855, _size852) = iprot.readListBegin()
+          for _i856 in xrange(_size852):
+            _elem857 = SQLForeignKey()
+            _elem857.read(iprot)
+            self.foreignKeys.append(_elem857)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.LIST:
           self.uniqueConstraints = []
-          (_etype854, _size851) = iprot.readListBegin()
-          for _i855 in xrange(_size851):
-            _elem856 = SQLUniqueConstraint()
-            _elem856.read(iprot)
-            self.uniqueConstraints.append(_elem856)
+          (_etype861, _size858) = iprot.readListBegin()
+          for _i862 in xrange(_size858):
+            _elem863 = SQLUniqueConstraint()
+            _elem863.read(iprot)
+            self.uniqueConstraints.append(_elem863)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 5:
         if ftype == TType.LIST:
           self.notNullConstraints = []
-          (_etype860, _size857) = iprot.readListBegin()
-          for _i861 in xrange(_size857):
-            _elem862 = SQLNotNullConstraint()
-            _elem862.read(iprot)
-            self.notNullConstraints.append(_elem862)
+          (_etype867, _size864) = iprot.readListBegin()
+          for _i868 in xrange(_size864):
+            _elem869 = SQLNotNullConstraint()
+            _elem869.read(iprot)
+            self.notNullConstraints.append(_elem869)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 6:
         if ftype == TType.LIST:
           self.defaultConstraints = []
-          (_etype866, _size863) = iprot.readListBegin()
-          for _i867 in xrange(_size863):
-            _elem868 = SQLDefaultConstraint()
-            _elem868.read(iprot)
-            self.defaultConstraints.append(_elem868)
+          (_etype873, _size870) = iprot.readListBegin()
+          for _i874 in xrange(_size870):
+            _elem875 = SQLDefaultConstraint()
+            _elem875.read(iprot)
+            self.defaultConstraints.append(_elem875)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 7:
         if ftype == TType.LIST:
           self.checkConstraints = []
-          (_etype872, _size869) = iprot.readListBegin()
-          for _i873 in xrange(_size869):
-            _elem874 = SQLCheckConstraint()
-            _elem874.read(iprot)
-            self.checkConstraints.append(_elem874)
+          (_etype879, _size876) = iprot.readListBegin()
+          for _i880 in xrange(_size876):
+            _elem881 = SQLCheckConstraint()
+            _elem881.read(iprot)
+            self.checkConstraints.append(_elem881)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16790,43 +17661,43 @@ class create_table_with_constraints_args:
     if self.primaryKeys is not None:
       oprot.writeFieldBegin('primaryKeys', TType.LIST, 2)
       oprot.writeListBegin(TType.STRUCT, len(self.primaryKeys))
-      for iter875 in self.primaryKeys:
-        iter875.write(oprot)
+      for iter882 in self.primaryKeys:
+        iter882.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.foreignKeys is not None:
       oprot.writeFieldBegin('foreignKeys', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.foreignKeys))
-      for iter876 in self.foreignKeys:
-        iter876.write(oprot)
+      for iter883 in self.foreignKeys:
+        iter883.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.uniqueConstraints is not None:
       oprot.writeFieldBegin('uniqueConstraints', TType.LIST, 4)
       oprot.writeListBegin(TType.STRUCT, len(self.uniqueConstraints))
-      for iter877 in self.uniqueConstraints:
-        iter877.write(oprot)
+      for iter884 in self.uniqueConstraints:
+        iter884.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.notNullConstraints is not None:
       oprot.writeFieldBegin('notNullConstraints', TType.LIST, 5)
       oprot.writeListBegin(TType.STRUCT, len(self.notNullConstraints))
-      for iter878 in self.notNullConstraints:
-        iter878.write(oprot)
+      for iter885 in self.notNullConstraints:
+        iter885.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.defaultConstraints is not None:
       oprot.writeFieldBegin('defaultConstraints', TType.LIST, 6)
       oprot.writeListBegin(TType.STRUCT, len(self.defaultConstraints))
-      for iter879 in self.defaultConstraints:
-        iter879.write(oprot)
+      for iter886 in self.defaultConstraints:
+        iter886.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.checkConstraints is not None:
       oprot.writeFieldBegin('checkConstraints', TType.LIST, 7)
       oprot.writeListBegin(TType.STRUCT, len(self.checkConstraints))
-      for iter880 in self.checkConstraints:
-        iter880.write(oprot)
+      for iter887 in self.checkConstraints:
+        iter887.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -18386,10 +19257,10 @@ class truncate_table_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.partNames = []
-          (_etype884, _size881) = iprot.readListBegin()
-          for _i885 in xrange(_size881):
-            _elem886 = iprot.readString()
-            self.partNames.append(_elem886)
+          (_etype891, _size888) = iprot.readListBegin()
+          for _i892 in xrange(_size888):
+            _elem893 = iprot.readString()
+            self.partNames.append(_elem893)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18414,8 +19285,8 @@ class truncate_table_args:
     if self.partNames is not None:
       oprot.writeFieldBegin('partNames', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.partNames))
-      for iter887 in self.partNames:
-        oprot.writeString(iter887)
+      for iter894 in self.partNames:
+        oprot.writeString(iter894)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -18615,10 +19486,10 @@ class get_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype891, _size888) = iprot.readListBegin()
-          for _i892 in xrange(_size888):
-            _elem893 = iprot.readString()
-            self.success.append(_elem893)
+          (_etype898, _size895) = iprot.readListBegin()
+          for _i899 in xrange(_size895):
+            _elem900 = iprot.readString()
+            self.success.append(_elem900)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18641,8 +19512,8 @@ class get_tables_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter894 in self.success:
-        oprot.writeString(iter894)
+      for iter901 in self.success:
+        oprot.writeString(iter901)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18792,10 +19663,10 @@ class get_tables_by_type_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype898, _size895) = iprot.readListBegin()
-          for _i899 in xrange(_size895):
-            _elem900 = iprot.readString()
-            self.success.append(_elem900)
+          (_etype905, _size902) = iprot.readListBegin()
+          for _i906 in xrange(_size902):
+            _elem907 = iprot.readString()
+            self.success.append(_elem907)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18818,8 +19689,8 @@ class get_tables_by_type_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter901 in self.success:
-        oprot.writeString(iter901)
+      for iter908 in self.success:
+        oprot.writeString(iter908)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18943,10 +19814,10 @@ class get_materialized_views_for_rewriting_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype905, _size902) = iprot.readListBegin()
-          for _i906 in xrange(_size902):
-            _elem907 = iprot.readString()
-            self.success.append(_elem907)
+          (_etype912, _size909) = iprot.readListBegin()
+          for _i913 in xrange(_size909):
+            _elem914 = iprot.readString()
+            self.success.append(_elem914)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18969,8 +19840,8 @@ class get_materialized_views_for_rewriting_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter908 in self.success:
-        oprot.writeString(iter908)
+      for iter915 in self.success:
+        oprot.writeString(iter915)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -19043,10 +19914,10 @@ class get_table_meta_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.tbl_types = []
-          (_etype912, _size909) = iprot.readListBegin()
-          for _i913 in xrange(_size909):
-            _elem914 = iprot.readString()
-            self.tbl_types.append(_elem914)
+          (_etype919, _size916) = iprot.readListBegin()
+          for _i920 in xrange(_size916):
+            _elem921 = iprot.readString()
+            self.tbl_types.append(_elem921)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19071,8 +19942,8 @@ class get_table_meta_args:
     if self.tbl_types is not None:
       oprot.writeFieldBegin('tbl_types', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.tbl_types))
-      for iter915 in self.tbl_types:
-        oprot.writeString(iter915)
+      for iter922 in self.tbl_types:
+        oprot.writeString(iter922)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -19128,11 +19999,11 @@ class get_table_meta_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype919, _size916) = iprot.readListBegin()
-          for _i920 in xrange(_size916):
-            _elem921 = TableMeta()
-            _elem921.read(iprot)
-            self.success.append(_elem921)
+          (_etype926, _size923) = iprot.readListBegin()
+          for _i927 in xrange(_size923):
+            _elem928 = TableMeta()
+            _elem928.read(iprot)
+            self.success.append(_elem928)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19155,8 +20026,8 @@ class get_table_meta_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter922 in self.success:
-        iter922.write(oprot)
+      for iter929 in self.success:
+        iter929.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -19280,10 +20151,10 @@ class get_all_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype926, _size923) = iprot.readListBegin()
-          for _i927 in xrange(_size923):
-            _elem928 = iprot.readString()
-            self.success.append(_elem928)
+          (_etype933, _size930) = iprot.readListBegin()
+          for _i934 in xrange(_size930):
+            _elem935 = iprot.readString()
+            self.success.append(_elem935)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19306,8 +20177,8 @@ class get_all_tables_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter929 in self.success:
-        oprot.writeString(iter929)
+      for iter936 in self.success:
+        oprot.writeString(iter936)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -19543,10 +20414,10 @@ class get_table_objects_by_name_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tbl_names = []
-          (_etype933, _size930) = iprot.readListBegin()
-          for _i934 in xrange(_size930):
-            _elem935 = iprot.readString()
-            self.tbl_names.append(_elem935)
+          (_etype940, _size937) = iprot.readListBegin()
+          for _i941 in xrange(_size937):
+            _elem942 = iprot.readString()
+            self.tbl_names.append(_elem942)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19567,8 +20438,8 @@ class get_table_objects_by_name_args:
     if self.tbl_names is not None:
       oprot.writeFieldBegin('tbl_names', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.tbl_names))
-      for iter936 in self.tbl_names:
-        oprot.writeString(iter936)
+      for iter943 in self.tbl_names:
+        oprot.writeString(iter943)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -19620,11 +20491,11 @@ class get_table_objects_by_name_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype940, _size937) = iprot.readListBegin()
-          for _i941 in xrange(_size937):
-            _elem942 = Table()
-            _elem942.read(iprot)
-            self.success.append(_elem942)
+          (_etype947, _size944) = iprot.readListBegin()
+          for _i948 in xrange(_size944):
+            _elem949 = Table()
+            _elem949.read(iprot)
+            self.success.append(_elem949)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19641,8 +20512,8 @@ class get_table_objects_by_name_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter943 in self.success:
-        iter943.write(oprot)
+      for iter950 in self.success:
+        iter950.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20034,10 +20905,10 @@ class get_materialization_invalidation_info_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tbl_names = []
-          (_etype947, _size944) = iprot.readListBegin()
-          for _i948 in xrange(_size944):
-            _elem949 = iprot.readString()
-            self.tbl_names.append(_elem949)
+          (_etype954, _size951) = iprot.readListBegin()
+          for _i955 in xrange(_size951):
+            _elem956 = iprot.readString()
+            self.tbl_names.append(_elem956)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20058,8 +20929,8 @@ class get_materialization_invalidation_info_args:
     if self.tbl_names is not None:
       oprot.writeFieldBegin('tbl_names', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.tbl_names))
-      for iter950 in self.tbl_names:
-        oprot.writeString(iter950)
+      for iter957 in self.tbl_names:
+        oprot.writeString(iter957)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20120,12 +20991,12 @@ class get_materialization_invalidation_info_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype952, _vtype953, _size951 ) = iprot.readMapBegin()
-          for _i955 in xrange(_size951):
-            _key956 = iprot.readString()
-            _val957 = Materialization()
-            _val957.read(iprot)
-            self.success[_key956] = _val957
+          (_ktype959, _vtype960, _size958 ) = iprot.readMapBegin()
+          for _i962 in xrange(_size958):
+            _key963 = iprot.readString()
+            _val964 = Materialization()
+            _val964.read(iprot)
+            self.success[_key963] = _val964
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -20160,9 +21031,9 @@ class get_materialization_invalidation_info_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.MAP, 0)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.success))
-      for kiter958,viter959 in self.success.items():
-        oprot.writeString(kiter958)
-        viter959.write(oprot)
+      for kiter965,viter966 in self.success.items():
+        oprot.writeString(kiter965)
+        viter966.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20206,6 +21077,7 @@ class get_materialization_invalidation_info_result:
 class update_creation_metadata_args:
   """
   Attributes:
+   - catName
    - dbname
    - tbl_name
    - creation_metadata
@@ -20213,12 +21085,14 @@ class update_creation_metadata_args:
 
   thrift_spec = (
     None, # 0
-    (1, TType.STRING, 'dbname', None, None, ), # 1
-    (2, TType.STRING, 'tbl_name', None, None, ), # 2
-    (3, TType.STRUCT, 'creation_metadata', (CreationMetadata, CreationMetadata.thrift_spec), None, ), # 3
+    (1, TType.STRING, 'catName', None, None, ), # 1
+    (2, TType.STRING, 'dbname', None, None, ), # 2
+    (3, TType.STRING, 'tbl_name', None, None, ), # 3
+    (4, TType.STRUCT, 'creation_metadata', (CreationMetadata, CreationMetadata.thrift_spec), None, ), # 4
   )
 
-  def __init__(self, dbname=None, tbl_name=None, creation_metadata=None,):
+  def __init__(self, catName=None, dbname=None, tbl_name=None, creation_metadata=None,):
+    self.catName = catName
     self.dbname = dbname
     self.tbl_name = tbl_name
     self.creation_metadata = creation_metadata
@@ -20234,15 +21108,20 @@ class update_creation_metadata_args:
         break
       if fid == 1:
         if ftype == TType.STRING:
-          self.dbname = iprot.readString()
+          self.catName = iprot.readString()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.STRING:
-          self.tbl_name = iprot.readString()
+          self.dbname = iprot.readString()
         else:
           iprot.skip(ftype)
       elif fid == 3:
+        if ftype == TType.STRING:
+          self.tbl_name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
         if ftype == TType.STRUCT:
           self.creation_metadata = CreationMetadata()
           self.creation_metadata.read(iprot)
@@ -20258,16 +21137,20 @@ class update_creation_metadata_args:
       oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
       return
     oprot.writeStructBegin('update_creation_metadata_args')
+    if self.catName is not None:
+      oprot.writeFieldBegin('catName', TType.STRING, 1)
+      oprot.writeString(self.catName)
+      oprot.writeFieldEnd()
     if self.dbname is not None:
-      oprot.writeFieldBegin('dbname', TType.STRING, 1)
+      oprot.writeFieldBegin('dbname', TType.STRING, 2)
       oprot.writeString(self.dbname)
       oprot.writeFieldEnd()
     if self.tbl_name is not None:
-      oprot.writeFieldBegin('tbl_name', TType.STRING, 2)
+      oprot.writeFieldBegin('tbl_name', TType.STRING, 3)
       oprot.writeString(self.tbl_name)
       oprot.writeFieldEnd()
     if self.creation_metadata is not None:
-      oprot.writeFieldBegin('creation_metadata', TType.STRUCT, 3)
+      oprot.writeFieldBegin('creation_metadata', TType.STRUCT, 4)
       self.creation_metadata.write(oprot)
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20279,6 +21162,7 @@ class update_creation_metadata_args:
 
   def __hash__(self):
     value = 17
+    value = (value * 31) ^ hash(self.catName)
     value = (value * 31) ^ hash(self.dbname)
     value = (value * 31) ^ hash(self.tbl_name)
     value = (value * 31) ^ hash(self.creation_metadata)
@@ -20514,10 +21398,10 @@ class get_table_names_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype963, _size960) = iprot.readListBegin()
-          for _i964 in xrange(_size960):
-            _elem965 = iprot.readString()
-            self.success.append(_elem965)
+          (_etype970, _size967) = iprot.readListBegin()
+          for _i971 in xrange(_size967):
+            _elem972 = iprot.readString()
+            self.success.append(_elem972)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20552,8 +21436,8 @@ class get_table_names_by_filter_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter966 in self.success:
-        oprot.writeString(iter966)
+      for iter973 in self.success:
+        oprot.writeString(iter973)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21523,11 +22407,11 @@ class add_partitions_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype970, _size967) = iprot.readListBegin()
-          for _i971 in xrange(_size967):
-            _elem972 = Partition()
-            _elem972.read(iprot)
-            self.new_parts.append(_elem972)
+          (_etype977, _size974) = iprot.readListBegin()
+          for _i978 in xrange(_size974):
+            _elem979 = Partition()
+            _elem979.read(iprot)
+            self.new_parts.append(_elem979)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21544,8 +22428,8 @@ class add_partitions_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter973 in self.new_parts:
-        iter973.write(oprot)
+      for iter980 in self.new_parts:
+        iter980.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -21703,11 +22587,11 @@ class add_partitions_pspec_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype977, _size974) = iprot.readListBegin()
-          for _i978 in xrange(_size974):
-            _elem979 = PartitionSpec()
-            _elem979.read(iprot)
-            self.new_parts.append(_elem979)
+          (_etype984, _size981) = iprot.readListBegin()
+          for _i985 in xrange(_size981):
+            _elem986 = PartitionSpec()
+            _elem986.read(iprot)
+            self.new_parts.append(_elem986)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21724,8 +22608,8 @@ class add_partitions_pspec_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter980 in self.new_parts:
-        iter980.write(oprot)
+      for iter987 in self.new_parts:
+        iter987.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -21899,10 +22783,10 @@ class append_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype984, _size981) = iprot.readListBegin()
-          for _i985 in xrange(_size981):
-            _elem986 = iprot.readString()
-            self.part_vals.append(_elem986)
+          (_etype991, _size988) = iprot.readListBegin()
+          for _i992 in xrange(_size988):
+            _elem993 = iprot.readString()
+            self.part_vals.append(_elem993)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21927,8 +22811,8 @@ class append_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter987 in self.part_vals:
-        oprot.writeString(iter987)
+      for iter994 in self.part_vals:
+        oprot.writeString(iter994)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -22281,10 +23165,10 @@ class append_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype991, _size988) = iprot.readListBegin()
-          for _i992 in xrange(_size988):
-            _elem993 = iprot.readString()
-            self.part_vals.append(_elem993)
+          (_etype998, _size995) = iprot.readListBegin()
+          for _i999 in xrange(_size995):
+            _elem1000 = iprot.readString()
+            self.part_vals.append(_elem1000)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22315,8 +23199,8 @@ class append_partition_with_environment_context_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter994 in self.part_vals:
-        oprot.writeString(iter994)
+      for iter1001 in self.part_vals:
+        oprot.writeString(iter1001)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -22911,10 +23795,10 @@ class drop_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype998, _size995) = iprot.readListBegin()
-          for _i999 in xrange(_size995):
-            _elem1000 = iprot.readString()
-            self.part_vals.append(_elem1000)
+          (_etype1005, _size1002) = iprot.readListBegin()
+          for _i1006 in xrange(_size1002):
+            _elem1007 = iprot.readString()
+            self.part_vals.append(_elem1007)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22944,8 +23828,8 @@ class drop_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1001 in self.part_vals:
-        oprot.writeString(iter1001)
+      for iter1008 in self.part_vals:
+        oprot.writeString(iter1008)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -23118,10 +24002,10 @@ class drop_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1005, _size1002) = iprot.readListBegin()
-          for _i1006 in xrange(_size1002):
-            _elem1007 = iprot.readString()
-            self.part_vals.append(_elem1007)
+          (_etype1012, _size1009) = iprot.readListBegin()
+          for _i1013 in xrange(_size1009):
+            _elem1014 = iprot.readString()
+            self.part_vals.append(_elem1014)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23157,8 +24041,8 @@ class drop_partition_with_environment_context_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1008 in self.part_vals:
-        oprot.writeString(iter1008)
+      for iter1015 in self.part_vals:
+        oprot.writeString(iter1015)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -23895,10 +24779,10 @@ class get_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1012, _size1009) = iprot.readListBegin()
-          for _i1013 in xrange(_size1009):
-            _elem1014 = iprot.readString()
-            self.part_vals.append(_elem1014)
+          (_etype1019, _size1016) = iprot.readListBegin()
+          for _i1020 in xrange(_size1016):
+            _elem1021 = iprot.readString()
+            self.part_vals.append(_elem1021)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23923,8 +24807,8 @@ class get_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1015 in self.part_vals:
-        oprot.writeString(iter1015)
+      for iter1022 in self.part_vals:
+        oprot.writeString(iter1022)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -24083,11 +24967,11 @@ class exchange_partition_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype1017, _vtype1018, _size1016 ) = iprot.readMapBegin()
-          for _i1020 in xrange(_size1016):
-            _key1021 = iprot.readString()
-            _v

<TRUNCATED>

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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php b/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
index d00d11b..a15a387 100644
--- a/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
+++ b/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
@@ -33,6 +33,32 @@ interface ThriftHiveMetastoreIf extends \FacebookServiceIf {
    */
   public function setMetaConf($key, $value);
   /**
+   * @param \metastore\CreateCatalogRequest $catalog
+   * @throws \metastore\AlreadyExistsException
+   * @throws \metastore\InvalidObjectException
+   * @throws \metastore\MetaException
+   */
+  public function create_catalog(\metastore\CreateCatalogRequest $catalog);
+  /**
+   * @param \metastore\GetCatalogRequest $catName
+   * @return \metastore\GetCatalogResponse
+   * @throws \metastore\NoSuchObjectException
+   * @throws \metastore\MetaException
+   */
+  public function get_catalog(\metastore\GetCatalogRequest $catName);
+  /**
+   * @return \metastore\GetCatalogsResponse
+   * @throws \metastore\MetaException
+   */
+  public function get_catalogs();
+  /**
+   * @param \metastore\DropCatalogRequest $catName
+   * @throws \metastore\NoSuchObjectException
+   * @throws \metastore\InvalidOperationException
+   * @throws \metastore\MetaException
+   */
+  public function drop_catalog(\metastore\DropCatalogRequest $catName);
+  /**
    * @param \metastore\Database $database
    * @throws \metastore\AlreadyExistsException
    * @throws \metastore\InvalidObjectException
@@ -310,6 +336,7 @@ interface ThriftHiveMetastoreIf extends \FacebookServiceIf {
    */
   public function get_materialization_invalidation_info($dbname, array $tbl_names);
   /**
+   * @param string $catName
    * @param string $dbname
    * @param string $tbl_name
    * @param \metastore\CreationMetadata $creation_metadata
@@ -317,7 +344,7 @@ interface ThriftHiveMetastoreIf extends \FacebookServiceIf {
    * @throws \metastore\InvalidOperationException
    * @throws \metastore\UnknownDBException
    */
-  public function update_creation_metadata($dbname, $tbl_name, \metastore\CreationMetadata $creation_metadata);
+  public function update_creation_metadata($catName, $dbname, $tbl_name, \metastore\CreationMetadata $creation_metadata);
   /**
    * @param string $dbname
    * @param string $filter
@@ -1600,6 +1627,230 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas
     return;
   }
 
+  public function create_catalog(\metastore\CreateCatalogRequest $catalog)
+  {
+    $this->send_create_catalog($catalog);
+    $this->recv_create_catalog();
+  }
+
+  public function send_create_catalog(\metastore\CreateCatalogRequest $catalog)
+  {
+    $args = new \metastore\ThriftHiveMetastore_create_catalog_args();
+    $args->catalog = $catalog;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
+    {
+      thrift_protocol_write_binary($this->output_, 'create_catalog', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('create_catalog', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
+
+  public function recv_create_catalog()
+  {
+    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_create_catalog_result', $this->input_->isStrictRead());
+    else
+    {
+      $rseqid = 0;
+      $fname = null;
+      $mtype = 0;
+
+      $this->input_->readMessageBegin($fname, $mtype, $rseqid);
+      if ($mtype == TMessageType::EXCEPTION) {
+        $x = new TApplicationException();
+        $x->read($this->input_);
+        $this->input_->readMessageEnd();
+        throw $x;
+      }
+      $result = new \metastore\ThriftHiveMetastore_create_catalog_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
+    }
+    if ($result->o1 !== null) {
+      throw $result->o1;
+    }
+    if ($result->o2 !== null) {
+      throw $result->o2;
+    }
+    if ($result->o3 !== null) {
+      throw $result->o3;
+    }
+    return;
+  }
+
+  public function get_catalog(\metastore\GetCatalogRequest $catName)
+  {
+    $this->send_get_catalog($catName);
+    return $this->recv_get_catalog();
+  }
+
+  public function send_get_catalog(\metastore\GetCatalogRequest $catName)
+  {
+    $args = new \metastore\ThriftHiveMetastore_get_catalog_args();
+    $args->catName = $catName;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
+    {
+      thrift_protocol_write_binary($this->output_, 'get_catalog', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('get_catalog', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
+
+  public function recv_get_catalog()
+  {
+    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_get_catalog_result', $this->input_->isStrictRead());
+    else
+    {
+      $rseqid = 0;
+      $fname = null;
+      $mtype = 0;
+
+      $this->input_->readMessageBegin($fname, $mtype, $rseqid);
+      if ($mtype == TMessageType::EXCEPTION) {
+        $x = new TApplicationException();
+        $x->read($this->input_);
+        $this->input_->readMessageEnd();
+        throw $x;
+      }
+      $result = new \metastore\ThriftHiveMetastore_get_catalog_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
+    }
+    if ($result->success !== null) {
+      return $result->success;
+    }
+    if ($result->o1 !== null) {
+      throw $result->o1;
+    }
+    if ($result->o2 !== null) {
+      throw $result->o2;
+    }
+    throw new \Exception("get_catalog failed: unknown result");
+  }
+
+  public function get_catalogs()
+  {
+    $this->send_get_catalogs();
+    return $this->recv_get_catalogs();
+  }
+
+  public function send_get_catalogs()
+  {
+    $args = new \metastore\ThriftHiveMetastore_get_catalogs_args();
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
+    {
+      thrift_protocol_write_binary($this->output_, 'get_catalogs', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('get_catalogs', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
+
+  public function recv_get_catalogs()
+  {
+    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_get_catalogs_result', $this->input_->isStrictRead());
+    else
+    {
+      $rseqid = 0;
+      $fname = null;
+      $mtype = 0;
+
+      $this->input_->readMessageBegin($fname, $mtype, $rseqid);
+      if ($mtype == TMessageType::EXCEPTION) {
+        $x = new TApplicationException();
+        $x->read($this->input_);
+        $this->input_->readMessageEnd();
+        throw $x;
+      }
+      $result = new \metastore\ThriftHiveMetastore_get_catalogs_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
+    }
+    if ($result->success !== null) {
+      return $result->success;
+    }
+    if ($result->o1 !== null) {
+      throw $result->o1;
+    }
+    throw new \Exception("get_catalogs failed: unknown result");
+  }
+
+  public function drop_catalog(\metastore\DropCatalogRequest $catName)
+  {
+    $this->send_drop_catalog($catName);
+    $this->recv_drop_catalog();
+  }
+
+  public function send_drop_catalog(\metastore\DropCatalogRequest $catName)
+  {
+    $args = new \metastore\ThriftHiveMetastore_drop_catalog_args();
+    $args->catName = $catName;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
+    {
+      thrift_protocol_write_binary($this->output_, 'drop_catalog', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('drop_catalog', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
+
+  public function recv_drop_catalog()
+  {
+    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_drop_catalog_result', $this->input_->isStrictRead());
+    else
+    {
+      $rseqid = 0;
+      $fname = null;
+      $mtype = 0;
+
+      $this->input_->readMessageBegin($fname, $mtype, $rseqid);
+      if ($mtype == TMessageType::EXCEPTION) {
+        $x = new TApplicationException();
+        $x->read($this->input_);
+        $this->input_->readMessageEnd();
+        throw $x;
+      }
+      $result = new \metastore\ThriftHiveMetastore_drop_catalog_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
+    }
+    if ($result->o1 !== null) {
+      throw $result->o1;
+    }
+    if ($result->o2 !== null) {
+      throw $result->o2;
+    }
+    if ($result->o3 !== null) {
+      throw $result->o3;
+    }
+    return;
+  }
+
   public function create_database(\metastore\Database $database)
   {
     $this->send_create_database($database);
@@ -3703,15 +3954,16 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas
     throw new \Exception("get_materialization_invalidation_info failed: unknown result");
   }
 
-  public function update_creation_metadata($dbname, $tbl_name, \metastore\CreationMetadata $creation_metadata)
+  public function update_creation_metadata($catName, $dbname, $tbl_name, \metastore\CreationMetadata $creation_metadata)
   {
-    $this->send_update_creation_metadata($dbname, $tbl_name, $creation_metadata);
+    $this->send_update_creation_metadata($catName, $dbname, $tbl_name, $creation_metadata);
     $this->recv_update_creation_metadata();
   }
 
-  public function send_update_creation_metadata($dbname, $tbl_name, \metastore\CreationMetadata $creation_metadata)
+  public function send_update_creation_metadata($catName, $dbname, $tbl_name, \metastore\CreationMetadata $creation_metadata)
   {
     $args = new \metastore\ThriftHiveMetastore_update_creation_metadata_args();
+    $args->catName = $catName;
     $args->dbname = $dbname;
     $args->tbl_name = $tbl_name;
     $args->creation_metadata = $creation_metadata;
@@ -12989,33 +13241,33 @@ class ThriftHiveMetastore_setMetaConf_result {
 
 }
 
-class ThriftHiveMetastore_create_database_args {
+class ThriftHiveMetastore_create_catalog_args {
   static $_TSPEC;
 
   /**
-   * @var \metastore\Database
+   * @var \metastore\CreateCatalogRequest
    */
-  public $database = null;
+  public $catalog = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
       self::$_TSPEC = array(
         1 => array(
-          'var' => 'database',
+          'var' => 'catalog',
           'type' => TType::STRUCT,
-          'class' => '\metastore\Database',
+          'class' => '\metastore\CreateCatalogRequest',
           ),
         );
     }
     if (is_array($vals)) {
-      if (isset($vals['database'])) {
-        $this->database = $vals['database'];
+      if (isset($vals['catalog'])) {
+        $this->catalog = $vals['catalog'];
       }
     }
   }
 
   public function getName() {
-    return 'ThriftHiveMetastore_create_database_args';
+    return 'ThriftHiveMetastore_create_catalog_args';
   }
 
   public function read($input)
@@ -13035,8 +13287,8 @@ class ThriftHiveMetastore_create_database_args {
       {
         case 1:
           if ($ftype == TType::STRUCT) {
-            $this->database = new \metastore\Database();
-            $xfer += $this->database->read($input);
+            $this->catalog = new \metastore\CreateCatalogRequest();
+            $xfer += $this->catalog->read($input);
           } else {
             $xfer += $input->skip($ftype);
           }
@@ -13053,13 +13305,13 @@ class ThriftHiveMetastore_create_database_args {
 
   public function write($output) {
     $xfer = 0;
-    $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_database_args');
-    if ($this->database !== null) {
-      if (!is_object($this->database)) {
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_catalog_args');
+    if ($this->catalog !== null) {
+      if (!is_object($this->catalog)) {
         throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
       }
-      $xfer += $output->writeFieldBegin('database', TType::STRUCT, 1);
-      $xfer += $this->database->write($output);
+      $xfer += $output->writeFieldBegin('catalog', TType::STRUCT, 1);
+      $xfer += $this->catalog->write($output);
       $xfer += $output->writeFieldEnd();
     }
     $xfer += $output->writeFieldStop();
@@ -13069,7 +13321,7 @@ class ThriftHiveMetastore_create_database_args {
 
 }
 
-class ThriftHiveMetastore_create_database_result {
+class ThriftHiveMetastore_create_catalog_result {
   static $_TSPEC;
 
   /**
@@ -13119,7 +13371,7 @@ class ThriftHiveMetastore_create_database_result {
   }
 
   public function getName() {
-    return 'ThriftHiveMetastore_create_database_result';
+    return 'ThriftHiveMetastore_create_catalog_result';
   }
 
   public function read($input)
@@ -13173,7 +13425,7 @@ class ThriftHiveMetastore_create_database_result {
 
   public function write($output) {
     $xfer = 0;
-    $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_database_result');
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_catalog_result');
     if ($this->o1 !== null) {
       $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
       $xfer += $this->o1->write($output);
@@ -13196,32 +13448,33 @@ class ThriftHiveMetastore_create_database_result {
 
 }
 
-class ThriftHiveMetastore_get_database_args {
+class ThriftHiveMetastore_get_catalog_args {
   static $_TSPEC;
 
   /**
-   * @var string
+   * @var \metastore\GetCatalogRequest
    */
-  public $name = null;
+  public $catName = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
       self::$_TSPEC = array(
         1 => array(
-          'var' => 'name',
-          'type' => TType::STRING,
+          'var' => 'catName',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\GetCatalogRequest',
           ),
         );
     }
     if (is_array($vals)) {
-      if (isset($vals['name'])) {
-        $this->name = $vals['name'];
+      if (isset($vals['catName'])) {
+        $this->catName = $vals['catName'];
       }
     }
   }
 
   public function getName() {
-    return 'ThriftHiveMetastore_get_database_args';
+    return 'ThriftHiveMetastore_get_catalog_args';
   }
 
   public function read($input)
@@ -13240,8 +13493,9 @@ class ThriftHiveMetastore_get_database_args {
       switch ($fid)
       {
         case 1:
-          if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->name);
+          if ($ftype == TType::STRUCT) {
+            $this->catName = new \metastore\GetCatalogRequest();
+            $xfer += $this->catName->read($input);
           } else {
             $xfer += $input->skip($ftype);
           }
@@ -13258,10 +13512,13 @@ class ThriftHiveMetastore_get_database_args {
 
   public function write($output) {
     $xfer = 0;
-    $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_database_args');
-    if ($this->name !== null) {
-      $xfer += $output->writeFieldBegin('name', TType::STRING, 1);
-      $xfer += $output->writeString($this->name);
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_catalog_args');
+    if ($this->catName !== null) {
+      if (!is_object($this->catName)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('catName', TType::STRUCT, 1);
+      $xfer += $this->catName->write($output);
       $xfer += $output->writeFieldEnd();
     }
     $xfer += $output->writeFieldStop();
@@ -13271,11 +13528,11 @@ class ThriftHiveMetastore_get_database_args {
 
 }
 
-class ThriftHiveMetastore_get_database_result {
+class ThriftHiveMetastore_get_catalog_result {
   static $_TSPEC;
 
   /**
-   * @var \metastore\Database
+   * @var \metastore\GetCatalogResponse
    */
   public $success = null;
   /**
@@ -13293,7 +13550,7 @@ class ThriftHiveMetastore_get_database_result {
         0 => array(
           'var' => 'success',
           'type' => TType::STRUCT,
-          'class' => '\metastore\Database',
+          'class' => '\metastore\GetCatalogResponse',
           ),
         1 => array(
           'var' => 'o1',
@@ -13321,7 +13578,7 @@ class ThriftHiveMetastore_get_database_result {
   }
 
   public function getName() {
-    return 'ThriftHiveMetastore_get_database_result';
+    return 'ThriftHiveMetastore_get_catalog_result';
   }
 
   public function read($input)
@@ -13341,7 +13598,7 @@ class ThriftHiveMetastore_get_database_result {
       {
         case 0:
           if ($ftype == TType::STRUCT) {
-            $this->success = new \metastore\Database();
+            $this->success = new \metastore\GetCatalogResponse();
             $xfer += $this->success->read($input);
           } else {
             $xfer += $input->skip($ftype);
@@ -13375,7 +13632,7 @@ class ThriftHiveMetastore_get_database_result {
 
   public function write($output) {
     $xfer = 0;
-    $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_database_result');
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_catalog_result');
     if ($this->success !== null) {
       if (!is_object($this->success)) {
         throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
@@ -13401,54 +13658,95 @@ class ThriftHiveMetastore_get_database_result {
 
 }
 
-class ThriftHiveMetastore_drop_database_args {
+class ThriftHiveMetastore_get_catalogs_args {
+  static $_TSPEC;
+
+
+  public function __construct() {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        );
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_get_catalogs_args';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_catalogs_args');
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_get_catalogs_result {
   static $_TSPEC;
 
   /**
-   * @var string
-   */
-  public $name = null;
-  /**
-   * @var bool
+   * @var \metastore\GetCatalogsResponse
    */
-  public $deleteData = null;
+  public $success = null;
   /**
-   * @var bool
+   * @var \metastore\MetaException
    */
-  public $cascade = null;
+  public $o1 = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
       self::$_TSPEC = array(
-        1 => array(
-          'var' => 'name',
-          'type' => TType::STRING,
-          ),
-        2 => array(
-          'var' => 'deleteData',
-          'type' => TType::BOOL,
+        0 => array(
+          'var' => 'success',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\GetCatalogsResponse',
           ),
-        3 => array(
-          'var' => 'cascade',
-          'type' => TType::BOOL,
+        1 => array(
+          'var' => 'o1',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\MetaException',
           ),
         );
     }
     if (is_array($vals)) {
-      if (isset($vals['name'])) {
-        $this->name = $vals['name'];
-      }
-      if (isset($vals['deleteData'])) {
-        $this->deleteData = $vals['deleteData'];
+      if (isset($vals['success'])) {
+        $this->success = $vals['success'];
       }
-      if (isset($vals['cascade'])) {
-        $this->cascade = $vals['cascade'];
+      if (isset($vals['o1'])) {
+        $this->o1 = $vals['o1'];
       }
     }
   }
 
   public function getName() {
-    return 'ThriftHiveMetastore_drop_database_args';
+    return 'ThriftHiveMetastore_get_catalogs_result';
   }
 
   public function read($input)
@@ -13466,23 +13764,18 @@ class ThriftHiveMetastore_drop_database_args {
       }
       switch ($fid)
       {
-        case 1:
-          if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->name);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        case 2:
-          if ($ftype == TType::BOOL) {
-            $xfer += $input->readBool($this->deleteData);
+        case 0:
+          if ($ftype == TType::STRUCT) {
+            $this->success = new \metastore\GetCatalogsResponse();
+            $xfer += $this->success->read($input);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 3:
-          if ($ftype == TType::BOOL) {
-            $xfer += $input->readBool($this->cascade);
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->o1 = new \metastore\MetaException();
+            $xfer += $this->o1->read($input);
           } else {
             $xfer += $input->skip($ftype);
           }
@@ -13499,20 +13792,98 @@ class ThriftHiveMetastore_drop_database_args {
 
   public function write($output) {
     $xfer = 0;
-    $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_database_args');
-    if ($this->name !== null) {
-      $xfer += $output->writeFieldBegin('name', TType::STRING, 1);
-      $xfer += $output->writeString($this->name);
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_catalogs_result');
+    if ($this->success !== null) {
+      if (!is_object($this->success)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0);
+      $xfer += $this->success->write($output);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->deleteData !== null) {
-      $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 2);
-      $xfer += $output->writeBool($this->deleteData);
+    if ($this->o1 !== null) {
+      $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
+      $xfer += $this->o1->write($output);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->cascade !== null) {
-      $xfer += $output->writeFieldBegin('cascade', TType::BOOL, 3);
-      $xfer += $output->writeBool($this->cascade);
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_drop_catalog_args {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\DropCatalogRequest
+   */
+  public $catName = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'catName',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\DropCatalogRequest',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['catName'])) {
+        $this->catName = $vals['catName'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_drop_catalog_args';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->catName = new \metastore\DropCatalogRequest();
+            $xfer += $this->catName->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_catalog_args');
+    if ($this->catName !== null) {
+      if (!is_object($this->catName)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('catName', TType::STRUCT, 1);
+      $xfer += $this->catName->write($output);
       $xfer += $output->writeFieldEnd();
     }
     $xfer += $output->writeFieldStop();
@@ -13522,7 +13893,667 @@ class ThriftHiveMetastore_drop_database_args {
 
 }
 
-class ThriftHiveMetastore_drop_database_result {
+class ThriftHiveMetastore_drop_catalog_result {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\NoSuchObjectException
+   */
+  public $o1 = null;
+  /**
+   * @var \metastore\InvalidOperationException
+   */
+  public $o2 = null;
+  /**
+   * @var \metastore\MetaException
+   */
+  public $o3 = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'o1',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\NoSuchObjectException',
+          ),
+        2 => array(
+          'var' => 'o2',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\InvalidOperationException',
+          ),
+        3 => array(
+          'var' => 'o3',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\MetaException',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['o1'])) {
+        $this->o1 = $vals['o1'];
+      }
+      if (isset($vals['o2'])) {
+        $this->o2 = $vals['o2'];
+      }
+      if (isset($vals['o3'])) {
+        $this->o3 = $vals['o3'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_drop_catalog_result';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->o1 = new \metastore\NoSuchObjectException();
+            $xfer += $this->o1->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRUCT) {
+            $this->o2 = new \metastore\InvalidOperationException();
+            $xfer += $this->o2->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::STRUCT) {
+            $this->o3 = new \metastore\MetaException();
+            $xfer += $this->o3->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_catalog_result');
+    if ($this->o1 !== null) {
+      $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
+      $xfer += $this->o1->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->o2 !== null) {
+      $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2);
+      $xfer += $this->o2->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->o3 !== null) {
+      $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3);
+      $xfer += $this->o3->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_create_database_args {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\Database
+   */
+  public $database = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'database',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\Database',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['database'])) {
+        $this->database = $vals['database'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_create_database_args';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->database = new \metastore\Database();
+            $xfer += $this->database->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_database_args');
+    if ($this->database !== null) {
+      if (!is_object($this->database)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('database', TType::STRUCT, 1);
+      $xfer += $this->database->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_create_database_result {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\AlreadyExistsException
+   */
+  public $o1 = null;
+  /**
+   * @var \metastore\InvalidObjectException
+   */
+  public $o2 = null;
+  /**
+   * @var \metastore\MetaException
+   */
+  public $o3 = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'o1',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\AlreadyExistsException',
+          ),
+        2 => array(
+          'var' => 'o2',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\InvalidObjectException',
+          ),
+        3 => array(
+          'var' => 'o3',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\MetaException',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['o1'])) {
+        $this->o1 = $vals['o1'];
+      }
+      if (isset($vals['o2'])) {
+        $this->o2 = $vals['o2'];
+      }
+      if (isset($vals['o3'])) {
+        $this->o3 = $vals['o3'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_create_database_result';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->o1 = new \metastore\AlreadyExistsException();
+            $xfer += $this->o1->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRUCT) {
+            $this->o2 = new \metastore\InvalidObjectException();
+            $xfer += $this->o2->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::STRUCT) {
+            $this->o3 = new \metastore\MetaException();
+            $xfer += $this->o3->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_database_result');
+    if ($this->o1 !== null) {
+      $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
+      $xfer += $this->o1->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->o2 !== null) {
+      $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2);
+      $xfer += $this->o2->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->o3 !== null) {
+      $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3);
+      $xfer += $this->o3->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_get_database_args {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $name = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'name',
+          'type' => TType::STRING,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['name'])) {
+        $this->name = $vals['name'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_get_database_args';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->name);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_database_args');
+    if ($this->name !== null) {
+      $xfer += $output->writeFieldBegin('name', TType::STRING, 1);
+      $xfer += $output->writeString($this->name);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_get_database_result {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\Database
+   */
+  public $success = null;
+  /**
+   * @var \metastore\NoSuchObjectException
+   */
+  public $o1 = null;
+  /**
+   * @var \metastore\MetaException
+   */
+  public $o2 = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        0 => array(
+          'var' => 'success',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\Database',
+          ),
+        1 => array(
+          'var' => 'o1',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\NoSuchObjectException',
+          ),
+        2 => array(
+          'var' => 'o2',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\MetaException',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['success'])) {
+        $this->success = $vals['success'];
+      }
+      if (isset($vals['o1'])) {
+        $this->o1 = $vals['o1'];
+      }
+      if (isset($vals['o2'])) {
+        $this->o2 = $vals['o2'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_get_database_result';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 0:
+          if ($ftype == TType::STRUCT) {
+            $this->success = new \metastore\Database();
+            $xfer += $this->success->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->o1 = new \metastore\NoSuchObjectException();
+            $xfer += $this->o1->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRUCT) {
+            $this->o2 = new \metastore\MetaException();
+            $xfer += $this->o2->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_database_result');
+    if ($this->success !== null) {
+      if (!is_object($this->success)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0);
+      $xfer += $this->success->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->o1 !== null) {
+      $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
+      $xfer += $this->o1->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->o2 !== null) {
+      $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2);
+      $xfer += $this->o2->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_drop_database_args {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $name = null;
+  /**
+   * @var bool
+   */
+  public $deleteData = null;
+  /**
+   * @var bool
+   */
+  public $cascade = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'name',
+          'type' => TType::STRING,
+          ),
+        2 => array(
+          'var' => 'deleteData',
+          'type' => TType::BOOL,
+          ),
+        3 => array(
+          'var' => 'cascade',
+          'type' => TType::BOOL,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['name'])) {
+        $this->name = $vals['name'];
+      }
+      if (isset($vals['deleteData'])) {
+        $this->deleteData = $vals['deleteData'];
+      }
+      if (isset($vals['cascade'])) {
+        $this->cascade = $vals['cascade'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_drop_database_args';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->name);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->deleteData);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->cascade);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_database_args');
+    if ($this->name !== null) {
+      $xfer += $output->writeFieldBegin('name', TType::STRING, 1);
+      $xfer += $output->writeString($this->name);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->deleteData !== null) {
+      $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 2);
+      $xfer += $output->writeBool($this->deleteData);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->cascade !== null) {
+      $xfer += $output->writeFieldBegin('cascade', TType::BOOL, 3);
+      $xfer += $output->writeBool($this->cascade);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_drop_database_result {
   static $_TSPEC;
 
   /**
@@ -13786,14 +14817,14 @@ class ThriftHiveMetastore_get_databases_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size792 = 0;
-            $_etype795 = 0;
-            $xfer += $input->readListBegin($_etype795, $_size792);
-            for ($_i796 = 0; $_i796 < $_size792; ++$_i796)
+            $_size799 = 0;
+            $_etype802 = 0;
+            $xfer += $input->readListBegin($_etype802, $_size799);
+            for ($_i803 = 0; $_i803 < $_size799; ++$_i803)
             {
-              $elem797 = null;
-              $xfer += $input->readString($elem797);
-              $this->success []= $elem797;
+              $elem804 = null;
+              $xfer += $input->readString($elem804);
+              $this->success []= $elem804;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13829,9 +14860,9 @@ class ThriftHiveMetastore_get_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter798)
+          foreach ($this->success as $iter805)
           {
-            $xfer += $output->writeString($iter798);
+            $xfer += $output->writeString($iter805);
           }
         }
         $output->writeListEnd();
@@ -13962,14 +14993,14 @@ class ThriftHiveMetastore_get_all_databases_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size799 = 0;
-            $_etype802 = 0;
-            $xfer += $input->readListBegin($_etype802, $_size799);
-            for ($_i803 = 0; $_i803 < $_size799; ++$_i803)
+            $_size806 = 0;
+            $_etype809 = 0;
+            $xfer += $input->readListBegin($_etype809, $_size806);
+            for ($_i810 = 0; $_i810 < $_size806; ++$_i810)
             {
-              $elem804 = null;
-              $xfer += $input->readString($elem804);
-              $this->success []= $elem804;
+              $elem811 = null;
+              $xfer += $input->readString($elem811);
+              $this->success []= $elem811;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14005,9 +15036,9 @@ class ThriftHiveMetastore_get_all_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter805)
+          foreach ($this->success as $iter812)
           {
-            $xfer += $output->writeString($iter805);
+            $xfer += $output->writeString($iter812);
           }
         }
         $output->writeListEnd();
@@ -15008,18 +16039,18 @@ class ThriftHiveMetastore_get_type_all_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size806 = 0;
-            $_ktype807 = 0;
-            $_vtype808 = 0;
-            $xfer += $input->readMapBegin($_ktype807, $_vtype808, $_size806);
-            for ($_i810 = 0; $_i810 < $_size806; ++$_i810)
+            $_size813 = 0;
+            $_ktype814 = 0;
+            $_vtype815 = 0;
+            $xfer += $input->readMapBegin($_ktype814, $_vtype815, $_size813);
+            for ($_i817 = 0; $_i817 < $_size813; ++$_i817)
             {
-              $key811 = '';
-              $val812 = new \metastore\Type();
-              $xfer += $input->readString($key811);
-              $val812 = new \metastore\Type();
-              $xfer += $val812->read($input);
-              $this->success[$key811] = $val812;
+              $key818 = '';
+              $val819 = new \metastore\Type();
+              $xfer += $input->readString($key818);
+              $val819 = new \metastore\Type();
+              $xfer += $val819->read($input);
+              $this->success[$key818] = $val819;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -15055,10 +16086,10 @@ class ThriftHiveMetastore_get_type_all_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $kiter813 => $viter814)
+          foreach ($this->success as $kiter820 => $viter821)
           {
-            $xfer += $output->writeString($kiter813);
-            $xfer += $viter814->write($output);
+            $xfer += $output->writeString($kiter820);
+            $xfer += $viter821->write($output);
           }
         }
         $output->writeMapEnd();
@@ -15262,15 +16293,15 @@ class ThriftHiveMetastore_get_fields_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size815 = 0;
-            $_etype818 = 0;
-            $xfer += $input->readListBegin($_etype818, $_size815);
-            for ($_i819 = 0; $_i819 < $_size815; ++$_i819)
+            $_size822 = 0;
+            $_etype825 = 0;
+            $xfer += $input->readListBegin($_etype825, $_size822);
+            for ($_i826 = 0; $_i826 < $_size822; ++$_i826)
             {
-              $elem820 = null;
-              $elem820 = new \metastore\FieldSchema();
-              $xfer += $elem820->read($input);
-              $this->success []= $elem820;
+              $elem827 = null;
+              $elem827 = new \metastore\FieldSchema();
+              $xfer += $elem827->read($input);
+              $this->success []= $elem827;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15322,9 +16353,9 @@ class ThriftHiveMetastore_get_fields_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter821)
+          foreach ($this->success as $iter828)
           {
-            $xfer += $iter821->write($output);
+            $xfer += $iter828->write($output);
           }
         }
         $output->writeListEnd();
@@ -15566,15 +16597,15 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size822 = 0;
-            $_etype825 = 0;
-            $xfer += $input->readListBegin($_etype825, $_size822);
-            for ($_i826 = 0; $_i826 < $_size822; ++$_i826)
+            $_size829 = 0;
+            $_etype832 = 0;
+            $xfer += $input->readListBegin($_etype832, $_size829);
+            for ($_i833 = 0; $_i833 < $_size829; ++$_i833)
             {
-              $elem827 = null;
-              $elem827 = new \metastore\FieldSchema();
-              $xfer += $elem827->read($input);
-              $this->success []= $elem827;
+              $elem834 = null;
+              $elem834 = new \metastore\FieldSchema();
+              $xfer += $elem834->read($input);
+              $this->success []= $elem834;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15626,9 +16657,9 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter828)
+          foreach ($this->success as $iter835)
           {
-            $xfer += $iter828->write($output);
+            $xfer += $iter835->write($output);
           }
         }
         $output->writeListEnd();
@@ -15842,15 +16873,15 @@ class ThriftHiveMetastore_get_schema_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size829 = 0;
-            $_etype832 = 0;
-            $xfer += $input->readListBegin($_etype832, $_size829);
-            for ($_i833 = 0; $_i833 < $_size829; ++$_i833)
+            $_size836 = 0;
+            $_etype839 = 0;
+            $xfer += $input->readListBegin($_etype839, $_size836);
+            for ($_i840 = 0; $_i840 < $_size836; ++$_i840)
             {
-              $elem834 = null;
-              $elem834 = new \metastore\FieldSchema();
-              $xfer += $elem834->read($input);
-              $this->success []= $elem834;
+              $elem841 = null;
+              $elem841 = new \metastore\FieldSchema();
+              $xfer += $elem841->read($input);
+              $this->success []= $elem841;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15902,9 +16933,9 @@ class ThriftHiveMetastore_get_schema_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter835)
+          foreach ($this->success as $iter842)
           {
-            $xfer += $iter835->write($output);
+            $xfer += $iter842->write($output);
           }
         }
         $output->writeListEnd();
@@ -16146,15 +17177,15 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size836 = 0;
-            $_etype839 = 0;
-            $xfer += $input->readListBegin($_etype839, $_size836);
-            for ($_i840 = 0; $_i840 < $_size836; ++$_i840)
+            $_size843 = 0;
+            $_etype846 = 0;
+            $xfer += $input->readListBegin($_etype846, $_size843);
+            for ($_i847 = 0; $_i847 < $_size843; ++$_i847)
             {
-              $elem841 = null;
-              $elem841 = new \metastore\FieldSchema();
-              $xfer += $elem841->read($input);
-              $this->success []= $elem841;
+              $elem848 = null;
+              $elem848 = new \metastore\FieldSchema();
+              $xfer += $elem848->read($input);
+              $this->success []= $elem848;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16206,9 +17237,9 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter842)
+          foreach ($this->success as $iter849)
           {
-            $xfer += $iter842->write($output);
+            $xfer += $iter849->write($output);
           }
         }
         $output->writeListEnd();
@@ -16880,15 +17911,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->primaryKeys = array();
-            $_size843 = 0;
-            $_etype846 = 0;
-            $xfer += $input->readListBegin($_etype846, $_size843);
-            for ($_i847 = 0; $_i847 < $_size843; ++$_i847)
+            $_size850 = 0;
+            $_etype853 = 0;
+            $xfer += $input->readListBegin($_etype853, $_size850);
+            for ($_i854 = 0; $_i854 < $_size850; ++$_i854)
             {
-              $elem848 = null;
-              $elem848 = new \metastore\SQLPrimaryKey();
-              $xfer += $elem848->read($input);
-              $this->primaryKeys []= $elem848;
+              $elem855 = null;
+              $elem855 = new \metastore\SQLPrimaryKey();
+              $xfer += $elem855->read($input);
+              $this->primaryKeys []= $elem855;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16898,15 +17929,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->foreignKeys = array();
-            $_size849 = 0;
-            $_etype852 = 0;
-            $xfer += $input->readListBegin($_etype852, $_size849);
-            for ($_i853 = 0; $_i853 < $_size849; ++$_i853)
+            $_size856 = 0;
+            $_etype859 = 0;
+            $xfer += $input->readListBegin($_etype859, $_size856);
+            for ($_i860 = 0; $_i860 < $_size856; ++$_i860)
             {
-              $elem854 = null;
-              $elem854 = new \metastore\SQLForeignKey();
-              $xfer += $elem854->read($input);
-              $this->foreignKeys []= $elem854;
+              $elem861 = null;
+              $elem861 = new \metastore\SQLForeignKey();
+              $xfer += $elem861->read($input);
+              $this->foreignKeys []= $elem861;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16916,15 +17947,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 4:
           if ($ftype == TType::LST) {
             $this->uniqueConstraints = array();
-            $_size855 = 0;
-            $_etype858 = 0;
-            $xfer += $input->readListBegin($_etype858, $_size855);
-            for ($_i859 = 0; $_i859 < $_size855; ++$_i859)
+            $_size862 = 0;
+            $_etype865 = 0;
+            $xfer += $input->readListBegin($_etype865, $_size862);
+            for ($_i866 = 0; $_i866 < $_size862; ++$_i866)
             {
-              $elem860 = null;
-              $elem860 = new \metastore\SQLUniqueConstraint();
-              $xfer += $elem860->read($input);
-              $this->uniqueConstraints []= $elem860;
+              $elem867 = null;
+              $elem867 = new \metastore\SQLUniqueConstraint();
+              $xfer += $elem867->read($input);
+              $this->uniqueConstraints []= $elem867;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16934,15 +17965,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->notNullConstraints = array();
-            $_size861 = 0;
-            $_etype864 = 0;
-            $xfer += $input->readListBegin($_etype864, $_size861);
-            for ($_i865 = 0; $_i865 < $_size861; ++$_i865)
+            $_size868 = 0;
+            $_etype871 = 0;
+            $xfer += $input->readListBegin($_etype871, $_size868);
+            for ($_i872 = 0; $_i872 < $_size868; ++$_i872)
             {
-              $elem866 = null;
-              $elem866 = new \metastore\SQLNotNullConstraint();
-              $xfer += $elem866->read($input);
-              $this->notNullConstraints []= $elem866;
+              $elem873 = null;
+              $elem873 = new \metastore\SQLNotNullConstraint();
+              $xfer += $elem873->read($input);
+              $this->notNullConstraints []= $elem873;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16952,15 +17983,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 6:
           if ($ftype == TType::LST) {
             $this->defaultConstraints = array();
-            $_size867 = 0;
-            $_etype870 = 0;
-            $xfer += $input->readListBegin($_etype870, $_size867);
-            for ($_i871 = 0; $_i871 < $_size867; ++$_i871)
+            $_size874 = 0;
+            $_etype877 = 0;
+            $xfer += $input->readListBegin($_etype877, $_size874);
+            for ($_i878 = 0; $_i878 < $_size874; ++$_i878)
             {
-              $elem872 = null;
-              $elem872 = new \metastore\SQLDefaultConstraint();
-              $xfer += $elem872->read($input);
-              $this->defaultConstraints []= $elem872;
+              $elem879 = null;
+              $elem879 = new \metastore\SQLDefaultConstraint();
+              $xfer += $elem879->read($input);
+              $this->defaultConstraints []= $elem879;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16970,15 +18001,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 7:
           if ($ftype == TType::LST) {
             $this->checkConstraints = array();
-            $_size873 = 0;
-            $_etype876 = 0;
-            $xfer += $input->readListBegin($_etype876, $_size873);
-            for ($_i877 = 0; $_i877 < $_size873; ++$_i877)
+            $_size880 = 0;
+            $_etype883 = 0;
+            $xfer += $input->readListBegin($_etype883, $_size880);
+            for ($_i884 = 0; $_i884 < $_size880; ++$_i884)
             {
-              $elem878 = null;
-              $elem878 = new \metastore\SQLCheckConstraint();
-              $xfer += $elem878->read($input);
-              $this->checkConstraints []= $elem878;
+              $elem885 = null;
+              $elem885 = new \metastore\SQLCheckConstraint();
+              $xfer += $elem885->read($input);
+              $this->checkConstraints []= $elem885;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17014,9 +18045,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->primaryKeys));
         {
-          foreach ($this->primaryKeys as $iter879)
+          foreach ($this->primaryKeys as $iter886)
           {
-            $xfer += $iter879->write($output);
+            $xfer += $iter886->write($output);
           }
         }
         $output->writeListEnd();
@@ -17031,9 +18062,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->foreignKeys));
         {
-          foreach ($this->foreignKeys as $iter880)
+          foreach ($this->foreignKeys as $iter887)
           {
-            $xfer += $iter880->write($output);
+            $xfer += $iter887->write($output);
           }
         }
         $output->writeListEnd();
@@ -17048,9 +18079,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->uniqueConstraints));
         {
-          foreach ($this->uniqueConstraints as $iter881)
+          foreach ($this->uniqueConstraints as $iter888)
           {
-            $xfer += $iter881->write($output);
+            $xfer += $iter888->write($output);
           }
         }
         $output->writeListEnd();
@@ -17065,9 +18096,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->notNullConstraints));
         {
-          foreach ($this->notNullConstraints as $iter882)
+          foreach ($this->notNullConstraints as $iter889)
           {
-            $xfer += $iter882->write($output);
+            $xfer += $iter889->write($output);
           }
         }
         $output->writeListEnd();
@@ -17082,9 +18113,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->defaultConstraints));
         {
-          foreach ($this->defaultConstraints as $iter883)
+          foreach ($this->defaultConstraints as $iter890)
           {
-            $xfer += $iter883->write($output);
+            $xfer += $iter890->write($output);
           }
         }
         $output->writeListEnd();
@@ -17099,9 +18130,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->checkConstraints));
         {
-          foreach ($this->checkConstraints as $iter884)
+          foreach ($this->checkConstraints as $iter891)
           {
-            $xfer += $iter884->write($output);
+            $xfer += $iter891->write($output);
           }
         }
         $output->writeListEnd();
@@ -19101,14 +20132,14 @@ class ThriftHiveMetastore_truncate_table_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->partNames = array();
-            $_size885 = 0;
-            $_etype888 = 0;
-            $xfer += $input->readListBegin($_etype888, $_size885);
-            for ($_i889 = 0; $_i889 < $_size885; ++$_i889)
+            $_size892 = 0;
+            $_etype895 = 0;
+            $xfer += $input->readListBegin($_etype895, $_size892);
+            for ($_i896 = 0; $_i896 < $_size892; ++$_i896)
             {
-              $elem890 = null;
-              $xfer += $input->readString($elem890);
-              $this->partNames []= $elem890;
+              $elem897 = null;
+              $xfer += $input->readString($elem897);
+              $this->partNames []= $elem897;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19146,9 +20177,9 @@ class ThriftHiveMetastore_truncate_table_args {
       {
         $output->writeListBegin(TType::STRING, count($this->partNames));
         {
-          foreach ($this->partNames as $iter891)
+          foreach ($this->partNames as $iter898)
           {
-            $xfer += $output->writeString($iter891);
+            $xfer += $output->writeString($iter898);
           }
         }
         $output->writeListEnd();
@@ -19399,14 +20430,14 @@ class ThriftHiveMetastore_get_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size892 = 0;
-            $_etype895 = 0;
-            $xfer += $input->readListBegin($_etype895, $_size892);
-            for ($_i896 = 0; $_i896 < $_size892; ++$_i896)
+            $_size899 = 0;
+            $_etype902 = 0;
+            $xfer += $input->readListBegin($_etype902, $_size899);
+            for ($_i903 = 0; $_i903 < $_size899; ++$_i903)
             {
-              $elem897 = null;
-              $xfer += $input->readString($elem897);
-              $this->success []= $elem897;
+              $elem904 = null;
+              $xfer += $input->readString($elem904);
+              $this->success []= $elem904;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19442,9 +20473,9 @@ class ThriftHiveMetastore_get_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter898)
+          foreach ($this->success as $iter905)
           {
-            $xfer += $output->writeString($iter898);
+            $xfer += $output->writeString($iter905);
           }
         }
         $output->writeListEnd();
@@ -19646,207 +20677,6 @@ class ThriftHiveMetastore_get_tables_by_type_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size899 = 0;
-            $_etype902 = 0;
-            $xfer += $input->readListBegin($_etype902, $_size899);
-            for ($_i903 = 0; $_i903 < $_size899; ++$_i903)
-            {
-              $elem904 = null;
-              $xfer += $input->readString($elem904);
-              $this->success []= $elem904;
-            }
-            $xfer += $input->readListEnd();
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        case 1:
-          if ($ftype == TType::STRUCT) {
-            $this->o1 = new \metastore\MetaException();
-            $xfer += $this->o1->read($input);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        default:
-          $xfer += $input->skip($ftype);
-          break;
-      }
-      $xfer += $input->readFieldEnd();
-    }
-    $xfer += $input->readStructEnd();
-    return $xfer;
-  }
-
-  public function write($output) {
-    $xfer = 0;
-    $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_tables_by_type_result');
-    if ($this->success !== null) {
-      if (!is_array($this->success)) {
-        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
-      }
-      $xfer += $output->writeFieldBegin('success', TType::LST, 0);
-      {
-        $output->writeListBegin(TType::STRING, count($this->success));
-        {
-          foreach ($this->success as $iter905)
-          {
-            $xfer += $output->writeString($iter905);
-          }
-        }
-        $output->writeListEnd();
-      }
-      $xfer += $output->writeFieldEnd();
-    }
-    if ($this->o1 !== null) {
-      $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
-      $xfer += $this->o1->write($output);
-      $xfer += $output->writeFieldEnd();
-    }
-    $xfer += $output->writeFieldStop();
-    $xfer += $output->writeStructEnd();
-    return $xfer;
-  }
-
-}
-
-class ThriftHiveMetastore_get_materialized_views_for_rewriting_args {
-  static $_TSPEC;
-
-  /**
-   * @var string
-   */
-  public $db_name = null;
-
-  public function __construct($vals=null) {
-    if (!isset(self::$_TSPEC)) {
-      self::$_TSPEC = array(
-        1 => array(
-          'var' => 'db_name',
-          'type' => TType::STRING,
-          ),
-        );
-    }
-    if (is_array($vals)) {
-      if (isset($vals['db_name'])) {
-        $this->db_name = $vals['db_name'];
-      }
-    }
-  }
-
-  public function getName() {
-    return 'ThriftHiveMetastore_get_materialized_views_for_rewriting_args';
-  }
-
-  public function read($input)
-  {
-    $xfer = 0;
-    $fname = null;
-    $ftype = 0;
-    $fid = 0;
-    $xfer += $input->readStructBegin($fname);
-    while (true)
-    {
-      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
-      if ($ftype == TType::STOP) {
-        break;
-      }
-      switch ($fid)
-      {
-        case 1:
-          if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->db_name);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        default:
-          $xfer += $input->skip($ftype);
-          break;
-      }
-      $xfer += $input->readFieldEnd();
-    }
-    $xfer += $input->readStructEnd();
-    return $xfer;
-  }
-
-  public function write($output) {
-    $xfer = 0;
-    $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_materialized_views_for_rewriting_args');
-    if ($this->db_name !== null) {
-      $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1);
-      $xfer += $output->writeString($this->db_name);
-      $xfer += $output->writeFieldEnd();
-    }
-    $xfer += $output->writeFieldStop();
-    $xfer += $output->writeStructEnd();
-    return $xfer;
-  }
-
-}
-
-class ThriftHiveMetastore_get_materialized_views_for_rewriting_result {
-  static $_TSPEC;
-
-  /**
-   * @var string[]
-   */
-  public $success = null;
-  /**
-   * @var \metastore\MetaException
-   */
-  public $o1 = null;
-
-  public function __construct($vals=null) {
-    if (!isset(self::$_TSPEC)) {
-      self::$_TSPEC = array(
-        0 => array(
-          'var' => 'success',
-          'type' => TType::LST,
-          'etype' => TType::STRING,
-          'elem' => array(
-            'type' => TType::STRING,
-            ),
-          ),
-        1 => array(
-          'var' => 'o1',
-          'type' => TType::STRUCT,
-          'class' => '\metastore\MetaException',
-          ),
-        );
-    }
-    if (is_array($vals)) {
-      if (isset($vals['success'])) {
-        $this->success = $vals['success'];
-      }
-      if (isset($vals['o1'])) {
-        $this->o1 = $vals['o1'];
-      }
-    }
-  }
-
-  public function getName() {
-    return 'ThriftHiveMetastore_get_materialized_views_for_rewriting_result';
-  }
-
-  public function read($input)
-  {
-    $xfer = 0;
-    $fname = null;
-    $ftype = 0;
-    $fid = 0;
-    $xfer += $input->readStructBegin($fname);
-    while (true)
-    {
-      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
-      if ($ftype == TType::STOP) {
-        break;
-      }
-      switch ($fid)
-      {
-        case 0:
-          if ($ftype == TType::LST) {
-            $this->success = array();
             $_size906 = 0;
             $_etype909 = 0;
             $xfer += $input->readListBegin($_etype909, $_size906);
@@ -19881,7 +20711,7 @@ class ThriftHiveMetastore_get_materialized_views_for_rewriting_result {
 
   public function write($output) {
     $xfer = 0;
-    $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_materialized_views_for_rewriting_result');
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_tables_by_type_result');
     if ($this->success !== null) {
       if (!is_array($this->success)) {
         throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
@@ -19911,6 +20741,207 @@ class ThriftHiveMetastore_get_materialized_views_for_rewriting_result {
 
 }
 
+class ThriftHiveMetastore_get_materialized_views_for_rewriting_args {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $db_name = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'db_name',
+          'type' => TType::STRING,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['db_name'])) {
+        $this->db_name = $vals['db_name'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_get_materialized_views_for_rewriting_args';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->db_name);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_materialized_views_for_rewriting_args');
+    if ($this->db_name !== null) {
+      $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1);
+      $xfer += $output->writeString($this->db_name);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_get_materialized_views_for_rewriting_result {
+  static $_TSPEC;
+
+  /**
+   * @var string[]
+   */
+  public $success = null;
+  /**
+   * @var \metastore\MetaException
+   */
+  public $o1 = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        0 => array(
+          'var' => 'success',
+          'type' => TType::LST,
+          'etype' => TType::STRING,
+          'elem' => array(
+            'type' => TType::STRING,
+            ),
+          ),
+        1 => array(
+          'var' => 'o1',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\MetaException',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['success'])) {
+        $this->success = $vals['success'];
+      }
+      if (isset($vals['o1'])) {
+        $this->o1 = $vals['o1'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_get_materialized_views_for_rewriting_result';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 0:
+          if ($ftype == TType::LST) {
+            $this->success = array();
+            $_size913 = 0;
+            $_etype916 = 0;
+            $xfer += $input->readListBegin($_etype916, $_size913);
+            for ($_i917 = 0; $_i917 < $_size913; ++$_i917)
+            {
+              $elem918 = null;
+              $xfer += $input->readString($elem918);
+              $this->success []= $elem918;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->o1 = new \metastore\MetaException();
+            $xfer += $this->o1->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_materialized_views_for_rewriting_result');
+    if ($this->success !== null) {
+      if (!is_array($this->success)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('success', TType::LST, 0);
+      {
+        $output->writeListBegin(TType::STRING, count($this->success));
+        {
+          foreach ($this->success as $iter919)
+          {
+            $xfer += $output->writeString($iter919);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->o1 !== null) {
+      $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
+      $xfer += $this->o1->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
 class ThriftHiveMetastore_get_table_meta_args {
   static $_TSPEC;
 
@@ -19997,14 +21028,14 @@ class ThriftHiveMetastore_get_table_meta_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->tbl_types = array();
-            $_size913 = 0;
-            $_etype916 = 0;
-            $xfer += $input->readListBegin($_etype916, $_size913);
-            for ($_i917 = 0; $_i917 < $_size913; ++$_i917)
+            $_size920 = 0;
+            $_etype923 = 0;
+            $xfer += $input->readListBegin($_etype923, $_size920);
+            for ($_i924 = 0; $_i924 < $_size920; ++$_i924)
             {
-              $elem918 = null;
-              $xfer += $input->readString($elem918);
-              $this->tbl_types []= $elem918;
+              $elem925 = null;
+              $xfer += $input->readString($elem925);
+              $this->tbl_types []= $elem925;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20042,9 +21073,9 @@ class ThriftHiveMetastore_get_table_meta_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_types));
         {
-          foreach ($this->tbl_types as $iter919)
+          foreach ($this->tbl_types as $iter926)
           {
-            $xfer += $output->writeString($iter919);
+            $xfer += $output->writeString($iter926);
           }
         }
         $output->writeListEnd();
@@ -20121,15 +21152,15 @@ class ThriftHiveMetastore_get_table_meta_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size920 = 0;
-            $_etype923 = 0;
-            $xfer += $input->readListBegin($_etype923, $_size920);
-            for ($_i924 = 0; $_i924 < $_size920; ++$_i924)
+            $_size927 = 0;
+            $_etype930 = 0;
+            $xfer += $input->readListBegin($_etype930, $_size927);
+            for ($_i931 = 0; $_i931 < $_size927; ++$_i931)
             {
-              $elem925 = null;
-              $elem925 = new \metastore\TableMeta();
-              $xfer += $elem925->read($input);
-              $this->success []= $elem925;
+              $elem932 = null;
+              $elem932 = new \metastore\TableMeta();
+              $xfer += $elem932->read($input);
+              $this->success []= $elem932;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20165,9 +21196,9 @@ class ThriftHiveMetastore_get_table_meta_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter926)
+          foreach ($this->success as $iter933)
           {
-            $xfer += $iter926->write($output);
+            $xfer += $iter933->write($output);
           }
         }
         $output->writeListEnd();
@@ -20323,14 +21354,14 @@ class ThriftHiveMetastore_get_all_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size927 = 0;
-            $_etype930 = 0;
-            $xfer += $input->readListBegin($_etype930, $_size927);
-            for ($_i931 = 0; $_i931 < $_size927; ++$_i931)
+            $_size934 = 0;
+            $_etype937 = 0;
+            $xfer += $input->readListBegin($_etype937, $_size934);
+            for ($_i938 = 0; $_i938 < $_size934; ++$_i938)
             {
-              $elem932 = null;
-              $xfer += $input->readString($elem932);
-              $this->success []= $elem932;
+              $elem939 = null;
+              $xfer += $input->readString($elem939);
+              $this->success []= $elem939;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20366,9 +21397,9 @@ class ThriftHiveMetastore_get_all_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter933)
+          foreach ($this->success as $iter940)
           {
-            $xfer += $output->writeString($iter933);
+            $xfer += $output->writeString($iter940);
           }
         }
         $output->writeListEnd();
@@ -20683,14 +21714,14 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->tbl_names = array();
-            $_size934 = 0;
-            $_etype937 = 0;
-            $xfer += $input->readListBegin($_etype937, $_size934);
-            for ($_i938 = 0; $_i938 < $_size934; ++$_i938)
+            $_size941 = 0;
+            $_etype944 = 0;
+            $xfer += $input->readListBegin($_etype944, $_size941);
+            for ($_i945 = 0; $_i945 < $_size941; ++$_i945)
             {
-              $elem939 = null;
-              $xfer += $input->readString($elem939);
-              $this->tbl_names []= $elem939;
+              $elem946 = null;
+              $xfer += $input->readString($elem946);
+              $this->tbl_names []= $elem946;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20723,9 +21754,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_names));
         {
-          foreach ($this->tbl_names as $iter940)
+          foreach ($this->tbl_names as $iter947)
           {
-            $xfer += $output->writeString($iter940);
+            $xfer += $output->writeString($iter947);
           }
         }
         $output->writeListEnd();
@@ -20790,15 +21821,15 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size941 = 0;
-            $_etype944 = 0;
-            $xfer += $input->readListBegin($_etype944, $_size941);
-            for ($_i945 = 0; $_i945 < $_size941; ++$_i945)
+            $_size948 = 0;
+            $_etype951 = 0;
+            $xfer += $input->readListBegin($_etype951, $_size948);
+            for ($_i952 = 0; $_i952 < $_size948; ++$_i952)
             {
-              $elem946 = null;
-              $elem946 = new \metastore\Table();
-              $xfer += $elem946->read($input);
-              $this->success []= $elem946;
+              $elem953 = null;
+              $elem953 = new \metastore\Table();
+              $xfer += $elem953->read($input);
+              $this->success []= $elem953;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20826,9 +21857,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter947)
+          foreach ($this->success as $iter954)
           {
-            $xfer += $iter947->write($output);
+            $xfer += $iter954->write($output);
           }
         }
         $output->writeListEnd();
@@ -21355,14 +22386,14 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->tbl_names = array();
-            $_size948 = 0;
-            $_etype951 = 0;
-            $xfer += $input->readListBegin($_etype951, $_size948);
-            for ($_i952 = 0; $_i952 < $_size948; ++$_i952)
+            $_size955 = 0;
+            $_etype958 = 0;
+            $xfer += $input->readListBegin($_etype958, $_size955);
+            for ($_i959 = 0; $_i959 < $_size955; ++$_i959)
             {
-              $elem953 = null;
-              $xfer += $input->readString($elem953);
-              $this->tbl_names []= $elem953;
+              $elem960 = null;
+              $xfer += $input->readString($elem960);
+              $this->tbl_names []= $elem960;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21395,9 +22426,9 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_names));
         {
-          foreach ($this->tbl_names as $iter954)
+          foreach ($this->tbl_names as $iter961)
           {
-            $xfer += $output->writeString($iter954);
+            $xfer += $output->writeString($iter961);
           }
         }
         $output->writeListEnd();
@@ -21502,18 +22533,18 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size955 = 0;
-            $_ktype956 = 0;
-            $_vtype957 = 0;
-            $xfer += $input->readMapBegin($_ktype956, $_vtype957, $_size955);
-            for ($_i959 = 0; $_i959 < $_size955; ++$_i959)
+            $_size962 = 0;
+            $_ktype963 = 0;
+            $_vtype964 = 0;
+            $xfer += $input->readMapBegin($_ktype963, $_vtype964, $_size962);
+            for ($_i966 = 0; $_i966 < $_size962; ++$_i966)
             {
-              $key960 = '';
-              $val961 = new \metastore\Materialization();
-              $xfer += $input->readString($key960);
-              $val961 = new \metastore\Materialization();
-              $xfer += $val961->read($input);
-              $this->success[$key960] = $val961;
+              $key967 = '';
+              $val968 = new \metastore\Materialization();
+              $xfer += $input->readString($key967);
+              $val968 = new \metastore\Materialization();
+              $xfer += $val968->read($input);
+              $this->success[$key967] = $val968;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -21565,10 +22596,10 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $kiter962 => $viter963)
+          foreach ($this->success as $kiter969 => $viter970)
           {
-            $xfer += $output->writeString($kiter962);
-            $xfer += $viter963->write($output);
+            $xfer += $output->writeString($kiter969);
+            $xfer += $viter970->write($output);
           }
         }
         $output->writeMapEnd();
@@ -21603,6 +22634,10 @@ class ThriftHiveMetastore_update_creation_metadata_args {
   /**
    * @var string
    */
+  public $catName = null;
+  /**
+   * @var string
+   */
   public $dbname = null;
   /**
    * @var string
@@ -21617,14 +22652,18 @@ class ThriftHiveMetastore_update_creation_metadata_args {
     if (!isset(self::$_TSPEC)) {
       self::$_TSPEC = array(
         1 => array(
-          'var' => 'dbname',
+          'var' => 'catName',
           'type' => TType::STRING,
           ),
         2 => array(
-          'var' => 'tbl_name',
+          'var' => 'dbname',
           'type' => TType::STRING,
           ),
         3 => array(
+          'var' => 'tbl_name',
+          'type' => TType::STRING,
+          ),
+        4 => array(
           'var' => 'creation_metadata',
           'type' => TType::STRUCT,
           'class' => '\metastore\CreationMetadata',
@@ -21632,6 +22671,9 @@ class ThriftHiveMetastore_update_creation_metadata_args {
         );
     }
     if (is_array($vals)) {
+      if (isset($vals['catName'])) {
+        $this->catName = $vals['catName'];
+      }
       if (isset($vals['dbname'])) {
         $this->dbname = $vals['dbname'];
       }
@@ -21665,19 +22707,26 @@ class ThriftHiveMetastore_update_creation_metadata_args {
       {
         case 1:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->dbname);
+            $xfer += $input->readString($this->catName);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
       

<TRUNCATED>

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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatisticsDesc.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatisticsDesc.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatisticsDesc.java
index 922094b..0e70758 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatisticsDesc.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatisticsDesc.java
@@ -43,6 +43,7 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)3);
   private static final org.apache.thrift.protocol.TField PART_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("partName", org.apache.thrift.protocol.TType.STRING, (short)4);
   private static final org.apache.thrift.protocol.TField LAST_ANALYZED_FIELD_DESC = new org.apache.thrift.protocol.TField("lastAnalyzed", org.apache.thrift.protocol.TType.I64, (short)5);
+  private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)6);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -55,6 +56,7 @@ import org.slf4j.LoggerFactory;
   private String tableName; // required
   private String partName; // optional
   private long lastAnalyzed; // optional
+  private String catName; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -62,7 +64,8 @@ import org.slf4j.LoggerFactory;
     DB_NAME((short)2, "dbName"),
     TABLE_NAME((short)3, "tableName"),
     PART_NAME((short)4, "partName"),
-    LAST_ANALYZED((short)5, "lastAnalyzed");
+    LAST_ANALYZED((short)5, "lastAnalyzed"),
+    CAT_NAME((short)6, "catName");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -87,6 +90,8 @@ import org.slf4j.LoggerFactory;
           return PART_NAME;
         case 5: // LAST_ANALYZED
           return LAST_ANALYZED;
+        case 6: // CAT_NAME
+          return CAT_NAME;
         default:
           return null;
       }
@@ -130,7 +135,7 @@ import org.slf4j.LoggerFactory;
   private static final int __ISTBLLEVEL_ISSET_ID = 0;
   private static final int __LASTANALYZED_ISSET_ID = 1;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.PART_NAME,_Fields.LAST_ANALYZED};
+  private static final _Fields optionals[] = {_Fields.PART_NAME,_Fields.LAST_ANALYZED,_Fields.CAT_NAME};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -144,6 +149,8 @@ import org.slf4j.LoggerFactory;
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.LAST_ANALYZED, new org.apache.thrift.meta_data.FieldMetaData("lastAnalyzed", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ColumnStatisticsDesc.class, metaDataMap);
   }
@@ -179,6 +186,9 @@ import org.slf4j.LoggerFactory;
       this.partName = other.partName;
     }
     this.lastAnalyzed = other.lastAnalyzed;
+    if (other.isSetCatName()) {
+      this.catName = other.catName;
+    }
   }
 
   public ColumnStatisticsDesc deepCopy() {
@@ -194,6 +204,7 @@ import org.slf4j.LoggerFactory;
     this.partName = null;
     setLastAnalyzedIsSet(false);
     this.lastAnalyzed = 0;
+    this.catName = null;
   }
 
   public boolean isIsTblLevel() {
@@ -309,6 +320,29 @@ import org.slf4j.LoggerFactory;
     __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __LASTANALYZED_ISSET_ID, value);
   }
 
+  public String getCatName() {
+    return this.catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
+  public void unsetCatName() {
+    this.catName = null;
+  }
+
+  /** Returns true if field catName is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatName() {
+    return this.catName != null;
+  }
+
+  public void setCatNameIsSet(boolean value) {
+    if (!value) {
+      this.catName = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case IS_TBL_LEVEL:
@@ -351,6 +385,14 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case CAT_NAME:
+      if (value == null) {
+        unsetCatName();
+      } else {
+        setCatName((String)value);
+      }
+      break;
+
     }
   }
 
@@ -371,6 +413,9 @@ import org.slf4j.LoggerFactory;
     case LAST_ANALYZED:
       return getLastAnalyzed();
 
+    case CAT_NAME:
+      return getCatName();
+
     }
     throw new IllegalStateException();
   }
@@ -392,6 +437,8 @@ import org.slf4j.LoggerFactory;
       return isSetPartName();
     case LAST_ANALYZED:
       return isSetLastAnalyzed();
+    case CAT_NAME:
+      return isSetCatName();
     }
     throw new IllegalStateException();
   }
@@ -454,6 +501,15 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_catName = true && this.isSetCatName();
+    boolean that_present_catName = true && that.isSetCatName();
+    if (this_present_catName || that_present_catName) {
+      if (!(this_present_catName && that_present_catName))
+        return false;
+      if (!this.catName.equals(that.catName))
+        return false;
+    }
+
     return true;
   }
 
@@ -486,6 +542,11 @@ import org.slf4j.LoggerFactory;
     if (present_lastAnalyzed)
       list.add(lastAnalyzed);
 
+    boolean present_catName = true && (isSetCatName());
+    list.add(present_catName);
+    if (present_catName)
+      list.add(catName);
+
     return list.hashCode();
   }
 
@@ -547,6 +608,16 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetCatName()).compareTo(other.isSetCatName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catName, other.catName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -602,6 +673,16 @@ import org.slf4j.LoggerFactory;
       sb.append(this.lastAnalyzed);
       first = false;
     }
+    if (isSetCatName()) {
+      if (!first) sb.append(", ");
+      sb.append("catName:");
+      if (this.catName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.catName);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -699,6 +780,14 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 6: // CAT_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.catName = iprot.readString();
+              struct.setCatNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -737,6 +826,13 @@ import org.slf4j.LoggerFactory;
         oprot.writeI64(struct.lastAnalyzed);
         oprot.writeFieldEnd();
       }
+      if (struct.catName != null) {
+        if (struct.isSetCatName()) {
+          oprot.writeFieldBegin(CAT_NAME_FIELD_DESC);
+          oprot.writeString(struct.catName);
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -764,13 +860,19 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetLastAnalyzed()) {
         optionals.set(1);
       }
-      oprot.writeBitSet(optionals, 2);
+      if (struct.isSetCatName()) {
+        optionals.set(2);
+      }
+      oprot.writeBitSet(optionals, 3);
       if (struct.isSetPartName()) {
         oprot.writeString(struct.partName);
       }
       if (struct.isSetLastAnalyzed()) {
         oprot.writeI64(struct.lastAnalyzed);
       }
+      if (struct.isSetCatName()) {
+        oprot.writeString(struct.catName);
+      }
     }
 
     @Override
@@ -782,7 +884,7 @@ import org.slf4j.LoggerFactory;
       struct.setDbNameIsSet(true);
       struct.tableName = iprot.readString();
       struct.setTableNameIsSet(true);
-      BitSet incoming = iprot.readBitSet(2);
+      BitSet incoming = iprot.readBitSet(3);
       if (incoming.get(0)) {
         struct.partName = iprot.readString();
         struct.setPartNameIsSet(true);
@@ -791,6 +893,10 @@ import org.slf4j.LoggerFactory;
         struct.lastAnalyzed = iprot.readI64();
         struct.setLastAnalyzedIsSet(true);
       }
+      if (incoming.get(2)) {
+        struct.catName = iprot.readString();
+        struct.setCatNameIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
index 1a27ff5..a106cd4 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
@@ -814,15 +814,15 @@ import org.slf4j.LoggerFactory;
           case 6: // PROPERTIES
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map650 = iprot.readMapBegin();
-                struct.properties = new HashMap<String,String>(2*_map650.size);
-                String _key651;
-                String _val652;
-                for (int _i653 = 0; _i653 < _map650.size; ++_i653)
+                org.apache.thrift.protocol.TMap _map658 = iprot.readMapBegin();
+                struct.properties = new HashMap<String,String>(2*_map658.size);
+                String _key659;
+                String _val660;
+                for (int _i661 = 0; _i661 < _map658.size; ++_i661)
                 {
-                  _key651 = iprot.readString();
-                  _val652 = iprot.readString();
-                  struct.properties.put(_key651, _val652);
+                  _key659 = iprot.readString();
+                  _val660 = iprot.readString();
+                  struct.properties.put(_key659, _val660);
                 }
                 iprot.readMapEnd();
               }
@@ -878,10 +878,10 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PROPERTIES_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.properties.size()));
-            for (Map.Entry<String, String> _iter654 : struct.properties.entrySet())
+            for (Map.Entry<String, String> _iter662 : struct.properties.entrySet())
             {
-              oprot.writeString(_iter654.getKey());
-              oprot.writeString(_iter654.getValue());
+              oprot.writeString(_iter662.getKey());
+              oprot.writeString(_iter662.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -928,10 +928,10 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetProperties()) {
         {
           oprot.writeI32(struct.properties.size());
-          for (Map.Entry<String, String> _iter655 : struct.properties.entrySet())
+          for (Map.Entry<String, String> _iter663 : struct.properties.entrySet())
           {
-            oprot.writeString(_iter655.getKey());
-            oprot.writeString(_iter655.getValue());
+            oprot.writeString(_iter663.getKey());
+            oprot.writeString(_iter663.getValue());
           }
         }
       }
@@ -957,15 +957,15 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TMap _map656 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.properties = new HashMap<String,String>(2*_map656.size);
-          String _key657;
-          String _val658;
-          for (int _i659 = 0; _i659 < _map656.size; ++_i659)
+          org.apache.thrift.protocol.TMap _map664 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.properties = new HashMap<String,String>(2*_map664.size);
+          String _key665;
+          String _val666;
+          for (int _i667 = 0; _i667 < _map664.size; ++_i667)
           {
-            _key657 = iprot.readString();
-            _val658 = iprot.readString();
-            struct.properties.put(_key657, _val658);
+            _key665 = iprot.readString();
+            _val666 = iprot.readString();
+            struct.properties.put(_key665, _val666);
           }
         }
         struct.setPropertiesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreateCatalogRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreateCatalogRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreateCatalogRequest.java
new file mode 100644
index 0000000..c260b3d
--- /dev/null
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreateCatalogRequest.java
@@ -0,0 +1,400 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class CreateCatalogRequest implements org.apache.thrift.TBase<CreateCatalogRequest, CreateCatalogRequest._Fields>, java.io.Serializable, Cloneable, Comparable<CreateCatalogRequest> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CreateCatalogRequest");
+
+  private static final org.apache.thrift.protocol.TField CATALOG_FIELD_DESC = new org.apache.thrift.protocol.TField("catalog", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new CreateCatalogRequestStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new CreateCatalogRequestTupleSchemeFactory());
+  }
+
+  private Catalog catalog; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    CATALOG((short)1, "catalog");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // CATALOG
+          return CATALOG;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.CATALOG, new org.apache.thrift.meta_data.FieldMetaData("catalog", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, Catalog.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CreateCatalogRequest.class, metaDataMap);
+  }
+
+  public CreateCatalogRequest() {
+  }
+
+  public CreateCatalogRequest(
+    Catalog catalog)
+  {
+    this();
+    this.catalog = catalog;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public CreateCatalogRequest(CreateCatalogRequest other) {
+    if (other.isSetCatalog()) {
+      this.catalog = new Catalog(other.catalog);
+    }
+  }
+
+  public CreateCatalogRequest deepCopy() {
+    return new CreateCatalogRequest(this);
+  }
+
+  @Override
+  public void clear() {
+    this.catalog = null;
+  }
+
+  public Catalog getCatalog() {
+    return this.catalog;
+  }
+
+  public void setCatalog(Catalog catalog) {
+    this.catalog = catalog;
+  }
+
+  public void unsetCatalog() {
+    this.catalog = null;
+  }
+
+  /** Returns true if field catalog is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatalog() {
+    return this.catalog != null;
+  }
+
+  public void setCatalogIsSet(boolean value) {
+    if (!value) {
+      this.catalog = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case CATALOG:
+      if (value == null) {
+        unsetCatalog();
+      } else {
+        setCatalog((Catalog)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case CATALOG:
+      return getCatalog();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case CATALOG:
+      return isSetCatalog();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof CreateCatalogRequest)
+      return this.equals((CreateCatalogRequest)that);
+    return false;
+  }
+
+  public boolean equals(CreateCatalogRequest that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_catalog = true && this.isSetCatalog();
+    boolean that_present_catalog = true && that.isSetCatalog();
+    if (this_present_catalog || that_present_catalog) {
+      if (!(this_present_catalog && that_present_catalog))
+        return false;
+      if (!this.catalog.equals(that.catalog))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_catalog = true && (isSetCatalog());
+    list.add(present_catalog);
+    if (present_catalog)
+      list.add(catalog);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(CreateCatalogRequest other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetCatalog()).compareTo(other.isSetCatalog());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatalog()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catalog, other.catalog);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("CreateCatalogRequest(");
+    boolean first = true;
+
+    sb.append("catalog:");
+    if (this.catalog == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.catalog);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+    if (catalog != null) {
+      catalog.validate();
+    }
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class CreateCatalogRequestStandardSchemeFactory implements SchemeFactory {
+    public CreateCatalogRequestStandardScheme getScheme() {
+      return new CreateCatalogRequestStandardScheme();
+    }
+  }
+
+  private static class CreateCatalogRequestStandardScheme extends StandardScheme<CreateCatalogRequest> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, CreateCatalogRequest struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // CATALOG
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.catalog = new Catalog();
+              struct.catalog.read(iprot);
+              struct.setCatalogIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, CreateCatalogRequest struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.catalog != null) {
+        oprot.writeFieldBegin(CATALOG_FIELD_DESC);
+        struct.catalog.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class CreateCatalogRequestTupleSchemeFactory implements SchemeFactory {
+    public CreateCatalogRequestTupleScheme getScheme() {
+      return new CreateCatalogRequestTupleScheme();
+    }
+  }
+
+  private static class CreateCatalogRequestTupleScheme extends TupleScheme<CreateCatalogRequest> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, CreateCatalogRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.isSetCatalog()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.isSetCatalog()) {
+        struct.catalog.write(oprot);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, CreateCatalogRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.catalog = new Catalog();
+        struct.catalog.read(iprot);
+        struct.setCatalogIsSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreationMetadata.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreationMetadata.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreationMetadata.java
index b744177..d28972c 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreationMetadata.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreationMetadata.java
@@ -38,10 +38,11 @@ import org.slf4j.LoggerFactory;
 @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class CreationMetadata implements org.apache.thrift.TBase<CreationMetadata, CreationMetadata._Fields>, java.io.Serializable, Cloneable, Comparable<CreationMetadata> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CreationMetadata");
 
-  private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbName", org.apache.thrift.protocol.TType.STRING, (short)1);
-  private static final org.apache.thrift.protocol.TField TBL_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tblName", org.apache.thrift.protocol.TType.STRING, (short)2);
-  private static final org.apache.thrift.protocol.TField TABLES_USED_FIELD_DESC = new org.apache.thrift.protocol.TField("tablesUsed", org.apache.thrift.protocol.TType.SET, (short)3);
-  private static final org.apache.thrift.protocol.TField VALID_TXN_LIST_FIELD_DESC = new org.apache.thrift.protocol.TField("validTxnList", org.apache.thrift.protocol.TType.STRING, (short)4);
+  private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbName", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField TBL_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tblName", org.apache.thrift.protocol.TType.STRING, (short)3);
+  private static final org.apache.thrift.protocol.TField TABLES_USED_FIELD_DESC = new org.apache.thrift.protocol.TField("tablesUsed", org.apache.thrift.protocol.TType.SET, (short)4);
+  private static final org.apache.thrift.protocol.TField VALID_TXN_LIST_FIELD_DESC = new org.apache.thrift.protocol.TField("validTxnList", org.apache.thrift.protocol.TType.STRING, (short)5);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -49,6 +50,7 @@ import org.slf4j.LoggerFactory;
     schemes.put(TupleScheme.class, new CreationMetadataTupleSchemeFactory());
   }
 
+  private String catName; // required
   private String dbName; // required
   private String tblName; // required
   private Set<String> tablesUsed; // required
@@ -56,10 +58,11 @@ import org.slf4j.LoggerFactory;
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-    DB_NAME((short)1, "dbName"),
-    TBL_NAME((short)2, "tblName"),
-    TABLES_USED((short)3, "tablesUsed"),
-    VALID_TXN_LIST((short)4, "validTxnList");
+    CAT_NAME((short)1, "catName"),
+    DB_NAME((short)2, "dbName"),
+    TBL_NAME((short)3, "tblName"),
+    TABLES_USED((short)4, "tablesUsed"),
+    VALID_TXN_LIST((short)5, "validTxnList");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -74,13 +77,15 @@ import org.slf4j.LoggerFactory;
      */
     public static _Fields findByThriftId(int fieldId) {
       switch(fieldId) {
-        case 1: // DB_NAME
+        case 1: // CAT_NAME
+          return CAT_NAME;
+        case 2: // DB_NAME
           return DB_NAME;
-        case 2: // TBL_NAME
+        case 3: // TBL_NAME
           return TBL_NAME;
-        case 3: // TABLES_USED
+        case 4: // TABLES_USED
           return TABLES_USED;
-        case 4: // VALID_TXN_LIST
+        case 5: // VALID_TXN_LIST
           return VALID_TXN_LIST;
         default:
           return null;
@@ -126,6 +131,8 @@ import org.slf4j.LoggerFactory;
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.DB_NAME, new org.apache.thrift.meta_data.FieldMetaData("dbName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.TBL_NAME, new org.apache.thrift.meta_data.FieldMetaData("tblName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
@@ -143,11 +150,13 @@ import org.slf4j.LoggerFactory;
   }
 
   public CreationMetadata(
+    String catName,
     String dbName,
     String tblName,
     Set<String> tablesUsed)
   {
     this();
+    this.catName = catName;
     this.dbName = dbName;
     this.tblName = tblName;
     this.tablesUsed = tablesUsed;
@@ -157,6 +166,9 @@ import org.slf4j.LoggerFactory;
    * Performs a deep copy on <i>other</i>.
    */
   public CreationMetadata(CreationMetadata other) {
+    if (other.isSetCatName()) {
+      this.catName = other.catName;
+    }
     if (other.isSetDbName()) {
       this.dbName = other.dbName;
     }
@@ -178,12 +190,36 @@ import org.slf4j.LoggerFactory;
 
   @Override
   public void clear() {
+    this.catName = null;
     this.dbName = null;
     this.tblName = null;
     this.tablesUsed = null;
     this.validTxnList = null;
   }
 
+  public String getCatName() {
+    return this.catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
+  public void unsetCatName() {
+    this.catName = null;
+  }
+
+  /** Returns true if field catName is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatName() {
+    return this.catName != null;
+  }
+
+  public void setCatNameIsSet(boolean value) {
+    if (!value) {
+      this.catName = null;
+    }
+  }
+
   public String getDbName() {
     return this.dbName;
   }
@@ -293,6 +329,14 @@ import org.slf4j.LoggerFactory;
 
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
+    case CAT_NAME:
+      if (value == null) {
+        unsetCatName();
+      } else {
+        setCatName((String)value);
+      }
+      break;
+
     case DB_NAME:
       if (value == null) {
         unsetDbName();
@@ -330,6 +374,9 @@ import org.slf4j.LoggerFactory;
 
   public Object getFieldValue(_Fields field) {
     switch (field) {
+    case CAT_NAME:
+      return getCatName();
+
     case DB_NAME:
       return getDbName();
 
@@ -353,6 +400,8 @@ import org.slf4j.LoggerFactory;
     }
 
     switch (field) {
+    case CAT_NAME:
+      return isSetCatName();
     case DB_NAME:
       return isSetDbName();
     case TBL_NAME:
@@ -378,6 +427,15 @@ import org.slf4j.LoggerFactory;
     if (that == null)
       return false;
 
+    boolean this_present_catName = true && this.isSetCatName();
+    boolean that_present_catName = true && that.isSetCatName();
+    if (this_present_catName || that_present_catName) {
+      if (!(this_present_catName && that_present_catName))
+        return false;
+      if (!this.catName.equals(that.catName))
+        return false;
+    }
+
     boolean this_present_dbName = true && this.isSetDbName();
     boolean that_present_dbName = true && that.isSetDbName();
     if (this_present_dbName || that_present_dbName) {
@@ -421,6 +479,11 @@ import org.slf4j.LoggerFactory;
   public int hashCode() {
     List<Object> list = new ArrayList<Object>();
 
+    boolean present_catName = true && (isSetCatName());
+    list.add(present_catName);
+    if (present_catName)
+      list.add(catName);
+
     boolean present_dbName = true && (isSetDbName());
     list.add(present_dbName);
     if (present_dbName)
@@ -452,6 +515,16 @@ import org.slf4j.LoggerFactory;
 
     int lastComparison = 0;
 
+    lastComparison = Boolean.valueOf(isSetCatName()).compareTo(other.isSetCatName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catName, other.catName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     lastComparison = Boolean.valueOf(isSetDbName()).compareTo(other.isSetDbName());
     if (lastComparison != 0) {
       return lastComparison;
@@ -512,6 +585,14 @@ import org.slf4j.LoggerFactory;
     StringBuilder sb = new StringBuilder("CreationMetadata(");
     boolean first = true;
 
+    sb.append("catName:");
+    if (this.catName == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.catName);
+    }
+    first = false;
+    if (!first) sb.append(", ");
     sb.append("dbName:");
     if (this.dbName == null) {
       sb.append("null");
@@ -551,6 +632,10 @@ import org.slf4j.LoggerFactory;
 
   public void validate() throws org.apache.thrift.TException {
     // check for required fields
+    if (!isSetCatName()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'catName' is unset! Struct:" + toString());
+    }
+
     if (!isSetDbName()) {
       throw new org.apache.thrift.protocol.TProtocolException("Required field 'dbName' is unset! Struct:" + toString());
     }
@@ -600,7 +685,15 @@ import org.slf4j.LoggerFactory;
           break;
         }
         switch (schemeField.id) {
-          case 1: // DB_NAME
+          case 1: // CAT_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.catName = iprot.readString();
+              struct.setCatNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // DB_NAME
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.dbName = iprot.readString();
               struct.setDbNameIsSet(true);
@@ -608,7 +701,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 2: // TBL_NAME
+          case 3: // TBL_NAME
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.tblName = iprot.readString();
               struct.setTblNameIsSet(true);
@@ -616,16 +709,16 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 3: // TABLES_USED
+          case 4: // TABLES_USED
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set676 = iprot.readSetBegin();
-                struct.tablesUsed = new HashSet<String>(2*_set676.size);
-                String _elem677;
-                for (int _i678 = 0; _i678 < _set676.size; ++_i678)
+                org.apache.thrift.protocol.TSet _set684 = iprot.readSetBegin();
+                struct.tablesUsed = new HashSet<String>(2*_set684.size);
+                String _elem685;
+                for (int _i686 = 0; _i686 < _set684.size; ++_i686)
                 {
-                  _elem677 = iprot.readString();
-                  struct.tablesUsed.add(_elem677);
+                  _elem685 = iprot.readString();
+                  struct.tablesUsed.add(_elem685);
                 }
                 iprot.readSetEnd();
               }
@@ -634,7 +727,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 4: // VALID_TXN_LIST
+          case 5: // VALID_TXN_LIST
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.validTxnList = iprot.readString();
               struct.setValidTxnListIsSet(true);
@@ -655,6 +748,11 @@ import org.slf4j.LoggerFactory;
       struct.validate();
 
       oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.catName != null) {
+        oprot.writeFieldBegin(CAT_NAME_FIELD_DESC);
+        oprot.writeString(struct.catName);
+        oprot.writeFieldEnd();
+      }
       if (struct.dbName != null) {
         oprot.writeFieldBegin(DB_NAME_FIELD_DESC);
         oprot.writeString(struct.dbName);
@@ -669,9 +767,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TABLES_USED_FIELD_DESC);
         {
           oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, struct.tablesUsed.size()));
-          for (String _iter679 : struct.tablesUsed)
+          for (String _iter687 : struct.tablesUsed)
           {
-            oprot.writeString(_iter679);
+            oprot.writeString(_iter687);
           }
           oprot.writeSetEnd();
         }
@@ -701,13 +799,14 @@ import org.slf4j.LoggerFactory;
     @Override
     public void write(org.apache.thrift.protocol.TProtocol prot, CreationMetadata struct) throws org.apache.thrift.TException {
       TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.catName);
       oprot.writeString(struct.dbName);
       oprot.writeString(struct.tblName);
       {
         oprot.writeI32(struct.tablesUsed.size());
-        for (String _iter680 : struct.tablesUsed)
+        for (String _iter688 : struct.tablesUsed)
         {
-          oprot.writeString(_iter680);
+          oprot.writeString(_iter688);
         }
       }
       BitSet optionals = new BitSet();
@@ -723,18 +822,20 @@ import org.slf4j.LoggerFactory;
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, CreationMetadata struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.catName = iprot.readString();
+      struct.setCatNameIsSet(true);
       struct.dbName = iprot.readString();
       struct.setDbNameIsSet(true);
       struct.tblName = iprot.readString();
       struct.setTblNameIsSet(true);
       {
-        org.apache.thrift.protocol.TSet _set681 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.tablesUsed = new HashSet<String>(2*_set681.size);
-        String _elem682;
-        for (int _i683 = 0; _i683 < _set681.size; ++_i683)
+        org.apache.thrift.protocol.TSet _set689 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.tablesUsed = new HashSet<String>(2*_set689.size);
+        String _elem690;
+        for (int _i691 = 0; _i691 < _set689.size; ++_i691)
         {
-          _elem682 = iprot.readString();
-          struct.tablesUsed.add(_elem682);
+          _elem690 = iprot.readString();
+          struct.tablesUsed.add(_elem690);
         }
       }
       struct.setTablesUsedIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Database.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Database.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Database.java
index 1a8c7b5..9cde9b8 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Database.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Database.java
@@ -45,6 +45,7 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField PRIVILEGES_FIELD_DESC = new org.apache.thrift.protocol.TField("privileges", org.apache.thrift.protocol.TType.STRUCT, (short)5);
   private static final org.apache.thrift.protocol.TField OWNER_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("ownerName", org.apache.thrift.protocol.TType.STRING, (short)6);
   private static final org.apache.thrift.protocol.TField OWNER_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("ownerType", org.apache.thrift.protocol.TType.I32, (short)7);
+  private static final org.apache.thrift.protocol.TField CATALOG_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catalogName", org.apache.thrift.protocol.TType.STRING, (short)8);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -59,6 +60,7 @@ import org.slf4j.LoggerFactory;
   private PrincipalPrivilegeSet privileges; // optional
   private String ownerName; // optional
   private PrincipalType ownerType; // optional
+  private String catalogName; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -72,7 +74,8 @@ import org.slf4j.LoggerFactory;
      * 
      * @see PrincipalType
      */
-    OWNER_TYPE((short)7, "ownerType");
+    OWNER_TYPE((short)7, "ownerType"),
+    CATALOG_NAME((short)8, "catalogName");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -101,6 +104,8 @@ import org.slf4j.LoggerFactory;
           return OWNER_NAME;
         case 7: // OWNER_TYPE
           return OWNER_TYPE;
+        case 8: // CATALOG_NAME
+          return CATALOG_NAME;
         default:
           return null;
       }
@@ -141,7 +146,7 @@ import org.slf4j.LoggerFactory;
   }
 
   // isset id assignments
-  private static final _Fields optionals[] = {_Fields.PRIVILEGES,_Fields.OWNER_NAME,_Fields.OWNER_TYPE};
+  private static final _Fields optionals[] = {_Fields.PRIVILEGES,_Fields.OWNER_NAME,_Fields.OWNER_TYPE,_Fields.CATALOG_NAME};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -161,6 +166,8 @@ import org.slf4j.LoggerFactory;
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.OWNER_TYPE, new org.apache.thrift.meta_data.FieldMetaData("ownerType", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, PrincipalType.class)));
+    tmpMap.put(_Fields.CATALOG_NAME, new org.apache.thrift.meta_data.FieldMetaData("catalogName", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(Database.class, metaDataMap);
   }
@@ -207,6 +214,9 @@ import org.slf4j.LoggerFactory;
     if (other.isSetOwnerType()) {
       this.ownerType = other.ownerType;
     }
+    if (other.isSetCatalogName()) {
+      this.catalogName = other.catalogName;
+    }
   }
 
   public Database deepCopy() {
@@ -222,6 +232,7 @@ import org.slf4j.LoggerFactory;
     this.privileges = null;
     this.ownerName = null;
     this.ownerType = null;
+    this.catalogName = null;
   }
 
   public String getName() {
@@ -404,6 +415,29 @@ import org.slf4j.LoggerFactory;
     }
   }
 
+  public String getCatalogName() {
+    return this.catalogName;
+  }
+
+  public void setCatalogName(String catalogName) {
+    this.catalogName = catalogName;
+  }
+
+  public void unsetCatalogName() {
+    this.catalogName = null;
+  }
+
+  /** Returns true if field catalogName is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatalogName() {
+    return this.catalogName != null;
+  }
+
+  public void setCatalogNameIsSet(boolean value) {
+    if (!value) {
+      this.catalogName = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case NAME:
@@ -462,6 +496,14 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case CATALOG_NAME:
+      if (value == null) {
+        unsetCatalogName();
+      } else {
+        setCatalogName((String)value);
+      }
+      break;
+
     }
   }
 
@@ -488,6 +530,9 @@ import org.slf4j.LoggerFactory;
     case OWNER_TYPE:
       return getOwnerType();
 
+    case CATALOG_NAME:
+      return getCatalogName();
+
     }
     throw new IllegalStateException();
   }
@@ -513,6 +558,8 @@ import org.slf4j.LoggerFactory;
       return isSetOwnerName();
     case OWNER_TYPE:
       return isSetOwnerType();
+    case CATALOG_NAME:
+      return isSetCatalogName();
     }
     throw new IllegalStateException();
   }
@@ -593,6 +640,15 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_catalogName = true && this.isSetCatalogName();
+    boolean that_present_catalogName = true && that.isSetCatalogName();
+    if (this_present_catalogName || that_present_catalogName) {
+      if (!(this_present_catalogName && that_present_catalogName))
+        return false;
+      if (!this.catalogName.equals(that.catalogName))
+        return false;
+    }
+
     return true;
   }
 
@@ -635,6 +691,11 @@ import org.slf4j.LoggerFactory;
     if (present_ownerType)
       list.add(ownerType.getValue());
 
+    boolean present_catalogName = true && (isSetCatalogName());
+    list.add(present_catalogName);
+    if (present_catalogName)
+      list.add(catalogName);
+
     return list.hashCode();
   }
 
@@ -716,6 +777,16 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetCatalogName()).compareTo(other.isSetCatalogName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatalogName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catalogName, other.catalogName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -797,6 +868,16 @@ import org.slf4j.LoggerFactory;
       }
       first = false;
     }
+    if (isSetCatalogName()) {
+      if (!first) sb.append(", ");
+      sb.append("catalogName:");
+      if (this.catalogName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.catalogName);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -870,15 +951,15 @@ import org.slf4j.LoggerFactory;
           case 4: // PARAMETERS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map94 = iprot.readMapBegin();
-                struct.parameters = new HashMap<String,String>(2*_map94.size);
-                String _key95;
-                String _val96;
-                for (int _i97 = 0; _i97 < _map94.size; ++_i97)
+                org.apache.thrift.protocol.TMap _map102 = iprot.readMapBegin();
+                struct.parameters = new HashMap<String,String>(2*_map102.size);
+                String _key103;
+                String _val104;
+                for (int _i105 = 0; _i105 < _map102.size; ++_i105)
                 {
-                  _key95 = iprot.readString();
-                  _val96 = iprot.readString();
-                  struct.parameters.put(_key95, _val96);
+                  _key103 = iprot.readString();
+                  _val104 = iprot.readString();
+                  struct.parameters.put(_key103, _val104);
                 }
                 iprot.readMapEnd();
               }
@@ -912,6 +993,14 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 8: // CATALOG_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.catalogName = iprot.readString();
+              struct.setCatalogNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -944,10 +1033,10 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PARAMETERS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.parameters.size()));
-          for (Map.Entry<String, String> _iter98 : struct.parameters.entrySet())
+          for (Map.Entry<String, String> _iter106 : struct.parameters.entrySet())
           {
-            oprot.writeString(_iter98.getKey());
-            oprot.writeString(_iter98.getValue());
+            oprot.writeString(_iter106.getKey());
+            oprot.writeString(_iter106.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -974,6 +1063,13 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldEnd();
         }
       }
+      if (struct.catalogName != null) {
+        if (struct.isSetCatalogName()) {
+          oprot.writeFieldBegin(CATALOG_NAME_FIELD_DESC);
+          oprot.writeString(struct.catalogName);
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -1013,7 +1109,10 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetOwnerType()) {
         optionals.set(6);
       }
-      oprot.writeBitSet(optionals, 7);
+      if (struct.isSetCatalogName()) {
+        optionals.set(7);
+      }
+      oprot.writeBitSet(optionals, 8);
       if (struct.isSetName()) {
         oprot.writeString(struct.name);
       }
@@ -1026,10 +1125,10 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetParameters()) {
         {
           oprot.writeI32(struct.parameters.size());
-          for (Map.Entry<String, String> _iter99 : struct.parameters.entrySet())
+          for (Map.Entry<String, String> _iter107 : struct.parameters.entrySet())
           {
-            oprot.writeString(_iter99.getKey());
-            oprot.writeString(_iter99.getValue());
+            oprot.writeString(_iter107.getKey());
+            oprot.writeString(_iter107.getValue());
           }
         }
       }
@@ -1042,12 +1141,15 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetOwnerType()) {
         oprot.writeI32(struct.ownerType.getValue());
       }
+      if (struct.isSetCatalogName()) {
+        oprot.writeString(struct.catalogName);
+      }
     }
 
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, Database struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
-      BitSet incoming = iprot.readBitSet(7);
+      BitSet incoming = iprot.readBitSet(8);
       if (incoming.get(0)) {
         struct.name = iprot.readString();
         struct.setNameIsSet(true);
@@ -1062,15 +1164,15 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(3)) {
         {
-          org.apache.thrift.protocol.TMap _map100 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.parameters = new HashMap<String,String>(2*_map100.size);
-          String _key101;
-          String _val102;
-          for (int _i103 = 0; _i103 < _map100.size; ++_i103)
+          org.apache.thrift.protocol.TMap _map108 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.parameters = new HashMap<String,String>(2*_map108.size);
+          String _key109;
+          String _val110;
+          for (int _i111 = 0; _i111 < _map108.size; ++_i111)
           {
-            _key101 = iprot.readString();
-            _val102 = iprot.readString();
-            struct.parameters.put(_key101, _val102);
+            _key109 = iprot.readString();
+            _val110 = iprot.readString();
+            struct.parameters.put(_key109, _val110);
           }
         }
         struct.setParametersIsSet(true);
@@ -1088,6 +1190,10 @@ import org.slf4j.LoggerFactory;
         struct.ownerType = org.apache.hadoop.hive.metastore.api.PrincipalType.findByValue(iprot.readI32());
         struct.setOwnerTypeIsSet(true);
       }
+      if (incoming.get(7)) {
+        struct.catalogName = iprot.readString();
+        struct.setCatalogNameIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DefaultConstraintsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DefaultConstraintsRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DefaultConstraintsRequest.java
index 5f4954d..69378c9 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DefaultConstraintsRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DefaultConstraintsRequest.java
@@ -38,8 +38,9 @@ import org.slf4j.LoggerFactory;
 @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class DefaultConstraintsRequest implements org.apache.thrift.TBase<DefaultConstraintsRequest, DefaultConstraintsRequest._Fields>, java.io.Serializable, Cloneable, Comparable<DefaultConstraintsRequest> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("DefaultConstraintsRequest");
 
-  private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("db_name", org.apache.thrift.protocol.TType.STRING, (short)1);
-  private static final org.apache.thrift.protocol.TField TBL_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tbl_name", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("db_name", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField TBL_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tbl_name", org.apache.thrift.protocol.TType.STRING, (short)3);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -47,13 +48,15 @@ import org.slf4j.LoggerFactory;
     schemes.put(TupleScheme.class, new DefaultConstraintsRequestTupleSchemeFactory());
   }
 
+  private String catName; // required
   private String db_name; // required
   private String tbl_name; // required
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-    DB_NAME((short)1, "db_name"),
-    TBL_NAME((short)2, "tbl_name");
+    CAT_NAME((short)1, "catName"),
+    DB_NAME((short)2, "db_name"),
+    TBL_NAME((short)3, "tbl_name");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -68,9 +71,11 @@ import org.slf4j.LoggerFactory;
      */
     public static _Fields findByThriftId(int fieldId) {
       switch(fieldId) {
-        case 1: // DB_NAME
+        case 1: // CAT_NAME
+          return CAT_NAME;
+        case 2: // DB_NAME
           return DB_NAME;
-        case 2: // TBL_NAME
+        case 3: // TBL_NAME
           return TBL_NAME;
         default:
           return null;
@@ -115,6 +120,8 @@ import org.slf4j.LoggerFactory;
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.DB_NAME, new org.apache.thrift.meta_data.FieldMetaData("db_name", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.TBL_NAME, new org.apache.thrift.meta_data.FieldMetaData("tbl_name", org.apache.thrift.TFieldRequirementType.REQUIRED, 
@@ -127,10 +134,12 @@ import org.slf4j.LoggerFactory;
   }
 
   public DefaultConstraintsRequest(
+    String catName,
     String db_name,
     String tbl_name)
   {
     this();
+    this.catName = catName;
     this.db_name = db_name;
     this.tbl_name = tbl_name;
   }
@@ -139,6 +148,9 @@ import org.slf4j.LoggerFactory;
    * Performs a deep copy on <i>other</i>.
    */
   public DefaultConstraintsRequest(DefaultConstraintsRequest other) {
+    if (other.isSetCatName()) {
+      this.catName = other.catName;
+    }
     if (other.isSetDb_name()) {
       this.db_name = other.db_name;
     }
@@ -153,10 +165,34 @@ import org.slf4j.LoggerFactory;
 
   @Override
   public void clear() {
+    this.catName = null;
     this.db_name = null;
     this.tbl_name = null;
   }
 
+  public String getCatName() {
+    return this.catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
+  public void unsetCatName() {
+    this.catName = null;
+  }
+
+  /** Returns true if field catName is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatName() {
+    return this.catName != null;
+  }
+
+  public void setCatNameIsSet(boolean value) {
+    if (!value) {
+      this.catName = null;
+    }
+  }
+
   public String getDb_name() {
     return this.db_name;
   }
@@ -205,6 +241,14 @@ import org.slf4j.LoggerFactory;
 
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
+    case CAT_NAME:
+      if (value == null) {
+        unsetCatName();
+      } else {
+        setCatName((String)value);
+      }
+      break;
+
     case DB_NAME:
       if (value == null) {
         unsetDb_name();
@@ -226,6 +270,9 @@ import org.slf4j.LoggerFactory;
 
   public Object getFieldValue(_Fields field) {
     switch (field) {
+    case CAT_NAME:
+      return getCatName();
+
     case DB_NAME:
       return getDb_name();
 
@@ -243,6 +290,8 @@ import org.slf4j.LoggerFactory;
     }
 
     switch (field) {
+    case CAT_NAME:
+      return isSetCatName();
     case DB_NAME:
       return isSetDb_name();
     case TBL_NAME:
@@ -264,6 +313,15 @@ import org.slf4j.LoggerFactory;
     if (that == null)
       return false;
 
+    boolean this_present_catName = true && this.isSetCatName();
+    boolean that_present_catName = true && that.isSetCatName();
+    if (this_present_catName || that_present_catName) {
+      if (!(this_present_catName && that_present_catName))
+        return false;
+      if (!this.catName.equals(that.catName))
+        return false;
+    }
+
     boolean this_present_db_name = true && this.isSetDb_name();
     boolean that_present_db_name = true && that.isSetDb_name();
     if (this_present_db_name || that_present_db_name) {
@@ -289,6 +347,11 @@ import org.slf4j.LoggerFactory;
   public int hashCode() {
     List<Object> list = new ArrayList<Object>();
 
+    boolean present_catName = true && (isSetCatName());
+    list.add(present_catName);
+    if (present_catName)
+      list.add(catName);
+
     boolean present_db_name = true && (isSetDb_name());
     list.add(present_db_name);
     if (present_db_name)
@@ -310,6 +373,16 @@ import org.slf4j.LoggerFactory;
 
     int lastComparison = 0;
 
+    lastComparison = Boolean.valueOf(isSetCatName()).compareTo(other.isSetCatName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catName, other.catName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     lastComparison = Boolean.valueOf(isSetDb_name()).compareTo(other.isSetDb_name());
     if (lastComparison != 0) {
       return lastComparison;
@@ -350,6 +423,14 @@ import org.slf4j.LoggerFactory;
     StringBuilder sb = new StringBuilder("DefaultConstraintsRequest(");
     boolean first = true;
 
+    sb.append("catName:");
+    if (this.catName == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.catName);
+    }
+    first = false;
+    if (!first) sb.append(", ");
     sb.append("db_name:");
     if (this.db_name == null) {
       sb.append("null");
@@ -371,6 +452,10 @@ import org.slf4j.LoggerFactory;
 
   public void validate() throws org.apache.thrift.TException {
     // check for required fields
+    if (!isSetCatName()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'catName' is unset! Struct:" + toString());
+    }
+
     if (!isSetDb_name()) {
       throw new org.apache.thrift.protocol.TProtocolException("Required field 'db_name' is unset! Struct:" + toString());
     }
@@ -416,7 +501,15 @@ import org.slf4j.LoggerFactory;
           break;
         }
         switch (schemeField.id) {
-          case 1: // DB_NAME
+          case 1: // CAT_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.catName = iprot.readString();
+              struct.setCatNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // DB_NAME
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.db_name = iprot.readString();
               struct.setDb_nameIsSet(true);
@@ -424,7 +517,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 2: // TBL_NAME
+          case 3: // TBL_NAME
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.tbl_name = iprot.readString();
               struct.setTbl_nameIsSet(true);
@@ -445,6 +538,11 @@ import org.slf4j.LoggerFactory;
       struct.validate();
 
       oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.catName != null) {
+        oprot.writeFieldBegin(CAT_NAME_FIELD_DESC);
+        oprot.writeString(struct.catName);
+        oprot.writeFieldEnd();
+      }
       if (struct.db_name != null) {
         oprot.writeFieldBegin(DB_NAME_FIELD_DESC);
         oprot.writeString(struct.db_name);
@@ -472,6 +570,7 @@ import org.slf4j.LoggerFactory;
     @Override
     public void write(org.apache.thrift.protocol.TProtocol prot, DefaultConstraintsRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.catName);
       oprot.writeString(struct.db_name);
       oprot.writeString(struct.tbl_name);
     }
@@ -479,6 +578,8 @@ import org.slf4j.LoggerFactory;
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, DefaultConstraintsRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.catName = iprot.readString();
+      struct.setCatNameIsSet(true);
       struct.db_name = iprot.readString();
       struct.setDb_nameIsSet(true);
       struct.tbl_name = iprot.readString();

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DefaultConstraintsResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DefaultConstraintsResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DefaultConstraintsResponse.java
index f7ee187..47b8d1c 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DefaultConstraintsResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DefaultConstraintsResponse.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // DEFAULT_CONSTRAINTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list344 = iprot.readListBegin();
-                struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list344.size);
-                SQLDefaultConstraint _elem345;
-                for (int _i346 = 0; _i346 < _list344.size; ++_i346)
+                org.apache.thrift.protocol.TList _list352 = iprot.readListBegin();
+                struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list352.size);
+                SQLDefaultConstraint _elem353;
+                for (int _i354 = 0; _i354 < _list352.size; ++_i354)
                 {
-                  _elem345 = new SQLDefaultConstraint();
-                  _elem345.read(iprot);
-                  struct.defaultConstraints.add(_elem345);
+                  _elem353 = new SQLDefaultConstraint();
+                  _elem353.read(iprot);
+                  struct.defaultConstraints.add(_elem353);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(DEFAULT_CONSTRAINTS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.defaultConstraints.size()));
-          for (SQLDefaultConstraint _iter347 : struct.defaultConstraints)
+          for (SQLDefaultConstraint _iter355 : struct.defaultConstraints)
           {
-            _iter347.write(oprot);
+            _iter355.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.defaultConstraints.size());
-        for (SQLDefaultConstraint _iter348 : struct.defaultConstraints)
+        for (SQLDefaultConstraint _iter356 : struct.defaultConstraints)
         {
-          _iter348.write(oprot);
+          _iter356.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, DefaultConstraintsResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list349 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list349.size);
-        SQLDefaultConstraint _elem350;
-        for (int _i351 = 0; _i351 < _list349.size; ++_i351)
+        org.apache.thrift.protocol.TList _list357 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list357.size);
+        SQLDefaultConstraint _elem358;
+        for (int _i359 = 0; _i359 < _list357.size; ++_i359)
         {
-          _elem350 = new SQLDefaultConstraint();
-          _elem350.read(iprot);
-          struct.defaultConstraints.add(_elem350);
+          _elem358 = new SQLDefaultConstraint();
+          _elem358.read(iprot);
+          struct.defaultConstraints.add(_elem358);
         }
       }
       struct.setDefaultConstraintsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropCatalogRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropCatalogRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropCatalogRequest.java
new file mode 100644
index 0000000..a11fe47
--- /dev/null
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DropCatalogRequest.java
@@ -0,0 +1,395 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class DropCatalogRequest implements org.apache.thrift.TBase<DropCatalogRequest, DropCatalogRequest._Fields>, java.io.Serializable, Cloneable, Comparable<DropCatalogRequest> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("DropCatalogRequest");
+
+  private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new DropCatalogRequestStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new DropCatalogRequestTupleSchemeFactory());
+  }
+
+  private String name; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    NAME((short)1, "name");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // NAME
+          return NAME;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(DropCatalogRequest.class, metaDataMap);
+  }
+
+  public DropCatalogRequest() {
+  }
+
+  public DropCatalogRequest(
+    String name)
+  {
+    this();
+    this.name = name;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public DropCatalogRequest(DropCatalogRequest other) {
+    if (other.isSetName()) {
+      this.name = other.name;
+    }
+  }
+
+  public DropCatalogRequest deepCopy() {
+    return new DropCatalogRequest(this);
+  }
+
+  @Override
+  public void clear() {
+    this.name = null;
+  }
+
+  public String getName() {
+    return this.name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  public void unsetName() {
+    this.name = null;
+  }
+
+  /** Returns true if field name is set (has been assigned a value) and false otherwise */
+  public boolean isSetName() {
+    return this.name != null;
+  }
+
+  public void setNameIsSet(boolean value) {
+    if (!value) {
+      this.name = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case NAME:
+      if (value == null) {
+        unsetName();
+      } else {
+        setName((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case NAME:
+      return getName();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case NAME:
+      return isSetName();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof DropCatalogRequest)
+      return this.equals((DropCatalogRequest)that);
+    return false;
+  }
+
+  public boolean equals(DropCatalogRequest that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_name = true && this.isSetName();
+    boolean that_present_name = true && that.isSetName();
+    if (this_present_name || that_present_name) {
+      if (!(this_present_name && that_present_name))
+        return false;
+      if (!this.name.equals(that.name))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_name = true && (isSetName());
+    list.add(present_name);
+    if (present_name)
+      list.add(name);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(DropCatalogRequest other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetName()).compareTo(other.isSetName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("DropCatalogRequest(");
+    boolean first = true;
+
+    sb.append("name:");
+    if (this.name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.name);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class DropCatalogRequestStandardSchemeFactory implements SchemeFactory {
+    public DropCatalogRequestStandardScheme getScheme() {
+      return new DropCatalogRequestStandardScheme();
+    }
+  }
+
+  private static class DropCatalogRequestStandardScheme extends StandardScheme<DropCatalogRequest> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, DropCatalogRequest struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.name = iprot.readString();
+              struct.setNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, DropCatalogRequest struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.name != null) {
+        oprot.writeFieldBegin(NAME_FIELD_DESC);
+        oprot.writeString(struct.name);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class DropCatalogRequestTupleSchemeFactory implements SchemeFactory {
+    public DropCatalogRequestTupleScheme getScheme() {
+      return new DropCatalogRequestTupleScheme();
+    }
+  }
+
+  private static class DropCatalogRequestTupleScheme extends TupleScheme<DropCatalogRequest> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, DropCatalogRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.isSetName()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.isSetName()) {
+        oprot.writeString(struct.name);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, DropCatalogRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.name = iprot.readString();
+        struct.setNameIsSet(true);
+      }
+    }
+  }
+
+}
+


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStoreSchemaMethods.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStoreSchemaMethods.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStoreSchemaMethods.java
index 2b8fbd1..137082f 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStoreSchemaMethods.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStoreSchemaMethods.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.hive.metastore.api.SchemaVersion;
 import org.apache.hadoop.hive.metastore.api.SchemaVersionDescriptor;
 import org.apache.hadoop.hive.metastore.api.SchemaVersionState;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
+import org.apache.hadoop.hive.metastore.client.builder.CatalogBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.ISchemaBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.SchemaVersionBuilder;
@@ -47,16 +48,19 @@ import org.junit.experimental.categories.Category;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
+import java.util.Random;
 
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
 import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
 
 @Category(MetastoreCheckinTest.class)
 public class TestObjectStoreSchemaMethods {
   private RawStore objectStore;
+  private Configuration conf;
 
   @Before
   public void setUp() throws Exception {
-    Configuration conf = MetastoreConf.newMetastoreConf();
+    conf = MetastoreConf.newMetastoreConf();
     MetastoreConf.setVar(conf, MetastoreConf.ConfVars.EXPRESSION_PROXY_CLASS,
         DefaultPartitionExpressionProxy.class.getName());
 
@@ -66,8 +70,8 @@ public class TestObjectStoreSchemaMethods {
 
   @Test
   public void iSchema() throws TException {
-    String dbName = createUniqueDatabaseForTest();
-    ISchema schema = objectStore.getISchema(new ISchemaName(dbName, "no.such.schema"));
+    Database db = createUniqueDatabaseForTest();
+    ISchema schema = objectStore.getISchema(new ISchemaName(db.getCatalogName(), db.getName(), "no.such.schema"));
     Assert.assertNull(schema);
 
     String schemaName = "schema1";
@@ -76,7 +80,7 @@ public class TestObjectStoreSchemaMethods {
     schema = new ISchemaBuilder()
         .setSchemaType(SchemaType.AVRO)
         .setName(schemaName)
-        .setDbName(dbName)
+        .inDb(db)
         .setCompatibility(SchemaCompatibility.FORWARD)
         .setValidationLevel(SchemaValidation.LATEST)
         .setCanEvolve(false)
@@ -85,7 +89,7 @@ public class TestObjectStoreSchemaMethods {
         .build();
     objectStore.createISchema(schema);
 
-    schema = objectStore.getISchema(new ISchemaName(dbName, schemaName));
+    schema = objectStore.getISchema(new ISchemaName(db.getCatalogName(), db.getName(), schemaName));
     Assert.assertNotNull(schema);
 
     Assert.assertEquals(SchemaType.AVRO, schema.getSchemaType());
@@ -103,9 +107,9 @@ public class TestObjectStoreSchemaMethods {
     schema.setCanEvolve(true);
     schema.setSchemaGroup(schemaGroup);
     schema.setDescription(description);
-    objectStore.alterISchema(new ISchemaName(dbName, schemaName), schema);
+    objectStore.alterISchema(new ISchemaName(db.getCatalogName(), db.getName(), schemaName), schema);
 
-    schema = objectStore.getISchema(new ISchemaName(dbName, schemaName));
+    schema = objectStore.getISchema(new ISchemaName(db.getCatalogName(), db.getName(), schemaName));
     Assert.assertNotNull(schema);
 
     Assert.assertEquals(SchemaType.AVRO, schema.getSchemaType());
@@ -116,8 +120,8 @@ public class TestObjectStoreSchemaMethods {
     Assert.assertEquals(schemaGroup, schema.getSchemaGroup());
     Assert.assertEquals(description, schema.getDescription());
 
-    objectStore.dropISchema(new ISchemaName(dbName, schemaName));
-    schema = objectStore.getISchema(new ISchemaName(dbName, schemaName));
+    objectStore.dropISchema(new ISchemaName(db.getCatalogName(), db.getName(), schemaName));
+    schema = objectStore.getISchema(new ISchemaName(db.getCatalogName(), db.getName(), schemaName));
     Assert.assertNull(schema);
   }
 
@@ -134,16 +138,16 @@ public class TestObjectStoreSchemaMethods {
 
   @Test(expected = AlreadyExistsException.class)
   public void schemaAlreadyExists() throws TException {
-    String dbName = createUniqueDatabaseForTest();
+    Database db = createUniqueDatabaseForTest();
     String schemaName = "schema2";
     ISchema schema = new ISchemaBuilder()
         .setSchemaType(SchemaType.HIVE)
         .setName(schemaName)
-        .setDbName(dbName)
+        .inDb(db)
         .build();
     objectStore.createISchema(schema);
 
-    schema = objectStore.getISchema(new ISchemaName(dbName, schemaName));
+    schema = objectStore.getISchema(new ISchemaName(db.getCatalogName(), db.getName(), schemaName));
     Assert.assertNotNull(schema);
 
     Assert.assertEquals(SchemaType.HIVE, schema.getSchemaType());
@@ -164,12 +168,12 @@ public class TestObjectStoreSchemaMethods {
         .setName(schemaName)
         .setDescription("a new description")
         .build();
-    objectStore.alterISchema(new ISchemaName(DEFAULT_DATABASE_NAME, schemaName), schema);
+    objectStore.alterISchema(new ISchemaName(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, schemaName), schema);
   }
 
   @Test(expected = NoSuchObjectException.class)
   public void dropNonExistentSchema() throws MetaException, NoSuchObjectException {
-    objectStore.dropISchema(new ISchemaName(DEFAULT_DATABASE_NAME, "no_such_schema"));
+    objectStore.dropISchema(new ISchemaName(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, "no_such_schema"));
   }
 
   @Test(expected = NoSuchObjectException.class)
@@ -177,7 +181,6 @@ public class TestObjectStoreSchemaMethods {
       NoSuchObjectException, InvalidObjectException {
     SchemaVersion schemaVersion = new SchemaVersionBuilder()
         .setSchemaName("noSchemaOfThisNameExists")
-        .setDbName(DEFAULT_DATABASE_NAME)
         .setVersion(1)
         .addCol("a", ColumnType.STRING_TYPE_NAME)
         .build();
@@ -186,16 +189,16 @@ public class TestObjectStoreSchemaMethods {
 
   @Test
   public void addSchemaVersion() throws TException {
-    String dbName = createUniqueDatabaseForTest();
+    Database db = createUniqueDatabaseForTest();
     String schemaName = "schema37";
     int version = 1;
-    SchemaVersion schemaVersion = objectStore.getSchemaVersion(new SchemaVersionDescriptor(new ISchemaName(dbName, schemaName), version));
+    SchemaVersion schemaVersion = objectStore.getSchemaVersion(new SchemaVersionDescriptor(new ISchemaName(db.getCatalogName(), db.getName(), schemaName), version));
     Assert.assertNull(schemaVersion);
 
     ISchema schema = new ISchemaBuilder()
         .setSchemaType(SchemaType.AVRO)
         .setName(schemaName)
-        .setDbName(dbName)
+        .inDb(db)
         .build();
     objectStore.createISchema(schema);
 
@@ -226,10 +229,11 @@ public class TestObjectStoreSchemaMethods {
         .build();
     objectStore.addSchemaVersion(schemaVersion);
 
-    schemaVersion = objectStore.getSchemaVersion(new SchemaVersionDescriptor(new ISchemaName(dbName, schemaName), version));
+    schemaVersion = objectStore.getSchemaVersion(new SchemaVersionDescriptor(new ISchemaName(db.getCatalogName(), db.getName(), schemaName), version));
     Assert.assertNotNull(schemaVersion);
     Assert.assertEquals(schemaName, schemaVersion.getSchema().getSchemaName());
-    Assert.assertEquals(dbName, schemaVersion.getSchema().getDbName());
+    Assert.assertEquals(db.getName(), schemaVersion.getSchema().getDbName());
+    Assert.assertEquals(db.getCatalogName(), schemaVersion.getSchema().getCatName());
     Assert.assertEquals(version, schemaVersion.getVersion());
     Assert.assertEquals(creationTime, schemaVersion.getCreatedAt());
     Assert.assertEquals(SchemaVersionState.INITIATED, schemaVersion.getState());
@@ -249,21 +253,21 @@ public class TestObjectStoreSchemaMethods {
     Assert.assertEquals("b", cols.get(1).getName());
     Assert.assertEquals(ColumnType.FLOAT_TYPE_NAME, cols.get(1).getType());
 
-    objectStore.dropSchemaVersion(new SchemaVersionDescriptor(new ISchemaName(dbName, schemaName), version));
-    schemaVersion = objectStore.getSchemaVersion(new SchemaVersionDescriptor(new ISchemaName(dbName, schemaName), version));
+    objectStore.dropSchemaVersion(new SchemaVersionDescriptor(new ISchemaName(db.getCatalogName(), db.getName(), schemaName), version));
+    schemaVersion = objectStore.getSchemaVersion(new SchemaVersionDescriptor(new ISchemaName(db.getCatalogName(), db.getName(), schemaName), version));
     Assert.assertNull(schemaVersion);
   }
 
   // Test that adding multiple versions of the same schema
   @Test
   public void multipleSchemaVersions() throws TException {
-    String dbName = createUniqueDatabaseForTest();
+    Database db = createUniqueDatabaseForTest();
     String schemaName = "schema195";
 
     ISchema schema = new ISchemaBuilder()
         .setSchemaType(SchemaType.AVRO)
         .setName(schemaName)
-        .setDbName(dbName)
+        .inDb(db)
         .build();
     objectStore.createISchema(schema);
     SchemaVersion schemaVersion = new SchemaVersionBuilder()
@@ -290,7 +294,7 @@ public class TestObjectStoreSchemaMethods {
         .build();
     objectStore.addSchemaVersion(schemaVersion);
 
-    schemaVersion = objectStore.getLatestSchemaVersion(new ISchemaName(dbName, schemaName));
+    schemaVersion = objectStore.getLatestSchemaVersion(new ISchemaName(db.getCatalogName(), db.getName(), schemaName));
     Assert.assertEquals(3, schemaVersion.getVersion());
     Assert.assertEquals(3, schemaVersion.getColsSize());
     List<FieldSchema> cols = schemaVersion.getCols();
@@ -302,14 +306,14 @@ public class TestObjectStoreSchemaMethods {
     Assert.assertEquals(ColumnType.DATE_TYPE_NAME, cols.get(1).getType());
     Assert.assertEquals(ColumnType.TIMESTAMP_TYPE_NAME, cols.get(2).getType());
 
-    schemaVersion = objectStore.getLatestSchemaVersion(new ISchemaName(dbName, "no.such.schema.with.this.name"));
+    schemaVersion = objectStore.getLatestSchemaVersion(new ISchemaName(db.getCatalogName(), db.getName(), "no.such.schema.with.this.name"));
     Assert.assertNull(schemaVersion);
 
     List<SchemaVersion> versions =
-        objectStore.getAllSchemaVersion(new ISchemaName(dbName, "there.really.isnt.a.schema.named.this"));
+        objectStore.getAllSchemaVersion(new ISchemaName(db.getCatalogName(), db.getName(), "there.really.isnt.a.schema.named.this"));
     Assert.assertNull(versions);
 
-    versions = objectStore.getAllSchemaVersion(new ISchemaName(dbName, schemaName));
+    versions = objectStore.getAllSchemaVersion(new ISchemaName(db.getCatalogName(), db.getName(), schemaName));
     Assert.assertEquals(3, versions.size());
     versions.sort(Comparator.comparingInt(SchemaVersion::getVersion));
     Assert.assertEquals(1, versions.get(0).getVersion());
@@ -339,16 +343,16 @@ public class TestObjectStoreSchemaMethods {
 
   @Test(expected = AlreadyExistsException.class)
   public void addDuplicateSchemaVersion() throws TException {
-    String dbName = createUniqueDatabaseForTest();
+    Database db = createUniqueDatabaseForTest();
     String schemaName = "schema1234";
     int version = 1;
-    SchemaVersion schemaVersion = objectStore.getSchemaVersion(new SchemaVersionDescriptor(new ISchemaName(dbName, schemaName), version));
+    SchemaVersion schemaVersion = objectStore.getSchemaVersion(new SchemaVersionDescriptor(new ISchemaName(db.getCatalogName(), db.getName(), schemaName), version));
     Assert.assertNull(schemaVersion);
 
     ISchema schema = new ISchemaBuilder()
         .setSchemaType(SchemaType.AVRO)
         .setName(schemaName)
-        .setDbName(dbName)
+        .inDb(db)
         .build();
     objectStore.createISchema(schema);
 
@@ -365,16 +369,16 @@ public class TestObjectStoreSchemaMethods {
 
   @Test
   public void alterSchemaVersion() throws TException {
-    String dbName = createUniqueDatabaseForTest();
+    Database db = createUniqueDatabaseForTest();
     String schemaName = "schema371234";
     int version = 1;
-    SchemaVersion schemaVersion = objectStore.getSchemaVersion(new SchemaVersionDescriptor(new ISchemaName(dbName, schemaName), version));
+    SchemaVersion schemaVersion = objectStore.getSchemaVersion(new SchemaVersionDescriptor(new ISchemaName(db.getCatalogName(), db.getName(), schemaName), version));
     Assert.assertNull(schemaVersion);
 
     ISchema schema = new ISchemaBuilder()
         .setSchemaType(SchemaType.AVRO)
         .setName(schemaName)
-        .setDbName(dbName)
+        .inDb(db)
         .build();
     objectStore.createISchema(schema);
 
@@ -387,10 +391,11 @@ public class TestObjectStoreSchemaMethods {
         .build();
     objectStore.addSchemaVersion(schemaVersion);
 
-    schemaVersion = objectStore.getSchemaVersion(new SchemaVersionDescriptor(new ISchemaName(dbName, schemaName), version));
+    schemaVersion = objectStore.getSchemaVersion(new SchemaVersionDescriptor(new ISchemaName(db.getCatalogName(), db.getName(), schemaName), version));
     Assert.assertNotNull(schemaVersion);
     Assert.assertEquals(schemaName, schemaVersion.getSchema().getSchemaName());
-    Assert.assertEquals(dbName, schemaVersion.getSchema().getDbName());
+    Assert.assertEquals(db.getName(), schemaVersion.getSchema().getDbName());
+    Assert.assertEquals(db.getCatalogName(), schemaVersion.getSchema().getCatName());
     Assert.assertEquals(version, schemaVersion.getVersion());
     Assert.assertEquals(SchemaVersionState.INITIATED, schemaVersion.getState());
 
@@ -402,12 +407,13 @@ public class TestObjectStoreSchemaMethods {
     serde.setSerializerClass(serializer);
     serde.setDeserializerClass(deserializer);
     schemaVersion.setSerDe(serde);
-    objectStore.alterSchemaVersion(new SchemaVersionDescriptor(new ISchemaName(dbName, schemaName), version), schemaVersion);
+    objectStore.alterSchemaVersion(new SchemaVersionDescriptor(new ISchemaName(db.getCatalogName(), db.getName(), schemaName), version), schemaVersion);
 
-    schemaVersion = objectStore.getSchemaVersion(new SchemaVersionDescriptor(new ISchemaName(dbName, schemaName), version));
+    schemaVersion = objectStore.getSchemaVersion(new SchemaVersionDescriptor(new ISchemaName(db.getCatalogName(), db.getName(), schemaName), version));
     Assert.assertNotNull(schemaVersion);
     Assert.assertEquals(schemaName, schemaVersion.getSchema().getSchemaName());
-    Assert.assertEquals(dbName, schemaVersion.getSchema().getDbName());
+    Assert.assertEquals(db.getName(), schemaVersion.getSchema().getDbName());
+    Assert.assertEquals(db.getCatalogName(), schemaVersion.getSchema().getCatName());
     Assert.assertEquals(version, schemaVersion.getVersion());
     Assert.assertEquals(SchemaVersionState.REVIEWED, schemaVersion.getState());
     Assert.assertEquals(serdeName, schemaVersion.getSerDe().getName());
@@ -428,22 +434,22 @@ public class TestObjectStoreSchemaMethods {
         .addCol("b", ColumnType.FLOAT_TYPE_NAME)
         .setState(SchemaVersionState.INITIATED)
         .build();
-    objectStore.alterSchemaVersion(new SchemaVersionDescriptor(new ISchemaName(DEFAULT_DATABASE_NAME, schemaName), version), schemaVersion);
+    objectStore.alterSchemaVersion(new SchemaVersionDescriptor(new ISchemaName(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, schemaName), version), schemaVersion);
   }
 
   @Test(expected = NoSuchObjectException.class)
   public void dropNonExistentSchemaVersion() throws NoSuchObjectException, MetaException {
-    objectStore.dropSchemaVersion(new SchemaVersionDescriptor(new ISchemaName(DEFAULT_DATABASE_NAME, "ther is no schema named this"), 23));
+    objectStore.dropSchemaVersion(new SchemaVersionDescriptor(new ISchemaName(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, "ther is no schema named this"), 23));
   }
 
   @Test
   public void schemaQuery() throws TException {
-    String dbName = createUniqueDatabaseForTest();
+    Database db = createUniqueDatabaseForTest();
     String schemaName1 = "a_schema1";
     ISchema schema1 = new ISchemaBuilder()
         .setSchemaType(SchemaType.AVRO)
         .setName(schemaName1)
-        .setDbName(dbName)
+        .inDb(db)
         .build();
     objectStore.createISchema(schema1);
 
@@ -451,7 +457,7 @@ public class TestObjectStoreSchemaMethods {
     ISchema schema2 = new ISchemaBuilder()
         .setSchemaType(SchemaType.AVRO)
         .setName(schemaName2)
-        .setDbName(dbName)
+        .inDb(db)
         .build();
     objectStore.createISchema(schema2);
 
@@ -497,7 +503,8 @@ public class TestObjectStoreSchemaMethods {
     results = objectStore.getSchemaVersionsByColumns("gamma", null, null);
     Assert.assertEquals(1, results.size());
     Assert.assertEquals(schemaName1, results.get(0).getSchema().getSchemaName());
-    Assert.assertEquals(dbName, results.get(0).getSchema().getDbName());
+    Assert.assertEquals(db.getName(), results.get(0).getSchema().getDbName());
+    Assert.assertEquals(db.getCatalogName(), results.get(0).getSchema().getCatName());
     Assert.assertEquals(2, results.get(0).getVersion());
 
     // fetch 2 in same schema
@@ -505,10 +512,12 @@ public class TestObjectStoreSchemaMethods {
     Assert.assertEquals(2, results.size());
     Collections.sort(results);
     Assert.assertEquals(schemaName1, results.get(0).getSchema().getSchemaName());
-    Assert.assertEquals(dbName, results.get(0).getSchema().getDbName());
+    Assert.assertEquals(db.getName(), results.get(0).getSchema().getDbName());
+    Assert.assertEquals(db.getCatalogName(), results.get(0).getSchema().getCatName());
     Assert.assertEquals(1, results.get(0).getVersion());
     Assert.assertEquals(schemaName1, results.get(1).getSchema().getSchemaName());
-    Assert.assertEquals(dbName, results.get(1).getSchema().getDbName());
+    Assert.assertEquals(db.getName(), results.get(1).getSchema().getDbName());
+    Assert.assertEquals(db.getCatalogName(), results.get(1).getSchema().getCatName());
     Assert.assertEquals(2, results.get(1).getVersion());
 
     // fetch across schemas
@@ -516,16 +525,20 @@ public class TestObjectStoreSchemaMethods {
     Assert.assertEquals(4, results.size());
     Collections.sort(results);
     Assert.assertEquals(schemaName1, results.get(0).getSchema().getSchemaName());
-    Assert.assertEquals(dbName, results.get(0).getSchema().getDbName());
+    Assert.assertEquals(db.getName(), results.get(0).getSchema().getDbName());
+    Assert.assertEquals(db.getCatalogName(), results.get(0).getSchema().getCatName());
     Assert.assertEquals(1, results.get(0).getVersion());
     Assert.assertEquals(schemaName1, results.get(1).getSchema().getSchemaName());
-    Assert.assertEquals(dbName, results.get(1).getSchema().getDbName());
+    Assert.assertEquals(db.getName(), results.get(1).getSchema().getDbName());
+    Assert.assertEquals(db.getCatalogName(), results.get(1).getSchema().getCatName());
     Assert.assertEquals(2, results.get(1).getVersion());
     Assert.assertEquals(schemaName2, results.get(2).getSchema().getSchemaName());
-    Assert.assertEquals(dbName, results.get(2).getSchema().getDbName());
+    Assert.assertEquals(db.getName(), results.get(2).getSchema().getDbName());
+    Assert.assertEquals(db.getCatalogName(), results.get(2).getSchema().getCatName());
     Assert.assertEquals(1, results.get(2).getVersion());
     Assert.assertEquals(schemaName2, results.get(3).getSchema().getSchemaName());
-    Assert.assertEquals(dbName, results.get(3).getSchema().getDbName());
+    Assert.assertEquals(db.getName(), results.get(3).getSchema().getDbName());
+    Assert.assertEquals(db.getCatalogName(), results.get(3).getSchema().getCatName());
     Assert.assertEquals(2, results.get(3).getVersion());
 
     // fetch by namespace
@@ -533,10 +546,12 @@ public class TestObjectStoreSchemaMethods {
     Assert.assertEquals(2, results.size());
     Collections.sort(results);
     Assert.assertEquals(schemaName1, results.get(0).getSchema().getSchemaName());
-    Assert.assertEquals(dbName, results.get(0).getSchema().getDbName());
+    Assert.assertEquals(db.getName(), results.get(0).getSchema().getDbName());
+    Assert.assertEquals(db.getCatalogName(), results.get(0).getSchema().getCatName());
     Assert.assertEquals(2, results.get(0).getVersion());
     Assert.assertEquals(schemaName2, results.get(1).getSchema().getSchemaName());
-    Assert.assertEquals(dbName, results.get(1).getSchema().getDbName());
+    Assert.assertEquals(db.getName(), results.get(1).getSchema().getDbName());
+    Assert.assertEquals(db.getCatalogName(), results.get(1).getSchema().getCatName());
     Assert.assertEquals(2, results.get(1).getVersion());
 
     // fetch by name and type
@@ -544,10 +559,12 @@ public class TestObjectStoreSchemaMethods {
     Assert.assertEquals(2, results.size());
     Collections.sort(results);
     Assert.assertEquals(schemaName2, results.get(0).getSchema().getSchemaName());
-    Assert.assertEquals(dbName, results.get(0).getSchema().getDbName());
+    Assert.assertEquals(db.getName(), results.get(0).getSchema().getDbName());
+    Assert.assertEquals(db.getCatalogName(), results.get(0).getSchema().getCatName());
     Assert.assertEquals(1, results.get(0).getVersion());
     Assert.assertEquals(schemaName2, results.get(1).getSchema().getSchemaName());
-    Assert.assertEquals(dbName, results.get(1).getSchema().getDbName());
+    Assert.assertEquals(db.getName(), results.get(1).getSchema().getDbName());
+    Assert.assertEquals(db.getCatalogName(), results.get(1).getSchema().getCatName());
     Assert.assertEquals(2, results.get(1).getVersion());
 
     // Make sure matching name but wrong type doesn't return
@@ -560,14 +577,26 @@ public class TestObjectStoreSchemaMethods {
   }
 
   private static int dbNum = 1;
-  private String createUniqueDatabaseForTest() throws MetaException, InvalidObjectException {
+  private static Random rand = new Random();
+  private Database createUniqueDatabaseForTest() throws MetaException, InvalidObjectException {
+    String catName;
+    if (rand.nextDouble() < 0.5) {
+      catName = "unique_cat_for_test_" + dbNum++;
+      objectStore.createCatalog(new CatalogBuilder()
+          .setName(catName)
+          .setLocation("there")
+          .build());
+    } else {
+      catName = DEFAULT_CATALOG_NAME;
+    }
     String dbName = "uniquedbfortest" + dbNum++;
     Database db = new DatabaseBuilder()
         .setName(dbName)
+        .setCatalogName(catName)
         .setLocation("somewhere")
         .setDescription("descriptive")
-        .build();
+        .build(conf);
     objectStore.createDatabase(db);
-    return dbName;
+    return db;
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestOldSchema.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestOldSchema.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestOldSchema.java
index f286da8..49033d3 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestOldSchema.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestOldSchema.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.common.ndv.hll.HyperLogLog;
 import org.apache.hadoop.hive.metastore.annotation.MetastoreUnitTest;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
+import org.apache.hadoop.hive.metastore.api.Catalog;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc;
@@ -42,6 +43,7 @@ import org.apache.hadoop.hive.metastore.api.Partition;
 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.client.builder.DatabaseBuilder;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
 import org.junit.After;
@@ -52,9 +54,12 @@ import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
+
 @Category(MetastoreUnitTest.class)
 public class TestOldSchema {
   private ObjectStore store = null;
+  private Configuration conf;
 
   private static final Logger LOG = LoggerFactory.getLogger(TestOldSchema.class.getName());
 
@@ -91,13 +96,14 @@ public class TestOldSchema {
 
   @Before
   public void setUp() throws Exception {
-    Configuration conf = MetastoreConf.newMetastoreConf();
+    conf = MetastoreConf.newMetastoreConf();
     MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.STATS_FETCH_BITVECTOR, false);
     MetaStoreTestUtils.setConfForStandloneMode(conf);
 
     store = new ObjectStore();
     store.setConf(conf);
     dropAllStoreObjects(store);
+    HiveMetaStore.HMSHandler.createDefaultCatalog(store, new Warehouse(conf));
 
     HyperLogLog hll = HyperLogLog.builder().build();
     hll.addLong(1);
@@ -121,7 +127,11 @@ public class TestOldSchema {
   public void testPartitionOps() throws Exception {
     String dbName = "default";
     String tableName = "snp";
-    Database db1 = new Database(dbName, "description", "locationurl", null);
+    Database db1 = new DatabaseBuilder()
+        .setName(dbName)
+        .setDescription("description")
+        .setLocation("locationurl")
+        .build(conf);
     store.createDatabase(db1);
     long now = System.currentTimeMillis();
     List<FieldSchema> cols = new ArrayList<>();
@@ -143,6 +153,7 @@ public class TestOldSchema {
       psd.setLocation("file:/tmp/default/hit/ds=" + partVal);
       Partition part = new Partition(partVal, dbName, tableName, (int) now, (int) now, psd,
           Collections.emptyMap());
+      part.setCatName(DEFAULT_CATALOG_NAME);
       store.addPartition(part);
       ColumnStatistics cs = new ColumnStatistics();
       ColumnStatisticsDesc desc = new ColumnStatisticsDesc(false, dbName, tableName);
@@ -185,7 +196,7 @@ public class TestOldSchema {
     for (int i = 0; i < 10; i++) {
       partNames.add("ds=" + i);
     }
-    AggrStats aggrStats = store.get_aggr_stats_for(dbName, tableName, partNames,
+    AggrStats aggrStats = store.get_aggr_stats_for(DEFAULT_CATALOG_NAME, dbName, tableName, partNames,
         Arrays.asList("col1"));
     statChecker.checkStats(aggrStats);
 
@@ -200,18 +211,18 @@ public class TestOldSchema {
     try {
       Deadline.registerIfNot(100000);
       Deadline.startTimer("getPartition");
-      List<String> dbs = store.getAllDatabases();
+      List<String> dbs = store.getAllDatabases(DEFAULT_CATALOG_NAME);
       for (int i = 0; i < dbs.size(); i++) {
         String db = dbs.get(i);
-        List<String> tbls = store.getAllTables(db);
+        List<String> tbls = store.getAllTables(DEFAULT_CATALOG_NAME, db);
         for (String tbl : tbls) {
-          List<Partition> parts = store.getPartitions(db, tbl, 100);
+          List<Partition> parts = store.getPartitions(DEFAULT_CATALOG_NAME, db, tbl, 100);
           for (Partition part : parts) {
-            store.dropPartition(db, tbl, part.getValues());
+            store.dropPartition(DEFAULT_CATALOG_NAME, db, tbl, part.getValues());
           }
-          store.dropTable(db, tbl);
+          store.dropTable(DEFAULT_CATALOG_NAME, db, tbl);
         }
-        store.dropDatabase(db);
+        store.dropDatabase(DEFAULT_CATALOG_NAME, db);
       }
     } catch (NoSuchObjectException e) {
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteHiveMetaStoreIpAddress.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteHiveMetaStoreIpAddress.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteHiveMetaStoreIpAddress.java
index 6b7d913..3d1723e 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteHiveMetaStoreIpAddress.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteHiveMetaStoreIpAddress.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.metastore;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
 import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
 import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
@@ -57,9 +58,9 @@ public class TestRemoteHiveMetaStoreIpAddress {
 
   @Test
   public void testIpAddress() throws Exception {
-    Database db = new Database();
-    db.setName("testIpAddressIp");
-    msc.createDatabase(db);
+    Database db = new DatabaseBuilder()
+        .setName("testIpAddressIp")
+        .create(msc, conf);
     msc.dropDatabase(db.getName());
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestRetryingHMSHandler.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestRetryingHMSHandler.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestRetryingHMSHandler.java
index 11f84f2..930e996 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestRetryingHMSHandler.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestRetryingHMSHandler.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
@@ -63,19 +64,17 @@ public class TestRetryingHMSHandler {
     String dbName = "hive4159";
     String tblName = "tmptbl";
 
-    Database db = new Database();
-    db.setName(dbName);
-    msc.createDatabase(db);
+    new DatabaseBuilder()
+        .setName(dbName)
+        .create(msc, conf);
 
     Assert.assertEquals(2, AlternateFailurePreListener.getCallCount());
 
-    Table tbl = new TableBuilder()
+    new TableBuilder()
         .setDbName(dbName)
         .setTableName(tblName)
         .addCol("c1", ColumnType.STRING_TYPE_NAME)
-        .build();
-
-    msc.createTable(tbl);
+        .create(msc, conf);
 
     Assert.assertEquals(4, AlternateFailurePreListener.getCallCount());
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestStats.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestStats.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestStats.java
new file mode 100644
index 0000000..6cca062
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestStats.java
@@ -0,0 +1,728 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
+import org.apache.hadoop.hive.metastore.api.AggrStats;
+import org.apache.hadoop.hive.metastore.api.BinaryColumnStatsData;
+import org.apache.hadoop.hive.metastore.api.BooleanColumnStatsData;
+import org.apache.hadoop.hive.metastore.api.Catalog;
+import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.Date;
+import org.apache.hadoop.hive.metastore.api.DateColumnStatsData;
+import org.apache.hadoop.hive.metastore.api.DoubleColumnStatsData;
+import org.apache.hadoop.hive.metastore.api.LongColumnStatsData;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.SetPartitionsStatsRequest;
+import org.apache.hadoop.hive.metastore.api.StringColumnStatsData;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.client.builder.CatalogBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.PartitionBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.thrift.TException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
+
+@Category(MetastoreCheckinTest.class)
+public class TestStats {
+  private static final Logger LOG = LoggerFactory.getLogger(TestStats.class);
+
+  private static final String NO_CAT = "DO_NOT_USE_A_CATALOG!";
+
+  private IMetaStoreClient client;
+  private Configuration conf;
+
+  @Before
+  public void setUp() throws MetaException {
+    conf = MetastoreConf.newMetastoreConf();
+    MetaStoreTestUtils.setConfForStandloneMode(conf);
+    MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.AGGREGATE_STATS_CACHE_ENABLED, false);
+    // Get new client
+    client = new HiveMetaStoreClient(conf);
+  }
+
+  @After
+  public void tearDown() throws TException {
+    // Drop any left over catalogs
+    List<String> catalogs = client.getCatalogs();
+    for (String catName : catalogs) {
+      if (!catName.equalsIgnoreCase(DEFAULT_CATALOG_NAME)) {
+        // First drop any databases in catalog
+        List<String> databases = client.getAllDatabases(catName);
+        for (String db : databases) {
+          client.dropDatabase(catName, db, true, false, true);
+        }
+        client.dropCatalog(catName);
+      } else {
+        List<String> databases = client.getAllDatabases(catName);
+        for (String db : databases) {
+          if (!db.equalsIgnoreCase(Warehouse.DEFAULT_DATABASE_NAME)) {
+            client.dropDatabase(catName, db, true, false, true);
+          }
+        }
+      }
+    }
+    try {
+      if (client != null) {
+        client.close();
+      }
+    } finally {
+      client = null;
+    }
+  }
+
+  private Map<String, Column> buildAllColumns() {
+    Map<String, Column> colMap = new HashMap<>(6);
+    Column[] cols = { new BinaryColumn(), new BooleanColumn(), new DateColumn(),
+                      new DoubleColumn(),  new LongColumn(), new StringColumn() };
+    for (Column c : cols) colMap.put(c.colName, c);
+    return colMap;
+  }
+
+  private List<String> createMetadata(String catName, String dbName, String tableName,
+                                      String partKey, List<String> partVals,
+                                      Map<String, Column> colMap)
+      throws TException {
+    if (!DEFAULT_CATALOG_NAME.equals(catName) && !NO_CAT.equals(catName)) {
+      Catalog cat = new CatalogBuilder()
+          .setName(catName)
+          .setLocation(MetaStoreTestUtils.getTestWarehouseDir(catName))
+          .build();
+      client.createCatalog(cat);
+    }
+
+    Database db;
+    if (!DEFAULT_DATABASE_NAME.equals(dbName)) {
+      DatabaseBuilder dbBuilder = new DatabaseBuilder()
+          .setName(dbName);
+      if (!NO_CAT.equals(catName)) dbBuilder.setCatalogName(catName);
+      db = dbBuilder.create(client, conf);
+    } else {
+      db = client.getDatabase(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME);
+    }
+
+    TableBuilder tb = new TableBuilder()
+        .inDb(db)
+        .setTableName(tableName);
+
+    for (Column col : colMap.values()) tb.addCol(col.colName, col.colType);
+
+    if (partKey != null) {
+      assert partVals != null && !partVals.isEmpty() :
+          "Must provide partition values for partitioned table";
+      tb.addPartCol(partKey, ColumnType.STRING_TYPE_NAME);
+    }
+    Table table = tb.create(client, conf);
+
+    if (partKey != null) {
+      for (String partVal : partVals) {
+        new PartitionBuilder()
+            .inTable(table)
+            .addValue(partVal)
+            .addToTable(client, conf);
+      }
+    }
+
+    SetPartitionsStatsRequest rqst = new SetPartitionsStatsRequest();
+    List<String> partNames = new ArrayList<>();
+    if (partKey == null) {
+      rqst.addToColStats(buildStatsForOneTableOrPartition(catName, dbName, tableName, null,
+          colMap.values()));
+    } else {
+      for (String partVal : partVals) {
+        String partName = partKey + "=" + partVal;
+        rqst.addToColStats(buildStatsForOneTableOrPartition(catName, dbName, tableName, partName,
+            colMap.values()));
+        partNames.add(partName);
+      }
+    }
+    client.setPartitionColumnStatistics(rqst);
+    return partNames;
+  }
+
+  private ColumnStatistics buildStatsForOneTableOrPartition(String catName, String dbName,
+                                                            String tableName, String partName,
+                                                            Collection<Column> cols) {
+    ColumnStatisticsDesc desc = new ColumnStatisticsDesc(partName == null, dbName, tableName);
+    if (!NO_CAT.equals(catName)) desc.setCatName(catName);
+    if (partName != null) desc.setPartName(partName);
+
+    List<ColumnStatisticsObj> objs = new ArrayList<>(cols.size());
+
+    for (Column col : cols) objs.add(col.generate());
+
+    return new ColumnStatistics(desc, objs);
+  }
+
+  private void dropStats(String catName, String dbName, String tableName, String partName,
+                         Collection<String> colNames)
+      throws TException {
+    for (String colName : colNames) {
+      if (partName == null) {
+        if (NO_CAT.equals(catName)) client.deleteTableColumnStatistics(dbName, tableName, colName);
+        else client.deleteTableColumnStatistics(catName, dbName, tableName, colName);
+      } else {
+        if (NO_CAT.equals(catName)) client.deletePartitionColumnStatistics(dbName, tableName, partName, colName);
+        else client.deletePartitionColumnStatistics(catName, dbName, tableName, partName, colName);
+      }
+    }
+  }
+
+  private void compareStatsForTable(String catName, String dbName, String tableName,
+                                    Map<String, Column> colMap) throws TException {
+    List<ColumnStatisticsObj> objs = catName.equals(NO_CAT) ?
+        client.getTableColumnStatistics(dbName, tableName, new ArrayList<>(colMap.keySet())) :
+        client.getTableColumnStatistics(catName, dbName, tableName, new ArrayList<>(colMap.keySet()));
+    compareStatsForOneTableOrPartition(objs, 0, colMap);
+  }
+
+  private void compareStatsForPartitions(String catName, String dbName, String tableName,
+                                         List<String> partNames, final Map<String, Column> colMap)
+      throws TException {
+    Map<String, List<ColumnStatisticsObj>> partObjs = catName.equals(NO_CAT) ?
+        client.getPartitionColumnStatistics(dbName, tableName, partNames, new ArrayList<>(colMap.keySet())) :
+        client.getPartitionColumnStatistics(catName, dbName, tableName, partNames, new ArrayList<>(colMap.keySet()));
+    for (int i = 0; i < partNames.size(); i++) {
+      compareStatsForOneTableOrPartition(partObjs.get(partNames.get(i)), i, colMap);
+    }
+    AggrStats aggr = catName.equals(NO_CAT) ?
+        client.getAggrColStatsFor(dbName, tableName, new ArrayList<>(colMap.keySet()), partNames) :
+        client.getAggrColStatsFor(catName, dbName, tableName, new ArrayList<>(colMap.keySet()), partNames);
+    Assert.assertEquals(partNames.size(), aggr.getPartsFound());
+    Assert.assertEquals(colMap.size(), aggr.getColStatsSize());
+    aggr.getColStats().forEach(cso -> colMap.get(cso.getColName()).compareAggr(cso));
+  }
+
+  private void compareStatsForOneTableOrPartition(List<ColumnStatisticsObj> objs,
+                                                  final int partOffset,
+                                                  final Map<String, Column> colMap)
+      throws TException {
+    Assert.assertEquals(objs.size(), colMap.size());
+    objs.forEach(cso -> colMap.get(cso.getColName()).compare(cso, partOffset));
+  }
+
+  @Test
+  public void tableInHiveCatalog() throws TException {
+    String dbName = "db_table_stats";
+    String tableName = "table_in_default_db_stats";
+    Map<String, Column> colMap = buildAllColumns();
+    createMetadata(DEFAULT_CATALOG_NAME, dbName, tableName, null, null, colMap);
+    compareStatsForTable(DEFAULT_CATALOG_NAME, dbName, tableName, colMap);
+    dropStats(DEFAULT_CATALOG_NAME, dbName, tableName, null, colMap.keySet());
+  }
+
+  @Test
+  public void partitionedTableInHiveCatalog() throws TException {
+    String dbName = "db_part_stats";
+    String tableName = "partitioned_table_in_default_db_stats";
+    Map<String, Column> colMap = buildAllColumns();
+    List<String> partNames = createMetadata(DEFAULT_CATALOG_NAME, dbName, tableName, "pk",
+        Arrays.asList("a1", "a2", "a3"), colMap);
+    compareStatsForPartitions(DEFAULT_CATALOG_NAME, dbName, tableName, partNames, colMap);
+    for (String partName : partNames) {
+      dropStats(DEFAULT_CATALOG_NAME, dbName, tableName, partName, colMap.keySet());
+    }
+  }
+
+  @Test
+  public void tableOtherCatalog() throws TException {
+    String catName = "cat_table_stats";
+    String dbName = "other_cat_db_table_stats";
+    String tableName = "table_in_default_db_stats";
+    Map<String, Column> colMap = buildAllColumns();
+    createMetadata(catName, dbName, tableName, null, null, colMap);
+    compareStatsForTable(catName, dbName, tableName, colMap);
+    dropStats(catName, dbName, tableName, null, colMap.keySet());
+  }
+
+  @Test
+  public void partitionedTableOtherCatalog() throws TException {
+    String catName = "cat_table_stats";
+    String dbName = "other_cat_db_part_stats";
+    String tableName = "partitioned_table_in_default_db_stats";
+    Map<String, Column> colMap = buildAllColumns();
+    List<String> partNames = createMetadata(catName, dbName, tableName, "pk",
+        Arrays.asList("a1", "a2", "a3"), colMap);
+    compareStatsForPartitions(catName, dbName, tableName, partNames, colMap);
+    for (String partName : partNames) {
+      dropStats(catName, dbName, tableName, partName, colMap.keySet());
+    }
+  }
+
+  @Test
+  public void tableDeprecatedCalls() throws TException {
+    String dbName = "old_db_table_stats";
+    String tableName = "table_in_default_db_stats";
+    Map<String, Column> colMap = buildAllColumns();
+    createMetadata(NO_CAT, dbName, tableName, null, null, colMap);
+    compareStatsForTable(NO_CAT, dbName, tableName, colMap);
+    dropStats(NO_CAT, dbName, tableName, null, colMap.keySet());
+  }
+
+  @Test
+  public void partitionedTableDeprecatedCalls() throws TException {
+    String dbName = "old_db_part_stats";
+    String tableName = "partitioned_table_in_default_db_stats";
+    Map<String, Column> colMap = buildAllColumns();
+    List<String> partNames = createMetadata(NO_CAT, dbName, tableName, "pk",
+        Arrays.asList("a1", "a2", "a3"), colMap);
+    compareStatsForPartitions(NO_CAT, dbName, tableName, partNames, colMap);
+    for (String partName : partNames) {
+      dropStats(NO_CAT, dbName, tableName, partName, colMap.keySet());
+    }
+  }
+
+  private abstract class Column {
+    final String colName;
+    final String colType;
+
+    Random rand = new Random();
+
+    List<Long> maxLens, numNulls, numDvs;
+    List<Double> avgLens;
+
+
+    public Column(String colName, String colType) {
+      this.colName = colName;
+      this.colType = colType;
+      maxLens = new ArrayList<>();
+      numNulls = new ArrayList<>();
+      avgLens = new ArrayList<>();
+      numDvs = new ArrayList<>();
+    }
+
+    abstract ColumnStatisticsObj generate();
+    abstract void compare(ColumnStatisticsObj obj, int offset);
+    abstract void compareAggr(ColumnStatisticsObj obj);
+
+    void compareCommon(ColumnStatisticsObj obj) {
+      Assert.assertEquals(colName, obj.getColName());
+      Assert.assertEquals(colType, obj.getColType());
+    }
+
+    long genMaxLen() {
+      return genPositiveLong(maxLens);
+    }
+
+    long getMaxLen() {
+      return maxLong(maxLens);
+    }
+
+    long genNumNulls() {
+      return genPositiveLong(numNulls);
+    }
+
+    long genNumDvs() {
+      return genPositiveLong(numDvs);
+    }
+
+    long getNumNulls() {
+      return sumLong(numNulls);
+    }
+
+    long getNumDvs() {
+      return maxLong(numDvs);
+    }
+
+    double genAvgLens() {
+      return genDouble(avgLens);
+    }
+
+    double getAvgLen() {
+      return maxDouble(avgLens);
+    }
+
+    protected long genNegativeLong(List<Long> addTo) {
+      long val = rand.nextInt(100);
+      if (val > 0) val *= -1;
+      addTo.add(val);
+      return val;
+    }
+
+    protected long genPositiveLong(List<Long> addTo) {
+      long val = rand.nextInt(100);
+      val = Math.abs(val) + 1; // make sure it isn't 0
+      addTo.add(val);
+      return val;
+    }
+
+    protected long maxLong(List<Long> maxOf) {
+      long max = Long.MIN_VALUE;
+      for (long maybe : maxOf) max = Math.max(max, maybe);
+      return max;
+    }
+
+    protected long sumLong(List<Long> sumOf) {
+      long sum = 0;
+      for (long element : sumOf) sum += element;
+      return sum;
+    }
+
+    protected double genDouble(List<Double> addTo) {
+      double val = rand.nextDouble() * rand.nextInt(100);
+      addTo.add(val);
+      return val;
+    }
+
+    protected double maxDouble(List<Double> maxOf) {
+      double max = Double.MIN_VALUE;
+      for (double maybe : maxOf) max = Math.max(max, maybe);
+      return max;
+    }
+
+  }
+
+  private class BinaryColumn extends Column {
+    public BinaryColumn() {
+      super("bincol", ColumnType.BINARY_TYPE_NAME);
+    }
+
+    @Override
+    ColumnStatisticsObj generate() {
+      BinaryColumnStatsData binData = new BinaryColumnStatsData(genMaxLen(), genAvgLens(), genNumNulls());
+      ColumnStatisticsData data = new ColumnStatisticsData();
+      data.setBinaryStats(binData);
+      return new ColumnStatisticsObj(colName, colType, data);
+    }
+
+    @Override
+    void compare(ColumnStatisticsObj obj, int offset) {
+      compareCommon(obj);
+      Assert.assertEquals("binary max length", maxLens.get(offset),
+          (Long)obj.getStatsData().getBinaryStats().getMaxColLen());
+      Assert.assertEquals("binary min length", avgLens.get(offset), obj.getStatsData().getBinaryStats().getAvgColLen(), 0.01);
+      Assert.assertEquals("binary num nulls", numNulls.get(offset), (Long)obj.getStatsData().getBinaryStats().getNumNulls());
+    }
+
+    @Override
+    void compareAggr(ColumnStatisticsObj obj) {
+      compareCommon(obj);
+      Assert.assertEquals("aggr binary max length", getMaxLen(), obj.getStatsData().getBinaryStats().getMaxColLen());
+      Assert.assertEquals("aggr binary min length", getAvgLen(), obj.getStatsData().getBinaryStats().getAvgColLen(), 0.01);
+      Assert.assertEquals("aggr binary num nulls", getNumNulls(), obj.getStatsData().getBinaryStats().getNumNulls());
+    }
+  }
+
+  private class BooleanColumn extends Column {
+    private List<Long> numTrues, numFalses;
+
+    public BooleanColumn() {
+      super("boolcol", ColumnType.BOOLEAN_TYPE_NAME);
+      numTrues = new ArrayList<>();
+      numFalses = new ArrayList<>();
+    }
+
+    @Override
+    ColumnStatisticsObj generate() {
+      BooleanColumnStatsData
+          boolData = new BooleanColumnStatsData(genNumTrues(), genNumFalses(), genNumNulls());
+      ColumnStatisticsData data = new ColumnStatisticsData();
+      data.setBooleanStats(boolData);
+      return new ColumnStatisticsObj(colName, colType, data);
+    }
+
+    @Override
+    void compare(ColumnStatisticsObj obj, int offset) {
+      compareCommon(obj);
+      Assert.assertEquals("boolean num trues", numTrues.get(offset), (Long)obj.getStatsData().getBooleanStats().getNumTrues());
+      Assert.assertEquals("boolean num falses", numFalses.get(offset), (Long)obj.getStatsData().getBooleanStats().getNumFalses());
+      Assert.assertEquals("boolean num nulls", numNulls.get(offset), (Long)obj.getStatsData().getBooleanStats().getNumNulls());
+    }
+
+    @Override
+    void compareAggr(ColumnStatisticsObj obj) {
+      compareCommon(obj);
+      Assert.assertEquals("aggr boolean num trues", getNumTrues(), obj.getStatsData().getBooleanStats().getNumTrues());
+      Assert.assertEquals("aggr boolean num falses", getNumFalses(), obj.getStatsData().getBooleanStats().getNumFalses());
+      Assert.assertEquals("aggr boolean num nulls", getNumNulls(), obj.getStatsData().getBooleanStats().getNumNulls());
+    }
+
+    private long genNumTrues() {
+      return genPositiveLong(numTrues);
+    }
+
+    private long genNumFalses() {
+      return genPositiveLong(numFalses);
+    }
+
+    private long getNumTrues() {
+      return sumLong(numTrues);
+    }
+
+    private long getNumFalses() {
+      return sumLong(numFalses);
+    }
+  }
+
+  private class DateColumn extends Column {
+    private List<Date> lowVals, highVals;
+
+    public DateColumn() {
+      super("datecol", ColumnType.DATE_TYPE_NAME);
+      lowVals = new ArrayList<>();
+      highVals = new ArrayList<>();
+    }
+
+    @Override
+    ColumnStatisticsObj generate() {
+      DateColumnStatsData dateData = new DateColumnStatsData(genNumNulls(), genNumDvs());
+      dateData.setLowValue(genLowValue());
+      dateData.setHighValue(genHighValue());
+      ColumnStatisticsData data = new ColumnStatisticsData();
+      data.setDateStats(dateData);
+      return new ColumnStatisticsObj(colName, colType, data);
+    }
+
+    @Override
+    void compare(ColumnStatisticsObj obj, int offset) {
+      compareCommon(obj);
+      Assert.assertEquals("date num nulls", numNulls.get(offset), (Long)obj.getStatsData().getDateStats().getNumNulls());
+      Assert.assertEquals("date num dvs", numDvs.get(offset), (Long)obj.getStatsData().getDateStats().getNumDVs());
+      Assert.assertEquals("date low val", lowVals.get(offset), obj.getStatsData().getDateStats().getLowValue());
+      Assert.assertEquals("date high val", highVals.get(offset), obj.getStatsData().getDateStats().getHighValue());
+    }
+
+    @Override
+    void compareAggr(ColumnStatisticsObj obj) {
+      compareCommon(obj);
+      Assert.assertEquals("aggr date num nulls", getNumNulls(), obj.getStatsData().getDateStats().getNumNulls());
+      Assert.assertEquals("aggr date num dvs", getNumDvs(), obj.getStatsData().getDateStats().getNumDVs());
+      Assert.assertEquals("aggr date low val", getLowVal(), obj.getStatsData().getDateStats().getLowValue());
+      Assert.assertEquals("aggr date high val", getHighVal(), obj.getStatsData().getDateStats().getHighValue());
+    }
+
+    private Date genLowValue() {
+      Date d = new Date(rand.nextInt(100) * -1);
+      lowVals.add(d);
+      return d;
+    }
+
+    private Date genHighValue() {
+      Date d = new Date(rand.nextInt(200));
+      highVals.add(d);
+      return d;
+    }
+
+    private Date getLowVal() {
+      long min = Long.MAX_VALUE;
+      for (Date d : lowVals) min = Math.min(min, d.getDaysSinceEpoch());
+      return new Date(min);
+    }
+
+    private Date getHighVal() {
+      long max = Long.MIN_VALUE;
+      for (Date d : highVals) max = Math.max(max, d.getDaysSinceEpoch());
+      return new Date(max);
+    }
+  }
+
+  private class DoubleColumn extends Column {
+    List<Double> lowVals, highVals;
+
+    public DoubleColumn() {
+      super("doublecol", ColumnType.DOUBLE_TYPE_NAME);
+      lowVals = new ArrayList<>();
+      highVals = new ArrayList<>();
+    }
+
+    @Override
+    ColumnStatisticsObj generate() {
+      DoubleColumnStatsData doubleData = new DoubleColumnStatsData(genNumNulls(), genNumDvs());
+      doubleData.setLowValue(genLowVal());
+      doubleData.setHighValue(genHighVal());
+      ColumnStatisticsData data = new ColumnStatisticsData();
+      data.setDoubleStats(doubleData);
+      return new ColumnStatisticsObj(colName, colType, data);
+    }
+
+    @Override
+    void compare(ColumnStatisticsObj obj, int offset) {
+      compareCommon(obj);
+      Assert.assertEquals("double num nulls", numNulls.get(offset),
+          (Long)obj.getStatsData().getDoubleStats().getNumNulls());
+      Assert.assertEquals("double num dvs", numDvs.get(offset),
+          (Long)obj.getStatsData().getDoubleStats().getNumDVs());
+      Assert.assertEquals("double low val", lowVals.get(offset),
+          obj.getStatsData().getDoubleStats().getLowValue(), 0.01);
+      Assert.assertEquals("double high val", highVals.get(offset),
+          obj.getStatsData().getDoubleStats().getHighValue(), 0.01);
+    }
+
+    @Override
+    void compareAggr(ColumnStatisticsObj obj) {
+      compareCommon(obj);
+      Assert.assertEquals("aggr double num nulls", getNumNulls(),
+          obj.getStatsData().getDoubleStats().getNumNulls());
+      Assert.assertEquals("aggr double num dvs", getNumDvs(),
+          obj.getStatsData().getDoubleStats().getNumDVs());
+      Assert.assertEquals("aggr double low val", getLowVal(),
+          obj.getStatsData().getDoubleStats().getLowValue(), 0.01);
+      Assert.assertEquals("aggr double high val", getHighVal(),
+          obj.getStatsData().getDoubleStats().getHighValue(), 0.01);
+
+    }
+
+    private double genLowVal() {
+      return genDouble(lowVals);
+    }
+
+    private double genHighVal() {
+      return genDouble(highVals);
+    }
+
+    private double getLowVal() {
+      double min = Double.MAX_VALUE;
+      for (Double d : lowVals) min = Math.min(min, d);
+      return min;
+    }
+
+    private double getHighVal() {
+      return maxDouble(highVals);
+    }
+  }
+
+  private class LongColumn extends Column {
+    List<Long> lowVals, highVals;
+
+    public LongColumn() {
+      super("bigintcol", ColumnType.BIGINT_TYPE_NAME);
+      lowVals = new ArrayList<>();
+      highVals = new ArrayList<>();
+    }
+
+    @Override
+    ColumnStatisticsObj generate() {
+      LongColumnStatsData longData = new LongColumnStatsData(genNumNulls(), genNumDvs());
+      longData.setLowValue(genLowVal());
+      longData.setHighValue(genHighVal());
+      ColumnStatisticsData data = new ColumnStatisticsData();
+      data.setLongStats(longData);
+      return new ColumnStatisticsObj(colName, colType, data);
+    }
+
+    @Override
+    void compare(ColumnStatisticsObj obj, int offset) {
+      compareCommon(obj);
+      Assert.assertEquals("long num nulls", numNulls.get(offset),
+          (Long)obj.getStatsData().getLongStats().getNumNulls());
+      Assert.assertEquals("long num dvs", numDvs.get(offset),
+          (Long)obj.getStatsData().getLongStats().getNumDVs());
+      Assert.assertEquals("long low val", (long)lowVals.get(offset),
+          obj.getStatsData().getLongStats().getLowValue());
+      Assert.assertEquals("long high val", (long)highVals.get(offset),
+          obj.getStatsData().getLongStats().getHighValue());
+    }
+
+    @Override
+    void compareAggr(ColumnStatisticsObj obj) {
+      compareCommon(obj);
+      Assert.assertEquals("aggr long num nulls", getNumNulls(),
+          obj.getStatsData().getLongStats().getNumNulls());
+      Assert.assertEquals("aggr long num dvs", getNumDvs(),
+          obj.getStatsData().getLongStats().getNumDVs());
+      Assert.assertEquals("aggr long low val", getLowVal(),
+          obj.getStatsData().getLongStats().getLowValue());
+      Assert.assertEquals("aggr long high val", getHighVal(),
+          obj.getStatsData().getLongStats().getHighValue());
+    }
+
+    private long genLowVal() {
+      return genNegativeLong(lowVals);
+    }
+
+    private long genHighVal() {
+      return genPositiveLong(highVals);
+    }
+
+    private long getLowVal() {
+      long min = Long.MAX_VALUE;
+      for (Long val : lowVals) min = Math.min(min, val);
+      return min;
+    }
+
+    private long getHighVal() {
+      return maxLong(highVals);
+    }
+  }
+
+  private class StringColumn extends Column {
+    public StringColumn() {
+      super("strcol", ColumnType.STRING_TYPE_NAME);
+    }
+
+    @Override
+    ColumnStatisticsObj generate() {
+      StringColumnStatsData strData = new StringColumnStatsData(genMaxLen(), genAvgLens(),
+          genNumNulls(), genNumDvs());
+      ColumnStatisticsData data = new ColumnStatisticsData();
+      data.setStringStats(strData);
+      return new ColumnStatisticsObj(colName, colType, data);
+    }
+
+    @Override
+    void compare(ColumnStatisticsObj obj, int offset) {
+      compareCommon(obj);
+      Assert.assertEquals("str num nulls", numNulls.get(offset),
+          (Long)obj.getStatsData().getStringStats().getNumNulls());
+      Assert.assertEquals("str num dvs", numDvs.get(offset),
+          (Long)obj.getStatsData().getStringStats().getNumDVs());
+      Assert.assertEquals("str low val", (long)maxLens.get(offset),
+          obj.getStatsData().getStringStats().getMaxColLen());
+      Assert.assertEquals("str high val", avgLens.get(offset),
+          obj.getStatsData().getStringStats().getAvgColLen(), 0.01);
+    }
+
+    @Override
+    void compareAggr(ColumnStatisticsObj obj) {
+      compareCommon(obj);
+      Assert.assertEquals("aggr str num nulls", getNumNulls(),
+          obj.getStatsData().getStringStats().getNumNulls());
+      Assert.assertEquals("aggr str num dvs", getNumDvs(),
+          obj.getStatsData().getStringStats().getNumDVs());
+      Assert.assertEquals("aggr str low val", getMaxLen(),
+          obj.getStatsData().getStringStats().getMaxColLen());
+      Assert.assertEquals("aggr str high val", getAvgLen(),
+          obj.getStatsData().getStringStats().getAvgColLen(), 0.01);
+
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/VerifyingObjectStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/VerifyingObjectStore.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/VerifyingObjectStore.java
index 150b6ca..c9a6a47 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/VerifyingObjectStore.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/VerifyingObjectStore.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hive.metastore;
 
 import static org.apache.commons.lang.StringUtils.repeat;
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
 
 import java.lang.reflect.AccessibleObject;
 import java.lang.reflect.Array;
@@ -50,35 +51,36 @@ public class VerifyingObjectStore extends ObjectStore {
   }
 
   @Override
-  public List<Partition> getPartitionsByFilter(String dbName, String tblName, String filter,
-      short maxParts) throws MetaException, NoSuchObjectException {
+  public List<Partition> getPartitionsByFilter(String catName, String dbName, String tblName,
+                                               String filter, short maxParts)
+      throws MetaException, NoSuchObjectException {
     List<Partition> sqlResults = getPartitionsByFilterInternal(
-        dbName, tblName, filter, maxParts, true, false);
+        catName, dbName, tblName, filter, maxParts, true, false);
     List<Partition> ormResults = getPartitionsByFilterInternal(
-        dbName, tblName, filter, maxParts, false, true);
+        catName, dbName, tblName, filter, maxParts, false, true);
     verifyLists(sqlResults, ormResults, Partition.class);
     return sqlResults;
   }
 
   @Override
-  public List<Partition> getPartitionsByNames(String dbName, String tblName,
+  public List<Partition> getPartitionsByNames(String catName, String dbName, String tblName,
       List<String> partNames) throws MetaException, NoSuchObjectException {
     List<Partition> sqlResults = getPartitionsByNamesInternal(
-        dbName, tblName, partNames, true, false);
+        catName, dbName, tblName, partNames, true, false);
     List<Partition> ormResults = getPartitionsByNamesInternal(
-        dbName, tblName, partNames, false, true);
+        catName, dbName, tblName, partNames, false, true);
     verifyLists(sqlResults, ormResults, Partition.class);
     return sqlResults;
   }
 
   @Override
-  public boolean getPartitionsByExpr(String dbName, String tblName, byte[] expr,
+  public boolean getPartitionsByExpr(String catName, String dbName, String tblName, byte[] expr,
       String defaultPartitionName, short maxParts, List<Partition> result) throws TException {
     List<Partition> ormParts = new LinkedList<>();
     boolean sqlResult = getPartitionsByExprInternal(
-        dbName, tblName, expr, defaultPartitionName, maxParts, result, true, false);
+        catName, dbName, tblName, expr, defaultPartitionName, maxParts, result, true, false);
     boolean ormResult = getPartitionsByExprInternal(
-        dbName, tblName, expr, defaultPartitionName, maxParts, ormParts, false, true);
+        catName, dbName, tblName, expr, defaultPartitionName, maxParts, ormParts, false, true);
     if (sqlResult != ormResult) {
       String msg = "The unknown flag is different - SQL " + sqlResult + ", ORM " + ormResult;
       LOG.error(msg);
@@ -90,32 +92,32 @@ public class VerifyingObjectStore extends ObjectStore {
 
   @Override
   public List<Partition> getPartitions(
-      String dbName, String tableName, int maxParts) throws MetaException, NoSuchObjectException {
-    List<Partition> sqlResults = getPartitionsInternal(dbName, tableName, maxParts, true, false);
-    List<Partition> ormResults = getPartitionsInternal(dbName, tableName, maxParts, false, true);
+      String catName, String dbName, String tableName, int maxParts) throws MetaException, NoSuchObjectException {
+    List<Partition> sqlResults = getPartitionsInternal(catName, dbName, tableName, maxParts, true, false);
+    List<Partition> ormResults = getPartitionsInternal(catName, dbName, tableName, maxParts, false, true);
     verifyLists(sqlResults, ormResults, Partition.class);
     return sqlResults;
   }
 
   @Override
-  public ColumnStatistics getTableColumnStatistics(String dbName,
+  public ColumnStatistics getTableColumnStatistics(String catName, String dbName,
       String tableName, List<String> colNames) throws MetaException, NoSuchObjectException {
     ColumnStatistics sqlResult = getTableColumnStatisticsInternal(
-        dbName, tableName, colNames, true, false);
+        catName, dbName, tableName, colNames, true, false);
     ColumnStatistics jdoResult = getTableColumnStatisticsInternal(
-        dbName, tableName, colNames, false, true);
+        catName, dbName, tableName, colNames, false, true);
     verifyObjects(sqlResult, jdoResult, ColumnStatistics.class);
     return sqlResult;
   }
 
   @Override
-  public List<ColumnStatistics> getPartitionColumnStatistics(String dbName,
+  public List<ColumnStatistics> getPartitionColumnStatistics(String catName, String dbName,
       String tableName, List<String> partNames, List<String> colNames)
       throws MetaException, NoSuchObjectException {
     List<ColumnStatistics> sqlResult = getPartitionColumnStatisticsInternal(
-        dbName, tableName, partNames, colNames, true, false);
+        catName, dbName, tableName, partNames, colNames, true, false);
     List<ColumnStatistics> jdoResult = getPartitionColumnStatisticsInternal(
-        dbName, tableName, partNames, colNames,  false, true);
+        catName, dbName, tableName, partNames, colNames,  false, true);
     verifyLists(sqlResult, jdoResult, ColumnStatistics.class);
     return sqlResult;
   }


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
index e10a655..b9e8e24 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
@@ -24,6 +24,10 @@ class ThriftHiveMetastoreIf : virtual public  ::facebook::fb303::FacebookService
   virtual ~ThriftHiveMetastoreIf() {}
   virtual void getMetaConf(std::string& _return, const std::string& key) = 0;
   virtual void setMetaConf(const std::string& key, const std::string& value) = 0;
+  virtual void create_catalog(const CreateCatalogRequest& catalog) = 0;
+  virtual void get_catalog(GetCatalogResponse& _return, const GetCatalogRequest& catName) = 0;
+  virtual void get_catalogs(GetCatalogsResponse& _return) = 0;
+  virtual void drop_catalog(const DropCatalogRequest& catName) = 0;
   virtual void create_database(const Database& database) = 0;
   virtual void get_database(Database& _return, const std::string& name) = 0;
   virtual void drop_database(const std::string& name, const bool deleteData, const bool cascade) = 0;
@@ -61,7 +65,7 @@ class ThriftHiveMetastoreIf : virtual public  ::facebook::fb303::FacebookService
   virtual void get_table_req(GetTableResult& _return, const GetTableRequest& req) = 0;
   virtual void get_table_objects_by_name_req(GetTablesResult& _return, const GetTablesRequest& req) = 0;
   virtual void get_materialization_invalidation_info(std::map<std::string, Materialization> & _return, const std::string& dbname, const std::vector<std::string> & tbl_names) = 0;
-  virtual void update_creation_metadata(const std::string& dbname, const std::string& tbl_name, const CreationMetadata& creation_metadata) = 0;
+  virtual void update_creation_metadata(const std::string& catName, const std::string& dbname, const std::string& tbl_name, const CreationMetadata& creation_metadata) = 0;
   virtual void get_table_names_by_filter(std::vector<std::string> & _return, const std::string& dbname, const std::string& filter, const int16_t max_tables) = 0;
   virtual void alter_table(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl) = 0;
   virtual void alter_table_with_environment_context(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const EnvironmentContext& environment_context) = 0;
@@ -253,6 +257,18 @@ class ThriftHiveMetastoreNull : virtual public ThriftHiveMetastoreIf , virtual p
   void setMetaConf(const std::string& /* key */, const std::string& /* value */) {
     return;
   }
+  void create_catalog(const CreateCatalogRequest& /* catalog */) {
+    return;
+  }
+  void get_catalog(GetCatalogResponse& /* _return */, const GetCatalogRequest& /* catName */) {
+    return;
+  }
+  void get_catalogs(GetCatalogsResponse& /* _return */) {
+    return;
+  }
+  void drop_catalog(const DropCatalogRequest& /* catName */) {
+    return;
+  }
   void create_database(const Database& /* database */) {
     return;
   }
@@ -366,7 +382,7 @@ class ThriftHiveMetastoreNull : virtual public ThriftHiveMetastoreIf , virtual p
   void get_materialization_invalidation_info(std::map<std::string, Materialization> & /* _return */, const std::string& /* dbname */, const std::vector<std::string> & /* tbl_names */) {
     return;
   }
-  void update_creation_metadata(const std::string& /* dbname */, const std::string& /* tbl_name */, const CreationMetadata& /* creation_metadata */) {
+  void update_creation_metadata(const std::string& /* catName */, const std::string& /* dbname */, const std::string& /* tbl_name */, const CreationMetadata& /* creation_metadata */) {
     return;
   }
   void get_table_names_by_filter(std::vector<std::string> & /* _return */, const std::string& /* dbname */, const std::string& /* filter */, const int16_t /* max_tables */) {
@@ -1087,6 +1103,466 @@ class ThriftHiveMetastore_setMetaConf_presult {
 
 };
 
+typedef struct _ThriftHiveMetastore_create_catalog_args__isset {
+  _ThriftHiveMetastore_create_catalog_args__isset() : catalog(false) {}
+  bool catalog :1;
+} _ThriftHiveMetastore_create_catalog_args__isset;
+
+class ThriftHiveMetastore_create_catalog_args {
+ public:
+
+  ThriftHiveMetastore_create_catalog_args(const ThriftHiveMetastore_create_catalog_args&);
+  ThriftHiveMetastore_create_catalog_args& operator=(const ThriftHiveMetastore_create_catalog_args&);
+  ThriftHiveMetastore_create_catalog_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_create_catalog_args() throw();
+  CreateCatalogRequest catalog;
+
+  _ThriftHiveMetastore_create_catalog_args__isset __isset;
+
+  void __set_catalog(const CreateCatalogRequest& val);
+
+  bool operator == (const ThriftHiveMetastore_create_catalog_args & rhs) const
+  {
+    if (!(catalog == rhs.catalog))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_create_catalog_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_create_catalog_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_create_catalog_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_create_catalog_pargs() throw();
+  const CreateCatalogRequest* catalog;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_create_catalog_result__isset {
+  _ThriftHiveMetastore_create_catalog_result__isset() : o1(false), o2(false), o3(false) {}
+  bool o1 :1;
+  bool o2 :1;
+  bool o3 :1;
+} _ThriftHiveMetastore_create_catalog_result__isset;
+
+class ThriftHiveMetastore_create_catalog_result {
+ public:
+
+  ThriftHiveMetastore_create_catalog_result(const ThriftHiveMetastore_create_catalog_result&);
+  ThriftHiveMetastore_create_catalog_result& operator=(const ThriftHiveMetastore_create_catalog_result&);
+  ThriftHiveMetastore_create_catalog_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_create_catalog_result() throw();
+  AlreadyExistsException o1;
+  InvalidObjectException o2;
+  MetaException o3;
+
+  _ThriftHiveMetastore_create_catalog_result__isset __isset;
+
+  void __set_o1(const AlreadyExistsException& val);
+
+  void __set_o2(const InvalidObjectException& val);
+
+  void __set_o3(const MetaException& val);
+
+  bool operator == (const ThriftHiveMetastore_create_catalog_result & rhs) const
+  {
+    if (!(o1 == rhs.o1))
+      return false;
+    if (!(o2 == rhs.o2))
+      return false;
+    if (!(o3 == rhs.o3))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_create_catalog_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_create_catalog_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_create_catalog_presult__isset {
+  _ThriftHiveMetastore_create_catalog_presult__isset() : o1(false), o2(false), o3(false) {}
+  bool o1 :1;
+  bool o2 :1;
+  bool o3 :1;
+} _ThriftHiveMetastore_create_catalog_presult__isset;
+
+class ThriftHiveMetastore_create_catalog_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_create_catalog_presult() throw();
+  AlreadyExistsException o1;
+  InvalidObjectException o2;
+  MetaException o3;
+
+  _ThriftHiveMetastore_create_catalog_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
+typedef struct _ThriftHiveMetastore_get_catalog_args__isset {
+  _ThriftHiveMetastore_get_catalog_args__isset() : catName(false) {}
+  bool catName :1;
+} _ThriftHiveMetastore_get_catalog_args__isset;
+
+class ThriftHiveMetastore_get_catalog_args {
+ public:
+
+  ThriftHiveMetastore_get_catalog_args(const ThriftHiveMetastore_get_catalog_args&);
+  ThriftHiveMetastore_get_catalog_args& operator=(const ThriftHiveMetastore_get_catalog_args&);
+  ThriftHiveMetastore_get_catalog_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_catalog_args() throw();
+  GetCatalogRequest catName;
+
+  _ThriftHiveMetastore_get_catalog_args__isset __isset;
+
+  void __set_catName(const GetCatalogRequest& val);
+
+  bool operator == (const ThriftHiveMetastore_get_catalog_args & rhs) const
+  {
+    if (!(catName == rhs.catName))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_catalog_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_catalog_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_get_catalog_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_catalog_pargs() throw();
+  const GetCatalogRequest* catName;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_catalog_result__isset {
+  _ThriftHiveMetastore_get_catalog_result__isset() : success(false), o1(false), o2(false) {}
+  bool success :1;
+  bool o1 :1;
+  bool o2 :1;
+} _ThriftHiveMetastore_get_catalog_result__isset;
+
+class ThriftHiveMetastore_get_catalog_result {
+ public:
+
+  ThriftHiveMetastore_get_catalog_result(const ThriftHiveMetastore_get_catalog_result&);
+  ThriftHiveMetastore_get_catalog_result& operator=(const ThriftHiveMetastore_get_catalog_result&);
+  ThriftHiveMetastore_get_catalog_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_catalog_result() throw();
+  GetCatalogResponse success;
+  NoSuchObjectException o1;
+  MetaException o2;
+
+  _ThriftHiveMetastore_get_catalog_result__isset __isset;
+
+  void __set_success(const GetCatalogResponse& val);
+
+  void __set_o1(const NoSuchObjectException& val);
+
+  void __set_o2(const MetaException& val);
+
+  bool operator == (const ThriftHiveMetastore_get_catalog_result & rhs) const
+  {
+    if (!(success == rhs.success))
+      return false;
+    if (!(o1 == rhs.o1))
+      return false;
+    if (!(o2 == rhs.o2))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_catalog_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_catalog_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_catalog_presult__isset {
+  _ThriftHiveMetastore_get_catalog_presult__isset() : success(false), o1(false), o2(false) {}
+  bool success :1;
+  bool o1 :1;
+  bool o2 :1;
+} _ThriftHiveMetastore_get_catalog_presult__isset;
+
+class ThriftHiveMetastore_get_catalog_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_catalog_presult() throw();
+  GetCatalogResponse* success;
+  NoSuchObjectException o1;
+  MetaException o2;
+
+  _ThriftHiveMetastore_get_catalog_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
+
+class ThriftHiveMetastore_get_catalogs_args {
+ public:
+
+  ThriftHiveMetastore_get_catalogs_args(const ThriftHiveMetastore_get_catalogs_args&);
+  ThriftHiveMetastore_get_catalogs_args& operator=(const ThriftHiveMetastore_get_catalogs_args&);
+  ThriftHiveMetastore_get_catalogs_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_catalogs_args() throw();
+
+  bool operator == (const ThriftHiveMetastore_get_catalogs_args & /* rhs */) const
+  {
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_catalogs_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_catalogs_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_get_catalogs_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_catalogs_pargs() throw();
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_catalogs_result__isset {
+  _ThriftHiveMetastore_get_catalogs_result__isset() : success(false), o1(false) {}
+  bool success :1;
+  bool o1 :1;
+} _ThriftHiveMetastore_get_catalogs_result__isset;
+
+class ThriftHiveMetastore_get_catalogs_result {
+ public:
+
+  ThriftHiveMetastore_get_catalogs_result(const ThriftHiveMetastore_get_catalogs_result&);
+  ThriftHiveMetastore_get_catalogs_result& operator=(const ThriftHiveMetastore_get_catalogs_result&);
+  ThriftHiveMetastore_get_catalogs_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_catalogs_result() throw();
+  GetCatalogsResponse success;
+  MetaException o1;
+
+  _ThriftHiveMetastore_get_catalogs_result__isset __isset;
+
+  void __set_success(const GetCatalogsResponse& val);
+
+  void __set_o1(const MetaException& val);
+
+  bool operator == (const ThriftHiveMetastore_get_catalogs_result & rhs) const
+  {
+    if (!(success == rhs.success))
+      return false;
+    if (!(o1 == rhs.o1))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_catalogs_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_catalogs_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_catalogs_presult__isset {
+  _ThriftHiveMetastore_get_catalogs_presult__isset() : success(false), o1(false) {}
+  bool success :1;
+  bool o1 :1;
+} _ThriftHiveMetastore_get_catalogs_presult__isset;
+
+class ThriftHiveMetastore_get_catalogs_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_catalogs_presult() throw();
+  GetCatalogsResponse* success;
+  MetaException o1;
+
+  _ThriftHiveMetastore_get_catalogs_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
+typedef struct _ThriftHiveMetastore_drop_catalog_args__isset {
+  _ThriftHiveMetastore_drop_catalog_args__isset() : catName(false) {}
+  bool catName :1;
+} _ThriftHiveMetastore_drop_catalog_args__isset;
+
+class ThriftHiveMetastore_drop_catalog_args {
+ public:
+
+  ThriftHiveMetastore_drop_catalog_args(const ThriftHiveMetastore_drop_catalog_args&);
+  ThriftHiveMetastore_drop_catalog_args& operator=(const ThriftHiveMetastore_drop_catalog_args&);
+  ThriftHiveMetastore_drop_catalog_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_drop_catalog_args() throw();
+  DropCatalogRequest catName;
+
+  _ThriftHiveMetastore_drop_catalog_args__isset __isset;
+
+  void __set_catName(const DropCatalogRequest& val);
+
+  bool operator == (const ThriftHiveMetastore_drop_catalog_args & rhs) const
+  {
+    if (!(catName == rhs.catName))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_drop_catalog_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_drop_catalog_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_drop_catalog_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_drop_catalog_pargs() throw();
+  const DropCatalogRequest* catName;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_drop_catalog_result__isset {
+  _ThriftHiveMetastore_drop_catalog_result__isset() : o1(false), o2(false), o3(false) {}
+  bool o1 :1;
+  bool o2 :1;
+  bool o3 :1;
+} _ThriftHiveMetastore_drop_catalog_result__isset;
+
+class ThriftHiveMetastore_drop_catalog_result {
+ public:
+
+  ThriftHiveMetastore_drop_catalog_result(const ThriftHiveMetastore_drop_catalog_result&);
+  ThriftHiveMetastore_drop_catalog_result& operator=(const ThriftHiveMetastore_drop_catalog_result&);
+  ThriftHiveMetastore_drop_catalog_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_drop_catalog_result() throw();
+  NoSuchObjectException o1;
+  InvalidOperationException o2;
+  MetaException o3;
+
+  _ThriftHiveMetastore_drop_catalog_result__isset __isset;
+
+  void __set_o1(const NoSuchObjectException& val);
+
+  void __set_o2(const InvalidOperationException& val);
+
+  void __set_o3(const MetaException& val);
+
+  bool operator == (const ThriftHiveMetastore_drop_catalog_result & rhs) const
+  {
+    if (!(o1 == rhs.o1))
+      return false;
+    if (!(o2 == rhs.o2))
+      return false;
+    if (!(o3 == rhs.o3))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_drop_catalog_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_drop_catalog_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_drop_catalog_presult__isset {
+  _ThriftHiveMetastore_drop_catalog_presult__isset() : o1(false), o2(false), o3(false) {}
+  bool o1 :1;
+  bool o2 :1;
+  bool o3 :1;
+} _ThriftHiveMetastore_drop_catalog_presult__isset;
+
+class ThriftHiveMetastore_drop_catalog_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_drop_catalog_presult() throw();
+  NoSuchObjectException o1;
+  InvalidOperationException o2;
+  MetaException o3;
+
+  _ThriftHiveMetastore_drop_catalog_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
 typedef struct _ThriftHiveMetastore_create_database_args__isset {
   _ThriftHiveMetastore_create_database_args__isset() : database(false) {}
   bool database :1;
@@ -5637,7 +6113,8 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_presult {
 };
 
 typedef struct _ThriftHiveMetastore_update_creation_metadata_args__isset {
-  _ThriftHiveMetastore_update_creation_metadata_args__isset() : dbname(false), tbl_name(false), creation_metadata(false) {}
+  _ThriftHiveMetastore_update_creation_metadata_args__isset() : catName(false), dbname(false), tbl_name(false), creation_metadata(false) {}
+  bool catName :1;
   bool dbname :1;
   bool tbl_name :1;
   bool creation_metadata :1;
@@ -5648,16 +6125,19 @@ class ThriftHiveMetastore_update_creation_metadata_args {
 
   ThriftHiveMetastore_update_creation_metadata_args(const ThriftHiveMetastore_update_creation_metadata_args&);
   ThriftHiveMetastore_update_creation_metadata_args& operator=(const ThriftHiveMetastore_update_creation_metadata_args&);
-  ThriftHiveMetastore_update_creation_metadata_args() : dbname(), tbl_name() {
+  ThriftHiveMetastore_update_creation_metadata_args() : catName(), dbname(), tbl_name() {
   }
 
   virtual ~ThriftHiveMetastore_update_creation_metadata_args() throw();
+  std::string catName;
   std::string dbname;
   std::string tbl_name;
   CreationMetadata creation_metadata;
 
   _ThriftHiveMetastore_update_creation_metadata_args__isset __isset;
 
+  void __set_catName(const std::string& val);
+
   void __set_dbname(const std::string& val);
 
   void __set_tbl_name(const std::string& val);
@@ -5666,6 +6146,8 @@ class ThriftHiveMetastore_update_creation_metadata_args {
 
   bool operator == (const ThriftHiveMetastore_update_creation_metadata_args & rhs) const
   {
+    if (!(catName == rhs.catName))
+      return false;
     if (!(dbname == rhs.dbname))
       return false;
     if (!(tbl_name == rhs.tbl_name))
@@ -5691,6 +6173,7 @@ class ThriftHiveMetastore_update_creation_metadata_pargs {
 
 
   virtual ~ThriftHiveMetastore_update_creation_metadata_pargs() throw();
+  const std::string* catName;
   const std::string* dbname;
   const std::string* tbl_name;
   const CreationMetadata* creation_metadata;
@@ -24933,6 +25416,18 @@ class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public
   void setMetaConf(const std::string& key, const std::string& value);
   void send_setMetaConf(const std::string& key, const std::string& value);
   void recv_setMetaConf();
+  void create_catalog(const CreateCatalogRequest& catalog);
+  void send_create_catalog(const CreateCatalogRequest& catalog);
+  void recv_create_catalog();
+  void get_catalog(GetCatalogResponse& _return, const GetCatalogRequest& catName);
+  void send_get_catalog(const GetCatalogRequest& catName);
+  void recv_get_catalog(GetCatalogResponse& _return);
+  void get_catalogs(GetCatalogsResponse& _return);
+  void send_get_catalogs();
+  void recv_get_catalogs(GetCatalogsResponse& _return);
+  void drop_catalog(const DropCatalogRequest& catName);
+  void send_drop_catalog(const DropCatalogRequest& catName);
+  void recv_drop_catalog();
   void create_database(const Database& database);
   void send_create_database(const Database& database);
   void recv_create_database();
@@ -25044,8 +25539,8 @@ class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public
   void get_materialization_invalidation_info(std::map<std::string, Materialization> & _return, const std::string& dbname, const std::vector<std::string> & tbl_names);
   void send_get_materialization_invalidation_info(const std::string& dbname, const std::vector<std::string> & tbl_names);
   void recv_get_materialization_invalidation_info(std::map<std::string, Materialization> & _return);
-  void update_creation_metadata(const std::string& dbname, const std::string& tbl_name, const CreationMetadata& creation_metadata);
-  void send_update_creation_metadata(const std::string& dbname, const std::string& tbl_name, const CreationMetadata& creation_metadata);
+  void update_creation_metadata(const std::string& catName, const std::string& dbname, const std::string& tbl_name, const CreationMetadata& creation_metadata);
+  void send_update_creation_metadata(const std::string& catName, const std::string& dbname, const std::string& tbl_name, const CreationMetadata& creation_metadata);
   void recv_update_creation_metadata();
   void get_table_names_by_filter(std::vector<std::string> & _return, const std::string& dbname, const std::string& filter, const int16_t max_tables);
   void send_get_table_names_by_filter(const std::string& dbname, const std::string& filter, const int16_t max_tables);
@@ -25527,6 +26022,10 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
   ProcessMap processMap_;
   void process_getMetaConf(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_setMetaConf(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
+  void process_create_catalog(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
+  void process_get_catalog(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
+  void process_get_catalogs(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
+  void process_drop_catalog(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_create_database(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_get_database(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_drop_database(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
@@ -25727,6 +26226,10 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
     iface_(iface) {
     processMap_["getMetaConf"] = &ThriftHiveMetastoreProcessor::process_getMetaConf;
     processMap_["setMetaConf"] = &ThriftHiveMetastoreProcessor::process_setMetaConf;
+    processMap_["create_catalog"] = &ThriftHiveMetastoreProcessor::process_create_catalog;
+    processMap_["get_catalog"] = &ThriftHiveMetastoreProcessor::process_get_catalog;
+    processMap_["get_catalogs"] = &ThriftHiveMetastoreProcessor::process_get_catalogs;
+    processMap_["drop_catalog"] = &ThriftHiveMetastoreProcessor::process_drop_catalog;
     processMap_["create_database"] = &ThriftHiveMetastoreProcessor::process_create_database;
     processMap_["get_database"] = &ThriftHiveMetastoreProcessor::process_get_database;
     processMap_["drop_database"] = &ThriftHiveMetastoreProcessor::process_drop_database;
@@ -25973,6 +26476,44 @@ class ThriftHiveMetastoreMultiface : virtual public ThriftHiveMetastoreIf, publi
     ifaces_[i]->setMetaConf(key, value);
   }
 
+  void create_catalog(const CreateCatalogRequest& catalog) {
+    size_t sz = ifaces_.size();
+    size_t i = 0;
+    for (; i < (sz - 1); ++i) {
+      ifaces_[i]->create_catalog(catalog);
+    }
+    ifaces_[i]->create_catalog(catalog);
+  }
+
+  void get_catalog(GetCatalogResponse& _return, const GetCatalogRequest& catName) {
+    size_t sz = ifaces_.size();
+    size_t i = 0;
+    for (; i < (sz - 1); ++i) {
+      ifaces_[i]->get_catalog(_return, catName);
+    }
+    ifaces_[i]->get_catalog(_return, catName);
+    return;
+  }
+
+  void get_catalogs(GetCatalogsResponse& _return) {
+    size_t sz = ifaces_.size();
+    size_t i = 0;
+    for (; i < (sz - 1); ++i) {
+      ifaces_[i]->get_catalogs(_return);
+    }
+    ifaces_[i]->get_catalogs(_return);
+    return;
+  }
+
+  void drop_catalog(const DropCatalogRequest& catName) {
+    size_t sz = ifaces_.size();
+    size_t i = 0;
+    for (; i < (sz - 1); ++i) {
+      ifaces_[i]->drop_catalog(catName);
+    }
+    ifaces_[i]->drop_catalog(catName);
+  }
+
   void create_database(const Database& database) {
     size_t sz = ifaces_.size();
     size_t i = 0;
@@ -26325,13 +26866,13 @@ class ThriftHiveMetastoreMultiface : virtual public ThriftHiveMetastoreIf, publi
     return;
   }
 
-  void update_creation_metadata(const std::string& dbname, const std::string& tbl_name, const CreationMetadata& creation_metadata) {
+  void update_creation_metadata(const std::string& catName, const std::string& dbname, const std::string& tbl_name, const CreationMetadata& creation_metadata) {
     size_t sz = ifaces_.size();
     size_t i = 0;
     for (; i < (sz - 1); ++i) {
-      ifaces_[i]->update_creation_metadata(dbname, tbl_name, creation_metadata);
+      ifaces_[i]->update_creation_metadata(catName, dbname, tbl_name, creation_metadata);
     }
-    ifaces_[i]->update_creation_metadata(dbname, tbl_name, creation_metadata);
+    ifaces_[i]->update_creation_metadata(catName, dbname, tbl_name, creation_metadata);
   }
 
   void get_table_names_by_filter(std::vector<std::string> & _return, const std::string& dbname, const std::string& filter, const int16_t max_tables) {
@@ -27861,6 +28402,18 @@ class ThriftHiveMetastoreConcurrentClient : virtual public ThriftHiveMetastoreIf
   void setMetaConf(const std::string& key, const std::string& value);
   int32_t send_setMetaConf(const std::string& key, const std::string& value);
   void recv_setMetaConf(const int32_t seqid);
+  void create_catalog(const CreateCatalogRequest& catalog);
+  int32_t send_create_catalog(const CreateCatalogRequest& catalog);
+  void recv_create_catalog(const int32_t seqid);
+  void get_catalog(GetCatalogResponse& _return, const GetCatalogRequest& catName);
+  int32_t send_get_catalog(const GetCatalogRequest& catName);
+  void recv_get_catalog(GetCatalogResponse& _return, const int32_t seqid);
+  void get_catalogs(GetCatalogsResponse& _return);
+  int32_t send_get_catalogs();
+  void recv_get_catalogs(GetCatalogsResponse& _return, const int32_t seqid);
+  void drop_catalog(const DropCatalogRequest& catName);
+  int32_t send_drop_catalog(const DropCatalogRequest& catName);
+  void recv_drop_catalog(const int32_t seqid);
   void create_database(const Database& database);
   int32_t send_create_database(const Database& database);
   void recv_create_database(const int32_t seqid);
@@ -27972,8 +28525,8 @@ class ThriftHiveMetastoreConcurrentClient : virtual public ThriftHiveMetastoreIf
   void get_materialization_invalidation_info(std::map<std::string, Materialization> & _return, const std::string& dbname, const std::vector<std::string> & tbl_names);
   int32_t send_get_materialization_invalidation_info(const std::string& dbname, const std::vector<std::string> & tbl_names);
   void recv_get_materialization_invalidation_info(std::map<std::string, Materialization> & _return, const int32_t seqid);
-  void update_creation_metadata(const std::string& dbname, const std::string& tbl_name, const CreationMetadata& creation_metadata);
-  int32_t send_update_creation_metadata(const std::string& dbname, const std::string& tbl_name, const CreationMetadata& creation_metadata);
+  void update_creation_metadata(const std::string& catName, const std::string& dbname, const std::string& tbl_name, const CreationMetadata& creation_metadata);
+  int32_t send_update_creation_metadata(const std::string& catName, const std::string& dbname, const std::string& tbl_name, const CreationMetadata& creation_metadata);
   void recv_update_creation_metadata(const int32_t seqid);
   void get_table_names_by_filter(std::vector<std::string> & _return, const std::string& dbname, const std::string& filter, const int16_t max_tables);
   int32_t send_get_table_names_by_filter(const std::string& dbname, const std::string& filter, const int16_t max_tables);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
index d7319e2..cfec64f 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
@@ -32,6 +32,26 @@ class ThriftHiveMetastoreHandler : virtual public ThriftHiveMetastoreIf {
     printf("setMetaConf\n");
   }
 
+  void create_catalog(const CreateCatalogRequest& catalog) {
+    // Your implementation goes here
+    printf("create_catalog\n");
+  }
+
+  void get_catalog(GetCatalogResponse& _return, const GetCatalogRequest& catName) {
+    // Your implementation goes here
+    printf("get_catalog\n");
+  }
+
+  void get_catalogs(GetCatalogsResponse& _return) {
+    // Your implementation goes here
+    printf("get_catalogs\n");
+  }
+
+  void drop_catalog(const DropCatalogRequest& catName) {
+    // Your implementation goes here
+    printf("drop_catalog\n");
+  }
+
   void create_database(const Database& database) {
     // Your implementation goes here
     printf("create_database\n");
@@ -217,7 +237,7 @@ class ThriftHiveMetastoreHandler : virtual public ThriftHiveMetastoreIf {
     printf("get_materialization_invalidation_info\n");
   }
 
-  void update_creation_metadata(const std::string& dbname, const std::string& tbl_name, const CreationMetadata& creation_metadata) {
+  void update_creation_metadata(const std::string& catName, const std::string& dbname, const std::string& tbl_name, const CreationMetadata& creation_metadata) {
     // Your implementation goes here
     printf("update_creation_metadata\n");
   }


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UniqueConstraintsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UniqueConstraintsRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UniqueConstraintsRequest.java
index 9ad8728..b5d4829 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UniqueConstraintsRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UniqueConstraintsRequest.java
@@ -38,8 +38,9 @@ import org.slf4j.LoggerFactory;
 @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class UniqueConstraintsRequest implements org.apache.thrift.TBase<UniqueConstraintsRequest, UniqueConstraintsRequest._Fields>, java.io.Serializable, Cloneable, Comparable<UniqueConstraintsRequest> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("UniqueConstraintsRequest");
 
-  private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("db_name", org.apache.thrift.protocol.TType.STRING, (short)1);
-  private static final org.apache.thrift.protocol.TField TBL_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tbl_name", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("db_name", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField TBL_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tbl_name", org.apache.thrift.protocol.TType.STRING, (short)3);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -47,13 +48,15 @@ import org.slf4j.LoggerFactory;
     schemes.put(TupleScheme.class, new UniqueConstraintsRequestTupleSchemeFactory());
   }
 
+  private String catName; // required
   private String db_name; // required
   private String tbl_name; // required
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-    DB_NAME((short)1, "db_name"),
-    TBL_NAME((short)2, "tbl_name");
+    CAT_NAME((short)1, "catName"),
+    DB_NAME((short)2, "db_name"),
+    TBL_NAME((short)3, "tbl_name");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -68,9 +71,11 @@ import org.slf4j.LoggerFactory;
      */
     public static _Fields findByThriftId(int fieldId) {
       switch(fieldId) {
-        case 1: // DB_NAME
+        case 1: // CAT_NAME
+          return CAT_NAME;
+        case 2: // DB_NAME
           return DB_NAME;
-        case 2: // TBL_NAME
+        case 3: // TBL_NAME
           return TBL_NAME;
         default:
           return null;
@@ -115,6 +120,8 @@ import org.slf4j.LoggerFactory;
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.DB_NAME, new org.apache.thrift.meta_data.FieldMetaData("db_name", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.TBL_NAME, new org.apache.thrift.meta_data.FieldMetaData("tbl_name", org.apache.thrift.TFieldRequirementType.REQUIRED, 
@@ -127,10 +134,12 @@ import org.slf4j.LoggerFactory;
   }
 
   public UniqueConstraintsRequest(
+    String catName,
     String db_name,
     String tbl_name)
   {
     this();
+    this.catName = catName;
     this.db_name = db_name;
     this.tbl_name = tbl_name;
   }
@@ -139,6 +148,9 @@ import org.slf4j.LoggerFactory;
    * Performs a deep copy on <i>other</i>.
    */
   public UniqueConstraintsRequest(UniqueConstraintsRequest other) {
+    if (other.isSetCatName()) {
+      this.catName = other.catName;
+    }
     if (other.isSetDb_name()) {
       this.db_name = other.db_name;
     }
@@ -153,10 +165,34 @@ import org.slf4j.LoggerFactory;
 
   @Override
   public void clear() {
+    this.catName = null;
     this.db_name = null;
     this.tbl_name = null;
   }
 
+  public String getCatName() {
+    return this.catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
+  public void unsetCatName() {
+    this.catName = null;
+  }
+
+  /** Returns true if field catName is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatName() {
+    return this.catName != null;
+  }
+
+  public void setCatNameIsSet(boolean value) {
+    if (!value) {
+      this.catName = null;
+    }
+  }
+
   public String getDb_name() {
     return this.db_name;
   }
@@ -205,6 +241,14 @@ import org.slf4j.LoggerFactory;
 
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
+    case CAT_NAME:
+      if (value == null) {
+        unsetCatName();
+      } else {
+        setCatName((String)value);
+      }
+      break;
+
     case DB_NAME:
       if (value == null) {
         unsetDb_name();
@@ -226,6 +270,9 @@ import org.slf4j.LoggerFactory;
 
   public Object getFieldValue(_Fields field) {
     switch (field) {
+    case CAT_NAME:
+      return getCatName();
+
     case DB_NAME:
       return getDb_name();
 
@@ -243,6 +290,8 @@ import org.slf4j.LoggerFactory;
     }
 
     switch (field) {
+    case CAT_NAME:
+      return isSetCatName();
     case DB_NAME:
       return isSetDb_name();
     case TBL_NAME:
@@ -264,6 +313,15 @@ import org.slf4j.LoggerFactory;
     if (that == null)
       return false;
 
+    boolean this_present_catName = true && this.isSetCatName();
+    boolean that_present_catName = true && that.isSetCatName();
+    if (this_present_catName || that_present_catName) {
+      if (!(this_present_catName && that_present_catName))
+        return false;
+      if (!this.catName.equals(that.catName))
+        return false;
+    }
+
     boolean this_present_db_name = true && this.isSetDb_name();
     boolean that_present_db_name = true && that.isSetDb_name();
     if (this_present_db_name || that_present_db_name) {
@@ -289,6 +347,11 @@ import org.slf4j.LoggerFactory;
   public int hashCode() {
     List<Object> list = new ArrayList<Object>();
 
+    boolean present_catName = true && (isSetCatName());
+    list.add(present_catName);
+    if (present_catName)
+      list.add(catName);
+
     boolean present_db_name = true && (isSetDb_name());
     list.add(present_db_name);
     if (present_db_name)
@@ -310,6 +373,16 @@ import org.slf4j.LoggerFactory;
 
     int lastComparison = 0;
 
+    lastComparison = Boolean.valueOf(isSetCatName()).compareTo(other.isSetCatName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catName, other.catName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     lastComparison = Boolean.valueOf(isSetDb_name()).compareTo(other.isSetDb_name());
     if (lastComparison != 0) {
       return lastComparison;
@@ -350,6 +423,14 @@ import org.slf4j.LoggerFactory;
     StringBuilder sb = new StringBuilder("UniqueConstraintsRequest(");
     boolean first = true;
 
+    sb.append("catName:");
+    if (this.catName == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.catName);
+    }
+    first = false;
+    if (!first) sb.append(", ");
     sb.append("db_name:");
     if (this.db_name == null) {
       sb.append("null");
@@ -371,6 +452,10 @@ import org.slf4j.LoggerFactory;
 
   public void validate() throws org.apache.thrift.TException {
     // check for required fields
+    if (!isSetCatName()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'catName' is unset! Struct:" + toString());
+    }
+
     if (!isSetDb_name()) {
       throw new org.apache.thrift.protocol.TProtocolException("Required field 'db_name' is unset! Struct:" + toString());
     }
@@ -416,7 +501,15 @@ import org.slf4j.LoggerFactory;
           break;
         }
         switch (schemeField.id) {
-          case 1: // DB_NAME
+          case 1: // CAT_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.catName = iprot.readString();
+              struct.setCatNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // DB_NAME
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.db_name = iprot.readString();
               struct.setDb_nameIsSet(true);
@@ -424,7 +517,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 2: // TBL_NAME
+          case 3: // TBL_NAME
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.tbl_name = iprot.readString();
               struct.setTbl_nameIsSet(true);
@@ -445,6 +538,11 @@ import org.slf4j.LoggerFactory;
       struct.validate();
 
       oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.catName != null) {
+        oprot.writeFieldBegin(CAT_NAME_FIELD_DESC);
+        oprot.writeString(struct.catName);
+        oprot.writeFieldEnd();
+      }
       if (struct.db_name != null) {
         oprot.writeFieldBegin(DB_NAME_FIELD_DESC);
         oprot.writeString(struct.db_name);
@@ -472,6 +570,7 @@ import org.slf4j.LoggerFactory;
     @Override
     public void write(org.apache.thrift.protocol.TProtocol prot, UniqueConstraintsRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.catName);
       oprot.writeString(struct.db_name);
       oprot.writeString(struct.tbl_name);
     }
@@ -479,6 +578,8 @@ import org.slf4j.LoggerFactory;
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, UniqueConstraintsRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.catName = iprot.readString();
+      struct.setCatNameIsSet(true);
       struct.db_name = iprot.readString();
       struct.setDb_nameIsSet(true);
       struct.tbl_name = iprot.readString();

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UniqueConstraintsResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UniqueConstraintsResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UniqueConstraintsResponse.java
index 4d45bee..80dea83 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UniqueConstraintsResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/UniqueConstraintsResponse.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // UNIQUE_CONSTRAINTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list328 = iprot.readListBegin();
-                struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list328.size);
-                SQLUniqueConstraint _elem329;
-                for (int _i330 = 0; _i330 < _list328.size; ++_i330)
+                org.apache.thrift.protocol.TList _list336 = iprot.readListBegin();
+                struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list336.size);
+                SQLUniqueConstraint _elem337;
+                for (int _i338 = 0; _i338 < _list336.size; ++_i338)
                 {
-                  _elem329 = new SQLUniqueConstraint();
-                  _elem329.read(iprot);
-                  struct.uniqueConstraints.add(_elem329);
+                  _elem337 = new SQLUniqueConstraint();
+                  _elem337.read(iprot);
+                  struct.uniqueConstraints.add(_elem337);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(UNIQUE_CONSTRAINTS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.uniqueConstraints.size()));
-          for (SQLUniqueConstraint _iter331 : struct.uniqueConstraints)
+          for (SQLUniqueConstraint _iter339 : struct.uniqueConstraints)
           {
-            _iter331.write(oprot);
+            _iter339.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.uniqueConstraints.size());
-        for (SQLUniqueConstraint _iter332 : struct.uniqueConstraints)
+        for (SQLUniqueConstraint _iter340 : struct.uniqueConstraints)
         {
-          _iter332.write(oprot);
+          _iter340.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, UniqueConstraintsResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list333 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list333.size);
-        SQLUniqueConstraint _elem334;
-        for (int _i335 = 0; _i335 < _list333.size; ++_i335)
+        org.apache.thrift.protocol.TList _list341 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list341.size);
+        SQLUniqueConstraint _elem342;
+        for (int _i343 = 0; _i343 < _list341.size; ++_i343)
         {
-          _elem334 = new SQLUniqueConstraint();
-          _elem334.read(iprot);
-          struct.uniqueConstraints.add(_elem334);
+          _elem342 = new SQLUniqueConstraint();
+          _elem342.read(iprot);
+          struct.uniqueConstraints.add(_elem342);
         }
       }
       struct.setUniqueConstraintsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java
index a7be2ec..3560567 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java
@@ -755,14 +755,14 @@ import org.slf4j.LoggerFactory;
           case 2: // POOLS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list816 = iprot.readListBegin();
-                struct.pools = new ArrayList<WMPool>(_list816.size);
-                WMPool _elem817;
-                for (int _i818 = 0; _i818 < _list816.size; ++_i818)
+                org.apache.thrift.protocol.TList _list824 = iprot.readListBegin();
+                struct.pools = new ArrayList<WMPool>(_list824.size);
+                WMPool _elem825;
+                for (int _i826 = 0; _i826 < _list824.size; ++_i826)
                 {
-                  _elem817 = new WMPool();
-                  _elem817.read(iprot);
-                  struct.pools.add(_elem817);
+                  _elem825 = new WMPool();
+                  _elem825.read(iprot);
+                  struct.pools.add(_elem825);
                 }
                 iprot.readListEnd();
               }
@@ -774,14 +774,14 @@ import org.slf4j.LoggerFactory;
           case 3: // MAPPINGS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list819 = iprot.readListBegin();
-                struct.mappings = new ArrayList<WMMapping>(_list819.size);
-                WMMapping _elem820;
-                for (int _i821 = 0; _i821 < _list819.size; ++_i821)
+                org.apache.thrift.protocol.TList _list827 = iprot.readListBegin();
+                struct.mappings = new ArrayList<WMMapping>(_list827.size);
+                WMMapping _elem828;
+                for (int _i829 = 0; _i829 < _list827.size; ++_i829)
                 {
-                  _elem820 = new WMMapping();
-                  _elem820.read(iprot);
-                  struct.mappings.add(_elem820);
+                  _elem828 = new WMMapping();
+                  _elem828.read(iprot);
+                  struct.mappings.add(_elem828);
                 }
                 iprot.readListEnd();
               }
@@ -793,14 +793,14 @@ import org.slf4j.LoggerFactory;
           case 4: // TRIGGERS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list822 = iprot.readListBegin();
-                struct.triggers = new ArrayList<WMTrigger>(_list822.size);
-                WMTrigger _elem823;
-                for (int _i824 = 0; _i824 < _list822.size; ++_i824)
+                org.apache.thrift.protocol.TList _list830 = iprot.readListBegin();
+                struct.triggers = new ArrayList<WMTrigger>(_list830.size);
+                WMTrigger _elem831;
+                for (int _i832 = 0; _i832 < _list830.size; ++_i832)
                 {
-                  _elem823 = new WMTrigger();
-                  _elem823.read(iprot);
-                  struct.triggers.add(_elem823);
+                  _elem831 = new WMTrigger();
+                  _elem831.read(iprot);
+                  struct.triggers.add(_elem831);
                 }
                 iprot.readListEnd();
               }
@@ -812,14 +812,14 @@ import org.slf4j.LoggerFactory;
           case 5: // POOL_TRIGGERS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list825 = iprot.readListBegin();
-                struct.poolTriggers = new ArrayList<WMPoolTrigger>(_list825.size);
-                WMPoolTrigger _elem826;
-                for (int _i827 = 0; _i827 < _list825.size; ++_i827)
+                org.apache.thrift.protocol.TList _list833 = iprot.readListBegin();
+                struct.poolTriggers = new ArrayList<WMPoolTrigger>(_list833.size);
+                WMPoolTrigger _elem834;
+                for (int _i835 = 0; _i835 < _list833.size; ++_i835)
                 {
-                  _elem826 = new WMPoolTrigger();
-                  _elem826.read(iprot);
-                  struct.poolTriggers.add(_elem826);
+                  _elem834 = new WMPoolTrigger();
+                  _elem834.read(iprot);
+                  struct.poolTriggers.add(_elem834);
                 }
                 iprot.readListEnd();
               }
@@ -850,9 +850,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(POOLS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.pools.size()));
-          for (WMPool _iter828 : struct.pools)
+          for (WMPool _iter836 : struct.pools)
           {
-            _iter828.write(oprot);
+            _iter836.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -863,9 +863,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(MAPPINGS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.mappings.size()));
-            for (WMMapping _iter829 : struct.mappings)
+            for (WMMapping _iter837 : struct.mappings)
             {
-              _iter829.write(oprot);
+              _iter837.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -877,9 +877,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TRIGGERS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.triggers.size()));
-            for (WMTrigger _iter830 : struct.triggers)
+            for (WMTrigger _iter838 : struct.triggers)
             {
-              _iter830.write(oprot);
+              _iter838.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -891,9 +891,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(POOL_TRIGGERS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.poolTriggers.size()));
-            for (WMPoolTrigger _iter831 : struct.poolTriggers)
+            for (WMPoolTrigger _iter839 : struct.poolTriggers)
             {
-              _iter831.write(oprot);
+              _iter839.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -920,9 +920,9 @@ import org.slf4j.LoggerFactory;
       struct.plan.write(oprot);
       {
         oprot.writeI32(struct.pools.size());
-        for (WMPool _iter832 : struct.pools)
+        for (WMPool _iter840 : struct.pools)
         {
-          _iter832.write(oprot);
+          _iter840.write(oprot);
         }
       }
       BitSet optionals = new BitSet();
@@ -939,27 +939,27 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetMappings()) {
         {
           oprot.writeI32(struct.mappings.size());
-          for (WMMapping _iter833 : struct.mappings)
+          for (WMMapping _iter841 : struct.mappings)
           {
-            _iter833.write(oprot);
+            _iter841.write(oprot);
           }
         }
       }
       if (struct.isSetTriggers()) {
         {
           oprot.writeI32(struct.triggers.size());
-          for (WMTrigger _iter834 : struct.triggers)
+          for (WMTrigger _iter842 : struct.triggers)
           {
-            _iter834.write(oprot);
+            _iter842.write(oprot);
           }
         }
       }
       if (struct.isSetPoolTriggers()) {
         {
           oprot.writeI32(struct.poolTriggers.size());
-          for (WMPoolTrigger _iter835 : struct.poolTriggers)
+          for (WMPoolTrigger _iter843 : struct.poolTriggers)
           {
-            _iter835.write(oprot);
+            _iter843.write(oprot);
           }
         }
       }
@@ -972,56 +972,56 @@ import org.slf4j.LoggerFactory;
       struct.plan.read(iprot);
       struct.setPlanIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list836 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.pools = new ArrayList<WMPool>(_list836.size);
-        WMPool _elem837;
-        for (int _i838 = 0; _i838 < _list836.size; ++_i838)
+        org.apache.thrift.protocol.TList _list844 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.pools = new ArrayList<WMPool>(_list844.size);
+        WMPool _elem845;
+        for (int _i846 = 0; _i846 < _list844.size; ++_i846)
         {
-          _elem837 = new WMPool();
-          _elem837.read(iprot);
-          struct.pools.add(_elem837);
+          _elem845 = new WMPool();
+          _elem845.read(iprot);
+          struct.pools.add(_elem845);
         }
       }
       struct.setPoolsIsSet(true);
       BitSet incoming = iprot.readBitSet(3);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list839 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.mappings = new ArrayList<WMMapping>(_list839.size);
-          WMMapping _elem840;
-          for (int _i841 = 0; _i841 < _list839.size; ++_i841)
+          org.apache.thrift.protocol.TList _list847 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.mappings = new ArrayList<WMMapping>(_list847.size);
+          WMMapping _elem848;
+          for (int _i849 = 0; _i849 < _list847.size; ++_i849)
           {
-            _elem840 = new WMMapping();
-            _elem840.read(iprot);
-            struct.mappings.add(_elem840);
+            _elem848 = new WMMapping();
+            _elem848.read(iprot);
+            struct.mappings.add(_elem848);
           }
         }
         struct.setMappingsIsSet(true);
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TList _list842 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.triggers = new ArrayList<WMTrigger>(_list842.size);
-          WMTrigger _elem843;
-          for (int _i844 = 0; _i844 < _list842.size; ++_i844)
+          org.apache.thrift.protocol.TList _list850 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.triggers = new ArrayList<WMTrigger>(_list850.size);
+          WMTrigger _elem851;
+          for (int _i852 = 0; _i852 < _list850.size; ++_i852)
           {
-            _elem843 = new WMTrigger();
-            _elem843.read(iprot);
-            struct.triggers.add(_elem843);
+            _elem851 = new WMTrigger();
+            _elem851.read(iprot);
+            struct.triggers.add(_elem851);
           }
         }
         struct.setTriggersIsSet(true);
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TList _list845 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.poolTriggers = new ArrayList<WMPoolTrigger>(_list845.size);
-          WMPoolTrigger _elem846;
-          for (int _i847 = 0; _i847 < _list845.size; ++_i847)
+          org.apache.thrift.protocol.TList _list853 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.poolTriggers = new ArrayList<WMPoolTrigger>(_list853.size);
+          WMPoolTrigger _elem854;
+          for (int _i855 = 0; _i855 < _list853.size; ++_i855)
           {
-            _elem846 = new WMPoolTrigger();
-            _elem846.read(iprot);
-            struct.poolTriggers.add(_elem846);
+            _elem854 = new WMPoolTrigger();
+            _elem854.read(iprot);
+            struct.poolTriggers.add(_elem854);
           }
         }
         struct.setPoolTriggersIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java
index d931b47..ffe8b68 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java
@@ -346,14 +346,14 @@ import org.slf4j.LoggerFactory;
           case 1: // RESOURCE_PLANS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list848 = iprot.readListBegin();
-                struct.resourcePlans = new ArrayList<WMResourcePlan>(_list848.size);
-                WMResourcePlan _elem849;
-                for (int _i850 = 0; _i850 < _list848.size; ++_i850)
+                org.apache.thrift.protocol.TList _list856 = iprot.readListBegin();
+                struct.resourcePlans = new ArrayList<WMResourcePlan>(_list856.size);
+                WMResourcePlan _elem857;
+                for (int _i858 = 0; _i858 < _list856.size; ++_i858)
                 {
-                  _elem849 = new WMResourcePlan();
-                  _elem849.read(iprot);
-                  struct.resourcePlans.add(_elem849);
+                  _elem857 = new WMResourcePlan();
+                  _elem857.read(iprot);
+                  struct.resourcePlans.add(_elem857);
                 }
                 iprot.readListEnd();
               }
@@ -380,9 +380,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(RESOURCE_PLANS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.resourcePlans.size()));
-            for (WMResourcePlan _iter851 : struct.resourcePlans)
+            for (WMResourcePlan _iter859 : struct.resourcePlans)
             {
-              _iter851.write(oprot);
+              _iter859.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetResourcePlans()) {
         {
           oprot.writeI32(struct.resourcePlans.size());
-          for (WMResourcePlan _iter852 : struct.resourcePlans)
+          for (WMResourcePlan _iter860 : struct.resourcePlans)
           {
-            _iter852.write(oprot);
+            _iter860.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list853 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.resourcePlans = new ArrayList<WMResourcePlan>(_list853.size);
-          WMResourcePlan _elem854;
-          for (int _i855 = 0; _i855 < _list853.size; ++_i855)
+          org.apache.thrift.protocol.TList _list861 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.resourcePlans = new ArrayList<WMResourcePlan>(_list861.size);
+          WMResourcePlan _elem862;
+          for (int _i863 = 0; _i863 < _list861.size; ++_i863)
           {
-            _elem854 = new WMResourcePlan();
-            _elem854.read(iprot);
-            struct.resourcePlans.add(_elem854);
+            _elem862 = new WMResourcePlan();
+            _elem862.read(iprot);
+            struct.resourcePlans.add(_elem862);
           }
         }
         struct.setResourcePlansIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java
index a674db2..9dfebf0 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java
@@ -346,14 +346,14 @@ import org.slf4j.LoggerFactory;
           case 1: // TRIGGERS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list872 = iprot.readListBegin();
-                struct.triggers = new ArrayList<WMTrigger>(_list872.size);
-                WMTrigger _elem873;
-                for (int _i874 = 0; _i874 < _list872.size; ++_i874)
+                org.apache.thrift.protocol.TList _list880 = iprot.readListBegin();
+                struct.triggers = new ArrayList<WMTrigger>(_list880.size);
+                WMTrigger _elem881;
+                for (int _i882 = 0; _i882 < _list880.size; ++_i882)
                 {
-                  _elem873 = new WMTrigger();
-                  _elem873.read(iprot);
-                  struct.triggers.add(_elem873);
+                  _elem881 = new WMTrigger();
+                  _elem881.read(iprot);
+                  struct.triggers.add(_elem881);
                 }
                 iprot.readListEnd();
               }
@@ -380,9 +380,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TRIGGERS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.triggers.size()));
-            for (WMTrigger _iter875 : struct.triggers)
+            for (WMTrigger _iter883 : struct.triggers)
             {
-              _iter875.write(oprot);
+              _iter883.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetTriggers()) {
         {
           oprot.writeI32(struct.triggers.size());
-          for (WMTrigger _iter876 : struct.triggers)
+          for (WMTrigger _iter884 : struct.triggers)
           {
-            _iter876.write(oprot);
+            _iter884.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list877 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.triggers = new ArrayList<WMTrigger>(_list877.size);
-          WMTrigger _elem878;
-          for (int _i879 = 0; _i879 < _list877.size; ++_i879)
+          org.apache.thrift.protocol.TList _list885 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.triggers = new ArrayList<WMTrigger>(_list885.size);
+          WMTrigger _elem886;
+          for (int _i887 = 0; _i887 < _list885.size; ++_i887)
           {
-            _elem878 = new WMTrigger();
-            _elem878.read(iprot);
-            struct.triggers.add(_elem878);
+            _elem886 = new WMTrigger();
+            _elem886.read(iprot);
+            struct.triggers.add(_elem886);
           }
         }
         struct.setTriggersIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java
index db11958..8f3b065 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java
@@ -441,13 +441,13 @@ import org.slf4j.LoggerFactory;
           case 1: // ERRORS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list856 = iprot.readListBegin();
-                struct.errors = new ArrayList<String>(_list856.size);
-                String _elem857;
-                for (int _i858 = 0; _i858 < _list856.size; ++_i858)
+                org.apache.thrift.protocol.TList _list864 = iprot.readListBegin();
+                struct.errors = new ArrayList<String>(_list864.size);
+                String _elem865;
+                for (int _i866 = 0; _i866 < _list864.size; ++_i866)
                 {
-                  _elem857 = iprot.readString();
-                  struct.errors.add(_elem857);
+                  _elem865 = iprot.readString();
+                  struct.errors.add(_elem865);
                 }
                 iprot.readListEnd();
               }
@@ -459,13 +459,13 @@ import org.slf4j.LoggerFactory;
           case 2: // WARNINGS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list859 = iprot.readListBegin();
-                struct.warnings = new ArrayList<String>(_list859.size);
-                String _elem860;
-                for (int _i861 = 0; _i861 < _list859.size; ++_i861)
+                org.apache.thrift.protocol.TList _list867 = iprot.readListBegin();
+                struct.warnings = new ArrayList<String>(_list867.size);
+                String _elem868;
+                for (int _i869 = 0; _i869 < _list867.size; ++_i869)
                 {
-                  _elem860 = iprot.readString();
-                  struct.warnings.add(_elem860);
+                  _elem868 = iprot.readString();
+                  struct.warnings.add(_elem868);
                 }
                 iprot.readListEnd();
               }
@@ -492,9 +492,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(ERRORS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.errors.size()));
-            for (String _iter862 : struct.errors)
+            for (String _iter870 : struct.errors)
             {
-              oprot.writeString(_iter862);
+              oprot.writeString(_iter870);
             }
             oprot.writeListEnd();
           }
@@ -506,9 +506,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(WARNINGS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.warnings.size()));
-            for (String _iter863 : struct.warnings)
+            for (String _iter871 : struct.warnings)
             {
-              oprot.writeString(_iter863);
+              oprot.writeString(_iter871);
             }
             oprot.writeListEnd();
           }
@@ -543,18 +543,18 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetErrors()) {
         {
           oprot.writeI32(struct.errors.size());
-          for (String _iter864 : struct.errors)
+          for (String _iter872 : struct.errors)
           {
-            oprot.writeString(_iter864);
+            oprot.writeString(_iter872);
           }
         }
       }
       if (struct.isSetWarnings()) {
         {
           oprot.writeI32(struct.warnings.size());
-          for (String _iter865 : struct.warnings)
+          for (String _iter873 : struct.warnings)
           {
-            oprot.writeString(_iter865);
+            oprot.writeString(_iter873);
           }
         }
       }
@@ -566,26 +566,26 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(2);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list866 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.errors = new ArrayList<String>(_list866.size);
-          String _elem867;
-          for (int _i868 = 0; _i868 < _list866.size; ++_i868)
+          org.apache.thrift.protocol.TList _list874 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.errors = new ArrayList<String>(_list874.size);
+          String _elem875;
+          for (int _i876 = 0; _i876 < _list874.size; ++_i876)
           {
-            _elem867 = iprot.readString();
-            struct.errors.add(_elem867);
+            _elem875 = iprot.readString();
+            struct.errors.add(_elem875);
           }
         }
         struct.setErrorsIsSet(true);
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TList _list869 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.warnings = new ArrayList<String>(_list869.size);
-          String _elem870;
-          for (int _i871 = 0; _i871 < _list869.size; ++_i871)
+          org.apache.thrift.protocol.TList _list877 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.warnings = new ArrayList<String>(_list877.size);
+          String _elem878;
+          for (int _i879 = 0; _i879 < _list877.size; ++_i879)
           {
-            _elem870 = iprot.readString();
-            struct.warnings.add(_elem870);
+            _elem878 = iprot.readString();
+            struct.warnings.add(_elem878);
           }
         }
         struct.setWarningsIsSet(true);


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprResult.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprResult.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprResult.java
index b5c1539..3f2ddcc 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprResult.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsByExprResult.java
@@ -439,14 +439,14 @@ import org.slf4j.LoggerFactory;
           case 1: // PARTITIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list408 = iprot.readListBegin();
-                struct.partitions = new ArrayList<Partition>(_list408.size);
-                Partition _elem409;
-                for (int _i410 = 0; _i410 < _list408.size; ++_i410)
+                org.apache.thrift.protocol.TList _list416 = iprot.readListBegin();
+                struct.partitions = new ArrayList<Partition>(_list416.size);
+                Partition _elem417;
+                for (int _i418 = 0; _i418 < _list416.size; ++_i418)
                 {
-                  _elem409 = new Partition();
-                  _elem409.read(iprot);
-                  struct.partitions.add(_elem409);
+                  _elem417 = new Partition();
+                  _elem417.read(iprot);
+                  struct.partitions.add(_elem417);
                 }
                 iprot.readListEnd();
               }
@@ -480,9 +480,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PARTITIONS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitions.size()));
-          for (Partition _iter411 : struct.partitions)
+          for (Partition _iter419 : struct.partitions)
           {
-            _iter411.write(oprot);
+            _iter419.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -510,9 +510,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.partitions.size());
-        for (Partition _iter412 : struct.partitions)
+        for (Partition _iter420 : struct.partitions)
         {
-          _iter412.write(oprot);
+          _iter420.write(oprot);
         }
       }
       oprot.writeBool(struct.hasUnknownPartitions);
@@ -522,14 +522,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, PartitionsByExprResult struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list413 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.partitions = new ArrayList<Partition>(_list413.size);
-        Partition _elem414;
-        for (int _i415 = 0; _i415 < _list413.size; ++_i415)
+        org.apache.thrift.protocol.TList _list421 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.partitions = new ArrayList<Partition>(_list421.size);
+        Partition _elem422;
+        for (int _i423 = 0; _i423 < _list421.size; ++_i423)
         {
-          _elem414 = new Partition();
-          _elem414.read(iprot);
-          struct.partitions.add(_elem414);
+          _elem422 = new Partition();
+          _elem422.read(iprot);
+          struct.partitions.add(_elem422);
         }
       }
       struct.setPartitionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java
index ad6f054..91cf567 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java
@@ -42,6 +42,7 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField TBL_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tblName", org.apache.thrift.protocol.TType.STRING, (short)2);
   private static final org.apache.thrift.protocol.TField COL_NAMES_FIELD_DESC = new org.apache.thrift.protocol.TField("colNames", org.apache.thrift.protocol.TType.LIST, (short)3);
   private static final org.apache.thrift.protocol.TField PART_NAMES_FIELD_DESC = new org.apache.thrift.protocol.TField("partNames", org.apache.thrift.protocol.TType.LIST, (short)4);
+  private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)5);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -53,13 +54,15 @@ import org.slf4j.LoggerFactory;
   private String tblName; // required
   private List<String> colNames; // required
   private List<String> partNames; // required
+  private String catName; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
     DB_NAME((short)1, "dbName"),
     TBL_NAME((short)2, "tblName"),
     COL_NAMES((short)3, "colNames"),
-    PART_NAMES((short)4, "partNames");
+    PART_NAMES((short)4, "partNames"),
+    CAT_NAME((short)5, "catName");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -82,6 +85,8 @@ import org.slf4j.LoggerFactory;
           return COL_NAMES;
         case 4: // PART_NAMES
           return PART_NAMES;
+        case 5: // CAT_NAME
+          return CAT_NAME;
         default:
           return null;
       }
@@ -122,6 +127,7 @@ import org.slf4j.LoggerFactory;
   }
 
   // isset id assignments
+  private static final _Fields optionals[] = {_Fields.CAT_NAME};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -135,6 +141,8 @@ import org.slf4j.LoggerFactory;
     tmpMap.put(_Fields.PART_NAMES, new org.apache.thrift.meta_data.FieldMetaData("partNames", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
             new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    tmpMap.put(_Fields.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(PartitionsStatsRequest.class, metaDataMap);
   }
@@ -173,6 +181,9 @@ import org.slf4j.LoggerFactory;
       List<String> __this__partNames = new ArrayList<String>(other.partNames);
       this.partNames = __this__partNames;
     }
+    if (other.isSetCatName()) {
+      this.catName = other.catName;
+    }
   }
 
   public PartitionsStatsRequest deepCopy() {
@@ -185,6 +196,7 @@ import org.slf4j.LoggerFactory;
     this.tblName = null;
     this.colNames = null;
     this.partNames = null;
+    this.catName = null;
   }
 
   public String getDbName() {
@@ -309,6 +321,29 @@ import org.slf4j.LoggerFactory;
     }
   }
 
+  public String getCatName() {
+    return this.catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
+  public void unsetCatName() {
+    this.catName = null;
+  }
+
+  /** Returns true if field catName is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatName() {
+    return this.catName != null;
+  }
+
+  public void setCatNameIsSet(boolean value) {
+    if (!value) {
+      this.catName = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case DB_NAME:
@@ -343,6 +378,14 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case CAT_NAME:
+      if (value == null) {
+        unsetCatName();
+      } else {
+        setCatName((String)value);
+      }
+      break;
+
     }
   }
 
@@ -360,6 +403,9 @@ import org.slf4j.LoggerFactory;
     case PART_NAMES:
       return getPartNames();
 
+    case CAT_NAME:
+      return getCatName();
+
     }
     throw new IllegalStateException();
   }
@@ -379,6 +425,8 @@ import org.slf4j.LoggerFactory;
       return isSetColNames();
     case PART_NAMES:
       return isSetPartNames();
+    case CAT_NAME:
+      return isSetCatName();
     }
     throw new IllegalStateException();
   }
@@ -432,6 +480,15 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_catName = true && this.isSetCatName();
+    boolean that_present_catName = true && that.isSetCatName();
+    if (this_present_catName || that_present_catName) {
+      if (!(this_present_catName && that_present_catName))
+        return false;
+      if (!this.catName.equals(that.catName))
+        return false;
+    }
+
     return true;
   }
 
@@ -459,6 +516,11 @@ import org.slf4j.LoggerFactory;
     if (present_partNames)
       list.add(partNames);
 
+    boolean present_catName = true && (isSetCatName());
+    list.add(present_catName);
+    if (present_catName)
+      list.add(catName);
+
     return list.hashCode();
   }
 
@@ -510,6 +572,16 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetCatName()).compareTo(other.isSetCatName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catName, other.catName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -561,6 +633,16 @@ import org.slf4j.LoggerFactory;
       sb.append(this.partNames);
     }
     first = false;
+    if (isSetCatName()) {
+      if (!first) sb.append(", ");
+      sb.append("catName:");
+      if (this.catName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.catName);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -639,13 +721,13 @@ import org.slf4j.LoggerFactory;
           case 3: // COL_NAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list450 = iprot.readListBegin();
-                struct.colNames = new ArrayList<String>(_list450.size);
-                String _elem451;
-                for (int _i452 = 0; _i452 < _list450.size; ++_i452)
+                org.apache.thrift.protocol.TList _list458 = iprot.readListBegin();
+                struct.colNames = new ArrayList<String>(_list458.size);
+                String _elem459;
+                for (int _i460 = 0; _i460 < _list458.size; ++_i460)
                 {
-                  _elem451 = iprot.readString();
-                  struct.colNames.add(_elem451);
+                  _elem459 = iprot.readString();
+                  struct.colNames.add(_elem459);
                 }
                 iprot.readListEnd();
               }
@@ -657,13 +739,13 @@ import org.slf4j.LoggerFactory;
           case 4: // PART_NAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list453 = iprot.readListBegin();
-                struct.partNames = new ArrayList<String>(_list453.size);
-                String _elem454;
-                for (int _i455 = 0; _i455 < _list453.size; ++_i455)
+                org.apache.thrift.protocol.TList _list461 = iprot.readListBegin();
+                struct.partNames = new ArrayList<String>(_list461.size);
+                String _elem462;
+                for (int _i463 = 0; _i463 < _list461.size; ++_i463)
                 {
-                  _elem454 = iprot.readString();
-                  struct.partNames.add(_elem454);
+                  _elem462 = iprot.readString();
+                  struct.partNames.add(_elem462);
                 }
                 iprot.readListEnd();
               }
@@ -672,6 +754,14 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 5: // CAT_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.catName = iprot.readString();
+              struct.setCatNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -699,9 +789,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(COL_NAMES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.colNames.size()));
-          for (String _iter456 : struct.colNames)
+          for (String _iter464 : struct.colNames)
           {
-            oprot.writeString(_iter456);
+            oprot.writeString(_iter464);
           }
           oprot.writeListEnd();
         }
@@ -711,14 +801,21 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PART_NAMES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partNames.size()));
-          for (String _iter457 : struct.partNames)
+          for (String _iter465 : struct.partNames)
           {
-            oprot.writeString(_iter457);
+            oprot.writeString(_iter465);
           }
           oprot.writeListEnd();
         }
         oprot.writeFieldEnd();
       }
+      if (struct.catName != null) {
+        if (struct.isSetCatName()) {
+          oprot.writeFieldBegin(CAT_NAME_FIELD_DESC);
+          oprot.writeString(struct.catName);
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -740,18 +837,26 @@ import org.slf4j.LoggerFactory;
       oprot.writeString(struct.tblName);
       {
         oprot.writeI32(struct.colNames.size());
-        for (String _iter458 : struct.colNames)
+        for (String _iter466 : struct.colNames)
         {
-          oprot.writeString(_iter458);
+          oprot.writeString(_iter466);
         }
       }
       {
         oprot.writeI32(struct.partNames.size());
-        for (String _iter459 : struct.partNames)
+        for (String _iter467 : struct.partNames)
         {
-          oprot.writeString(_iter459);
+          oprot.writeString(_iter467);
         }
       }
+      BitSet optionals = new BitSet();
+      if (struct.isSetCatName()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.isSetCatName()) {
+        oprot.writeString(struct.catName);
+      }
     }
 
     @Override
@@ -762,27 +867,32 @@ import org.slf4j.LoggerFactory;
       struct.tblName = iprot.readString();
       struct.setTblNameIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list460 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.colNames = new ArrayList<String>(_list460.size);
-        String _elem461;
-        for (int _i462 = 0; _i462 < _list460.size; ++_i462)
+        org.apache.thrift.protocol.TList _list468 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.colNames = new ArrayList<String>(_list468.size);
+        String _elem469;
+        for (int _i470 = 0; _i470 < _list468.size; ++_i470)
         {
-          _elem461 = iprot.readString();
-          struct.colNames.add(_elem461);
+          _elem469 = iprot.readString();
+          struct.colNames.add(_elem469);
         }
       }
       struct.setColNamesIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list463 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.partNames = new ArrayList<String>(_list463.size);
-        String _elem464;
-        for (int _i465 = 0; _i465 < _list463.size; ++_i465)
+        org.apache.thrift.protocol.TList _list471 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.partNames = new ArrayList<String>(_list471.size);
+        String _elem472;
+        for (int _i473 = 0; _i473 < _list471.size; ++_i473)
         {
-          _elem464 = iprot.readString();
-          struct.partNames.add(_elem464);
+          _elem472 = iprot.readString();
+          struct.partNames.add(_elem472);
         }
       }
       struct.setPartNamesIsSet(true);
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.catName = iprot.readString();
+        struct.setCatNameIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsResult.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsResult.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsResult.java
index d84af22..4caec8f 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsResult.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsResult.java
@@ -363,26 +363,26 @@ import org.slf4j.LoggerFactory;
           case 1: // PART_STATS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map424 = iprot.readMapBegin();
-                struct.partStats = new HashMap<String,List<ColumnStatisticsObj>>(2*_map424.size);
-                String _key425;
-                List<ColumnStatisticsObj> _val426;
-                for (int _i427 = 0; _i427 < _map424.size; ++_i427)
+                org.apache.thrift.protocol.TMap _map432 = iprot.readMapBegin();
+                struct.partStats = new HashMap<String,List<ColumnStatisticsObj>>(2*_map432.size);
+                String _key433;
+                List<ColumnStatisticsObj> _val434;
+                for (int _i435 = 0; _i435 < _map432.size; ++_i435)
                 {
-                  _key425 = iprot.readString();
+                  _key433 = iprot.readString();
                   {
-                    org.apache.thrift.protocol.TList _list428 = iprot.readListBegin();
-                    _val426 = new ArrayList<ColumnStatisticsObj>(_list428.size);
-                    ColumnStatisticsObj _elem429;
-                    for (int _i430 = 0; _i430 < _list428.size; ++_i430)
+                    org.apache.thrift.protocol.TList _list436 = iprot.readListBegin();
+                    _val434 = new ArrayList<ColumnStatisticsObj>(_list436.size);
+                    ColumnStatisticsObj _elem437;
+                    for (int _i438 = 0; _i438 < _list436.size; ++_i438)
                     {
-                      _elem429 = new ColumnStatisticsObj();
-                      _elem429.read(iprot);
-                      _val426.add(_elem429);
+                      _elem437 = new ColumnStatisticsObj();
+                      _elem437.read(iprot);
+                      _val434.add(_elem437);
                     }
                     iprot.readListEnd();
                   }
-                  struct.partStats.put(_key425, _val426);
+                  struct.partStats.put(_key433, _val434);
                 }
                 iprot.readMapEnd();
               }
@@ -408,14 +408,14 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PART_STATS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, struct.partStats.size()));
-          for (Map.Entry<String, List<ColumnStatisticsObj>> _iter431 : struct.partStats.entrySet())
+          for (Map.Entry<String, List<ColumnStatisticsObj>> _iter439 : struct.partStats.entrySet())
           {
-            oprot.writeString(_iter431.getKey());
+            oprot.writeString(_iter439.getKey());
             {
-              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter431.getValue().size()));
-              for (ColumnStatisticsObj _iter432 : _iter431.getValue())
+              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter439.getValue().size()));
+              for (ColumnStatisticsObj _iter440 : _iter439.getValue())
               {
-                _iter432.write(oprot);
+                _iter440.write(oprot);
               }
               oprot.writeListEnd();
             }
@@ -443,14 +443,14 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.partStats.size());
-        for (Map.Entry<String, List<ColumnStatisticsObj>> _iter433 : struct.partStats.entrySet())
+        for (Map.Entry<String, List<ColumnStatisticsObj>> _iter441 : struct.partStats.entrySet())
         {
-          oprot.writeString(_iter433.getKey());
+          oprot.writeString(_iter441.getKey());
           {
-            oprot.writeI32(_iter433.getValue().size());
-            for (ColumnStatisticsObj _iter434 : _iter433.getValue())
+            oprot.writeI32(_iter441.getValue().size());
+            for (ColumnStatisticsObj _iter442 : _iter441.getValue())
             {
-              _iter434.write(oprot);
+              _iter442.write(oprot);
             }
           }
         }
@@ -461,25 +461,25 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, PartitionsStatsResult struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TMap _map435 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, iprot.readI32());
-        struct.partStats = new HashMap<String,List<ColumnStatisticsObj>>(2*_map435.size);
-        String _key436;
-        List<ColumnStatisticsObj> _val437;
-        for (int _i438 = 0; _i438 < _map435.size; ++_i438)
+        org.apache.thrift.protocol.TMap _map443 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, iprot.readI32());
+        struct.partStats = new HashMap<String,List<ColumnStatisticsObj>>(2*_map443.size);
+        String _key444;
+        List<ColumnStatisticsObj> _val445;
+        for (int _i446 = 0; _i446 < _map443.size; ++_i446)
         {
-          _key436 = iprot.readString();
+          _key444 = iprot.readString();
           {
-            org.apache.thrift.protocol.TList _list439 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            _val437 = new ArrayList<ColumnStatisticsObj>(_list439.size);
-            ColumnStatisticsObj _elem440;
-            for (int _i441 = 0; _i441 < _list439.size; ++_i441)
+            org.apache.thrift.protocol.TList _list447 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            _val445 = new ArrayList<ColumnStatisticsObj>(_list447.size);
+            ColumnStatisticsObj _elem448;
+            for (int _i449 = 0; _i449 < _list447.size; ++_i449)
             {
-              _elem440 = new ColumnStatisticsObj();
-              _elem440.read(iprot);
-              _val437.add(_elem440);
+              _elem448 = new ColumnStatisticsObj();
+              _elem448.read(iprot);
+              _val445.add(_elem448);
             }
           }
-          struct.partStats.put(_key436, _val437);
+          struct.partStats.put(_key444, _val445);
         }
       }
       struct.setPartStatsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrimaryKeysRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrimaryKeysRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrimaryKeysRequest.java
index 8930f34..591348d 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrimaryKeysRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrimaryKeysRequest.java
@@ -40,6 +40,7 @@ import org.slf4j.LoggerFactory;
 
   private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("db_name", org.apache.thrift.protocol.TType.STRING, (short)1);
   private static final org.apache.thrift.protocol.TField TBL_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tbl_name", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)3);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -49,11 +50,13 @@ import org.slf4j.LoggerFactory;
 
   private String db_name; // required
   private String tbl_name; // required
+  private String catName; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
     DB_NAME((short)1, "db_name"),
-    TBL_NAME((short)2, "tbl_name");
+    TBL_NAME((short)2, "tbl_name"),
+    CAT_NAME((short)3, "catName");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -72,6 +75,8 @@ import org.slf4j.LoggerFactory;
           return DB_NAME;
         case 2: // TBL_NAME
           return TBL_NAME;
+        case 3: // CAT_NAME
+          return CAT_NAME;
         default:
           return null;
       }
@@ -112,6 +117,7 @@ import org.slf4j.LoggerFactory;
   }
 
   // isset id assignments
+  private static final _Fields optionals[] = {_Fields.CAT_NAME};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -119,6 +125,8 @@ import org.slf4j.LoggerFactory;
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.TBL_NAME, new org.apache.thrift.meta_data.FieldMetaData("tbl_name", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(PrimaryKeysRequest.class, metaDataMap);
   }
@@ -145,6 +153,9 @@ import org.slf4j.LoggerFactory;
     if (other.isSetTbl_name()) {
       this.tbl_name = other.tbl_name;
     }
+    if (other.isSetCatName()) {
+      this.catName = other.catName;
+    }
   }
 
   public PrimaryKeysRequest deepCopy() {
@@ -155,6 +166,7 @@ import org.slf4j.LoggerFactory;
   public void clear() {
     this.db_name = null;
     this.tbl_name = null;
+    this.catName = null;
   }
 
   public String getDb_name() {
@@ -203,6 +215,29 @@ import org.slf4j.LoggerFactory;
     }
   }
 
+  public String getCatName() {
+    return this.catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
+  public void unsetCatName() {
+    this.catName = null;
+  }
+
+  /** Returns true if field catName is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatName() {
+    return this.catName != null;
+  }
+
+  public void setCatNameIsSet(boolean value) {
+    if (!value) {
+      this.catName = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case DB_NAME:
@@ -221,6 +256,14 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case CAT_NAME:
+      if (value == null) {
+        unsetCatName();
+      } else {
+        setCatName((String)value);
+      }
+      break;
+
     }
   }
 
@@ -232,6 +275,9 @@ import org.slf4j.LoggerFactory;
     case TBL_NAME:
       return getTbl_name();
 
+    case CAT_NAME:
+      return getCatName();
+
     }
     throw new IllegalStateException();
   }
@@ -247,6 +293,8 @@ import org.slf4j.LoggerFactory;
       return isSetDb_name();
     case TBL_NAME:
       return isSetTbl_name();
+    case CAT_NAME:
+      return isSetCatName();
     }
     throw new IllegalStateException();
   }
@@ -282,6 +330,15 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_catName = true && this.isSetCatName();
+    boolean that_present_catName = true && that.isSetCatName();
+    if (this_present_catName || that_present_catName) {
+      if (!(this_present_catName && that_present_catName))
+        return false;
+      if (!this.catName.equals(that.catName))
+        return false;
+    }
+
     return true;
   }
 
@@ -299,6 +356,11 @@ import org.slf4j.LoggerFactory;
     if (present_tbl_name)
       list.add(tbl_name);
 
+    boolean present_catName = true && (isSetCatName());
+    list.add(present_catName);
+    if (present_catName)
+      list.add(catName);
+
     return list.hashCode();
   }
 
@@ -330,6 +392,16 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetCatName()).compareTo(other.isSetCatName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catName, other.catName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -365,6 +437,16 @@ import org.slf4j.LoggerFactory;
       sb.append(this.tbl_name);
     }
     first = false;
+    if (isSetCatName()) {
+      if (!first) sb.append(", ");
+      sb.append("catName:");
+      if (this.catName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.catName);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -432,6 +514,14 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 3: // CAT_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.catName = iprot.readString();
+              struct.setCatNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -455,6 +545,13 @@ import org.slf4j.LoggerFactory;
         oprot.writeString(struct.tbl_name);
         oprot.writeFieldEnd();
       }
+      if (struct.catName != null) {
+        if (struct.isSetCatName()) {
+          oprot.writeFieldBegin(CAT_NAME_FIELD_DESC);
+          oprot.writeString(struct.catName);
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -474,6 +571,14 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       oprot.writeString(struct.db_name);
       oprot.writeString(struct.tbl_name);
+      BitSet optionals = new BitSet();
+      if (struct.isSetCatName()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.isSetCatName()) {
+        oprot.writeString(struct.catName);
+      }
     }
 
     @Override
@@ -483,6 +588,11 @@ import org.slf4j.LoggerFactory;
       struct.setDb_nameIsSet(true);
       struct.tbl_name = iprot.readString();
       struct.setTbl_nameIsSet(true);
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.catName = iprot.readString();
+        struct.setCatNameIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrimaryKeysResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrimaryKeysResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrimaryKeysResponse.java
index 7eed56f..0b776fb 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrimaryKeysResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PrimaryKeysResponse.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // PRIMARY_KEYS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list312 = iprot.readListBegin();
-                struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list312.size);
-                SQLPrimaryKey _elem313;
-                for (int _i314 = 0; _i314 < _list312.size; ++_i314)
+                org.apache.thrift.protocol.TList _list320 = iprot.readListBegin();
+                struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list320.size);
+                SQLPrimaryKey _elem321;
+                for (int _i322 = 0; _i322 < _list320.size; ++_i322)
                 {
-                  _elem313 = new SQLPrimaryKey();
-                  _elem313.read(iprot);
-                  struct.primaryKeys.add(_elem313);
+                  _elem321 = new SQLPrimaryKey();
+                  _elem321.read(iprot);
+                  struct.primaryKeys.add(_elem321);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PRIMARY_KEYS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.primaryKeys.size()));
-          for (SQLPrimaryKey _iter315 : struct.primaryKeys)
+          for (SQLPrimaryKey _iter323 : struct.primaryKeys)
           {
-            _iter315.write(oprot);
+            _iter323.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.primaryKeys.size());
-        for (SQLPrimaryKey _iter316 : struct.primaryKeys)
+        for (SQLPrimaryKey _iter324 : struct.primaryKeys)
         {
-          _iter316.write(oprot);
+          _iter324.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, PrimaryKeysResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list317 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list317.size);
-        SQLPrimaryKey _elem318;
-        for (int _i319 = 0; _i319 < _list317.size; ++_i319)
+        org.apache.thrift.protocol.TList _list325 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list325.size);
+        SQLPrimaryKey _elem326;
+        for (int _i327 = 0; _i327 < _list325.size; ++_i327)
         {
-          _elem318 = new SQLPrimaryKey();
-          _elem318.read(iprot);
-          struct.primaryKeys.add(_elem318);
+          _elem326 = new SQLPrimaryKey();
+          _elem326.read(iprot);
+          struct.primaryKeys.add(_elem326);
         }
       }
       struct.setPrimaryKeysIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
index 490d718..e8cba59 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
@@ -547,13 +547,13 @@ import org.slf4j.LoggerFactory;
           case 1: // FILE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list752 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list752.size);
-                long _elem753;
-                for (int _i754 = 0; _i754 < _list752.size; ++_i754)
+                org.apache.thrift.protocol.TList _list760 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list760.size);
+                long _elem761;
+                for (int _i762 = 0; _i762 < _list760.size; ++_i762)
                 {
-                  _elem753 = iprot.readI64();
-                  struct.fileIds.add(_elem753);
+                  _elem761 = iprot.readI64();
+                  struct.fileIds.add(_elem761);
                 }
                 iprot.readListEnd();
               }
@@ -565,13 +565,13 @@ import org.slf4j.LoggerFactory;
           case 2: // METADATA
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list755 = iprot.readListBegin();
-                struct.metadata = new ArrayList<ByteBuffer>(_list755.size);
-                ByteBuffer _elem756;
-                for (int _i757 = 0; _i757 < _list755.size; ++_i757)
+                org.apache.thrift.protocol.TList _list763 = iprot.readListBegin();
+                struct.metadata = new ArrayList<ByteBuffer>(_list763.size);
+                ByteBuffer _elem764;
+                for (int _i765 = 0; _i765 < _list763.size; ++_i765)
                 {
-                  _elem756 = iprot.readBinary();
-                  struct.metadata.add(_elem756);
+                  _elem764 = iprot.readBinary();
+                  struct.metadata.add(_elem764);
                 }
                 iprot.readListEnd();
               }
@@ -605,9 +605,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FILE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.fileIds.size()));
-          for (long _iter758 : struct.fileIds)
+          for (long _iter766 : struct.fileIds)
           {
-            oprot.writeI64(_iter758);
+            oprot.writeI64(_iter766);
           }
           oprot.writeListEnd();
         }
@@ -617,9 +617,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(METADATA_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.metadata.size()));
-          for (ByteBuffer _iter759 : struct.metadata)
+          for (ByteBuffer _iter767 : struct.metadata)
           {
-            oprot.writeBinary(_iter759);
+            oprot.writeBinary(_iter767);
           }
           oprot.writeListEnd();
         }
@@ -651,16 +651,16 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter760 : struct.fileIds)
+        for (long _iter768 : struct.fileIds)
         {
-          oprot.writeI64(_iter760);
+          oprot.writeI64(_iter768);
         }
       }
       {
         oprot.writeI32(struct.metadata.size());
-        for (ByteBuffer _iter761 : struct.metadata)
+        for (ByteBuffer _iter769 : struct.metadata)
         {
-          oprot.writeBinary(_iter761);
+          oprot.writeBinary(_iter769);
         }
       }
       BitSet optionals = new BitSet();
@@ -677,24 +677,24 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, PutFileMetadataRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list762 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list762.size);
-        long _elem763;
-        for (int _i764 = 0; _i764 < _list762.size; ++_i764)
+        org.apache.thrift.protocol.TList _list770 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list770.size);
+        long _elem771;
+        for (int _i772 = 0; _i772 < _list770.size; ++_i772)
         {
-          _elem763 = iprot.readI64();
-          struct.fileIds.add(_elem763);
+          _elem771 = iprot.readI64();
+          struct.fileIds.add(_elem771);
         }
       }
       struct.setFileIdsIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list765 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.metadata = new ArrayList<ByteBuffer>(_list765.size);
-        ByteBuffer _elem766;
-        for (int _i767 = 0; _i767 < _list765.size; ++_i767)
+        org.apache.thrift.protocol.TList _list773 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.metadata = new ArrayList<ByteBuffer>(_list773.size);
+        ByteBuffer _elem774;
+        for (int _i775 = 0; _i775 < _list773.size; ++_i775)
         {
-          _elem766 = iprot.readBinary();
-          struct.metadata.add(_elem766);
+          _elem774 = iprot.readBinary();
+          struct.metadata.add(_elem774);
         }
       }
       struct.setMetadataIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RequestPartsSpec.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RequestPartsSpec.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RequestPartsSpec.java
index d922683..bad44ad 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RequestPartsSpec.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RequestPartsSpec.java
@@ -168,13 +168,13 @@ import org.slf4j.LoggerFactory;
           if (field.type == NAMES_FIELD_DESC.type) {
             List<String> names;
             {
-              org.apache.thrift.protocol.TList _list490 = iprot.readListBegin();
-              names = new ArrayList<String>(_list490.size);
-              String _elem491;
-              for (int _i492 = 0; _i492 < _list490.size; ++_i492)
+              org.apache.thrift.protocol.TList _list498 = iprot.readListBegin();
+              names = new ArrayList<String>(_list498.size);
+              String _elem499;
+              for (int _i500 = 0; _i500 < _list498.size; ++_i500)
               {
-                _elem491 = iprot.readString();
-                names.add(_elem491);
+                _elem499 = iprot.readString();
+                names.add(_elem499);
               }
               iprot.readListEnd();
             }
@@ -187,14 +187,14 @@ import org.slf4j.LoggerFactory;
           if (field.type == EXPRS_FIELD_DESC.type) {
             List<DropPartitionsExpr> exprs;
             {
-              org.apache.thrift.protocol.TList _list493 = iprot.readListBegin();
-              exprs = new ArrayList<DropPartitionsExpr>(_list493.size);
-              DropPartitionsExpr _elem494;
-              for (int _i495 = 0; _i495 < _list493.size; ++_i495)
+              org.apache.thrift.protocol.TList _list501 = iprot.readListBegin();
+              exprs = new ArrayList<DropPartitionsExpr>(_list501.size);
+              DropPartitionsExpr _elem502;
+              for (int _i503 = 0; _i503 < _list501.size; ++_i503)
               {
-                _elem494 = new DropPartitionsExpr();
-                _elem494.read(iprot);
-                exprs.add(_elem494);
+                _elem502 = new DropPartitionsExpr();
+                _elem502.read(iprot);
+                exprs.add(_elem502);
               }
               iprot.readListEnd();
             }
@@ -219,9 +219,9 @@ import org.slf4j.LoggerFactory;
         List<String> names = (List<String>)value_;
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, names.size()));
-          for (String _iter496 : names)
+          for (String _iter504 : names)
           {
-            oprot.writeString(_iter496);
+            oprot.writeString(_iter504);
           }
           oprot.writeListEnd();
         }
@@ -230,9 +230,9 @@ import org.slf4j.LoggerFactory;
         List<DropPartitionsExpr> exprs = (List<DropPartitionsExpr>)value_;
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, exprs.size()));
-          for (DropPartitionsExpr _iter497 : exprs)
+          for (DropPartitionsExpr _iter505 : exprs)
           {
-            _iter497.write(oprot);
+            _iter505.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -250,13 +250,13 @@ import org.slf4j.LoggerFactory;
         case NAMES:
           List<String> names;
           {
-            org.apache.thrift.protocol.TList _list498 = iprot.readListBegin();
-            names = new ArrayList<String>(_list498.size);
-            String _elem499;
-            for (int _i500 = 0; _i500 < _list498.size; ++_i500)
+            org.apache.thrift.protocol.TList _list506 = iprot.readListBegin();
+            names = new ArrayList<String>(_list506.size);
+            String _elem507;
+            for (int _i508 = 0; _i508 < _list506.size; ++_i508)
             {
-              _elem499 = iprot.readString();
-              names.add(_elem499);
+              _elem507 = iprot.readString();
+              names.add(_elem507);
             }
             iprot.readListEnd();
           }
@@ -264,14 +264,14 @@ import org.slf4j.LoggerFactory;
         case EXPRS:
           List<DropPartitionsExpr> exprs;
           {
-            org.apache.thrift.protocol.TList _list501 = iprot.readListBegin();
-            exprs = new ArrayList<DropPartitionsExpr>(_list501.size);
-            DropPartitionsExpr _elem502;
-            for (int _i503 = 0; _i503 < _list501.size; ++_i503)
+            org.apache.thrift.protocol.TList _list509 = iprot.readListBegin();
+            exprs = new ArrayList<DropPartitionsExpr>(_list509.size);
+            DropPartitionsExpr _elem510;
+            for (int _i511 = 0; _i511 < _list509.size; ++_i511)
             {
-              _elem502 = new DropPartitionsExpr();
-              _elem502.read(iprot);
-              exprs.add(_elem502);
+              _elem510 = new DropPartitionsExpr();
+              _elem510.read(iprot);
+              exprs.add(_elem510);
             }
             iprot.readListEnd();
           }
@@ -291,9 +291,9 @@ import org.slf4j.LoggerFactory;
         List<String> names = (List<String>)value_;
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, names.size()));
-          for (String _iter504 : names)
+          for (String _iter512 : names)
           {
-            oprot.writeString(_iter504);
+            oprot.writeString(_iter512);
           }
           oprot.writeListEnd();
         }
@@ -302,9 +302,9 @@ import org.slf4j.LoggerFactory;
         List<DropPartitionsExpr> exprs = (List<DropPartitionsExpr>)value_;
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, exprs.size()));
-          for (DropPartitionsExpr _iter505 : exprs)
+          for (DropPartitionsExpr _iter513 : exprs)
           {
-            _iter505.write(oprot);
+            _iter513.write(oprot);
           }
           oprot.writeListEnd();
         }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLCheckConstraint.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLCheckConstraint.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLCheckConstraint.java
index 24ce47d..9a5d328 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLCheckConstraint.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLCheckConstraint.java
@@ -38,14 +38,15 @@ import org.slf4j.LoggerFactory;
 @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class SQLCheckConstraint implements org.apache.thrift.TBase<SQLCheckConstraint, SQLCheckConstraint._Fields>, java.io.Serializable, Cloneable, Comparable<SQLCheckConstraint> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SQLCheckConstraint");
 
-  private static final org.apache.thrift.protocol.TField TABLE_DB_FIELD_DESC = new org.apache.thrift.protocol.TField("table_db", org.apache.thrift.protocol.TType.STRING, (short)1);
-  private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("table_name", org.apache.thrift.protocol.TType.STRING, (short)2);
-  private static final org.apache.thrift.protocol.TField COLUMN_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("column_name", org.apache.thrift.protocol.TType.STRING, (short)3);
-  private static final org.apache.thrift.protocol.TField CHECK_EXPRESSION_FIELD_DESC = new org.apache.thrift.protocol.TField("check_expression", org.apache.thrift.protocol.TType.STRING, (short)4);
-  private static final org.apache.thrift.protocol.TField DC_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dc_name", org.apache.thrift.protocol.TType.STRING, (short)5);
-  private static final org.apache.thrift.protocol.TField ENABLE_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("enable_cstr", org.apache.thrift.protocol.TType.BOOL, (short)6);
-  private static final org.apache.thrift.protocol.TField VALIDATE_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("validate_cstr", org.apache.thrift.protocol.TType.BOOL, (short)7);
-  private static final org.apache.thrift.protocol.TField RELY_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("rely_cstr", org.apache.thrift.protocol.TType.BOOL, (short)8);
+  private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField TABLE_DB_FIELD_DESC = new org.apache.thrift.protocol.TField("table_db", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("table_name", org.apache.thrift.protocol.TType.STRING, (short)3);
+  private static final org.apache.thrift.protocol.TField COLUMN_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("column_name", org.apache.thrift.protocol.TType.STRING, (short)4);
+  private static final org.apache.thrift.protocol.TField CHECK_EXPRESSION_FIELD_DESC = new org.apache.thrift.protocol.TField("check_expression", org.apache.thrift.protocol.TType.STRING, (short)5);
+  private static final org.apache.thrift.protocol.TField DC_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dc_name", org.apache.thrift.protocol.TType.STRING, (short)6);
+  private static final org.apache.thrift.protocol.TField ENABLE_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("enable_cstr", org.apache.thrift.protocol.TType.BOOL, (short)7);
+  private static final org.apache.thrift.protocol.TField VALIDATE_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("validate_cstr", org.apache.thrift.protocol.TType.BOOL, (short)8);
+  private static final org.apache.thrift.protocol.TField RELY_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("rely_cstr", org.apache.thrift.protocol.TType.BOOL, (short)9);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -53,6 +54,7 @@ import org.slf4j.LoggerFactory;
     schemes.put(TupleScheme.class, new SQLCheckConstraintTupleSchemeFactory());
   }
 
+  private String catName; // required
   private String table_db; // required
   private String table_name; // required
   private String column_name; // required
@@ -64,14 +66,15 @@ import org.slf4j.LoggerFactory;
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-    TABLE_DB((short)1, "table_db"),
-    TABLE_NAME((short)2, "table_name"),
-    COLUMN_NAME((short)3, "column_name"),
-    CHECK_EXPRESSION((short)4, "check_expression"),
-    DC_NAME((short)5, "dc_name"),
-    ENABLE_CSTR((short)6, "enable_cstr"),
-    VALIDATE_CSTR((short)7, "validate_cstr"),
-    RELY_CSTR((short)8, "rely_cstr");
+    CAT_NAME((short)1, "catName"),
+    TABLE_DB((short)2, "table_db"),
+    TABLE_NAME((short)3, "table_name"),
+    COLUMN_NAME((short)4, "column_name"),
+    CHECK_EXPRESSION((short)5, "check_expression"),
+    DC_NAME((short)6, "dc_name"),
+    ENABLE_CSTR((short)7, "enable_cstr"),
+    VALIDATE_CSTR((short)8, "validate_cstr"),
+    RELY_CSTR((short)9, "rely_cstr");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -86,21 +89,23 @@ import org.slf4j.LoggerFactory;
      */
     public static _Fields findByThriftId(int fieldId) {
       switch(fieldId) {
-        case 1: // TABLE_DB
+        case 1: // CAT_NAME
+          return CAT_NAME;
+        case 2: // TABLE_DB
           return TABLE_DB;
-        case 2: // TABLE_NAME
+        case 3: // TABLE_NAME
           return TABLE_NAME;
-        case 3: // COLUMN_NAME
+        case 4: // COLUMN_NAME
           return COLUMN_NAME;
-        case 4: // CHECK_EXPRESSION
+        case 5: // CHECK_EXPRESSION
           return CHECK_EXPRESSION;
-        case 5: // DC_NAME
+        case 6: // DC_NAME
           return DC_NAME;
-        case 6: // ENABLE_CSTR
+        case 7: // ENABLE_CSTR
           return ENABLE_CSTR;
-        case 7: // VALIDATE_CSTR
+        case 8: // VALIDATE_CSTR
           return VALIDATE_CSTR;
-        case 8: // RELY_CSTR
+        case 9: // RELY_CSTR
           return RELY_CSTR;
         default:
           return null;
@@ -149,6 +154,8 @@ import org.slf4j.LoggerFactory;
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.TABLE_DB, new org.apache.thrift.meta_data.FieldMetaData("table_db", org.apache.thrift.TFieldRequirementType.DEFAULT, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("table_name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -173,6 +180,7 @@ import org.slf4j.LoggerFactory;
   }
 
   public SQLCheckConstraint(
+    String catName,
     String table_db,
     String table_name,
     String column_name,
@@ -183,6 +191,7 @@ import org.slf4j.LoggerFactory;
     boolean rely_cstr)
   {
     this();
+    this.catName = catName;
     this.table_db = table_db;
     this.table_name = table_name;
     this.column_name = column_name;
@@ -201,6 +210,9 @@ import org.slf4j.LoggerFactory;
    */
   public SQLCheckConstraint(SQLCheckConstraint other) {
     __isset_bitfield = other.__isset_bitfield;
+    if (other.isSetCatName()) {
+      this.catName = other.catName;
+    }
     if (other.isSetTable_db()) {
       this.table_db = other.table_db;
     }
@@ -227,6 +239,7 @@ import org.slf4j.LoggerFactory;
 
   @Override
   public void clear() {
+    this.catName = null;
     this.table_db = null;
     this.table_name = null;
     this.column_name = null;
@@ -240,6 +253,29 @@ import org.slf4j.LoggerFactory;
     this.rely_cstr = false;
   }
 
+  public String getCatName() {
+    return this.catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
+  public void unsetCatName() {
+    this.catName = null;
+  }
+
+  /** Returns true if field catName is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatName() {
+    return this.catName != null;
+  }
+
+  public void setCatNameIsSet(boolean value) {
+    if (!value) {
+      this.catName = null;
+    }
+  }
+
   public String getTable_db() {
     return this.table_db;
   }
@@ -423,6 +459,14 @@ import org.slf4j.LoggerFactory;
 
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
+    case CAT_NAME:
+      if (value == null) {
+        unsetCatName();
+      } else {
+        setCatName((String)value);
+      }
+      break;
+
     case TABLE_DB:
       if (value == null) {
         unsetTable_db();
@@ -492,6 +536,9 @@ import org.slf4j.LoggerFactory;
 
   public Object getFieldValue(_Fields field) {
     switch (field) {
+    case CAT_NAME:
+      return getCatName();
+
     case TABLE_DB:
       return getTable_db();
 
@@ -527,6 +574,8 @@ import org.slf4j.LoggerFactory;
     }
 
     switch (field) {
+    case CAT_NAME:
+      return isSetCatName();
     case TABLE_DB:
       return isSetTable_db();
     case TABLE_NAME:
@@ -560,6 +609,15 @@ import org.slf4j.LoggerFactory;
     if (that == null)
       return false;
 
+    boolean this_present_catName = true && this.isSetCatName();
+    boolean that_present_catName = true && that.isSetCatName();
+    if (this_present_catName || that_present_catName) {
+      if (!(this_present_catName && that_present_catName))
+        return false;
+      if (!this.catName.equals(that.catName))
+        return false;
+    }
+
     boolean this_present_table_db = true && this.isSetTable_db();
     boolean that_present_table_db = true && that.isSetTable_db();
     if (this_present_table_db || that_present_table_db) {
@@ -639,6 +697,11 @@ import org.slf4j.LoggerFactory;
   public int hashCode() {
     List<Object> list = new ArrayList<Object>();
 
+    boolean present_catName = true && (isSetCatName());
+    list.add(present_catName);
+    if (present_catName)
+      list.add(catName);
+
     boolean present_table_db = true && (isSetTable_db());
     list.add(present_table_db);
     if (present_table_db)
@@ -690,6 +753,16 @@ import org.slf4j.LoggerFactory;
 
     int lastComparison = 0;
 
+    lastComparison = Boolean.valueOf(isSetCatName()).compareTo(other.isSetCatName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catName, other.catName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     lastComparison = Boolean.valueOf(isSetTable_db()).compareTo(other.isSetTable_db());
     if (lastComparison != 0) {
       return lastComparison;
@@ -790,6 +863,14 @@ import org.slf4j.LoggerFactory;
     StringBuilder sb = new StringBuilder("SQLCheckConstraint(");
     boolean first = true;
 
+    sb.append("catName:");
+    if (this.catName == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.catName);
+    }
+    first = false;
+    if (!first) sb.append(", ");
     sb.append("table_db:");
     if (this.table_db == null) {
       sb.append("null");
@@ -886,7 +967,15 @@ import org.slf4j.LoggerFactory;
           break;
         }
         switch (schemeField.id) {
-          case 1: // TABLE_DB
+          case 1: // CAT_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.catName = iprot.readString();
+              struct.setCatNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // TABLE_DB
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.table_db = iprot.readString();
               struct.setTable_dbIsSet(true);
@@ -894,7 +983,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 2: // TABLE_NAME
+          case 3: // TABLE_NAME
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.table_name = iprot.readString();
               struct.setTable_nameIsSet(true);
@@ -902,7 +991,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 3: // COLUMN_NAME
+          case 4: // COLUMN_NAME
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.column_name = iprot.readString();
               struct.setColumn_nameIsSet(true);
@@ -910,7 +999,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 4: // CHECK_EXPRESSION
+          case 5: // CHECK_EXPRESSION
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.check_expression = iprot.readString();
               struct.setCheck_expressionIsSet(true);
@@ -918,7 +1007,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 5: // DC_NAME
+          case 6: // DC_NAME
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.dc_name = iprot.readString();
               struct.setDc_nameIsSet(true);
@@ -926,7 +1015,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 6: // ENABLE_CSTR
+          case 7: // ENABLE_CSTR
             if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
               struct.enable_cstr = iprot.readBool();
               struct.setEnable_cstrIsSet(true);
@@ -934,7 +1023,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 7: // VALIDATE_CSTR
+          case 8: // VALIDATE_CSTR
             if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
               struct.validate_cstr = iprot.readBool();
               struct.setValidate_cstrIsSet(true);
@@ -942,7 +1031,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 8: // RELY_CSTR
+          case 9: // RELY_CSTR
             if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
               struct.rely_cstr = iprot.readBool();
               struct.setRely_cstrIsSet(true);
@@ -963,6 +1052,11 @@ import org.slf4j.LoggerFactory;
       struct.validate();
 
       oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.catName != null) {
+        oprot.writeFieldBegin(CAT_NAME_FIELD_DESC);
+        oprot.writeString(struct.catName);
+        oprot.writeFieldEnd();
+      }
       if (struct.table_db != null) {
         oprot.writeFieldBegin(TABLE_DB_FIELD_DESC);
         oprot.writeString(struct.table_db);
@@ -1015,31 +1109,37 @@ import org.slf4j.LoggerFactory;
     public void write(org.apache.thrift.protocol.TProtocol prot, SQLCheckConstraint struct) throws org.apache.thrift.TException {
       TTupleProtocol oprot = (TTupleProtocol) prot;
       BitSet optionals = new BitSet();
-      if (struct.isSetTable_db()) {
+      if (struct.isSetCatName()) {
         optionals.set(0);
       }
-      if (struct.isSetTable_name()) {
+      if (struct.isSetTable_db()) {
         optionals.set(1);
       }
-      if (struct.isSetColumn_name()) {
+      if (struct.isSetTable_name()) {
         optionals.set(2);
       }
-      if (struct.isSetCheck_expression()) {
+      if (struct.isSetColumn_name()) {
         optionals.set(3);
       }
-      if (struct.isSetDc_name()) {
+      if (struct.isSetCheck_expression()) {
         optionals.set(4);
       }
-      if (struct.isSetEnable_cstr()) {
+      if (struct.isSetDc_name()) {
         optionals.set(5);
       }
-      if (struct.isSetValidate_cstr()) {
+      if (struct.isSetEnable_cstr()) {
         optionals.set(6);
       }
-      if (struct.isSetRely_cstr()) {
+      if (struct.isSetValidate_cstr()) {
         optionals.set(7);
       }
-      oprot.writeBitSet(optionals, 8);
+      if (struct.isSetRely_cstr()) {
+        optionals.set(8);
+      }
+      oprot.writeBitSet(optionals, 9);
+      if (struct.isSetCatName()) {
+        oprot.writeString(struct.catName);
+      }
       if (struct.isSetTable_db()) {
         oprot.writeString(struct.table_db);
       }
@@ -1069,36 +1169,40 @@ import org.slf4j.LoggerFactory;
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, SQLCheckConstraint struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
-      BitSet incoming = iprot.readBitSet(8);
+      BitSet incoming = iprot.readBitSet(9);
       if (incoming.get(0)) {
+        struct.catName = iprot.readString();
+        struct.setCatNameIsSet(true);
+      }
+      if (incoming.get(1)) {
         struct.table_db = iprot.readString();
         struct.setTable_dbIsSet(true);
       }
-      if (incoming.get(1)) {
+      if (incoming.get(2)) {
         struct.table_name = iprot.readString();
         struct.setTable_nameIsSet(true);
       }
-      if (incoming.get(2)) {
+      if (incoming.get(3)) {
         struct.column_name = iprot.readString();
         struct.setColumn_nameIsSet(true);
       }
-      if (incoming.get(3)) {
+      if (incoming.get(4)) {
         struct.check_expression = iprot.readString();
         struct.setCheck_expressionIsSet(true);
       }
-      if (incoming.get(4)) {
+      if (incoming.get(5)) {
         struct.dc_name = iprot.readString();
         struct.setDc_nameIsSet(true);
       }
-      if (incoming.get(5)) {
+      if (incoming.get(6)) {
         struct.enable_cstr = iprot.readBool();
         struct.setEnable_cstrIsSet(true);
       }
-      if (incoming.get(6)) {
+      if (incoming.get(7)) {
         struct.validate_cstr = iprot.readBool();
         struct.setValidate_cstrIsSet(true);
       }
-      if (incoming.get(7)) {
+      if (incoming.get(8)) {
         struct.rely_cstr = iprot.readBool();
         struct.setRely_cstrIsSet(true);
       }


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
index 1b3ebcf..d39690f 100755
--- a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
+++ b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
@@ -26,6 +26,10 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   print('Functions:')
   print('  string getMetaConf(string key)')
   print('  void setMetaConf(string key, string value)')
+  print('  void create_catalog(CreateCatalogRequest catalog)')
+  print('  GetCatalogResponse get_catalog(GetCatalogRequest catName)')
+  print('  GetCatalogsResponse get_catalogs()')
+  print('  void drop_catalog(DropCatalogRequest catName)')
   print('  void create_database(Database database)')
   print('  Database get_database(string name)')
   print('  void drop_database(string name, bool deleteData, bool cascade)')
@@ -63,7 +67,7 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   print('  GetTableResult get_table_req(GetTableRequest req)')
   print('  GetTablesResult get_table_objects_by_name_req(GetTablesRequest req)')
   print('   get_materialization_invalidation_info(string dbname,  tbl_names)')
-  print('  void update_creation_metadata(string dbname, string tbl_name, CreationMetadata creation_metadata)')
+  print('  void update_creation_metadata(string catName, string dbname, string tbl_name, CreationMetadata creation_metadata)')
   print('   get_table_names_by_filter(string dbname, string filter, i16 max_tables)')
   print('  void alter_table(string dbname, string tbl_name, Table new_tbl)')
   print('  void alter_table_with_environment_context(string dbname, string tbl_name, Table new_tbl, EnvironmentContext environment_context)')
@@ -301,6 +305,30 @@ elif cmd == 'setMetaConf':
     sys.exit(1)
   pp.pprint(client.setMetaConf(args[0],args[1],))
 
+elif cmd == 'create_catalog':
+  if len(args) != 1:
+    print('create_catalog requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.create_catalog(eval(args[0]),))
+
+elif cmd == 'get_catalog':
+  if len(args) != 1:
+    print('get_catalog requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.get_catalog(eval(args[0]),))
+
+elif cmd == 'get_catalogs':
+  if len(args) != 0:
+    print('get_catalogs requires 0 args')
+    sys.exit(1)
+  pp.pprint(client.get_catalogs())
+
+elif cmd == 'drop_catalog':
+  if len(args) != 1:
+    print('drop_catalog requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.drop_catalog(eval(args[0]),))
+
 elif cmd == 'create_database':
   if len(args) != 1:
     print('create_database requires 1 args')
@@ -524,10 +552,10 @@ elif cmd == 'get_materialization_invalidation_info':
   pp.pprint(client.get_materialization_invalidation_info(args[0],eval(args[1]),))
 
 elif cmd == 'update_creation_metadata':
-  if len(args) != 3:
-    print('update_creation_metadata requires 3 args')
+  if len(args) != 4:
+    print('update_creation_metadata requires 4 args')
     sys.exit(1)
-  pp.pprint(client.update_creation_metadata(args[0],args[1],eval(args[2]),))
+  pp.pprint(client.update_creation_metadata(args[0],args[1],args[2],eval(args[3]),))
 
 elif cmd == 'get_table_names_by_filter':
   if len(args) != 3:


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

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

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

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

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

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

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


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCachedStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCachedStore.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCachedStore.java
index a72fc0b..d451f96 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCachedStore.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCachedStore.java
@@ -28,9 +28,11 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.ThreadFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.common.ndv.hll.HyperLogLog;
+import org.apache.hadoop.hive.metastore.HiveMetaStore;
 import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
 import org.apache.hadoop.hive.metastore.ObjectStore;
 import org.apache.hadoop.hive.metastore.TableType;
+import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
 import org.apache.hadoop.hive.metastore.api.BooleanColumnStatsData;
@@ -46,6 +48,7 @@ import org.apache.hadoop.hive.metastore.api.PrincipalType;
 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.client.builder.DatabaseBuilder;
 import org.apache.hadoop.hive.metastore.columnstats.cache.LongColumnStatsDataInspector;
 import org.apache.hadoop.hive.metastore.columnstats.cache.StringColumnStatsDataInspector;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
@@ -54,16 +57,19 @@ import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
+
 @Category(MetastoreCheckinTest.class)
 public class TestCachedStore {
 
   private ObjectStore objectStore;
   private CachedStore cachedStore;
   private SharedCache sharedCache;
+  private Configuration conf;
 
   @Before
   public void setUp() throws Exception {
-    Configuration conf = MetastoreConf.newMetastoreConf();
+    conf = MetastoreConf.newMetastoreConf();
     MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.HIVE_IN_TEST, true);
     MetaStoreTestUtils.setConfForStandloneMode(conf);
     objectStore = new ObjectStore();
@@ -76,6 +82,9 @@ public class TestCachedStore {
     sharedCache.getDatabaseCache().clear();
     sharedCache.getTableCache().clear();
     sharedCache.getSdCache().clear();
+
+    // Create the 'hive' catalog
+    HiveMetaStore.HMSHandler.createDefaultCatalog(objectStore, new Warehouse(conf));
   }
 
   /**********************************************************************************************
@@ -89,67 +98,67 @@ public class TestCachedStore {
     String dbOwner = "user1";
     Database db = createTestDb(dbName, dbOwner);
     objectStore.createDatabase(db);
-    db = objectStore.getDatabase(dbName);
+    db = objectStore.getDatabase(DEFAULT_CATALOG_NAME, dbName);
     // Prewarm CachedStore
     CachedStore.setCachePrewarmedState(false);
     CachedStore.prewarm(objectStore);
 
     // Read database via CachedStore
-    Database dbRead = cachedStore.getDatabase(dbName);
+    Database dbRead = cachedStore.getDatabase(DEFAULT_CATALOG_NAME, dbName);
     Assert.assertEquals(db, dbRead);
 
     // Add another db via CachedStore
     final String dbName1 = "testDatabaseOps1";
     Database db1 = createTestDb(dbName1, dbOwner);
     cachedStore.createDatabase(db1);
-    db1 = cachedStore.getDatabase(dbName1);
+    db1 = cachedStore.getDatabase(DEFAULT_CATALOG_NAME, dbName1);
 
     // Read db via ObjectStore
-    dbRead = objectStore.getDatabase(dbName1);
+    dbRead = objectStore.getDatabase(DEFAULT_CATALOG_NAME, dbName1);
     Assert.assertEquals(db1, dbRead);
 
     // Alter the db via CachedStore (can only alter owner or parameters)
     dbOwner = "user2";
     db = new Database(db);
     db.setOwnerName(dbOwner);
-    cachedStore.alterDatabase(dbName, db);
-    db = cachedStore.getDatabase(dbName);
+    cachedStore.alterDatabase(DEFAULT_CATALOG_NAME, dbName, db);
+    db = cachedStore.getDatabase(DEFAULT_CATALOG_NAME, dbName);
 
     // Read db via ObjectStore
-    dbRead = objectStore.getDatabase(dbName);
+    dbRead = objectStore.getDatabase(DEFAULT_CATALOG_NAME, dbName);
     Assert.assertEquals(db, dbRead);
 
     // Add another db via ObjectStore
     final String dbName2 = "testDatabaseOps2";
     Database db2 = createTestDb(dbName2, dbOwner);
     objectStore.createDatabase(db2);
-    db2 = objectStore.getDatabase(dbName2);
+    db2 = objectStore.getDatabase(DEFAULT_CATALOG_NAME, dbName2);
 
     // Alter db "testDatabaseOps" via ObjectStore
     dbOwner = "user1";
     db = new Database(db);
     db.setOwnerName(dbOwner);
-    objectStore.alterDatabase(dbName, db);
-    db = objectStore.getDatabase(dbName);
+    objectStore.alterDatabase(DEFAULT_CATALOG_NAME, dbName, db);
+    db = objectStore.getDatabase(DEFAULT_CATALOG_NAME, dbName);
 
     // Drop db "testDatabaseOps1" via ObjectStore
-    objectStore.dropDatabase(dbName1);
+    objectStore.dropDatabase(DEFAULT_CATALOG_NAME, dbName1);
 
     // We update twice to accurately detect if cache is dirty or not
     updateCache(cachedStore);
     updateCache(cachedStore);
 
     // Read the newly added db via CachedStore
-    dbRead = cachedStore.getDatabase(dbName2);
+    dbRead = cachedStore.getDatabase(DEFAULT_CATALOG_NAME, dbName2);
     Assert.assertEquals(db2, dbRead);
 
     // Read the altered db via CachedStore (altered user from "user2" to "user1")
-    dbRead = cachedStore.getDatabase(dbName);
+    dbRead = cachedStore.getDatabase(DEFAULT_CATALOG_NAME, dbName);
     Assert.assertEquals(db, dbRead);
 
     // Try to read the dropped db after cache update
     try {
-      dbRead = cachedStore.getDatabase(dbName1);
+      dbRead = cachedStore.getDatabase(DEFAULT_CATALOG_NAME, dbName1);
       Assert.fail("The database: " + dbName1
           + " should have been removed from the cache after running the update service");
     } catch (NoSuchObjectException e) {
@@ -157,8 +166,8 @@ public class TestCachedStore {
     }
 
     // Clean up
-    objectStore.dropDatabase(dbName);
-    objectStore.dropDatabase(dbName2);
+    objectStore.dropDatabase(DEFAULT_CATALOG_NAME, dbName);
+    objectStore.dropDatabase(DEFAULT_CATALOG_NAME, dbName2);
     sharedCache.getDatabaseCache().clear();
     sharedCache.getTableCache().clear();
     sharedCache.getSdCache().clear();
@@ -171,7 +180,7 @@ public class TestCachedStore {
     String dbOwner = "user1";
     Database db = createTestDb(dbName, dbOwner);
     objectStore.createDatabase(db);
-    db = objectStore.getDatabase(dbName);
+    db = objectStore.getDatabase(DEFAULT_CATALOG_NAME, dbName);
 
     // Add a table via ObjectStore
     String tblName = "tbl";
@@ -184,16 +193,16 @@ public class TestCachedStore {
     List<FieldSchema> ptnCols = new ArrayList<FieldSchema>();
     Table tbl = createTestTbl(dbName, tblName, tblOwner, cols, ptnCols);
     objectStore.createTable(tbl);
-    tbl = objectStore.getTable(dbName, tblName);
+    tbl = objectStore.getTable(DEFAULT_CATALOG_NAME, dbName, tblName);
 
     // Prewarm CachedStore
     CachedStore.setCachePrewarmedState(false);
     CachedStore.prewarm(objectStore);
 
     // Read database, table via CachedStore
-    Database dbRead= cachedStore.getDatabase(dbName);
+    Database dbRead= cachedStore.getDatabase(DEFAULT_CATALOG_NAME, dbName);
     Assert.assertEquals(db, dbRead);
-    Table tblRead = cachedStore.getTable(dbName, tblName);
+    Table tblRead = cachedStore.getTable(DEFAULT_CATALOG_NAME, dbName, tblName);
     Assert.assertEquals(tbl, tblRead);
 
     // Add a new table via CachedStore
@@ -201,10 +210,10 @@ public class TestCachedStore {
     Table tbl1 = new Table(tbl);
     tbl1.setTableName(tblName1);
     cachedStore.createTable(tbl1);
-    tbl1 = cachedStore.getTable(dbName, tblName1);
+    tbl1 = cachedStore.getTable(DEFAULT_CATALOG_NAME, dbName, tblName1);
 
     // Read via object store
-    tblRead = objectStore.getTable(dbName, tblName1);
+    tblRead = objectStore.getTable(DEFAULT_CATALOG_NAME, dbName, tblName1);
     Assert.assertEquals(tbl1, tblRead);
 
     // Add a new table via ObjectStore
@@ -212,43 +221,43 @@ public class TestCachedStore {
     Table tbl2 = new Table(tbl);
     tbl2.setTableName(tblName2);
     objectStore.createTable(tbl2);
-    tbl2 = objectStore.getTable(dbName, tblName2);
+    tbl2 = objectStore.getTable(DEFAULT_CATALOG_NAME, dbName, tblName2);
 
     // Alter table "tbl" via ObjectStore
     tblOwner = "user2";
     tbl.setOwner(tblOwner);
-    objectStore.alterTable(dbName, tblName, tbl);
-    tbl = objectStore.getTable(dbName, tblName);
+    objectStore.alterTable(DEFAULT_CATALOG_NAME, dbName, tblName, tbl);
+    tbl = objectStore.getTable(DEFAULT_CATALOG_NAME, dbName, tblName);
 
     // Drop table "tbl1" via ObjectStore
-    objectStore.dropTable(dbName, tblName1);
+    objectStore.dropTable(DEFAULT_CATALOG_NAME, dbName, tblName1);
 
     // We update twice to accurately detect if cache is dirty or not
     updateCache(cachedStore);
     updateCache(cachedStore);
 
     // Read "tbl2" via CachedStore
-    tblRead = cachedStore.getTable(dbName, tblName2);
+    tblRead = cachedStore.getTable(DEFAULT_CATALOG_NAME, dbName, tblName2);
     Assert.assertEquals(tbl2, tblRead);
 
     // Read the altered "tbl" via CachedStore
-    tblRead = cachedStore.getTable(dbName, tblName);
+    tblRead = cachedStore.getTable(DEFAULT_CATALOG_NAME, dbName, tblName);
     Assert.assertEquals(tbl, tblRead);
 
     // Try to read the dropped "tbl1" via CachedStore (should throw exception)
-    tblRead = cachedStore.getTable(dbName, tblName1);
+    tblRead = cachedStore.getTable(DEFAULT_CATALOG_NAME, dbName, tblName1);
     Assert.assertNull(tblRead);
 
     // Should return "tbl" and "tbl2"
-    List<String> tblNames = cachedStore.getTables(dbName, "*");
+    List<String> tblNames = cachedStore.getTables(DEFAULT_CATALOG_NAME, dbName, "*");
     Assert.assertTrue(tblNames.contains(tblName));
     Assert.assertTrue(!tblNames.contains(tblName1));
     Assert.assertTrue(tblNames.contains(tblName2));
 
     // Clean up
-    objectStore.dropTable(dbName, tblName);
-    objectStore.dropTable(dbName, tblName2);
-    objectStore.dropDatabase(dbName);
+    objectStore.dropTable(DEFAULT_CATALOG_NAME, dbName, tblName);
+    objectStore.dropTable(DEFAULT_CATALOG_NAME, dbName, tblName2);
+    objectStore.dropDatabase(DEFAULT_CATALOG_NAME, dbName);
     sharedCache.getDatabaseCache().clear();
     sharedCache.getTableCache().clear();
     sharedCache.getSdCache().clear();
@@ -261,7 +270,7 @@ public class TestCachedStore {
     String dbOwner = "user1";
     Database db = createTestDb(dbName, dbOwner);
     objectStore.createDatabase(db);
-    db = objectStore.getDatabase(dbName);
+    db = objectStore.getDatabase(DEFAULT_CATALOG_NAME, dbName);
 
     // Add a table via ObjectStore
     String tblName = "tbl";
@@ -276,76 +285,81 @@ public class TestCachedStore {
     ptnCols.add(ptnCol1);
     Table tbl = createTestTbl(dbName, tblName, tblOwner, cols, ptnCols);
     objectStore.createTable(tbl);
-    tbl = objectStore.getTable(dbName, tblName);
+    tbl = objectStore.getTable(DEFAULT_CATALOG_NAME, dbName, tblName);
 
     final String ptnColVal1 = "aaa";
     Map<String, String> partParams = new HashMap<String, String>();
     Partition ptn1 =
         new Partition(Arrays.asList(ptnColVal1), dbName, tblName, 0, 0, tbl.getSd(), partParams);
+    ptn1.setCatName(DEFAULT_CATALOG_NAME);
     objectStore.addPartition(ptn1);
-    ptn1 = objectStore.getPartition(dbName, tblName, Arrays.asList(ptnColVal1));
+    ptn1 = objectStore.getPartition(DEFAULT_CATALOG_NAME, dbName, tblName, Arrays.asList(ptnColVal1));
+    ptn1.setCatName(DEFAULT_CATALOG_NAME);
     final String ptnColVal2 = "bbb";
     Partition ptn2 =
         new Partition(Arrays.asList(ptnColVal2), dbName, tblName, 0, 0, tbl.getSd(), partParams);
+    ptn2.setCatName(DEFAULT_CATALOG_NAME);
     objectStore.addPartition(ptn2);
-    ptn2 = objectStore.getPartition(dbName, tblName, Arrays.asList(ptnColVal2));
+    ptn2 = objectStore.getPartition(DEFAULT_CATALOG_NAME, dbName, tblName, Arrays.asList(ptnColVal2));
 
     // Prewarm CachedStore
     CachedStore.setCachePrewarmedState(false);
     CachedStore.prewarm(objectStore);
 
     // Read database, table, partition via CachedStore
-    Database dbRead = cachedStore.getDatabase(dbName);
+    Database dbRead = cachedStore.getDatabase(DEFAULT_CATALOG_NAME, dbName);
     Assert.assertEquals(db, dbRead);
-    Table tblRead = cachedStore.getTable(dbName, tblName);
+    Table tblRead = cachedStore.getTable(DEFAULT_CATALOG_NAME, dbName, tblName);
     Assert.assertEquals(tbl, tblRead);
-    Partition ptn1Read = cachedStore.getPartition(dbName, tblName, Arrays.asList(ptnColVal1));
+    Partition ptn1Read = cachedStore.getPartition(DEFAULT_CATALOG_NAME, dbName, tblName, Arrays.asList(ptnColVal1));
     Assert.assertEquals(ptn1, ptn1Read);
-    Partition ptn2Read = cachedStore.getPartition(dbName, tblName, Arrays.asList(ptnColVal2));
+    Partition ptn2Read = cachedStore.getPartition(DEFAULT_CATALOG_NAME, dbName, tblName, Arrays.asList(ptnColVal2));
     Assert.assertEquals(ptn2, ptn2Read);
 
     // Add a new partition via ObjectStore
     final String ptnColVal3 = "ccc";
     Partition ptn3 =
         new Partition(Arrays.asList(ptnColVal3), dbName, tblName, 0, 0, tbl.getSd(), partParams);
+    ptn3.setCatName(DEFAULT_CATALOG_NAME);
     objectStore.addPartition(ptn3);
-    ptn3 = objectStore.getPartition(dbName, tblName, Arrays.asList(ptnColVal3));
+    ptn3 = objectStore.getPartition(DEFAULT_CATALOG_NAME, dbName, tblName, Arrays.asList(ptnColVal3));
 
     // Alter an existing partition ("aaa") via ObjectStore
     final String ptnColVal1Alt = "aaaAlt";
     Partition ptn1Atl =
         new Partition(Arrays.asList(ptnColVal1Alt), dbName, tblName, 0, 0, tbl.getSd(), partParams);
-    objectStore.alterPartition(dbName, tblName, Arrays.asList(ptnColVal1), ptn1Atl);
-    ptn1Atl = objectStore.getPartition(dbName, tblName, Arrays.asList(ptnColVal1Alt));
+    ptn1Atl.setCatName(DEFAULT_CATALOG_NAME);
+    objectStore.alterPartition(DEFAULT_CATALOG_NAME, dbName, tblName, Arrays.asList(ptnColVal1), ptn1Atl);
+    ptn1Atl = objectStore.getPartition(DEFAULT_CATALOG_NAME, dbName, tblName, Arrays.asList(ptnColVal1Alt));
 
     // Drop an existing partition ("bbb") via ObjectStore
-    objectStore.dropPartition(dbName, tblName, Arrays.asList(ptnColVal2));
+    objectStore.dropPartition(DEFAULT_CATALOG_NAME, dbName, tblName, Arrays.asList(ptnColVal2));
 
     // We update twice to accurately detect if cache is dirty or not
     updateCache(cachedStore);
     updateCache(cachedStore);
 
     // Read the newly added partition via CachedStore
-    Partition ptnRead = cachedStore.getPartition(dbName, tblName, Arrays.asList(ptnColVal3));
+    Partition ptnRead = cachedStore.getPartition(DEFAULT_CATALOG_NAME, dbName, tblName, Arrays.asList(ptnColVal3));
     Assert.assertEquals(ptn3, ptnRead);
 
     // Read the altered partition via CachedStore
-    ptnRead = cachedStore.getPartition(dbName, tblName, Arrays.asList(ptnColVal1Alt));
+    ptnRead = cachedStore.getPartition(DEFAULT_CATALOG_NAME, dbName, tblName, Arrays.asList(ptnColVal1Alt));
     Assert.assertEquals(ptn1Atl, ptnRead);
 
     // Try to read the dropped partition via CachedStore
     try {
-      ptnRead = cachedStore.getPartition(dbName, tblName, Arrays.asList(ptnColVal2));
+      ptnRead = cachedStore.getPartition(DEFAULT_CATALOG_NAME, dbName, tblName, Arrays.asList(ptnColVal2));
       Assert.fail("The partition: " + ptnColVal2
           + " should have been removed from the cache after running the update service");
     } catch (NoSuchObjectException e) {
       // Expected
     }
     // Clean up
-    objectStore.dropPartition(dbName, tblName, Arrays.asList(ptnColVal1Alt));
-    objectStore.dropPartition(dbName, tblName, Arrays.asList(ptnColVal3));
-    objectStore.dropTable(dbName, tblName);
-    objectStore.dropDatabase(dbName);
+    objectStore.dropPartition(DEFAULT_CATALOG_NAME, dbName, tblName, Arrays.asList(ptnColVal1Alt));
+    objectStore.dropPartition(DEFAULT_CATALOG_NAME, dbName, tblName, Arrays.asList(ptnColVal3));
+    objectStore.dropTable(DEFAULT_CATALOG_NAME, dbName, tblName);
+    objectStore.dropDatabase(DEFAULT_CATALOG_NAME, dbName);
     sharedCache.getDatabaseCache().clear();
     sharedCache.getTableCache().clear();
     sharedCache.getSdCache().clear();
@@ -358,7 +372,7 @@ public class TestCachedStore {
     String dbOwner = "user1";
     Database db = createTestDb(dbName, dbOwner);
     objectStore.createDatabase(db);
-    db = objectStore.getDatabase(dbName);
+    db = objectStore.getDatabase(DEFAULT_CATALOG_NAME, dbName);
 
     // Add a table via ObjectStore
     final String tblName = "tbl";
@@ -389,7 +403,7 @@ public class TestCachedStore {
     ptnCols.add(ptnCol1);
     Table tbl = createTestTbl(dbName, tblName, tblOwner, cols, ptnCols);
     objectStore.createTable(tbl);
-    tbl = objectStore.getTable(dbName, tblName);
+    tbl = objectStore.getTable(DEFAULT_CATALOG_NAME, dbName, tblName);
 
     // Add ColumnStatistics for tbl to metastore DB via ObjectStore
     ColumnStatistics stats = new ColumnStatistics();
@@ -440,13 +454,13 @@ public class TestCachedStore {
 
     // Read table stats via CachedStore
     ColumnStatistics newStats =
-        cachedStore.getTableColumnStatistics(dbName, tblName,
+        cachedStore.getTableColumnStatistics(DEFAULT_CATALOG_NAME, dbName, tblName,
             Arrays.asList(col1.getName(), col2.getName(), col3.getName()));
     Assert.assertEquals(stats, newStats);
 
     // Clean up
-    objectStore.dropTable(dbName, tblName);
-    objectStore.dropDatabase(dbName);
+    objectStore.dropTable(DEFAULT_CATALOG_NAME, dbName, tblName);
+    objectStore.dropDatabase(DEFAULT_CATALOG_NAME, dbName);
     sharedCache.getDatabaseCache().clear();
     sharedCache.getTableCache().clear();
     sharedCache.getSdCache().clear();
@@ -466,11 +480,11 @@ public class TestCachedStore {
     sharedCache.addDatabaseToCache(db2);
     sharedCache.addDatabaseToCache(db3);
     Assert.assertEquals(sharedCache.getCachedDatabaseCount(), 3);
-    sharedCache.alterDatabaseInCache("db1", newDb1);
+    sharedCache.alterDatabaseInCache(DEFAULT_CATALOG_NAME, "db1", newDb1);
     Assert.assertEquals(sharedCache.getCachedDatabaseCount(), 3);
-    sharedCache.removeDatabaseFromCache("db2");
+    sharedCache.removeDatabaseFromCache(DEFAULT_CATALOG_NAME, "db2");
     Assert.assertEquals(sharedCache.getCachedDatabaseCount(), 2);
-    List<String> dbs = sharedCache.listCachedDatabases();
+    List<String> dbs = sharedCache.listCachedDatabases(DEFAULT_CATALOG_NAME);
     Assert.assertEquals(dbs.size(), 2);
     Assert.assertTrue(dbs.contains("newdb1"));
     Assert.assertTrue(dbs.contains("db3"));
@@ -528,26 +542,26 @@ public class TestCachedStore {
     newTbl1.setSd(newSd1);
     newTbl1.setPartitionKeys(new ArrayList<>());
 
-    sharedCache.addTableToCache("db1", "tbl1", tbl1);
-    sharedCache.addTableToCache("db1", "tbl2", tbl2);
-    sharedCache.addTableToCache("db1", "tbl3", tbl3);
-    sharedCache.addTableToCache("db2", "tbl1", tbl1);
+    sharedCache.addTableToCache(DEFAULT_CATALOG_NAME, "db1", "tbl1", tbl1);
+    sharedCache.addTableToCache(DEFAULT_CATALOG_NAME, "db1", "tbl2", tbl2);
+    sharedCache.addTableToCache(DEFAULT_CATALOG_NAME, "db1", "tbl3", tbl3);
+    sharedCache.addTableToCache(DEFAULT_CATALOG_NAME, "db2", "tbl1", tbl1);
 
     Assert.assertEquals(sharedCache.getCachedTableCount(), 4);
     Assert.assertEquals(sharedCache.getSdCache().size(), 2);
 
-    Table t = sharedCache.getTableFromCache("db1", "tbl1");
+    Table t = sharedCache.getTableFromCache(DEFAULT_CATALOG_NAME, "db1", "tbl1");
     Assert.assertEquals(t.getSd().getLocation(), "loc1");
 
-    sharedCache.removeTableFromCache("db1", "tbl1");
+    sharedCache.removeTableFromCache(DEFAULT_CATALOG_NAME, "db1", "tbl1");
     Assert.assertEquals(sharedCache.getCachedTableCount(), 3);
     Assert.assertEquals(sharedCache.getSdCache().size(), 2);
 
-    sharedCache.alterTableInCache("db2", "tbl1", newTbl1);
+    sharedCache.alterTableInCache(DEFAULT_CATALOG_NAME, "db2", "tbl1", newTbl1);
     Assert.assertEquals(sharedCache.getCachedTableCount(), 3);
     Assert.assertEquals(sharedCache.getSdCache().size(), 3);
 
-    sharedCache.removeTableFromCache("db1", "tbl2");
+    sharedCache.removeTableFromCache(DEFAULT_CATALOG_NAME, "db1", "tbl2");
     Assert.assertEquals(sharedCache.getCachedTableCount(), 2);
     Assert.assertEquals(sharedCache.getSdCache().size(), 2);
   }
@@ -568,9 +582,9 @@ public class TestCachedStore {
     cols.add(col2);
     List<FieldSchema> ptnCols = new ArrayList<FieldSchema>();
     Table tbl1 = createTestTbl(dbName, tbl1Name, owner, cols, ptnCols);
-    sharedCache.addTableToCache(dbName, tbl1Name, tbl1);
+    sharedCache.addTableToCache(DEFAULT_CATALOG_NAME, dbName, tbl1Name, tbl1);
     Table tbl2 = createTestTbl(dbName, tbl2Name, owner, cols, ptnCols);
-    sharedCache.addTableToCache(dbName, tbl2Name, tbl2);
+    sharedCache.addTableToCache(DEFAULT_CATALOG_NAME, dbName, tbl2Name, tbl2);
 
     Partition part1 = new Partition();
     StorageDescriptor sd1 = new StorageDescriptor();
@@ -622,20 +636,20 @@ public class TestCachedStore {
     newPart1.setSd(newSd1);
     newPart1.setValues(Arrays.asList("201701"));
 
-    sharedCache.addPartitionToCache(dbName, tbl1Name, part1);
-    sharedCache.addPartitionToCache(dbName, tbl1Name, part2);
-    sharedCache.addPartitionToCache(dbName, tbl1Name, part3);
-    sharedCache.addPartitionToCache(dbName, tbl2Name, part1);
+    sharedCache.addPartitionToCache(DEFAULT_CATALOG_NAME, dbName, tbl1Name, part1);
+    sharedCache.addPartitionToCache(DEFAULT_CATALOG_NAME, dbName, tbl1Name, part2);
+    sharedCache.addPartitionToCache(DEFAULT_CATALOG_NAME, dbName, tbl1Name, part3);
+    sharedCache.addPartitionToCache(DEFAULT_CATALOG_NAME, dbName, tbl2Name, part1);
 
-    Partition t = sharedCache.getPartitionFromCache(dbName, tbl1Name, Arrays.asList("201701"));
+    Partition t = sharedCache.getPartitionFromCache(DEFAULT_CATALOG_NAME, dbName, tbl1Name, Arrays.asList("201701"));
     Assert.assertEquals(t.getSd().getLocation(), "loc1");
 
-    sharedCache.removePartitionFromCache(dbName, tbl2Name, Arrays.asList("201701"));
-    t = sharedCache.getPartitionFromCache(dbName, tbl2Name, Arrays.asList("201701"));
+    sharedCache.removePartitionFromCache(DEFAULT_CATALOG_NAME, dbName, tbl2Name, Arrays.asList("201701"));
+    t = sharedCache.getPartitionFromCache(DEFAULT_CATALOG_NAME, dbName, tbl2Name, Arrays.asList("201701"));
     Assert.assertNull(t);
 
-    sharedCache.alterPartitionInCache(dbName, tbl1Name, Arrays.asList("201701"), newPart1);
-    t = sharedCache.getPartitionFromCache(dbName, tbl1Name, Arrays.asList("201701"));
+    sharedCache.alterPartitionInCache(DEFAULT_CATALOG_NAME, dbName, tbl1Name, Arrays.asList("201701"), newPart1);
+    t = sharedCache.getPartitionFromCache(DEFAULT_CATALOG_NAME, dbName, tbl1Name, Arrays.asList("201701"));
     Assert.assertEquals(t.getSd().getLocation(), "loc1new");
   }
 
@@ -645,7 +659,10 @@ public class TestCachedStore {
     String tblName = "tbl";
     String colName = "f1";
 
-    Database db = new Database(dbName, null, "some_location", null);
+    Database db = new DatabaseBuilder()
+        .setName(dbName)
+        .setLocation("some_location")
+        .build(conf);
     cachedStore.createDatabase(db);
 
     List<FieldSchema> cols = new ArrayList<>();
@@ -659,6 +676,7 @@ public class TestCachedStore {
     Table tbl =
         new Table(tblName, dbName, null, 0, 0, 0, sd, partCols, new HashMap<>(),
             null, null, TableType.MANAGED_TABLE.toString());
+    tbl.setCatName(DEFAULT_CATALOG_NAME);
     cachedStore.createTable(tbl);
 
     List<String> partVals1 = new ArrayList<>();
@@ -668,9 +686,11 @@ public class TestCachedStore {
 
     Partition ptn1 =
         new Partition(partVals1, dbName, tblName, 0, 0, sd, new HashMap<>());
+    ptn1.setCatName(DEFAULT_CATALOG_NAME);
     cachedStore.addPartition(ptn1);
     Partition ptn2 =
         new Partition(partVals2, dbName, tblName, 0, 0, sd, new HashMap<>());
+    ptn2.setCatName(DEFAULT_CATALOG_NAME);
     cachedStore.addPartition(ptn2);
 
     ColumnStatistics stats = new ColumnStatistics();
@@ -699,9 +719,9 @@ public class TestCachedStore {
     List<String> aggrPartVals = new ArrayList<>();
     aggrPartVals.add("1");
     aggrPartVals.add("2");
-    AggrStats aggrStats = cachedStore.get_aggr_stats_for(dbName, tblName, aggrPartVals, colNames);
+    AggrStats aggrStats = cachedStore.get_aggr_stats_for(DEFAULT_CATALOG_NAME, dbName, tblName, aggrPartVals, colNames);
     Assert.assertEquals(aggrStats.getColStats().get(0).getStatsData().getLongStats().getNumNulls(), 100);
-    aggrStats = cachedStore.get_aggr_stats_for(dbName, tblName, aggrPartVals, colNames);
+    aggrStats = cachedStore.get_aggr_stats_for(DEFAULT_CATALOG_NAME, dbName, tblName, aggrPartVals, colNames);
     Assert.assertEquals(aggrStats.getColStats().get(0).getStatsData().getLongStats().getNumNulls(), 100);
   }
 
@@ -712,6 +732,7 @@ public class TestCachedStore {
     String colName = "f1";
 
     Database db = new Database(dbName, null, "some_location", null);
+    db.setCatalogName(DEFAULT_CATALOG_NAME);
     cachedStore.createDatabase(db);
 
     List<FieldSchema> cols = new ArrayList<>();
@@ -725,6 +746,7 @@ public class TestCachedStore {
     Table tbl =
         new Table(tblName, dbName, null, 0, 0, 0, sd, partCols, new HashMap<>(),
             null, null, TableType.MANAGED_TABLE.toString());
+    tbl.setCatName(DEFAULT_CATALOG_NAME);
     cachedStore.createTable(tbl);
 
     List<String> partVals1 = new ArrayList<>();
@@ -734,9 +756,11 @@ public class TestCachedStore {
 
     Partition ptn1 =
         new Partition(partVals1, dbName, tblName, 0, 0, sd, new HashMap<>());
+    ptn1.setCatName(DEFAULT_CATALOG_NAME);
     cachedStore.addPartition(ptn1);
     Partition ptn2 =
         new Partition(partVals2, dbName, tblName, 0, 0, sd, new HashMap<>());
+    ptn2.setCatName(DEFAULT_CATALOG_NAME);
     cachedStore.addPartition(ptn2);
 
     ColumnStatistics stats = new ColumnStatistics();
@@ -767,10 +791,10 @@ public class TestCachedStore {
     List<String> aggrPartVals = new ArrayList<>();
     aggrPartVals.add("1");
     aggrPartVals.add("2");
-    AggrStats aggrStats = cachedStore.get_aggr_stats_for(dbName, tblName, aggrPartVals, colNames);
+    AggrStats aggrStats = cachedStore.get_aggr_stats_for(DEFAULT_CATALOG_NAME, dbName, tblName, aggrPartVals, colNames);
     Assert.assertEquals(aggrStats.getColStats().get(0).getStatsData().getLongStats().getNumNulls(), 100);
     Assert.assertEquals(aggrStats.getColStats().get(0).getStatsData().getLongStats().getNumDVs(), 40);
-    aggrStats = cachedStore.get_aggr_stats_for(dbName, tblName, aggrPartVals, colNames);
+    aggrStats = cachedStore.get_aggr_stats_for(DEFAULT_CATALOG_NAME, dbName, tblName, aggrPartVals, colNames);
     Assert.assertEquals(aggrStats.getColStats().get(0).getStatsData().getLongStats().getNumNulls(), 100);
     Assert.assertEquals(aggrStats.getColStats().get(0).getStatsData().getLongStats().getNumDVs(), 40);
   }
@@ -782,6 +806,7 @@ public class TestCachedStore {
     String colName = "f1";
 
     Database db = new Database(dbName, null, "some_location", null);
+    db.setCatalogName(DEFAULT_CATALOG_NAME);
     cachedStore.createDatabase(db);
 
     List<FieldSchema> cols = new ArrayList<>();
@@ -795,6 +820,7 @@ public class TestCachedStore {
     Table tbl =
         new Table(tblName, dbName, null, 0, 0, 0, sd, partCols, new HashMap<>(),
             null, null, TableType.MANAGED_TABLE.toString());
+    tbl.setCatName(DEFAULT_CATALOG_NAME);
     cachedStore.createTable(tbl);
 
     List<String> partVals1 = new ArrayList<>();
@@ -804,9 +830,11 @@ public class TestCachedStore {
 
     Partition ptn1 =
         new Partition(partVals1, dbName, tblName, 0, 0, sd, new HashMap<>());
+    ptn1.setCatName(DEFAULT_CATALOG_NAME);
     cachedStore.addPartition(ptn1);
     Partition ptn2 =
         new Partition(partVals2, dbName, tblName, 0, 0, sd, new HashMap<>());
+    ptn2.setCatName(DEFAULT_CATALOG_NAME);
     cachedStore.addPartition(ptn2);
 
     ColumnStatistics stats = new ColumnStatistics();
@@ -851,10 +879,10 @@ public class TestCachedStore {
     List<String> aggrPartVals = new ArrayList<>();
     aggrPartVals.add("1");
     aggrPartVals.add("2");
-    AggrStats aggrStats = cachedStore.get_aggr_stats_for(dbName, tblName, aggrPartVals, colNames);
+    AggrStats aggrStats = cachedStore.get_aggr_stats_for(DEFAULT_CATALOG_NAME, dbName, tblName, aggrPartVals, colNames);
     Assert.assertEquals(aggrStats.getColStats().get(0).getStatsData().getLongStats().getNumNulls(), 100);
     Assert.assertEquals(aggrStats.getColStats().get(0).getStatsData().getLongStats().getNumDVs(), 5);
-    aggrStats = cachedStore.get_aggr_stats_for(dbName, tblName, aggrPartVals, colNames);
+    aggrStats = cachedStore.get_aggr_stats_for(DEFAULT_CATALOG_NAME, dbName, tblName, aggrPartVals, colNames);
     Assert.assertEquals(aggrStats.getColStats().get(0).getStatsData().getLongStats().getNumNulls(), 100);
     Assert.assertEquals(aggrStats.getColStats().get(0).getStatsData().getLongStats().getNumDVs(), 5);
   }
@@ -885,7 +913,7 @@ public class TestCachedStore {
     }
     executor.invokeAll(tasks);
     for (String dbName : dbNames) {
-      Database db = sharedCache.getDatabaseFromCache(dbName);
+      Database db = sharedCache.getDatabaseFromCache(DEFAULT_CATALOG_NAME, dbName);
       Assert.assertNotNull(db);
       Assert.assertEquals(dbName, db.getName());
     }
@@ -906,7 +934,7 @@ public class TestCachedStore {
       Callable<Object> c = new Callable<Object>() {
         public Object call() {
           Table tbl = createTestTbl(dbNames.get(0), tblName, "user1", cols, ptnCols);
-          sharedCache.addTableToCache(dbNames.get(0), tblName, tbl);
+          sharedCache.addTableToCache(DEFAULT_CATALOG_NAME, dbNames.get(0), tblName, tbl);
           return null;
         }
       };
@@ -914,7 +942,7 @@ public class TestCachedStore {
     }
     executor.invokeAll(tasks);
     for (String tblName : tblNames) {
-      Table tbl = sharedCache.getTableFromCache(dbNames.get(0), tblName);
+      Table tbl = sharedCache.getTableFromCache(DEFAULT_CATALOG_NAME, dbNames.get(0), tblName);
       Assert.assertNotNull(tbl);
       Assert.assertEquals(tblName, tbl.getTableName());
     }
@@ -923,14 +951,14 @@ public class TestCachedStore {
     List<String> ptnVals = new ArrayList<String>(Arrays.asList("aaa", "bbb", "ccc", "ddd", "eee"));
     tasks.clear();
     for (String tblName : tblNames) {
-      Table tbl = sharedCache.getTableFromCache(dbNames.get(0), tblName);
+      Table tbl = sharedCache.getTableFromCache(DEFAULT_CATALOG_NAME, dbNames.get(0), tblName);
       for (String ptnVal : ptnVals) {
         Map<String, String> partParams = new HashMap<String, String>();
         Callable<Object> c = new Callable<Object>() {
           public Object call() {
             Partition ptn = new Partition(Arrays.asList(ptnVal), dbNames.get(0), tblName, 0, 0,
                 tbl.getSd(), partParams);
-            sharedCache.addPartitionToCache(dbNames.get(0), tblName, ptn);
+            sharedCache.addPartitionToCache(DEFAULT_CATALOG_NAME, dbNames.get(0), tblName, ptn);
             return null;
           }
         };
@@ -940,7 +968,7 @@ public class TestCachedStore {
     executor.invokeAll(tasks);
     for (String tblName : tblNames) {
       for (String ptnVal : ptnVals) {
-        Partition ptn = sharedCache.getPartitionFromCache(dbNames.get(0), tblName, Arrays.asList(ptnVal));
+        Partition ptn = sharedCache.getPartitionFromCache(DEFAULT_CATALOG_NAME, dbNames.get(0), tblName, Arrays.asList(ptnVal));
         Assert.assertNotNull(ptn);
         Assert.assertEquals(tblName, ptn.getTableName());
         Assert.assertEquals(tblName, ptn.getTableName());
@@ -957,7 +985,7 @@ public class TestCachedStore {
       for (String ptnVal : ptnVals) {
         Callable<Object> c = new Callable<Object>() {
           public Object call() {
-            sharedCache.removePartitionFromCache(dbNames.get(0), tblName, Arrays.asList(ptnVal));
+            sharedCache.removePartitionFromCache(DEFAULT_CATALOG_NAME, dbNames.get(0), tblName, Arrays.asList(ptnVal));
             return null;
           }
         };
@@ -965,14 +993,14 @@ public class TestCachedStore {
       }
     }
     for (String tblName : addPtnTblNames) {
-      Table tbl = sharedCache.getTableFromCache(dbNames.get(0), tblName);
+      Table tbl = sharedCache.getTableFromCache(DEFAULT_CATALOG_NAME, dbNames.get(0), tblName);
       for (String ptnVal : newPtnVals) {
         Map<String, String> partParams = new HashMap<String, String>();
         Callable<Object> c = new Callable<Object>() {
           public Object call() {
             Partition ptn = new Partition(Arrays.asList(ptnVal), dbNames.get(0), tblName, 0, 0,
                 tbl.getSd(), partParams);
-            sharedCache.addPartitionToCache(dbNames.get(0), tblName, ptn);
+            sharedCache.addPartitionToCache(DEFAULT_CATALOG_NAME, dbNames.get(0), tblName, ptn);
             return null;
           }
         };
@@ -982,7 +1010,7 @@ public class TestCachedStore {
     executor.invokeAll(tasks);
     for (String tblName : addPtnTblNames) {
       for (String ptnVal : newPtnVals) {
-        Partition ptn = sharedCache.getPartitionFromCache(dbNames.get(0), tblName, Arrays.asList(ptnVal));
+        Partition ptn = sharedCache.getPartitionFromCache(DEFAULT_CATALOG_NAME, dbNames.get(0), tblName, Arrays.asList(ptnVal));
         Assert.assertNotNull(ptn);
         Assert.assertEquals(tblName, ptn.getTableName());
         Assert.assertEquals(tblName, ptn.getTableName());
@@ -990,7 +1018,7 @@ public class TestCachedStore {
       }
     }
     for (String tblName : dropPtnTblNames) {
-      List<Partition> ptns = sharedCache.listCachedPartitions(dbNames.get(0), tblName, 100);
+      List<Partition> ptns = sharedCache.listCachedPartitions(DEFAULT_CATALOG_NAME, dbNames.get(0), tblName, 100);
       Assert.assertEquals(0, ptns.size());
     }
     sharedCache.getDatabaseCache().clear();
@@ -1005,6 +1033,7 @@ public class TestCachedStore {
     Database db = new Database(dbName, dbDescription, dbLocation, dbParams);
     db.setOwnerName(dbOwner);
     db.setOwnerType(PrincipalType.USER);
+    db.setCatalogName(DEFAULT_CATALOG_NAME);
     return db;
   }
 
@@ -1019,6 +1048,7 @@ public class TestCachedStore {
     sd.setStoredAsSubDirectories(false);
     Table tbl = new Table(tblName, dbName, tblOwner, 0, 0, 0, sd, ptnCols, tblParams, null, null,
         TableType.MANAGED_TABLE.toString());
+    tbl.setCatName(DEFAULT_CATALOG_NAME);
     return tbl;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCatalogCaching.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCatalogCaching.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCatalogCaching.java
new file mode 100644
index 0000000..423dce8
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCatalogCaching.java
@@ -0,0 +1,142 @@
+/*
+ * 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.cache;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.HiveMetaStore;
+import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
+import org.apache.hadoop.hive.metastore.ObjectStore;
+import org.apache.hadoop.hive.metastore.Warehouse;
+import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
+import org.apache.hadoop.hive.metastore.api.Catalog;
+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.client.builder.CatalogBuilder;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.util.Comparator;
+import java.util.List;
+
+/**
+ * Tests that catalogs are properly cached.
+ */
+@Category(MetastoreCheckinTest.class)
+public class TestCatalogCaching {
+  private static final String CAT1_NAME = "cat1";
+  private static final String CAT2_NAME = "cat2";
+
+  private ObjectStore objectStore;
+  private Configuration conf;
+  private CachedStore cachedStore;
+
+  @Before
+  public void createObjectStore() throws MetaException, InvalidOperationException {
+    conf = MetastoreConf.newMetastoreConf();
+    MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.HIVE_IN_TEST, true);
+    MetaStoreTestUtils.setConfForStandloneMode(conf);
+    objectStore = new ObjectStore();
+    objectStore.setConf(conf);
+
+    // Create three catalogs
+    HiveMetaStore.HMSHandler.createDefaultCatalog(objectStore, new Warehouse(conf));
+
+    Catalog cat1 = new CatalogBuilder()
+        .setName(CAT1_NAME)
+        .setLocation("/tmp/cat1")
+        .build();
+    objectStore.createCatalog(cat1);
+    Catalog cat2 = new CatalogBuilder()
+        .setName(CAT2_NAME)
+        .setLocation("/tmp/cat2")
+        .build();
+    objectStore.createCatalog(cat2);
+  }
+
+  @After
+  public void clearCatalogCache() throws MetaException, NoSuchObjectException {
+    List<String> catalogs = objectStore.getCatalogs();
+    for (String catalog : catalogs) objectStore.dropCatalog(catalog);
+  }
+
+  @Test
+  public void defaultHiveOnly() throws Exception {
+    // By default just the Hive catalog should be cached.
+    cachedStore = new CachedStore();
+    cachedStore.setConf(conf);
+    CachedStore.stopCacheUpdateService(1);
+    cachedStore.resetCatalogCache();
+
+    CachedStore.prewarm(objectStore);
+
+    // Only the hive catalog should be cached
+    List<String> cachedCatalogs = cachedStore.getCatalogs();
+    Assert.assertEquals(1, cachedCatalogs.size());
+    Assert.assertEquals(Warehouse.DEFAULT_CATALOG_NAME, cachedCatalogs.get(0));
+  }
+
+  @Test
+  public void cacheAll() throws Exception {
+    // Set the config value to empty string, which should result in all catalogs being cached.
+    Configuration newConf = new Configuration(conf);
+    MetastoreConf.setVar(newConf, MetastoreConf.ConfVars.CATALOGS_TO_CACHE, "");
+    cachedStore = new CachedStore();
+    cachedStore.setConf(newConf);
+    CachedStore.stopCacheUpdateService(1);
+    objectStore.setConf(newConf); // have to override it with the new conf since this is where
+                                  // prewarm gets the conf object
+    cachedStore.resetCatalogCache();
+
+    CachedStore.prewarm(objectStore);
+
+    // All the catalogs should be cached
+    List<String> cachedCatalogs = cachedStore.getCatalogs();
+    Assert.assertEquals(3, cachedCatalogs.size());
+    cachedCatalogs.sort(Comparator.naturalOrder());
+    Assert.assertEquals(CAT1_NAME, cachedCatalogs.get(0));
+    Assert.assertEquals(CAT2_NAME, cachedCatalogs.get(1));
+    Assert.assertEquals(Warehouse.DEFAULT_CATALOG_NAME, cachedCatalogs.get(2));
+  }
+
+  @Test
+  public void cacheSome() throws Exception {
+    // Set the config value to 2 catalogs other than hive
+    Configuration newConf = new Configuration(conf);
+    MetastoreConf.setVar(newConf, MetastoreConf.ConfVars.CATALOGS_TO_CACHE, CAT1_NAME + "," + CAT2_NAME);
+    cachedStore = new CachedStore();
+    cachedStore.setConf(newConf);
+    CachedStore.stopCacheUpdateService(1);
+    objectStore.setConf(newConf); // have to override it with the new conf since this is where
+                                  // prewarm gets the conf object
+    cachedStore.resetCatalogCache();
+
+    CachedStore.prewarm(objectStore);
+
+    // All the catalogs should be cached
+    List<String> cachedCatalogs = cachedStore.getCatalogs();
+    Assert.assertEquals(2, cachedCatalogs.size());
+    cachedCatalogs.sort(Comparator.naturalOrder());
+    Assert.assertEquals(CAT1_NAME, cachedCatalogs.get(0));
+    Assert.assertEquals(CAT2_NAME, cachedCatalogs.get(1));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/MetaStoreFactoryForTests.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/MetaStoreFactoryForTests.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/MetaStoreFactoryForTests.java
index 84c187b..1a57df2 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/MetaStoreFactoryForTests.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/MetaStoreFactoryForTests.java
@@ -87,6 +87,7 @@ public final class MetaStoreFactoryForTests {
     // Create Embedded MetaStore
     conf.set("javax.jdo.option.ConnectionURL",
         "jdbc:derby:memory:${test.tmp.dir}/junit_metastore_db1;create=true");
+    MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.TRY_DIRECT_SQL, false);
     AbstractMetaStoreService embedded =
         new MiniHMS.Builder()
             .setConf(conf)
@@ -97,6 +98,7 @@ public final class MetaStoreFactoryForTests {
     // Create Remote MetaStore
     conf.set("javax.jdo.option.ConnectionURL",
         "jdbc:derby:memory:${test.tmp.dir}/junit_metastore_db2;create=true");
+    MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.TRY_DIRECT_SQL, true);
     AbstractMetaStoreService remote =
         new MiniHMS.Builder()
             .setConf(conf)

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestAddPartitions.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestAddPartitions.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestAddPartitions.java
index 4d9cb1b..8555eee 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestAddPartitions.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestAddPartitions.java
@@ -19,12 +19,16 @@
 package org.apache.hadoop.hive.metastore.client;
 
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
 import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
+import org.apache.hadoop.hive.metastore.api.Catalog;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
@@ -34,6 +38,7 @@ import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.SkewedInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.client.builder.CatalogBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.PartitionBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
@@ -81,10 +86,9 @@ public class TestAddPartitions extends MetaStoreClientTest {
     // Clean up the database
     client.dropDatabase(DB_NAME, true, true, true);
     metaStore.cleanWarehouseDirs();
-    Database db = new DatabaseBuilder().
+    new DatabaseBuilder().
         setName(DB_NAME).
-        build();
-    client.createDatabase(db);
+        create(client, metaStore.getConf());
   }
 
   @After
@@ -123,6 +127,72 @@ public class TestAddPartitions extends MetaStoreClientTest {
   }
 
   @Test
+  public void addPartitionOtherCatalog() throws TException {
+    String catName = "add_partition_catalog";
+    Catalog cat = new CatalogBuilder()
+        .setName(catName)
+        .setLocation(MetaStoreTestUtils.getTestWarehouseDir(catName))
+        .build();
+    client.createCatalog(cat);
+
+    String dbName = "add_partition_database_in_other_catalog";
+    Database db = new DatabaseBuilder()
+        .setName(dbName)
+        .setCatalogName(catName)
+        .create(client, metaStore.getConf());
+
+    String tableName = "table_in_other_catalog";
+    Table table = new TableBuilder()
+        .inDb(db)
+        .setTableName(tableName)
+        .addCol("id", "int")
+        .addCol("name", "string")
+        .addPartCol("partcol", "string")
+        .create(client, metaStore.getConf());
+
+    Partition[] parts = new Partition[5];
+    for (int i = 0; i < parts.length; i++) {
+      parts[i] = new PartitionBuilder()
+          .inTable(table)
+          .addValue("a" + i)
+          .build(metaStore.getConf());
+    }
+    client.add_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(catName, dbName, tableName,
+          Collections.singletonList("a" + i));
+      Assert.assertEquals(catName, fetched.getCatName());
+      Assert.assertEquals(dbName, fetched.getDbName());
+      Assert.assertEquals(tableName, fetched.getTableName());
+    }
+
+    client.dropDatabase(catName, dbName, true, true, true);
+    client.dropCatalog(catName);
+  }
+
+  @Test(expected = InvalidObjectException.class)
+  public void noSuchCatalog() throws TException {
+    String tableName = "table_for_no_such_catalog";
+    Table table = new TableBuilder()
+        .setTableName(tableName)
+        .addCol("id", "int")
+        .addCol("name", "string")
+        .addPartCol("partcol", "string")
+        .create(client, metaStore.getConf());
+
+    Partition part = new PartitionBuilder()
+        .inTable(table)
+        .addValue("a")
+        .build(metaStore.getConf());
+    // Explicitly mis-set the catalog name
+    part.setCatName("nosuch");
+    client.add_partition(part);
+  }
+
+  @Test
   public void testAddPartitionWithDefaultAttributes() throws Exception {
 
     Table table = createTable();
@@ -134,7 +204,7 @@ public class TestAddPartitions extends MetaStoreClientTest {
         .setCols(getYearPartCol())
         .addCol("test_id", "int", "test col id")
         .addCol("test_value", "string", "test col value")
-        .build();
+        .build(metaStore.getConf());
 
     client.add_partition(partition);
 
@@ -270,7 +340,7 @@ public class TestAddPartitions extends MetaStoreClientTest {
 
     createTable();
     Partition partition = buildPartition(DB_NAME, TABLE_NAME, DEFAULT_YEAR_VALUE);
-    partition.getSd().setCols(new ArrayList<FieldSchema>());
+    partition.getSd().setCols(new ArrayList<>());
     client.add_partition(partition);
 
     // TODO: Not sure that this is the correct behavior. It doesn't make sense to create the
@@ -372,8 +442,7 @@ public class TestAddPartitions extends MetaStoreClientTest {
         .addCol("test_value", DEFAULT_COL_TYPE, "test col value")
         .addPartCol(YEAR_COL_NAME, DEFAULT_COL_TYPE)
         .setLocation(null)
-        .build();
-    client.createTable(table);
+        .create(client, metaStore.getConf());
     Partition partition = buildPartition(DB_NAME, TABLE_NAME, DEFAULT_YEAR_VALUE);
     client.add_partition(partition);
   }
@@ -427,8 +496,7 @@ public class TestAddPartitions extends MetaStoreClientTest {
         .setTableName(TABLE_NAME)
         .addCol("test_id", "int", "test col id")
         .addCol("test_value", "string", "test col value")
-        .build();
-    client.createTable(origTable);
+        .create(client, metaStore.getConf());
     Partition partition = buildPartition(DB_NAME, TABLE_NAME, DEFAULT_YEAR_VALUE);
     client.add_partition(partition);
   }
@@ -442,7 +510,7 @@ public class TestAddPartitions extends MetaStoreClientTest {
         .setTableName(TABLE_NAME)
         .addValue(DEFAULT_YEAR_VALUE)
         .setLocation(metaStore.getWarehouseRoot() + "/addparttest")
-        .build();
+        .build(metaStore.getConf());
     client.add_partition(partition);
   }
 
@@ -455,7 +523,7 @@ public class TestAddPartitions extends MetaStoreClientTest {
         .setTableName(TABLE_NAME)
         .addValue("1000")
         .addCol("time", "int")
-        .build();
+        .build(metaStore.getConf());
 
     client.add_partition(partition);
     Partition part = client.getPartition(DB_NAME, TABLE_NAME, "year=1000");
@@ -474,7 +542,7 @@ public class TestAddPartitions extends MetaStoreClientTest {
         .setTableName(TABLE_NAME)
         .addCol(YEAR_COL_NAME, DEFAULT_COL_TYPE)
         .setLocation(metaStore.getWarehouseRoot() + "/addparttest")
-        .build();
+        .build(metaStore.getConf());
     client.add_partition(partition);
   }
 
@@ -588,7 +656,7 @@ public class TestAddPartitions extends MetaStoreClientTest {
         .setCols(getYearPartCol())
         .addCol("test_id", "int", "test col id")
         .addCol("test_value", "string", "test col value")
-        .build();
+        .build(metaStore.getConf());
 
     client.add_partitions(Lists.newArrayList(partition));
 
@@ -622,7 +690,7 @@ public class TestAddPartitions extends MetaStoreClientTest {
   @Test
   public void testAddPartitionsEmptyList() throws Exception {
 
-    client.add_partitions(new ArrayList<Partition>());
+    client.add_partitions(new ArrayList<>());
   }
 
   @Test(expected = MetaException.class)
@@ -873,7 +941,7 @@ public class TestAddPartitions extends MetaStoreClientTest {
 
     createTable();
     Partition partition = buildPartition(DB_NAME, TABLE_NAME, DEFAULT_YEAR_VALUE);
-    partition.getSd().setCols(new ArrayList<FieldSchema>());
+    partition.getSd().setCols(new ArrayList<>());
     client.add_partitions(Lists.newArrayList(partition));
 
     // TODO: Not sure that this is the correct behavior. It doesn't make sense to create the
@@ -976,8 +1044,7 @@ public class TestAddPartitions extends MetaStoreClientTest {
         .addCol("test_value", "string", "test col value")
         .addPartCol(YEAR_COL_NAME, DEFAULT_COL_TYPE)
         .setLocation(null)
-        .build();
-    client.createTable(table);
+        .create(client, metaStore.getConf());
     Partition partition = buildPartition(DB_NAME, TABLE_NAME, DEFAULT_YEAR_VALUE);
     List<Partition> partitions = Lists.newArrayList(partition);
     client.add_partitions(partitions);
@@ -1044,7 +1111,7 @@ public class TestAddPartitions extends MetaStoreClientTest {
         .setTableName(TABLE_NAME)
         .addCol(YEAR_COL_NAME, DEFAULT_COL_TYPE)
         .setLocation(metaStore.getWarehouseRoot() + "/addparttest")
-        .build();
+        .build(metaStore.getConf());
     List<Partition> partitions = new ArrayList<>();
     partitions.add(partition);
     client.add_partitions(partitions);
@@ -1160,7 +1227,7 @@ public class TestAddPartitions extends MetaStoreClientTest {
   public void testAddPartsEmptyList() throws Exception {
 
     List<Partition> addedPartitions =
-        client.add_partitions(new ArrayList<Partition>(), false, true);
+        client.add_partitions(new ArrayList<>(), false, true);
     Assert.assertNotNull(addedPartitions);
     Assert.assertTrue(addedPartitions.isEmpty());
   }
@@ -1276,8 +1343,7 @@ public class TestAddPartitions extends MetaStoreClientTest {
 
   // Helper methods
   private void createDB(String dbName) throws TException {
-    Database db = new DatabaseBuilder().setName(dbName).build();
-    client.createDatabase(db);
+    new DatabaseBuilder().setName(dbName).create(client, metaStore.getConf());
   }
 
   private Table createTable() throws Exception {
@@ -1302,13 +1368,12 @@ public class TestAddPartitions extends MetaStoreClientTest {
         .setStoredAsSubDirectories(false)
         .addSerdeParam("partTestSerdeParamKey", "partTestSerdeParamValue")
         .setLocation(location)
-        .build();
-    client.createTable(table);
+        .create(client, metaStore.getConf());
     return client.getTable(dbName, tableName);
   }
 
   private void createExternalTable(String tableName, String location) throws Exception {
-    Table table = new TableBuilder()
+    new TableBuilder()
         .setDbName(DB_NAME)
         .setTableName(tableName)
         .addCol("test_id", "int", "test col id")
@@ -1316,8 +1381,7 @@ public class TestAddPartitions extends MetaStoreClientTest {
         .addPartCol(YEAR_COL_NAME, DEFAULT_COL_TYPE)
         .addTableParam("EXTERNAL", "TRUE")
         .setLocation(location)
-        .build();
-    client.createTable(table);
+        .create(client, metaStore.getConf());
   }
 
   private Partition buildPartition(String dbName, String tableName, String value)
@@ -1337,7 +1401,7 @@ public class TestAddPartitions extends MetaStoreClientTest {
         .addCol("test_value", "string", "test col value")
         .addPartParam(DEFAULT_PARAM_KEY, DEFAULT_PARAM_VALUE)
         .setLocation(location)
-        .build();
+        .build(metaStore.getConf());
     return partition;
   }
 
@@ -1357,7 +1421,7 @@ public class TestAddPartitions extends MetaStoreClientTest {
         .setLastAccessTime(123456)
         .addCol("test_id", "int", "test col id")
         .addCol("test_value", "string", "test col value")
-        .build();
+        .build(metaStore.getConf());
     return partition;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestAddPartitionsFromPartSpec.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestAddPartitionsFromPartSpec.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestAddPartitionsFromPartSpec.java
index 1122057..b32954f 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestAddPartitionsFromPartSpec.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestAddPartitionsFromPartSpec.java
@@ -88,10 +88,9 @@ public class TestAddPartitionsFromPartSpec extends MetaStoreClientTest {
     // Clean up the database
     client.dropDatabase(DB_NAME, true, true, true);
     metaStore.cleanWarehouseDirs();
-    Database db = new DatabaseBuilder().
+    new DatabaseBuilder().
         setName(DB_NAME).
-        build();
-    client.createDatabase(db);
+        create(client, metaStore.getConf());
   }
 
   @After
@@ -169,6 +168,8 @@ public class TestAddPartitionsFromPartSpec extends MetaStoreClientTest {
     verifyPartitionSharedSD(table, "year=2005/month=may", Lists.newArrayList("2005", "may"), 4);
   }
 
+  // TODO add tests for partitions in other catalogs
+
   @Test(expected = NullPointerException.class)
   public void testAddPartitionSpecNullSpec() throws Exception {
 
@@ -679,8 +680,7 @@ public class TestAddPartitionsFromPartSpec extends MetaStoreClientTest {
         .addCol("test_value", DEFAULT_COL_TYPE, "test col value")
         .addPartCol(YEAR_COL_NAME, DEFAULT_COL_TYPE)
         .setLocation(null)
-        .build();
-    client.createTable(table);
+        .create(client, metaStore.getConf());
 
     Partition partition = buildPartition(DB_NAME, TABLE_NAME, DEFAULT_YEAR_VALUE);
     PartitionSpecProxy partitionSpecProxy =
@@ -714,7 +714,7 @@ public class TestAddPartitionsFromPartSpec extends MetaStoreClientTest {
         .setTableName(TABLE_NAME)
         .addCol(YEAR_COL_NAME, DEFAULT_COL_TYPE)
         .setLocation(metaStore.getWarehouseRoot() + "/addpartspectest")
-        .build();
+        .build(metaStore.getConf());
 
     PartitionSpecProxy partitionSpecProxy =
         buildPartitionSpec(DB_NAME, TABLE_NAME, null, Lists.newArrayList(partition));
@@ -821,8 +821,7 @@ public class TestAddPartitionsFromPartSpec extends MetaStoreClientTest {
 
   // Helper methods
   private void createDB(String dbName) throws TException {
-    Database db = new DatabaseBuilder().setName(dbName).build();
-    client.createDatabase(db);
+    Database db = new DatabaseBuilder().setName(dbName).create(client, metaStore.getConf());
   }
 
   private Table createTable() throws Exception {
@@ -844,8 +843,7 @@ public class TestAddPartitionsFromPartSpec extends MetaStoreClientTest {
         .setStoredAsSubDirectories(false)
         .addSerdeParam("partTestSerdeParamKey", "partTestSerdeParamValue")
         .setLocation(location)
-        .build();
-    client.createTable(table);
+        .create(client, metaStore.getConf());
     return client.getTable(dbName, tableName);
   }
 
@@ -866,7 +864,7 @@ public class TestAddPartitionsFromPartSpec extends MetaStoreClientTest {
         .addCol("test_value", "string", "test col value")
         .addPartParam(DEFAULT_PARAM_KEY, DEFAULT_PARAM_VALUE)
         .setLocation(location)
-        .build();
+        .build(metaStore.getConf());
     return partition;
   }
 
@@ -886,7 +884,7 @@ public class TestAddPartitionsFromPartSpec extends MetaStoreClientTest {
         .setLastAccessTime(DEFAULT_CREATE_TIME)
         .addCol("test_id", "int", "test col id")
         .addCol("test_value", "string", "test col value")
-        .build();
+        .build(metaStore.getConf());
     return partition;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestAlterPartitions.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestAlterPartitions.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestAlterPartitions.java
index 747f66d..770da1a 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestAlterPartitions.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestAlterPartitions.java
@@ -19,11 +19,15 @@
 package org.apache.hadoop.hive.metastore.client;
 
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
 import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
+import org.apache.hadoop.hive.metastore.api.Catalog;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
@@ -31,6 +35,7 @@ import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.client.builder.CatalogBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.PartitionBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
@@ -41,6 +46,8 @@ import org.apache.thrift.transport.TTransportException;
 import com.google.common.collect.Lists;
 
 import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -48,6 +55,7 @@ import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
 import static java.util.stream.Collectors.joining;
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotEquals;
@@ -60,7 +68,7 @@ import static org.junit.Assert.fail;
 @RunWith(Parameterized.class)
 @Category(MetastoreCheckinTest.class)
 public class TestAlterPartitions extends MetaStoreClientTest {
-  public static final int NEW_CREATE_TIME = 123456789;
+  private static final int NEW_CREATE_TIME = 123456789;
   private AbstractMetaStoreService metaStore;
   private IMetaStoreClient client;
 
@@ -95,13 +103,12 @@ public class TestAlterPartitions extends MetaStoreClientTest {
   }
 
   private void createDB(String dbName) throws TException {
-    Database db = new DatabaseBuilder().
+    new DatabaseBuilder().
             setName(dbName).
-            build();
-    client.createDatabase(db);
+            create(client, metaStore.getConf());
   }
 
-  private static Table createTestTable(IMetaStoreClient client, String dbName, String tableName,
+  private Table createTestTable(IMetaStoreClient client, String dbName, String tableName,
                                        List<String> partCols, boolean setPartitionLevelPrivilages)
           throws Exception {
     TableBuilder builder = new TableBuilder()
@@ -111,7 +118,7 @@ public class TestAlterPartitions extends MetaStoreClientTest {
             .addCol("name", "string");
 
     partCols.forEach(col -> builder.addPartCol(col, "string"));
-    Table table = builder.build();
+    Table table = builder.build(metaStore.getConf());
 
     if (setPartitionLevelPrivilages) {
       table.putToParameters("PARTITION_LEVEL_PRIVILEGE", "true");
@@ -121,14 +128,14 @@ public class TestAlterPartitions extends MetaStoreClientTest {
     return table;
   }
 
-  private static void addPartition(IMetaStoreClient client, Table table, List<String> values)
+  private void addPartition(IMetaStoreClient client, Table table, List<String> values)
           throws TException {
-    PartitionBuilder partitionBuilder = new PartitionBuilder().fromTable(table);
+    PartitionBuilder partitionBuilder = new PartitionBuilder().inTable(table);
     values.forEach(val -> partitionBuilder.addValue(val));
-    client.add_partition(partitionBuilder.build());
+    client.add_partition(partitionBuilder.build(metaStore.getConf()));
   }
 
-  private static List<List<String>> createTable4PartColsParts(IMetaStoreClient client) throws
+  private List<List<String>> createTable4PartColsParts(IMetaStoreClient client) throws
           Exception {
     Table t = createTestTable(client, DB_NAME, TABLE_NAME, PARTCOL_SCHEMA, false);
     List<List<String>> testValues = Lists.newArrayList(
@@ -197,7 +204,6 @@ public class TestAlterPartitions extends MetaStoreClientTest {
   /**
    * Testing alter_partition(String,String,Partition) ->
    *         alter_partition_with_environment_context(String,String,Partition,null).
-   * @throws Exception
    */
   @Test
   public void testAlterPartition() throws Exception {
@@ -217,12 +223,152 @@ public class TestAlterPartitions extends MetaStoreClientTest {
 
   }
 
+  @Test
+  public void otherCatalog() throws TException {
+    String catName = "alter_partition_catalog";
+    Catalog cat = new CatalogBuilder()
+        .setName(catName)
+        .setLocation(MetaStoreTestUtils.getTestWarehouseDir(catName))
+        .build();
+    client.createCatalog(cat);
+
+    String dbName = "alter_partition_database_in_other_catalog";
+    Database db = new DatabaseBuilder()
+        .setName(dbName)
+        .setCatalogName(catName)
+        .create(client, metaStore.getConf());
+
+    String tableName = "table_in_other_catalog";
+    Table table = new TableBuilder()
+        .inDb(db)
+        .setTableName(tableName)
+        .addCol("id", "int")
+        .addCol("name", "string")
+        .addPartCol("partcol", "string")
+        .create(client, metaStore.getConf());
+
+    Partition[] parts = new Partition[5];
+    for (int i = 0; i < 5; i++) {
+      parts[i] = new PartitionBuilder()
+          .inTable(table)
+          .addValue("a" + i)
+          .setLocation(MetaStoreTestUtils.getTestWarehouseDir("b" + i))
+          .build(metaStore.getConf());
+    }
+    client.add_partitions(Arrays.asList(parts));
+
+    Partition newPart =
+        client.getPartition(catName, dbName, tableName, Collections.singletonList("a0"));
+    newPart.getParameters().put("test_key", "test_value");
+    client.alter_partition(catName, dbName, tableName, newPart);
+
+    Partition fetched =
+        client.getPartition(catName, dbName, tableName, Collections.singletonList("a0"));
+    Assert.assertEquals(catName, fetched.getCatName());
+    Assert.assertEquals("test_value", fetched.getParameters().get("test_key"));
+
+    newPart =
+        client.getPartition(catName, dbName, tableName, Collections.singletonList("a1"));
+    newPart.setLastAccessTime(3);
+    Partition newPart1 =
+        client.getPartition(catName, dbName, tableName, Collections.singletonList("a2"));
+    newPart1.getSd().setLocation(MetaStoreTestUtils.getTestWarehouseDir("somewhere"));
+    client.alter_partitions(catName, dbName, tableName, Arrays.asList(newPart, newPart1));
+    fetched =
+        client.getPartition(catName, dbName, tableName, Collections.singletonList("a1"));
+    Assert.assertEquals(catName, fetched.getCatName());
+    Assert.assertEquals(3L, fetched.getLastAccessTime());
+    fetched =
+        client.getPartition(catName, dbName, tableName, Collections.singletonList("a2"));
+    Assert.assertEquals(catName, fetched.getCatName());
+    Assert.assertTrue(fetched.getSd().getLocation().contains("somewhere"));
+
+    newPart =
+        client.getPartition(catName, 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(catName, dbName, tableName, newPart, ec);
+    fetched =
+        client.getPartition(catName, dbName, tableName, Collections.singletonList("a4"));
+    Assert.assertEquals(catName, fetched.getCatName());
+    Assert.assertEquals("test_value", fetched.getParameters().get("test_key"));
+
+
+    client.dropDatabase(catName, dbName, true, true, true);
+    client.dropCatalog(catName);
+  }
+
+  @SuppressWarnings("deprecation")
+  @Test
+  public void deprecatedCalls() throws TException {
+    String tableName = "deprecated_table";
+    Table table = new TableBuilder()
+        .setTableName(tableName)
+        .addCol("id", "int")
+        .addCol("name", "string")
+        .addPartCol("partcol", "string")
+        .create(client, metaStore.getConf());
+
+    Partition[] parts = new Partition[5];
+    for (int i = 0; i < 5; i++) {
+      parts[i] = new PartitionBuilder()
+          .inTable(table)
+          .addValue("a" + i)
+          .setLocation(MetaStoreTestUtils.getTestWarehouseDir("a" + i))
+          .build(metaStore.getConf());
+    }
+    client.add_partitions(Arrays.asList(parts));
+
+    Partition newPart =
+        client.getPartition(DEFAULT_DATABASE_NAME, tableName, Collections.singletonList("a0"));
+    newPart.getParameters().put("test_key", "test_value");
+    client.alter_partition(DEFAULT_DATABASE_NAME, tableName, newPart);
+
+    Partition fetched =
+        client.getPartition(DEFAULT_DATABASE_NAME, tableName, Collections.singletonList("a0"));
+    Assert.assertEquals("test_value", fetched.getParameters().get("test_key"));
+
+    newPart =
+        client.getPartition(DEFAULT_DATABASE_NAME, tableName, Collections.singletonList("a1"));
+    newPart.setLastAccessTime(3);
+    Partition newPart1 =
+        client.getPartition(DEFAULT_DATABASE_NAME, tableName, Collections.singletonList("a2"));
+    newPart1.getSd().setLocation("somewhere");
+    client.alter_partitions(DEFAULT_DATABASE_NAME, tableName, Arrays.asList(newPart, newPart1));
+    fetched =
+        client.getPartition(DEFAULT_DATABASE_NAME, tableName, Collections.singletonList("a1"));
+    Assert.assertEquals(3L, fetched.getLastAccessTime());
+    fetched =
+        client.getPartition(DEFAULT_DATABASE_NAME, tableName, Collections.singletonList("a2"));
+    Assert.assertTrue(fetched.getSd().getLocation().contains("somewhere"));
+
+    newPart =
+        client.getPartition(DEFAULT_DATABASE_NAME, tableName, Collections.singletonList("a3"));
+    newPart.setValues(Collections.singletonList("b3"));
+    client.renamePartition(DEFAULT_DATABASE_NAME, tableName, Collections.singletonList("a3"), newPart);
+    fetched =
+        client.getPartition(DEFAULT_DATABASE_NAME, tableName, Collections.singletonList("b3"));
+    Assert.assertEquals(1, fetched.getValuesSize());
+    Assert.assertEquals("b3", fetched.getValues().get(0));
+
+    newPart =
+        client.getPartition(DEFAULT_DATABASE_NAME, tableName, Collections.singletonList("a4"));
+    newPart.getParameters().put("test_key", "test_value");
+    EnvironmentContext ec = new EnvironmentContext();
+    ec.setProperties(Collections.singletonMap("a", "b"));
+    client.alter_partition(DEFAULT_DATABASE_NAME, tableName, newPart, ec);
+    fetched =
+        client.getPartition(DEFAULT_DATABASE_NAME, tableName, Collections.singletonList("a4"));
+    Assert.assertEquals("test_value", fetched.getParameters().get("test_key"));
+  }
+
   @Test(expected = InvalidOperationException.class)
   public void testAlterPartitionUnknownPartition() throws Exception {
     createTable4PartColsParts(client);
     Table t = client.getTable(DB_NAME, TABLE_NAME);
     PartitionBuilder builder = new PartitionBuilder();
-    Partition part = builder.fromTable(t).addValue("1111").addValue("11").addValue("11").build();
+    Partition part = builder.inTable(t).addValue("1111").addValue("11").addValue("11").build(metaStore.getConf());
     client.alter_partition(DB_NAME, TABLE_NAME, part);
   }
 
@@ -231,7 +377,7 @@ public class TestAlterPartitions extends MetaStoreClientTest {
     createTable4PartColsParts(client);
     Table t = client.getTable(DB_NAME, TABLE_NAME);
     PartitionBuilder builder = new PartitionBuilder();
-    Partition part = builder.fromTable(t).addValue("2017").build();
+    Partition part = builder.inTable(t).addValue("2017").build(metaStore.getConf());
     client.alter_partition(DB_NAME, TABLE_NAME, part);
   }
 
@@ -240,11 +386,18 @@ public class TestAlterPartitions extends MetaStoreClientTest {
     createTable4PartColsParts(client);
     Table t = client.getTable(DB_NAME, TABLE_NAME);
     PartitionBuilder builder = new PartitionBuilder();
-    Partition part = builder.fromTable(t).build();
+    Partition part = builder.inTable(t).build(metaStore.getConf());
     client.alter_partition(DB_NAME, TABLE_NAME, part);
   }
 
   @Test(expected = InvalidOperationException.class)
+  public void testAlterPartitionBogusCatalogName() throws Exception {
+    createTable4PartColsParts(client);
+    List<Partition> partitions = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+    client.alter_partition("nosuch", DB_NAME, TABLE_NAME, partitions.get(3));
+  }
+
+  @Test(expected = InvalidOperationException.class)
   public void testAlterPartitionNoDbName() throws Exception {
     createTable4PartColsParts(client);
     List<Partition> partitions = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
@@ -315,7 +468,6 @@ public class TestAlterPartitions extends MetaStoreClientTest {
   /**
    * Testing alter_partition(String,String,Partition,EnvironmentContext) ->
    *         alter_partition_with_environment_context(String,String,Partition,EnvironmentContext).
-   * @throws Exception
    */
   @Test
   public void testAlterPartitionWithEnvironmentCtx() throws Exception {
@@ -349,7 +501,7 @@ public class TestAlterPartitions extends MetaStoreClientTest {
     createTable4PartColsParts(client);
     Table t = client.getTable(DB_NAME, TABLE_NAME);
     PartitionBuilder builder = new PartitionBuilder();
-    Partition part = builder.fromTable(t).addValue("1111").addValue("11").addValue("11").build();
+    Partition part = builder.inTable(t).addValue("1111").addValue("11").addValue("11").build(metaStore.getConf());
     client.alter_partition(DB_NAME, TABLE_NAME, part, new EnvironmentContext());
   }
 
@@ -358,7 +510,7 @@ public class TestAlterPartitions extends MetaStoreClientTest {
     createTable4PartColsParts(client);
     Table t = client.getTable(DB_NAME, TABLE_NAME);
     PartitionBuilder builder = new PartitionBuilder();
-    Partition part = builder.fromTable(t).addValue("2017").build();
+    Partition part = builder.inTable(t).addValue("2017").build(metaStore.getConf());
     client.alter_partition(DB_NAME, TABLE_NAME, part, new EnvironmentContext());
   }
 
@@ -367,7 +519,7 @@ public class TestAlterPartitions extends MetaStoreClientTest {
     createTable4PartColsParts(client);
     Table t = client.getTable(DB_NAME, TABLE_NAME);
     PartitionBuilder builder = new PartitionBuilder();
-    Partition part = builder.fromTable(t).build();
+    Partition part = builder.inTable(t).build(metaStore.getConf());
     client.alter_partition(DB_NAME, TABLE_NAME, part, new EnvironmentContext());
   }
 
@@ -444,7 +596,6 @@ public class TestAlterPartitions extends MetaStoreClientTest {
    * Testing
    *    alter_partitions(String,String,List(Partition)) ->
    *    alter_partitions_with_environment_context(String,String,List(Partition),null).
-   * @throws Exception
    */
   @Test
   public void testAlterPartitions() throws Exception {
@@ -478,7 +629,7 @@ public class TestAlterPartitions extends MetaStoreClientTest {
       createTable4PartColsParts(client);
       Table t = client.getTable(DB_NAME, TABLE_NAME);
       PartitionBuilder builder = new PartitionBuilder();
-      Partition part = builder.fromTable(t).addValue("1111").addValue("11").addValue("11").build();
+      Partition part = builder.inTable(t).addValue("1111").addValue("11").addValue("11").build(metaStore.getConf());
       part1 = client.listPartitions(DB_NAME, TABLE_NAME, (short) -1).get(0);
       makeTestChangesOnPartition(part1);
       client.alter_partitions(DB_NAME, TABLE_NAME, Lists.newArrayList(part, part1));
@@ -494,7 +645,7 @@ public class TestAlterPartitions extends MetaStoreClientTest {
     createTable4PartColsParts(client);
     Table t = client.getTable(DB_NAME, TABLE_NAME);
     PartitionBuilder builder = new PartitionBuilder();
-    Partition part = builder.fromTable(t).addValue("2017").build();
+    Partition part = builder.inTable(t).addValue("2017").build(metaStore.getConf());
     Partition part1 = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1).get(0);
     client.alter_partitions(DB_NAME, TABLE_NAME, Lists.newArrayList(part, part1));
   }
@@ -504,12 +655,19 @@ public class TestAlterPartitions extends MetaStoreClientTest {
     createTable4PartColsParts(client);
     Table t = client.getTable(DB_NAME, TABLE_NAME);
     PartitionBuilder builder = new PartitionBuilder();
-    Partition part = builder.fromTable(t).build();
+    Partition part = builder.inTable(t).build(metaStore.getConf());
     Partition part1 = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1).get(0);
     client.alter_partitions(DB_NAME, TABLE_NAME, Lists.newArrayList(part, part1));
   }
 
   @Test(expected = InvalidOperationException.class)
+  public void testAlterPartitionsBogusCatalogName() throws Exception {
+    createTable4PartColsParts(client);
+    Partition part = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1).get(0);
+    client.alter_partitions("nosuch", DB_NAME, TABLE_NAME, Lists.newArrayList(part));
+  }
+
+  @Test(expected = InvalidOperationException.class)
   public void testAlterPartitionsNoDbName() throws Exception {
     createTable4PartColsParts(client);
     Partition part = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1).get(0);
@@ -596,7 +754,6 @@ public class TestAlterPartitions extends MetaStoreClientTest {
    * Testing
    *    alter_partitions(String,String,List(Partition),EnvironmentContext) ->
    *    alter_partitions_with_environment_context(String,String,List(Partition),EnvironmentContext).
-   * @throws Exception
    */
   @Test
   public void testAlterPartitionsWithEnvironmentCtx() throws Exception {
@@ -642,7 +799,7 @@ public class TestAlterPartitions extends MetaStoreClientTest {
     createTable4PartColsParts(client);
     Table t = client.getTable(DB_NAME, TABLE_NAME);
     PartitionBuilder builder = new PartitionBuilder();
-    Partition part = builder.fromTable(t).addValue("1111").addValue("11").addValue("11").build();
+    Partition part = builder.inTable(t).addValue("1111").addValue("11").addValue("11").build(metaStore.getConf());
     Partition part1 = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1).get(0);
     client.alter_partitions(DB_NAME, TABLE_NAME, Lists.newArrayList(part, part1),
             new EnvironmentContext());
@@ -653,7 +810,7 @@ public class TestAlterPartitions extends MetaStoreClientTest {
     createTable4PartColsParts(client);
     Table t = client.getTable(DB_NAME, TABLE_NAME);
     PartitionBuilder builder = new PartitionBuilder();
-    Partition part = builder.fromTable(t).addValue("2017").build();
+    Partition part = builder.inTable(t).addValue("2017").build(metaStore.getConf());
     Partition part1 = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1).get(0);
     client.alter_partitions(DB_NAME, TABLE_NAME, Lists.newArrayList(part, part1),
             new EnvironmentContext());
@@ -664,13 +821,20 @@ public class TestAlterPartitions extends MetaStoreClientTest {
     createTable4PartColsParts(client);
     Table t = client.getTable(DB_NAME, TABLE_NAME);
     PartitionBuilder builder = new PartitionBuilder();
-    Partition part = builder.fromTable(t).build();
+    Partition part = builder.inTable(t).build(metaStore.getConf());
     Partition part1 = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1).get(0);
     client.alter_partitions(DB_NAME, TABLE_NAME, Lists.newArrayList(part, part1),
             new EnvironmentContext());
   }
 
   @Test(expected = InvalidOperationException.class)
+  public void testAlterPartitionsWithEnvironmentCtxBogusCatalogName() throws Exception {
+    createTable4PartColsParts(client);
+    Partition part = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1).get(0);
+    client.alter_partitions("nosuch", DB_NAME, TABLE_NAME, Lists.newArrayList(part), new EnvironmentContext());
+  }
+
+  @Test(expected = InvalidOperationException.class)
   public void testAlterPartitionsWithEnvironmentCtxNoDbName() throws Exception {
     createTable4PartColsParts(client);
     Partition part = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1).get(0);
@@ -757,7 +921,6 @@ public class TestAlterPartitions extends MetaStoreClientTest {
    * Testing
    *    renamePartition(String,String,List(String),Partition) ->
    *    renamePartition(String,String,List(String),Partition).
-   * @throws Exception
    */
   @Test
   public void testRenamePartition() throws Exception {
@@ -870,6 +1033,16 @@ public class TestAlterPartitions extends MetaStoreClientTest {
   }
 
   @Test(expected = InvalidOperationException.class)
+  public void testRenamePartitionBogusCatalogName() throws Exception {
+    List<List<String>> oldValues = createTable4PartColsParts(client);
+    List<Partition> oldParts = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);
+
+    Partition partToRename = oldParts.get(3);
+    partToRename.setValues(Lists.newArrayList("2018", "01", "16"));
+    client.renamePartition("nosuch", DB_NAME, TABLE_NAME, oldValues.get(3), partToRename);
+  }
+
+  @Test(expected = InvalidOperationException.class)
   public void testRenamePartitionNoDbName() throws Exception {
     List<List<String>> oldValues = createTable4PartColsParts(client);
     List<Partition> oldParts = client.listPartitions(DB_NAME, TABLE_NAME, (short)-1);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestAppendPartitions.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestAppendPartitions.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestAppendPartitions.java
index 79d0953..75b26f2 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestAppendPartitions.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestAppendPartitions.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hive.metastore.client;
 
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -27,9 +28,11 @@ import java.util.stream.Collectors;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
+import org.apache.hadoop.hive.metastore.api.Catalog;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
@@ -37,10 +40,13 @@ import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.client.builder.CatalogBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.PartitionBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
 import org.apache.hadoop.hive.metastore.minihms.AbstractMetaStoreService;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransportException;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -78,10 +84,9 @@ public class TestAppendPartitions extends MetaStoreClientTest {
     // Clean up the database
     client.dropDatabase(DB_NAME, true, true, true);
     metaStore.cleanWarehouseDirs();
-    Database db = new DatabaseBuilder()
+    new DatabaseBuilder()
         .setName(DB_NAME)
-        .build();
-    client.createDatabase(db);
+        .create(client, metaStore.getConf());
 
     tableWithPartitions = createTableWithPartitions();
     externalTable = createExternalTable();
@@ -221,18 +226,22 @@ public class TestAppendPartitions extends MetaStoreClientTest {
     client.appendPartition(tableWithPartitions.getDbName(), null, partitionValues);
   }
 
-  @Test(expected = MetaException.class)
+  @Test(expected = InvalidObjectException.class)
   public void testAppendPartitionEmptyPartValues() throws Exception {
 
     Table table = tableWithPartitions;
-    client.appendPartition(table.getDbName(), table.getTableName(), new ArrayList<String>());
+    client.appendPartition(table.getDbName(), table.getTableName(), new ArrayList<>());
   }
 
-  @Test(expected = MetaException.class)
+  @Test
   public void testAppendPartitionNullPartValues() throws Exception {
-
-    Table table = tableWithPartitions;
-    client.appendPartition(table.getDbName(), table.getTableName(), (List<String>) null);
+    try {
+      Table table = tableWithPartitions;
+      client.appendPartition(table.getDbName(), table.getTableName(), (List<String>) null);
+      Assert.fail("Exception should have been thrown.");
+    } catch (TTransportException | InvalidObjectException e) {
+      // TODO: NPE should not be thrown
+    }
   }
 
   @Test
@@ -436,6 +445,57 @@ public class TestAppendPartitions extends MetaStoreClientTest {
     client.appendPartition(table.getDbName(), table.getTableName(), partitionName);
   }
 
+  @Test
+  public void otherCatalog() throws TException {
+    String catName = "append_partition_catalog";
+    Catalog cat = new CatalogBuilder()
+        .setName(catName)
+        .setLocation(MetaStoreTestUtils.getTestWarehouseDir(catName))
+        .build();
+    client.createCatalog(cat);
+
+    String dbName = "append_partition_database_in_other_catalog";
+    Database db = new DatabaseBuilder()
+        .setName(dbName)
+        .setCatalogName(catName)
+        .create(client, metaStore.getConf());
+
+    String tableName = "table_in_other_catalog";
+    new TableBuilder()
+        .inDb(db)
+        .setTableName(tableName)
+        .addCol("id", "int")
+        .addCol("name", "string")
+        .addPartCol("partcol", "string")
+        .create(client, metaStore.getConf());
+
+    Partition created =
+        client.appendPartition(catName, dbName, tableName, Collections.singletonList("a1"));
+    Assert.assertEquals(1, created.getValuesSize());
+    Assert.assertEquals("a1", created.getValues().get(0));
+    Partition fetched =
+        client.getPartition(catName, dbName, tableName, Collections.singletonList("a1"));
+    Assert.assertEquals(created, fetched);
+
+    created = client.appendPartition(catName, dbName, tableName, "partcol=a2");
+    Assert.assertEquals(1, created.getValuesSize());
+    Assert.assertEquals("a2", created.getValues().get(0));
+    fetched = client.getPartition(catName, dbName, tableName, Collections.singletonList("a2"));
+    Assert.assertEquals(created, fetched);
+  }
+
+  @Test(expected = InvalidObjectException.class)
+  public void testAppendPartitionBogusCatalog() throws Exception {
+    client.appendPartition("nosuch", DB_NAME, tableWithPartitions.getTableName(),
+        Lists.newArrayList("2017", "may"));
+  }
+
+  @Test(expected = InvalidObjectException.class)
+  public void testAppendPartitionByNameBogusCatalog() throws Exception {
+    client.appendPartition("nosuch", DB_NAME, tableWithPartitions.getTableName(),
+        "year=2017/month=april");
+  }
+
   // Helper methods
 
   private Table createTableWithPartitions() throws Exception {
@@ -471,7 +531,7 @@ public class TestAppendPartitions extends MetaStoreClientTest {
 
   private Table createTable(String tableName, List<FieldSchema> partCols, Map<String,
       String> tableParams, String tableType, String location) throws Exception {
-    Table table = new TableBuilder()
+    new TableBuilder()
         .setDbName(DB_NAME)
         .setTableName(tableName)
         .addCol("test_id", "int", "test col id")
@@ -480,17 +540,15 @@ public class TestAppendPartitions extends MetaStoreClientTest {
         .setTableParams(tableParams)
         .setType(tableType)
         .setLocation(location)
-        .build();
-    client.createTable(table);
+        .create(client, metaStore.getConf());
     return client.getTable(DB_NAME, tableName);
   }
 
   private void createPartition(Table table, List<String> values) throws Exception {
-    Partition partition = new PartitionBuilder()
-        .fromTable(table)
+    new PartitionBuilder()
+        .inTable(table)
         .setValues(values)
-        .build();
-    client.add_partition(partition);
+        .addToTable(client, metaStore.getConf());
   }
 
   private static List<FieldSchema> getYearAndMonthPartCols() {


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

Posted by ga...@apache.org.
HIVE-18755 Modifications to the metastore for catalogs (Alan Gates, reviewed by Thejas Nair)


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

Branch: refs/heads/master
Commit: ba8a99e115b6726e0de3ed2ac60b25037156b084
Parents: 5109602
Author: Alan Gates <ga...@hortonworks.com>
Authored: Fri Mar 30 18:24:21 2018 -0700
Committer: Alan Gates <ga...@hortonworks.com>
Committed: Fri Mar 30 18:24:21 2018 -0700

----------------------------------------------------------------------
 .../cli/SemanticAnalysis/CreateTableHook.java   |    2 +-
 .../apache/hive/hcatalog/cli/TestPermsGrp.java  |    4 +-
 .../listener/DbNotificationListener.java        |   19 +
 .../hcatalog/listener/NotificationListener.java |    2 +-
 .../listener/DummyRawStoreFailEvent.java        |  317 +-
 .../hive/metastore/TestAcidTableSetup.java      |   14 +-
 .../hive/metastore/TestMetaStoreMetrics.java    |    2 +-
 .../hive/ql/parse/TestReplicationScenarios.java |   13 +-
 .../upgrade/derby/053-HIVE-18755.derby.sql      |   54 +
 .../derby/upgrade-2.3.0-to-3.0.0.derby.sql      |    1 +
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |    5 +-
 .../hive/ql/exec/MaterializedViewTask.java      |    6 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |   25 +-
 .../ql/metadata/SessionHiveMetaStoreClient.java |   28 +-
 .../hive/ql/parse/BaseSemanticAnalyzer.java     |   91 +-
 .../hive/ql/parse/DDLSemanticAnalyzer.java      |   20 +-
 .../hive/ql/parse/MacroSemanticAnalyzer.java    |    2 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |    7 +-
 .../HiveAuthorizationProviderBase.java          |    4 +-
 .../AuthorizationMetaStoreFilterHook.java       |    3 +-
 .../hive/ql/txn/compactor/CompactorThread.java  |    6 +-
 .../alter_table_constraint_duplicate_pk.q.out   |    2 +-
 .../clientnegative/create_view_failure2.q.out   |    2 +-
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  | 6501 ++++++++++------
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.h    |  575 +-
 .../ThriftHiveMetastore_server.skeleton.cpp     |   22 +-
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp | 7270 ++++++++++-------
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |  700 +-
 .../hive/metastore/api/AbortTxnsRequest.java    |   32 +-
 .../api/AddCheckConstraintRequest.java          |   36 +-
 .../api/AddDefaultConstraintRequest.java        |   36 +-
 .../metastore/api/AddDynamicPartitions.java     |   32 +-
 .../metastore/api/AddForeignKeyRequest.java     |   36 +-
 .../api/AddNotNullConstraintRequest.java        |   36 +-
 .../metastore/api/AddPartitionsRequest.java     |  150 +-
 .../hive/metastore/api/AddPartitionsResult.java |   36 +-
 .../metastore/api/AddPrimaryKeyRequest.java     |   36 +-
 .../api/AddUniqueConstraintRequest.java         |   36 +-
 .../hadoop/hive/metastore/api/AggrStats.java    |   36 +-
 .../api/AllocateTableWriteIdsRequest.java       |   32 +-
 .../api/AllocateTableWriteIdsResponse.java      |   36 +-
 .../hadoop/hive/metastore/api/Catalog.java      |  606 ++
 .../metastore/api/CheckConstraintsRequest.java  |  117 +-
 .../metastore/api/CheckConstraintsResponse.java |   36 +-
 .../metastore/api/ClearFileMetadataRequest.java |   32 +-
 .../hive/metastore/api/ClientCapabilities.java  |   32 +-
 .../hive/metastore/api/ColumnStatistics.java    |   36 +-
 .../metastore/api/ColumnStatisticsDesc.java     |  114 +-
 .../hive/metastore/api/CompactionRequest.java   |   44 +-
 .../metastore/api/CreateCatalogRequest.java     |  400 +
 .../hive/metastore/api/CreationMetadata.java    |  165 +-
 .../hadoop/hive/metastore/api/Database.java     |  158 +-
 .../api/DefaultConstraintsRequest.java          |  117 +-
 .../api/DefaultConstraintsResponse.java         |   36 +-
 .../hive/metastore/api/DropCatalogRequest.java  |  395 +
 .../metastore/api/DropConstraintRequest.java    |  112 +-
 .../metastore/api/DropPartitionsRequest.java    |  114 +-
 .../metastore/api/DropPartitionsResult.java     |   36 +-
 .../hive/metastore/api/EnvironmentContext.java  |   44 +-
 .../metastore/api/FindSchemasByColsResp.java    |   36 +-
 .../hive/metastore/api/FireEventRequest.java    |  146 +-
 .../hive/metastore/api/ForeignKeysRequest.java  |  113 +-
 .../hive/metastore/api/ForeignKeysResponse.java |   36 +-
 .../hadoop/hive/metastore/api/Function.java     |  149 +-
 .../metastore/api/GetAllFunctionsResponse.java  |   36 +-
 .../hive/metastore/api/GetCatalogRequest.java   |  395 +
 .../hive/metastore/api/GetCatalogResponse.java  |  400 +
 .../hive/metastore/api/GetCatalogsResponse.java |  444 ++
 .../api/GetFileMetadataByExprRequest.java       |   32 +-
 .../api/GetFileMetadataByExprResult.java        |   48 +-
 .../metastore/api/GetFileMetadataRequest.java   |   32 +-
 .../metastore/api/GetFileMetadataResult.java    |   44 +-
 .../metastore/api/GetOpenTxnsInfoResponse.java  |   36 +-
 .../hive/metastore/api/GetOpenTxnsResponse.java |   32 +-
 .../hive/metastore/api/GetTableRequest.java     |  114 +-
 .../hive/metastore/api/GetTablesRequest.java    |  146 +-
 .../hive/metastore/api/GetTablesResult.java     |   36 +-
 .../metastore/api/GetValidWriteIdsRequest.java  |   32 +-
 .../metastore/api/GetValidWriteIdsResponse.java |   36 +-
 .../api/HeartbeatTxnRangeResponse.java          |   64 +-
 .../hive/metastore/api/HiveObjectRef.java       |  113 +-
 .../hadoop/hive/metastore/api/ISchema.java      |  178 +-
 .../hadoop/hive/metastore/api/ISchemaName.java  |  130 +-
 .../metastore/api/InsertEventRequestData.java   |   64 +-
 .../hadoop/hive/metastore/api/LockRequest.java  |   36 +-
 .../hive/metastore/api/Materialization.java     |   32 +-
 .../api/NotNullConstraintsRequest.java          |  117 +-
 .../api/NotNullConstraintsResponse.java         |   36 +-
 .../hive/metastore/api/NotificationEvent.java   |  114 +-
 .../api/NotificationEventResponse.java          |   36 +-
 .../api/NotificationEventsCountRequest.java     |  112 +-
 .../hive/metastore/api/OpenTxnsResponse.java    |   32 +-
 .../hadoop/hive/metastore/api/Partition.java    |  190 +-
 .../api/PartitionListComposingSpec.java         |   36 +-
 .../hive/metastore/api/PartitionSpec.java       |  114 +-
 .../api/PartitionSpecWithSharedSD.java          |   36 +-
 .../metastore/api/PartitionValuesRequest.java   |  186 +-
 .../metastore/api/PartitionValuesResponse.java  |   36 +-
 .../hive/metastore/api/PartitionValuesRow.java  |   32 +-
 .../hive/metastore/api/PartitionWithoutSD.java  |   76 +-
 .../metastore/api/PartitionsByExprRequest.java  |  114 +-
 .../metastore/api/PartitionsByExprResult.java   |   36 +-
 .../metastore/api/PartitionsStatsRequest.java   |  176 +-
 .../metastore/api/PartitionsStatsResult.java    |   76 +-
 .../hive/metastore/api/PrimaryKeysRequest.java  |  112 +-
 .../hive/metastore/api/PrimaryKeysResponse.java |   36 +-
 .../metastore/api/PutFileMetadataRequest.java   |   64 +-
 .../hive/metastore/api/RequestPartsSpec.java    |   68 +-
 .../hive/metastore/api/SQLCheckConstraint.java  |  202 +-
 .../metastore/api/SQLDefaultConstraint.java     |  202 +-
 .../hive/metastore/api/SQLForeignKey.java       |  113 +-
 .../metastore/api/SQLNotNullConstraint.java     |  190 +-
 .../hive/metastore/api/SQLPrimaryKey.java       |  113 +-
 .../hive/metastore/api/SQLUniqueConstraint.java |  202 +-
 .../hadoop/hive/metastore/api/Schema.java       |   80 +-
 .../hive/metastore/api/SchemaVersion.java       |   36 +-
 .../hadoop/hive/metastore/api/SerDeInfo.java    |   44 +-
 .../api/SetPartitionsStatsRequest.java          |   36 +-
 .../hive/metastore/api/ShowCompactResponse.java |   36 +-
 .../hive/metastore/api/ShowLocksResponse.java   |   36 +-
 .../hadoop/hive/metastore/api/SkewedInfo.java   |  164 +-
 .../hive/metastore/api/StorageDescriptor.java   |  148 +-
 .../apache/hadoop/hive/metastore/api/Table.java |  194 +-
 .../hadoop/hive/metastore/api/TableMeta.java    |  114 +-
 .../hive/metastore/api/TableStatsRequest.java   |  144 +-
 .../hive/metastore/api/TableStatsResult.java    |   36 +-
 .../hive/metastore/api/TableValidWriteIds.java  |   32 +-
 .../hive/metastore/api/ThriftHiveMetastore.java | 7313 ++++++++++++++----
 .../metastore/api/UniqueConstraintsRequest.java |  117 +-
 .../api/UniqueConstraintsResponse.java          |   36 +-
 .../hive/metastore/api/WMFullResourcePlan.java  |  144 +-
 .../api/WMGetAllResourcePlanResponse.java       |   36 +-
 .../WMGetTriggersForResourePlanResponse.java    |   36 +-
 .../api/WMValidateResourcePlanResponse.java     |   64 +-
 .../gen-php/metastore/ThriftHiveMetastore.php   | 3016 +++++---
 .../src/gen/thrift/gen-php/metastore/Types.php  | 3627 ++++++---
 .../hive_metastore/ThriftHiveMetastore-remote   |   36 +-
 .../hive_metastore/ThriftHiveMetastore.py       | 2106 +++--
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  | 2793 ++++---
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |  325 +-
 .../gen/thrift/gen-rb/thrift_hive_metastore.rb  |  280 +-
 .../hive/metastore/AggregateStatsCache.java     |   46 +-
 .../hadoop/hive/metastore/AlterHandler.java     |   29 +-
 .../DefaultMetaStoreFilterHookImpl.java         |   11 +-
 .../hadoop/hive/metastore/HiveAlterHandler.java |  134 +-
 .../hadoop/hive/metastore/HiveMetaStore.java    | 1253 ++-
 .../hive/metastore/HiveMetaStoreClient.java     | 1304 ++--
 .../hadoop/hive/metastore/IHMSHandler.java      |   25 +-
 .../hadoop/hive/metastore/IMetaStoreClient.java | 2119 ++++-
 .../MaterializationsInvalidationCache.java      |   11 +-
 .../hive/metastore/MetaStoreDirectSql.java      |  290 +-
 .../hive/metastore/MetaStoreEventListener.java  |    8 +
 .../hive/metastore/MetaStoreFilterHook.java     |   67 +-
 .../metastore/MetaStoreListenerNotifier.java    |    6 +
 .../hadoop/hive/metastore/ObjectStore.java      | 1805 +++--
 .../apache/hadoop/hive/metastore/RawStore.java  | 1008 ++-
 .../hive/metastore/StatObjectConverter.java     |    6 +
 .../TransactionalValidationListener.java        |    4 +-
 .../apache/hadoop/hive/metastore/Warehouse.java |   80 +-
 .../hadoop/hive/metastore/cache/CacheUtils.java |   40 +-
 .../hive/metastore/cache/CachedStore.java       |  938 ++-
 .../hive/metastore/cache/SharedCache.java       |  380 +-
 .../client/builder/CatalogBuilder.java          |   62 +
 .../client/builder/ConstraintBuilder.java       |   47 +-
 .../client/builder/DatabaseBuilder.java         |   39 +-
 .../client/builder/FunctionBuilder.java         |   37 +-
 .../client/builder/ISchemaBuilder.java          |   12 +-
 .../client/builder/PartitionBuilder.java        |   28 +-
 .../builder/SQLCheckConstraintBuilder.java      |   51 +
 .../builder/SQLDefaultConstraintBuilder.java    |   51 +
 .../client/builder/SQLForeignKeyBuilder.java    |   48 +-
 .../builder/SQLNotNullConstraintBuilder.java    |   23 +-
 .../client/builder/SQLPrimaryKeyBuilder.java    |   18 +-
 .../builder/SQLUniqueConstraintBuilder.java     |   17 +-
 .../client/builder/SchemaVersionBuilder.java    |   14 +-
 .../metastore/client/builder/TableBuilder.java  |   60 +-
 .../hive/metastore/conf/MetastoreConf.java      |    6 +
 .../metastore/events/CreateCatalogEvent.java    |   39 +
 .../hive/metastore/events/DropCatalogEvent.java |   39 +
 .../metastore/events/DropConstraintEvent.java   |    8 +-
 .../hive/metastore/events/InsertEvent.java      |    6 +-
 .../metastore/events/PreCreateCatalogEvent.java |   39 +
 .../metastore/events/PreDropCatalogEvent.java   |   39 +
 .../hive/metastore/events/PreEventContext.java  |    5 +-
 .../events/PreLoadPartitionDoneEvent.java       |    8 +-
 .../metastore/events/PreReadCatalogEvent.java   |   39 +
 .../messaging/CreateCatalogMessage.java         |   25 +
 .../metastore/messaging/DropCatalogMessage.java |   25 +
 .../hive/metastore/messaging/EventMessage.java  |    4 +-
 .../metastore/messaging/MessageFactory.java     |    7 +
 .../json/JSONCreateCatalogMessage.java          |   79 +
 .../messaging/json/JSONDropCatalogMessage.java  |   66 +
 .../messaging/json/JSONMessageFactory.java      |   13 +
 .../hadoop/hive/metastore/model/MCatalog.java   |   58 +
 .../hive/metastore/model/MCreationMetadata.java |   12 +-
 .../hadoop/hive/metastore/model/MDatabase.java  |   12 +-
 .../hive/metastore/model/MNotificationLog.java  |   12 +-
 .../model/MPartitionColumnStatistics.java       |    9 +
 .../hive/metastore/model/MPartitionEvent.java   |    9 +-
 .../metastore/model/MTableColumnStatistics.java |    9 +
 .../spec/CompositePartitionSpecProxy.java       |   32 +
 .../spec/PartitionListComposingSpecProxy.java   |   18 +
 .../partition/spec/PartitionSpecProxy.java      |   19 +
 .../spec/PartitionSpecWithSharedSDProxy.java    |   19 +-
 .../hadoop/hive/metastore/tools/SmokeTest.java  |   17 +-
 .../hive/metastore/utils/MetaStoreUtils.java    |  143 +-
 .../src/main/resources/package.jdo              |   39 +-
 .../main/sql/derby/hive-schema-3.0.0.derby.sql  |  103 +-
 .../sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql  |   58 +
 .../main/sql/mssql/hive-schema-3.0.0.mssql.sql  |   27 +-
 .../sql/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql  |   59 +
 .../main/sql/mysql/hive-schema-3.0.0.mysql.sql  |   20 +-
 .../sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql  |   57 +
 .../sql/oracle/hive-schema-3.0.0.oracle.sql     |   22 +-
 .../oracle/upgrade-2.3.0-to-3.0.0.oracle.sql    |   58 +-
 .../sql/postgres/hive-schema-3.0.0.postgres.sql |   18 +-
 .../upgrade-2.3.0-to-3.0.0.postgres.sql         |   56 +
 .../src/main/thrift/hive_metastore.thrift       |  202 +-
 .../DummyRawStoreControlledCommit.java          |  308 +-
 .../DummyRawStoreForJdoConnection.java          |  201 +-
 .../HiveMetaStoreClientPreCatalog.java          | 3286 ++++++++
 .../InjectableBehaviourObjectStore.java         |    8 +-
 .../hive/metastore/MetaStoreTestUtils.java      |   29 +
 .../hive/metastore/TestAggregateStatsCache.java |   26 +-
 .../metastore/TestCatalogNonDefaultClient.java  |   74 +
 .../metastore/TestCatalogNonDefaultSvr.java     |   70 +
 .../hive/metastore/TestCatalogOldClient.java    |   44 +
 .../hadoop/hive/metastore/TestFilterHooks.java  |   39 +-
 .../hive/metastore/TestHiveAlterHandler.java    |   16 +-
 .../hive/metastore/TestHiveMetaStore.java       |  227 +-
 .../TestHiveMetaStorePartitionSpecs.java        |    9 +-
 .../TestHiveMetaStoreSchemaMethods.java         |  214 +-
 .../metastore/TestHiveMetaStoreTimeout.java     |   21 +-
 ...TestHiveMetaStoreWithEnvironmentContext.java |   10 +-
 .../hive/metastore/TestMarkPartition.java       |   10 +-
 .../TestMetaStoreEndFunctionListener.java       |   11 +-
 .../metastore/TestMetaStoreEventListener.java   |   21 +-
 .../TestMetaStoreEventListenerOnlyOnCommit.java |   24 +-
 ...stMetaStoreMaterializationsCacheCleaner.java |    9 +-
 .../metastore/TestNonCatCallsWithCatalog.java   | 1126 +++
 .../hadoop/hive/metastore/TestObjectStore.java  |  204 +-
 .../metastore/TestObjectStoreSchemaMethods.java |  145 +-
 .../hadoop/hive/metastore/TestOldSchema.java    |   29 +-
 .../TestRemoteHiveMetaStoreIpAddress.java       |    7 +-
 .../hive/metastore/TestRetryingHMSHandler.java  |   13 +-
 .../apache/hadoop/hive/metastore/TestStats.java |  728 ++
 .../hive/metastore/VerifyingObjectStore.java    |   40 +-
 .../hive/metastore/cache/TestCachedStore.java   |  226 +-
 .../metastore/cache/TestCatalogCaching.java     |  142 +
 .../client/MetaStoreFactoryForTests.java        |    2 +
 .../metastore/client/TestAddPartitions.java     |  120 +-
 .../client/TestAddPartitionsFromPartSpec.java   |   22 +-
 .../metastore/client/TestAlterPartitions.java   |  227 +-
 .../metastore/client/TestAppendPartitions.java  |   90 +-
 .../hive/metastore/client/TestCatalogs.java     |  217 +
 .../metastore/client/TestCheckConstraint.java   |  360 +
 .../hive/metastore/client/TestDatabases.java    |  184 +-
 .../metastore/client/TestDefaultConstraint.java |  360 +
 .../metastore/client/TestDropPartitions.java    |   96 +-
 .../client/TestExchangePartitions.java          |   13 +-
 .../hive/metastore/client/TestForeignKey.java   |  535 ++
 .../hive/metastore/client/TestFunctions.java    |  126 +-
 .../metastore/client/TestGetPartitions.java     |  120 +-
 .../hive/metastore/client/TestGetTableMeta.java |   89 +-
 .../metastore/client/TestListPartitions.java    |  236 +-
 .../metastore/client/TestNotNullConstraint.java |  352 +
 .../hive/metastore/client/TestPrimaryKey.java   |  465 ++
 .../TestTablesCreateDropAlterTruncate.java      |  337 +-
 .../metastore/client/TestTablesGetExists.java   |  129 +-
 .../hive/metastore/client/TestTablesList.java   |   73 +-
 .../metastore/client/TestUniqueConstraint.java  |  353 +
 .../minihms/AbstractMetaStoreService.java       |    4 +
 .../tools/TestSchemaToolForMetastore.java       |   31 +-
 .../src/test/resources/log4j2.properties        |   74 +-
 274 files changed, 51952 insertions(+), 17293 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/hcatalog/core/src/main/java/org/apache/hive/hcatalog/cli/SemanticAnalysis/CreateTableHook.java
----------------------------------------------------------------------
diff --git a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/cli/SemanticAnalysis/CreateTableHook.java b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/cli/SemanticAnalysis/CreateTableHook.java
index b6fe9ce..a377805 100644
--- a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/cli/SemanticAnalysis/CreateTableHook.java
+++ b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/cli/SemanticAnalysis/CreateTableHook.java
@@ -106,7 +106,7 @@ final class CreateTableHook extends HCatSemanticAnalyzerBase {
 
       case HiveParser.TOK_TABLEPARTCOLS:
         List<FieldSchema> partCols = BaseSemanticAnalyzer
-          .getColumns(child, false);
+          .getColumns(child, false, context.getConf());
         for (FieldSchema fs : partCols) {
           if (!fs.getType().equalsIgnoreCase("string")) {
             throw new SemanticException(

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestPermsGrp.java
----------------------------------------------------------------------
diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestPermsGrp.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestPermsGrp.java
index a686579..8a2151c 100644
--- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestPermsGrp.java
+++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestPermsGrp.java
@@ -156,7 +156,7 @@ public class TestPermsGrp extends TestCase {
         fail();
       } catch (Exception e) {
         assertTrue(e instanceof NoSuchObjectException);
-        assertEquals("default.simptbl table not found", e.getMessage());
+        assertEquals("hive.default.simptbl table not found", e.getMessage());
       }
 
       // test for invalid group name
@@ -177,7 +177,7 @@ public class TestPermsGrp extends TestCase {
         fail();
       } catch (Exception e) {
         assertTrue(e instanceof NoSuchObjectException);
-        assertEquals("default.simptbl table not found", e.getMessage());
+        assertEquals("hive.default.simptbl table not found", e.getMessage());
       }
       try {
         // neither dir should get created.

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
----------------------------------------------------------------------
diff --git a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
index d647181..8523428 100644
--- a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
+++ b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
@@ -74,6 +74,8 @@ import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.Lists;
 
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
+
 /**
  * An implementation of {@link org.apache.hadoop.hive.metastore.MetaStoreEventListener} that
  * stores events in the database.
@@ -140,6 +142,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
     NotificationEvent event =
         new NotificationEvent(0, now(), EventType.CREATE_TABLE.toString(), msgFactory
             .buildCreateTableMessage(t, new FileIterator(t.getSd().getLocation())).toString());
+    event.setCatName(t.isSetCatName() ? t.getCatName() : DEFAULT_CATALOG_NAME);
     event.setDbName(t.getDbName());
     event.setTableName(t.getTableName());
     process(event, tableEvent);
@@ -155,6 +158,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
     NotificationEvent event =
         new NotificationEvent(0, now(), EventType.DROP_TABLE.toString(), msgFactory
             .buildDropTableMessage(t).toString());
+    event.setCatName(t.isSetCatName() ? t.getCatName() : DEFAULT_CATALOG_NAME);
     event.setDbName(t.getDbName());
     event.setTableName(t.getTableName());
     process(event, tableEvent);
@@ -171,6 +175,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
     NotificationEvent event =
         new NotificationEvent(0, now(), EventType.ALTER_TABLE.toString(), msgFactory
             .buildAlterTableMessage(before, after, tableEvent.getIsTruncateOp()).toString());
+    event.setCatName(after.isSetCatName() ? after.getCatName() : DEFAULT_CATALOG_NAME);
     event.setDbName(after.getDbName());
     event.setTableName(after.getTableName());
     process(event, tableEvent);
@@ -279,6 +284,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
             new PartitionFilesIterator(partitionEvent.getPartitionIterator(), t)).toString();
     NotificationEvent event =
         new NotificationEvent(0, now(), EventType.ADD_PARTITION.toString(), msg);
+    event.setCatName(t.isSetCatName() ? t.getCatName() : DEFAULT_CATALOG_NAME);
     event.setDbName(t.getDbName());
     event.setTableName(t.getTableName());
     process(event, partitionEvent);
@@ -294,6 +300,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
     NotificationEvent event =
         new NotificationEvent(0, now(), EventType.DROP_PARTITION.toString(), msgFactory
             .buildDropPartitionMessage(t, partitionEvent.getPartitionIterator()).toString());
+    event.setCatName(t.isSetCatName() ? t.getCatName() : DEFAULT_CATALOG_NAME);
     event.setDbName(t.getDbName());
     event.setTableName(t.getTableName());
     process(event, partitionEvent);
@@ -310,6 +317,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
     NotificationEvent event =
         new NotificationEvent(0, now(), EventType.ALTER_PARTITION.toString(), msgFactory
             .buildAlterPartitionMessage(partitionEvent.getTable(), before, after, partitionEvent.getIsTruncateOp()).toString());
+    event.setCatName(before.isSetCatName() ? before.getCatName() : DEFAULT_CATALOG_NAME);
     event.setDbName(before.getDbName());
     event.setTableName(before.getTableName());
     process(event, partitionEvent);
@@ -325,6 +333,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
     NotificationEvent event =
         new NotificationEvent(0, now(), EventType.CREATE_DATABASE.toString(), msgFactory
             .buildCreateDatabaseMessage(db).toString());
+    event.setCatName(db.isSetCatalogName() ? db.getCatalogName() : DEFAULT_CATALOG_NAME);
     event.setDbName(db.getName());
     process(event, dbEvent);
   }
@@ -339,6 +348,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
     NotificationEvent event =
         new NotificationEvent(0, now(), EventType.DROP_DATABASE.toString(), msgFactory
             .buildDropDatabaseMessage(db).toString());
+    event.setCatName(db.isSetCatalogName() ? db.getCatalogName() : DEFAULT_CATALOG_NAME);
     event.setDbName(db.getName());
     process(event, dbEvent);
   }
@@ -354,6 +364,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
     NotificationEvent event =
             new NotificationEvent(0, now(), EventType.ALTER_DATABASE.toString(), msgFactory
                     .buildAlterDatabaseMessage(oldDb, newDb).toString());
+    event.setCatName(oldDb.isSetCatalogName() ? oldDb.getCatalogName() : DEFAULT_CATALOG_NAME);
     event.setDbName(oldDb.getName());
     process(event, dbEvent);
   }
@@ -368,6 +379,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
     NotificationEvent event =
         new NotificationEvent(0, now(), EventType.CREATE_FUNCTION.toString(), msgFactory
             .buildCreateFunctionMessage(fn).toString());
+    event.setCatName(fn.isSetCatName() ? fn.getCatName() : DEFAULT_CATALOG_NAME);
     event.setDbName(fn.getDbName());
     process(event, fnEvent);
   }
@@ -382,6 +394,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
     NotificationEvent event =
         new NotificationEvent(0, now(), EventType.DROP_FUNCTION.toString(), msgFactory
             .buildDropFunctionMessage(fn).toString());
+    event.setCatName(fn.isSetCatName() ? fn.getCatName() : DEFAULT_CATALOG_NAME);
     event.setDbName(fn.getDbName());
     process(event, fnEvent);
   }
@@ -419,6 +432,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
                 insertEvent.getPartitionObj(), insertEvent.isReplace(),
             new FileChksumIterator(insertEvent.getFiles(), insertEvent.getFileChecksums()))
                 .toString());
+    event.setCatName(tableObj.isSetCatName() ? tableObj.getCatName() : DEFAULT_CATALOG_NAME);
     event.setDbName(tableObj.getDbName());
     event.setTableName(tableObj.getTableName());
     process(event, insertEvent);
@@ -445,6 +459,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
       NotificationEvent event =
           new NotificationEvent(0, now(), EventType.ADD_PRIMARYKEY.toString(), msgFactory
               .buildAddPrimaryKeyMessage(addPrimaryKeyEvent.getPrimaryKeyCols()).toString());
+      event.setCatName(cols.get(0).isSetCatName() ? cols.get(0).getCatName() : DEFAULT_CATALOG_NAME);
       event.setDbName(cols.get(0).getTable_db());
       event.setTableName(cols.get(0).getTable_name());
       process(event, addPrimaryKeyEvent);
@@ -462,6 +477,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
       NotificationEvent event =
           new NotificationEvent(0, now(), EventType.ADD_FOREIGNKEY.toString(), msgFactory
               .buildAddForeignKeyMessage(addForeignKeyEvent.getForeignKeyCols()).toString());
+      event.setCatName(cols.get(0).isSetCatName() ? cols.get(0).getCatName() : DEFAULT_CATALOG_NAME);
       event.setDbName(cols.get(0).getPktable_db());
       event.setTableName(cols.get(0).getPktable_name());
       process(event, addForeignKeyEvent);
@@ -479,6 +495,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
       NotificationEvent event =
           new NotificationEvent(0, now(), EventType.ADD_UNIQUECONSTRAINT.toString(), msgFactory
               .buildAddUniqueConstraintMessage(addUniqueConstraintEvent.getUniqueConstraintCols()).toString());
+      event.setCatName(cols.get(0).isSetCatName() ? cols.get(0).getCatName() : DEFAULT_CATALOG_NAME);
       event.setDbName(cols.get(0).getTable_db());
       event.setTableName(cols.get(0).getTable_name());
       process(event, addUniqueConstraintEvent);
@@ -496,6 +513,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
       NotificationEvent event =
           new NotificationEvent(0, now(), EventType.ADD_NOTNULLCONSTRAINT.toString(), msgFactory
               .buildAddNotNullConstraintMessage(addNotNullConstraintEvent.getNotNullConstraintCols()).toString());
+      event.setCatName(cols.get(0).isSetCatName() ? cols.get(0).getCatName() : DEFAULT_CATALOG_NAME);
       event.setDbName(cols.get(0).getTable_db());
       event.setTableName(cols.get(0).getTable_name());
       process(event, addNotNullConstraintEvent);
@@ -514,6 +532,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
     NotificationEvent event =
         new NotificationEvent(0, now(), EventType.DROP_CONSTRAINT.toString(), msgFactory
             .buildDropConstraintMessage(dbName, tableName, constraintName).toString());
+    event.setCatName(dropConstraintEvent.getCatName());
     event.setDbName(dbName);
     event.setTableName(tableName);
     process(event, dropConstraintEvent);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/NotificationListener.java
----------------------------------------------------------------------
diff --git a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/NotificationListener.java b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/NotificationListener.java
index 53246a0..649d901 100644
--- a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/NotificationListener.java
+++ b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/NotificationListener.java
@@ -207,7 +207,7 @@ public class NotificationListener extends MetaStoreEventListener {
       Configuration conf = handler.getConf();
       Table newTbl;
       try {
-        newTbl = handler.get_table_core(tbl.getDbName(), tbl.getTableName())
+        newTbl = handler.get_table_core(tbl.getCatName(), tbl.getDbName(), tbl.getTableName())
           .deepCopy();
         newTbl.getParameters().put(
           HCatConstants.HCAT_MSGBUS_TOPIC_NAME,

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
----------------------------------------------------------------------
diff --git a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
index 5cc407a..4697f60 100644
--- a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
+++ b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
@@ -20,6 +20,7 @@ package org.apache.hive.hcatalog.listener;
 
 import org.apache.hadoop.hive.metastore.api.ISchemaName;
 import org.apache.hadoop.hive.metastore.api.SchemaVersionDescriptor;
+import org.apache.hadoop.hive.metastore.api.Catalog;
 import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
@@ -146,6 +147,40 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
   }
 
   @Override
+  public void createCatalog(Catalog cat) throws MetaException {
+    if (shouldEventSucceed) {
+      objectStore.createCatalog(cat);
+    } else {
+      throw new RuntimeException("Failed event");
+    }
+  }
+
+  @Override
+  public void alterCatalog(String catName, Catalog cat) throws MetaException,
+      InvalidOperationException {
+    objectStore.alterCatalog(catName, cat);
+  }
+
+  @Override
+  public Catalog getCatalog(String catalogName) throws NoSuchObjectException, MetaException {
+    return objectStore.getCatalog(catalogName);
+  }
+
+  @Override
+  public List<String> getCatalogs() throws MetaException {
+    return objectStore.getCatalogs();
+  }
+
+  @Override
+  public void dropCatalog(String catalogName) throws NoSuchObjectException, MetaException {
+    if (shouldEventSucceed) {
+      objectStore.dropCatalog(catalogName);
+    } else {
+      throw new RuntimeException("Event failed.");
+    }
+  }
+
+  @Override
   public void createDatabase(Database db) throws InvalidObjectException, MetaException {
     if (shouldEventSucceed) {
       objectStore.createDatabase(db);
@@ -155,34 +190,34 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
   }
 
   @Override
-  public Database getDatabase(String dbName) throws NoSuchObjectException {
-    return objectStore.getDatabase(dbName);
+  public Database getDatabase(String catName, String dbName) throws NoSuchObjectException {
+    return objectStore.getDatabase(catName, dbName);
   }
 
   @Override
-  public boolean dropDatabase(String dbName)
+  public boolean dropDatabase(String catName, String dbName)
       throws NoSuchObjectException, MetaException {
     if (shouldEventSucceed) {
-      return objectStore.dropDatabase(dbName);
+      return objectStore.dropDatabase(catName, dbName);
     } else {
       throw new RuntimeException("Event failed.");
     }
   }
 
   @Override
-  public boolean alterDatabase(String dbName, Database db)
+  public boolean alterDatabase(String catName, String dbName, Database db)
       throws NoSuchObjectException, MetaException {
-    return objectStore.alterDatabase(dbName, db);
+    return objectStore.alterDatabase(catName, dbName, db);
   }
 
   @Override
-  public List<String> getDatabases(String pattern) throws MetaException {
-    return objectStore.getDatabases(pattern);
+  public List<String> getDatabases(String catName, String pattern) throws MetaException {
+    return objectStore.getDatabases(catName, pattern);
   }
 
   @Override
-  public List<String> getAllDatabases() throws MetaException {
-    return objectStore.getAllDatabases();
+  public List<String> getAllDatabases(String catName) throws MetaException {
+    return objectStore.getAllDatabases(catName);
   }
 
   @Override
@@ -210,19 +245,19 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
   }
 
   @Override
-  public boolean dropTable(String dbName, String tableName)
+  public boolean dropTable(String catName, String dbName, String tableName)
       throws MetaException, NoSuchObjectException,
       InvalidObjectException, InvalidInputException {
     if (shouldEventSucceed) {
-      return objectStore.dropTable(dbName, tableName);
+      return objectStore.dropTable(catName, dbName, tableName);
     } else {
       throw new RuntimeException("Event failed.");
     }
   }
 
   @Override
-  public Table getTable(String dbName, String tableName) throws MetaException {
-    return objectStore.getTable(dbName, tableName);
+  public Table getTable(String catName, String dbName, String tableName) throws MetaException {
+    return objectStore.getTable(catName, dbName, tableName);
   }
 
   @Override
@@ -232,162 +267,159 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
   }
 
   @Override
-  public Partition getPartition(String dbName, String tableName, List<String> partVals)
+  public Partition getPartition(String catName, String dbName, String tableName, List<String> partVals)
       throws MetaException, NoSuchObjectException {
-    return objectStore.getPartition(dbName, tableName, partVals);
+    return objectStore.getPartition(catName, dbName, tableName, partVals);
   }
 
   @Override
-  public boolean dropPartition(String dbName, String tableName, List<String> partVals)
+  public boolean dropPartition(String catName, String dbName, String tableName, List<String> partVals)
       throws MetaException, NoSuchObjectException,
       InvalidObjectException, InvalidInputException {
     if (shouldEventSucceed) {
-      return objectStore.dropPartition(dbName, tableName, partVals);
+      return objectStore.dropPartition(catName, dbName, tableName, partVals);
     } else {
       throw new RuntimeException("Event failed.");
     }
   }
 
   @Override
-  public List<Partition> getPartitions(String dbName, String tableName, int max)
+  public List<Partition> getPartitions(String catName, String dbName, String tableName, int max)
       throws MetaException, NoSuchObjectException {
-    return objectStore.getPartitions(dbName, tableName, max);
+    return objectStore.getPartitions(catName, dbName, tableName, max);
   }
 
   @Override
-  public void updateCreationMetadata(String dbname, String tablename, CreationMetadata cm)
+  public void updateCreationMetadata(String catName, String dbname, String tablename, CreationMetadata cm)
       throws MetaException {
-    objectStore.updateCreationMetadata(dbname, tablename, cm);
+    objectStore.updateCreationMetadata(catName, dbname, tablename, cm);
   }
 
-  @Override
-  public void alterTable(String dbName, String name, Table newTable)
+  public void alterTable(String catName, String dbName, String name, Table newTable)
       throws InvalidObjectException, MetaException {
     if (shouldEventSucceed) {
-      objectStore.alterTable(dbName, name, newTable);
+      objectStore.alterTable(catName, dbName, name, newTable);
     } else {
       throw new RuntimeException("Event failed.");
     }
   }
 
   @Override
-  public List<String> getTables(String dbName, String pattern) throws MetaException {
-    return objectStore.getTables(dbName, pattern);
+  public List<String> getTables(String catName, String dbName, String pattern) throws MetaException {
+    return objectStore.getTables(catName, dbName, pattern);
   }
 
   @Override
-  public List<String> getTables(String dbName, String pattern, TableType tableType) throws MetaException {
-    return objectStore.getTables(dbName, pattern, tableType);
+  public List<String> getTables(String catName, String dbName, String pattern, TableType tableType) throws MetaException {
+    return objectStore.getTables(catName, dbName, pattern, tableType);
   }
 
   @Override
-  public List<String> getMaterializedViewsForRewriting(String dbName)
+  public List<String> getMaterializedViewsForRewriting(String catName, String dbName)
       throws MetaException, NoSuchObjectException {
-    return objectStore.getMaterializedViewsForRewriting(dbName);
+    return objectStore.getMaterializedViewsForRewriting(catName, dbName);
   }
 
   @Override
-  public List<TableMeta> getTableMeta(String dbNames, String tableNames, List<String> tableTypes)
+  public List<TableMeta> getTableMeta(String catName, String dbNames, String tableNames, List<String> tableTypes)
       throws MetaException {
-    return objectStore.getTableMeta(dbNames, tableNames, tableTypes);
+    return objectStore.getTableMeta(catName, dbNames, tableNames, tableTypes);
   }
 
   @Override
-  public List<Table> getTableObjectsByName(String dbName, List<String> tableNames)
+  public List<Table> getTableObjectsByName(String catName, String dbName, List<String> tableNames)
       throws MetaException, UnknownDBException {
-    return objectStore.getTableObjectsByName(dbName, tableNames);
+    return objectStore.getTableObjectsByName(catName, dbName, tableNames);
   }
 
   @Override
-  public List<String> getAllTables(String dbName) throws MetaException {
-    return objectStore.getAllTables(dbName);
+  public List<String> getAllTables(String catName, String dbName) throws MetaException {
+    return objectStore.getAllTables(catName, dbName);
   }
 
   @Override
-  public List<String> listTableNamesByFilter(String dbName, String filter,
+  public List<String> listTableNamesByFilter(String catName, String dbName, String filter,
                                              short maxTables) throws MetaException, UnknownDBException {
-    return objectStore.listTableNamesByFilter(dbName, filter, maxTables);
+    return objectStore.listTableNamesByFilter(catName, dbName, filter, maxTables);
   }
 
   @Override
-  public List<String> listPartitionNames(String dbName, String tblName, short maxParts)
+  public List<String> listPartitionNames(String catName, String dbName, String tblName, short maxParts)
       throws MetaException {
-    return objectStore.listPartitionNames(dbName, tblName, maxParts);
+    return objectStore.listPartitionNames(catName, dbName, tblName, maxParts);
   }
 
   @Override
-  public PartitionValuesResponse listPartitionValues(String db_name, String tbl_name, List<FieldSchema> cols, boolean applyDistinct, String filter, boolean ascending, List<FieldSchema> order, long maxParts) throws MetaException {
+  public PartitionValuesResponse listPartitionValues(String catName, String db_name,
+                                                     String tbl_name, List<FieldSchema> cols,
+                                                     boolean applyDistinct, String filter,
+                                                     boolean ascending, List<FieldSchema> order,
+                                                     long maxParts) throws MetaException {
     return null;
   }
 
   @Override
-  public List<String> listPartitionNamesByFilter(String dbName, String tblName,
-                                                 String filter, short maxParts) throws MetaException {
-    return objectStore.listPartitionNamesByFilter(dbName, tblName, filter, maxParts);
-  }
-
-  @Override
-  public void alterPartition(String dbName, String tblName, List<String> partVals,
+  public void alterPartition(String catName, String dbName, String tblName, List<String> partVals,
                              Partition newPart) throws InvalidObjectException, MetaException {
     if (shouldEventSucceed) {
-      objectStore.alterPartition(dbName, tblName, partVals, newPart);
+      objectStore.alterPartition(catName, dbName, tblName, partVals, newPart);
     } else {
       throw new RuntimeException("Event failed.");
     }
   }
 
   @Override
-  public void alterPartitions(String dbName, String tblName,
+  public void alterPartitions(String catName, String dbName, String tblName,
                               List<List<String>> partValsList, List<Partition> newParts)
       throws InvalidObjectException, MetaException {
-    objectStore.alterPartitions(dbName, tblName, partValsList, newParts);
+    objectStore.alterPartitions(catName, dbName, tblName, partValsList, newParts);
   }
 
   @Override
-  public List<Partition> getPartitionsByFilter(String dbName, String tblName,
+  public List<Partition> getPartitionsByFilter(String catName, String dbName, String tblName,
                                                String filter, short maxParts) throws MetaException, NoSuchObjectException {
-    return objectStore.getPartitionsByFilter(dbName, tblName, filter, maxParts);
+    return objectStore.getPartitionsByFilter(catName, dbName, tblName, filter, maxParts);
   }
 
   @Override
-  public int getNumPartitionsByFilter(String dbName, String tblName,
+  public int getNumPartitionsByFilter(String catName, String dbName, String tblName,
                                       String filter) throws MetaException, NoSuchObjectException {
-    return objectStore.getNumPartitionsByFilter(dbName, tblName, filter);
+    return objectStore.getNumPartitionsByFilter(catName, dbName, tblName, filter);
   }
 
   @Override
-  public int getNumPartitionsByExpr(String dbName, String tblName,
+  public int getNumPartitionsByExpr(String catName, String dbName, String tblName,
                                     byte[] expr) throws MetaException, NoSuchObjectException {
-    return objectStore.getNumPartitionsByExpr(dbName, tblName, expr);
+    return objectStore.getNumPartitionsByExpr(catName, dbName, tblName, expr);
   }
 
   @Override
-  public List<Partition> getPartitionsByNames(String dbName, String tblName,
+  public List<Partition> getPartitionsByNames(String catName, String dbName, String tblName,
                                               List<String> partNames) throws MetaException, NoSuchObjectException {
-    return objectStore.getPartitionsByNames(dbName, tblName, partNames);
+    return objectStore.getPartitionsByNames(catName, dbName, tblName, partNames);
   }
 
   @Override
-  public boolean getPartitionsByExpr(String dbName, String tblName, byte[] expr,
+  public boolean getPartitionsByExpr(String catName, String dbName, String tblName, byte[] expr,
                                      String defaultPartitionName, short maxParts, List<Partition> result) throws TException {
-    return objectStore.getPartitionsByExpr(
+    return objectStore.getPartitionsByExpr(catName,
         dbName, tblName, expr, defaultPartitionName, maxParts, result);
   }
 
   @Override
-  public Table markPartitionForEvent(String dbName, String tblName,
+  public Table markPartitionForEvent(String catName, String dbName, String tblName,
                                      Map<String, String> partVals, PartitionEventType evtType)
       throws MetaException, UnknownTableException, InvalidPartitionException,
       UnknownPartitionException {
-    return objectStore.markPartitionForEvent(dbName, tblName, partVals, evtType);
+    return objectStore.markPartitionForEvent(catName, dbName, tblName, partVals, evtType);
   }
 
   @Override
-  public boolean isPartitionMarkedForEvent(String dbName, String tblName,
+  public boolean isPartitionMarkedForEvent(String catName, String dbName, String tblName,
                                            Map<String, String> partName, PartitionEventType evtType)
       throws MetaException, UnknownTableException, InvalidPartitionException,
       UnknownPartitionException {
-    return objectStore.isPartitionMarkedForEvent(dbName, tblName, partName, evtType);
+    return objectStore.isPartitionMarkedForEvent(catName, dbName, tblName, partName, evtType);
   }
 
   @Override
@@ -423,32 +455,32 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
   }
 
   @Override
-  public PrincipalPrivilegeSet getDBPrivilegeSet(String dbName, String userName,
+  public PrincipalPrivilegeSet getDBPrivilegeSet(String catName, String dbName, String userName,
                                                  List<String> groupNames) throws InvalidObjectException, MetaException {
-    return objectStore.getDBPrivilegeSet(dbName, userName, groupNames);
+    return objectStore.getDBPrivilegeSet(catName, dbName, userName, groupNames);
   }
 
   @Override
-  public PrincipalPrivilegeSet getTablePrivilegeSet(String dbName, String tableName,
+  public PrincipalPrivilegeSet getTablePrivilegeSet(String catName, String dbName, String tableName,
                                                     String userName, List<String> groupNames)
       throws InvalidObjectException, MetaException {
-    return objectStore.getTablePrivilegeSet(dbName, tableName, userName, groupNames);
+    return objectStore.getTablePrivilegeSet(catName, dbName, tableName, userName, groupNames);
   }
 
   @Override
-  public PrincipalPrivilegeSet getPartitionPrivilegeSet(String dbName, String tableName,
+  public PrincipalPrivilegeSet getPartitionPrivilegeSet(String catName, String dbName, String tableName,
                                                         String partition, String userName, List<String> groupNames)
       throws InvalidObjectException, MetaException {
-    return objectStore.getPartitionPrivilegeSet(dbName, tableName, partition,
+    return objectStore.getPartitionPrivilegeSet(catName, dbName, tableName, partition,
         userName, groupNames);
   }
 
   @Override
-  public PrincipalPrivilegeSet getColumnPrivilegeSet(String dbName, String tableName,
+  public PrincipalPrivilegeSet getColumnPrivilegeSet(String catName, String dbName, String tableName,
                                                      String partitionName, String columnName, String userName,
                                                      List<String> groupNames)
       throws InvalidObjectException, MetaException {
-    return objectStore.getColumnPrivilegeSet(dbName, tableName, partitionName,
+    return objectStore.getColumnPrivilegeSet(catName, dbName, tableName, partitionName,
         columnName, userName, groupNames);
   }
 
@@ -460,40 +492,40 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
 
   @Override
   public List<HiveObjectPrivilege> listPrincipalDBGrants(String principalName,
-                                                         PrincipalType principalType, String dbName) {
-    return objectStore.listPrincipalDBGrants(principalName, principalType, dbName);
+                                                         PrincipalType principalType, String catName, String dbName) {
+    return objectStore.listPrincipalDBGrants(principalName, principalType, catName, dbName);
   }
 
   @Override
   public List<HiveObjectPrivilege> listAllTableGrants(String principalName,
-                                                      PrincipalType principalType, String dbName, String tableName) {
+                                                      PrincipalType principalType, String catName, String dbName, String tableName) {
     return objectStore.listAllTableGrants(principalName, principalType,
-        dbName, tableName);
+        catName, dbName, tableName);
   }
 
   @Override
   public List<HiveObjectPrivilege> listPrincipalPartitionGrants(String principalName,
-                                                                PrincipalType principalType, String dbName, String tableName,
+                                                                PrincipalType principalType, String catName, String dbName, String tableName,
                                                                 List<String> partValues,
                                                                 String partName) {
     return objectStore.listPrincipalPartitionGrants(principalName, principalType,
-        dbName, tableName, partValues, partName);
+        catName, dbName, tableName, partValues, partName);
   }
 
   @Override
   public List<HiveObjectPrivilege> listPrincipalTableColumnGrants(String principalName,
-                                                                  PrincipalType principalType, String dbName,
+                                                                  PrincipalType principalType, String catName, String dbName,
                                                                   String tableName, String columnName) {
     return objectStore.listPrincipalTableColumnGrants(principalName, principalType,
-        dbName, tableName, columnName);
+        catName, dbName, tableName, columnName);
   }
 
   @Override
   public List<HiveObjectPrivilege> listPrincipalPartitionColumnGrants(
-      String principalName, PrincipalType principalType, String dbName, String tableName,
+      String principalName, PrincipalType principalType, String catName, String dbName, String tableName,
       List<String> partVals, String partName, String columnName) {
     return objectStore.listPrincipalPartitionColumnGrants(principalName, principalType,
-        dbName, tableName, partVals, partName, columnName);
+        catName, dbName, tableName, partVals, partName, columnName);
   }
 
   @Override
@@ -535,34 +567,34 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
   }
 
   @Override
-  public Partition getPartitionWithAuth(String dbName, String tblName,
+  public Partition getPartitionWithAuth(String catName, String dbName, String tblName,
                                         List<String> partVals, String userName, List<String> groupNames)
       throws MetaException, NoSuchObjectException, InvalidObjectException {
-    return objectStore.getPartitionWithAuth(dbName, tblName, partVals, userName,
+    return objectStore.getPartitionWithAuth(catName, dbName, tblName, partVals, userName,
         groupNames);
   }
 
   @Override
-  public List<Partition> getPartitionsWithAuth(String dbName, String tblName,
+  public List<Partition> getPartitionsWithAuth(String catName, String dbName, String tblName,
                                                short maxParts, String userName, List<String> groupNames)
       throws MetaException, NoSuchObjectException, InvalidObjectException {
-    return objectStore.getPartitionsWithAuth(dbName, tblName, maxParts, userName,
+    return objectStore.getPartitionsWithAuth(catName, dbName, tblName, maxParts, userName,
         groupNames);
   }
 
   @Override
-  public List<String> listPartitionNamesPs(String dbName, String tblName,
+  public List<String> listPartitionNamesPs(String catName, String dbName, String tblName,
                                            List<String> partVals, short maxParts)
       throws MetaException, NoSuchObjectException {
-    return objectStore.listPartitionNamesPs(dbName, tblName, partVals, maxParts);
+    return objectStore.listPartitionNamesPs(catName, dbName, tblName, partVals, maxParts);
   }
 
   @Override
-  public List<Partition> listPartitionsPsWithAuth(String dbName, String tblName,
+  public List<Partition> listPartitionsPsWithAuth(String catName, String dbName, String tblName,
                                                   List<String> partVals, short maxParts, String userName,
                                                   List<String> groupNames)
       throws MetaException, InvalidObjectException, NoSuchObjectException {
-    return objectStore.listPartitionsPsWithAuth(dbName, tblName, partVals, maxParts,
+    return objectStore.listPartitionsPsWithAuth(catName, dbName, tblName, partVals, maxParts,
         userName, groupNames);
   }
 
@@ -607,53 +639,52 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
   }
 
   @Override
-  public List<HiveObjectPrivilege> listDBGrantsAll(String dbName) {
-    return objectStore.listDBGrantsAll(dbName);
+  public List<HiveObjectPrivilege> listDBGrantsAll(String catName, String dbName) {
+    return objectStore.listDBGrantsAll(catName, dbName);
   }
 
   @Override
-  public List<HiveObjectPrivilege> listPartitionColumnGrantsAll(String dbName, String tableName,
+  public List<HiveObjectPrivilege> listPartitionColumnGrantsAll(String catName, String dbName, String tableName,
                                                                 String partitionName, String columnName) {
-    return objectStore.listPartitionColumnGrantsAll(dbName, tableName, partitionName, columnName);
+    return objectStore.listPartitionColumnGrantsAll(catName, dbName, tableName, partitionName, columnName);
   }
 
   @Override
-  public List<HiveObjectPrivilege> listTableGrantsAll(String dbName, String tableName) {
-    return objectStore.listTableGrantsAll(dbName, tableName);
+  public List<HiveObjectPrivilege> listTableGrantsAll(String catName, String dbName, String tableName) {
+    return objectStore.listTableGrantsAll(catName, dbName, tableName);
   }
 
   @Override
-  public List<HiveObjectPrivilege> listPartitionGrantsAll(String dbName, String tableName,
+  public List<HiveObjectPrivilege> listPartitionGrantsAll(String catName, String dbName, String tableName,
                                                           String partitionName) {
-    return objectStore.listPartitionGrantsAll(dbName, tableName, partitionName);
+    return objectStore.listPartitionGrantsAll(catName, dbName, tableName, partitionName);
   }
 
   @Override
-  public List<HiveObjectPrivilege> listTableColumnGrantsAll(String dbName, String tableName,
+  public List<HiveObjectPrivilege> listTableColumnGrantsAll(String catName, String dbName, String tableName,
                                                             String columnName) {
-    return objectStore.listTableColumnGrantsAll(dbName, tableName, columnName);
+    return objectStore.listTableColumnGrantsAll(catName, dbName, tableName, columnName);
   }
 
   @Override
-  public ColumnStatistics getTableColumnStatistics(String dbName, String tableName,
+  public ColumnStatistics getTableColumnStatistics(String catName, String dbName, String tableName,
                                                    List<String> colNames) throws MetaException, NoSuchObjectException {
-    return objectStore.getTableColumnStatistics(dbName, tableName, colNames);
+    return objectStore.getTableColumnStatistics(catName, dbName, tableName, colNames);
   }
 
   @Override
-  public boolean deleteTableColumnStatistics(String dbName, String tableName,
+  public boolean deleteTableColumnStatistics(String catName, String dbName, String tableName,
                                              String colName)
-      throws NoSuchObjectException, MetaException, InvalidObjectException,
-      InvalidInputException {
-    return objectStore.deleteTableColumnStatistics(dbName, tableName, colName);
+      throws NoSuchObjectException, MetaException, InvalidObjectException, InvalidInputException {
+    return objectStore.deleteTableColumnStatistics(catName, dbName, tableName, colName);
   }
 
   @Override
-  public boolean deletePartitionColumnStatistics(String dbName, String tableName,
+  public boolean deletePartitionColumnStatistics(String catName, String dbName, String tableName,
                                                  String partName, List<String> partVals, String colName)
       throws NoSuchObjectException, MetaException, InvalidObjectException,
       InvalidInputException {
-    return objectStore.deletePartitionColumnStatistics(dbName, tableName, partName,
+    return objectStore.deletePartitionColumnStatistics(catName, dbName, tableName, partName,
         partVals, colName);
   }
 
@@ -689,7 +720,7 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
 
   @Override
   public List<String> getAllTokenIdentifiers() {
-    return new ArrayList<String>();
+    return new ArrayList<>();
   }
 
   @Override
@@ -727,35 +758,35 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
   }
 
   @Override
-  public List<ColumnStatistics> getPartitionColumnStatistics(String dbName,
+  public List<ColumnStatistics> getPartitionColumnStatistics(String catName, String dbName,
                                                              String tblName, List<String> colNames,
                                                              List<String> partNames)
       throws MetaException, NoSuchObjectException {
-    return objectStore.getPartitionColumnStatistics(dbName, tblName  , colNames, partNames);
+    return objectStore.getPartitionColumnStatistics(catName, dbName, tblName  , colNames, partNames);
   }
 
   @Override
-  public boolean doesPartitionExist(String dbName, String tableName,
+  public boolean doesPartitionExist(String catName, String dbName, String tableName,
                                     List<String> partVals) throws MetaException, NoSuchObjectException {
-    return objectStore.doesPartitionExist(dbName, tableName, partVals);
+    return objectStore.doesPartitionExist(catName, dbName, tableName, partVals);
   }
 
   @Override
-  public boolean addPartitions(String dbName, String tblName, List<Partition> parts)
+  public boolean addPartitions(String catName, String dbName, String tblName, List<Partition> parts)
       throws InvalidObjectException, MetaException {
-    return objectStore.addPartitions(dbName, tblName, parts);
+    return objectStore.addPartitions(catName, dbName, tblName, parts);
   }
 
   @Override
-  public boolean addPartitions(String dbName, String tblName, PartitionSpecProxy partitionSpec,
+  public boolean addPartitions(String catName, String dbName, String tblName, PartitionSpecProxy partitionSpec,
                                boolean ifNotExists) throws InvalidObjectException, MetaException {
     return false;
   }
 
   @Override
-  public void dropPartitions(String dbName, String tblName, List<String> partNames)
+  public void dropPartitions(String catName, String dbName, String tblName, List<String> partNames)
       throws MetaException, NoSuchObjectException {
-    objectStore.dropPartitions(dbName, tblName, partNames);
+    objectStore.dropPartitions(catName, dbName, tblName, partNames);
   }
 
   @Override
@@ -769,42 +800,42 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
   }
 
   @Override
-  public void alterFunction(String dbName, String funcName, Function newFunction)
+  public void alterFunction(String catName, String dbName, String funcName, Function newFunction)
       throws InvalidObjectException, MetaException {
-    objectStore.alterFunction(dbName, funcName, newFunction);
+    objectStore.alterFunction(catName, dbName, funcName, newFunction);
   }
 
   @Override
-  public void dropFunction(String dbName, String funcName)
+  public void dropFunction(String catName, String dbName, String funcName)
       throws MetaException, NoSuchObjectException, InvalidObjectException,
       InvalidInputException {
     if (shouldEventSucceed) {
-      objectStore.dropFunction(dbName, funcName);
+      objectStore.dropFunction(catName, dbName, funcName);
     } else {
       throw new RuntimeException("Event failed.");
     }
   }
 
   @Override
-  public Function getFunction(String dbName, String funcName)
+  public Function getFunction(String catName, String dbName, String funcName)
       throws MetaException {
-    return objectStore.getFunction(dbName, funcName);
+    return objectStore.getFunction(catName, dbName, funcName);
   }
 
   @Override
-  public List<Function> getAllFunctions()
+  public List<Function> getAllFunctions(String catName)
       throws MetaException {
     return Collections.emptyList();
   }
 
   @Override
-  public List<String> getFunctions(String dbName, String pattern)
+  public List<String> getFunctions(String catName, String dbName, String pattern)
       throws MetaException {
-    return objectStore.getFunctions(dbName, pattern);
+    return objectStore.getFunctions(catName, dbName, pattern);
   }
 
   @Override
-  public AggrStats get_aggr_stats_for(String dbName,
+  public AggrStats get_aggr_stats_for(String catName, String dbName,
                                       String tblName, List<String> partNames, List<String> colNames)
       throws MetaException {
     return null;
@@ -882,38 +913,38 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
   }
 
   @Override
-  public List<SQLPrimaryKey> getPrimaryKeys(String db_name, String tbl_name)
+  public List<SQLPrimaryKey> getPrimaryKeys(String catName, String db_name, String tbl_name)
       throws MetaException {
     return null;
   }
 
   @Override
-  public List<SQLForeignKey> getForeignKeys(String parent_db_name,
+  public List<SQLForeignKey> getForeignKeys(String catName, String parent_db_name,
                                             String parent_tbl_name, String foreign_db_name, String foreign_tbl_name)
       throws MetaException {
     return null;
   }
 
   @Override
-  public List<SQLUniqueConstraint> getUniqueConstraints(String db_name, String tbl_name)
+  public List<SQLUniqueConstraint> getUniqueConstraints(String catName, String db_name, String tbl_name)
       throws MetaException {
     return null;
   }
 
   @Override
-  public List<SQLNotNullConstraint> getNotNullConstraints(String db_name, String tbl_name)
+  public List<SQLNotNullConstraint> getNotNullConstraints(String catName, String db_name, String tbl_name)
       throws MetaException {
     return null;
   }
 
   @Override
-  public List<SQLCheckConstraint> getCheckConstraints(String db_name, String tbl_name)
+  public List<SQLCheckConstraint> getCheckConstraints(String catName, String db_name, String tbl_name)
       throws MetaException {
     return null;
   }
 
   @Override
-  public List<SQLDefaultConstraint> getDefaultConstraints(String db_name, String tbl_name)
+  public List<SQLDefaultConstraint> getDefaultConstraints(String catName, String db_name, String tbl_name)
       throws MetaException {
     return null;
   }
@@ -930,8 +961,9 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
   }
 
   @Override
-  public void dropConstraint(String dbName, String tableName,
-                             String constraintName) throws NoSuchObjectException {
+  public void dropConstraint(String catName, String dbName, String tableName,
+                             String constraintName, boolean missingOk)
+      throws NoSuchObjectException {
   }
 
   @Override
@@ -982,7 +1014,7 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
   }
 
   @Override
-  public WMFullResourcePlan getResourcePlan(String name) throws NoSuchObjectException {
+  public WMFullResourcePlan getResourcePlan(String name) throws NoSuchObjectException, MetaException {
     return objectStore.getResourcePlan(name);
   }
 
@@ -1083,6 +1115,13 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
     objectStore.dropWMTriggerToPoolMapping(resourcePlanName, triggerName, poolPath);
   }
 
+  @Override
+  public List<ColStatsObjWithSourceInfo> getPartitionColStatsForDatabase(String catName, String dbName)
+      throws MetaException, NoSuchObjectException {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
   public void createISchema(ISchema schema) throws AlreadyExistsException, MetaException,
       NoSuchObjectException {
     objectStore.createISchema(schema);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestAcidTableSetup.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestAcidTableSetup.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestAcidTableSetup.java
index 62bd94a..505b3c0 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestAcidTableSetup.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestAcidTableSetup.java
@@ -85,7 +85,7 @@ public class TestAcidTableSetup {
           .setTableName(tblName)
           .setTableParams(params)
           .setCols(type.getFields())
-          .build();
+          .build(conf);
       client.createTable(t);
       fail("Expected exception");
     } catch (MetaException e) {
@@ -102,7 +102,7 @@ public class TestAcidTableSetup {
           .setTableName(tblName)
           .setTableParams(params)
           .setCols(type.getFields())
-          .build();
+          .build(conf);
       client.createTable(t);
       fail("Expected exception");
     } catch (MetaException e) {
@@ -119,7 +119,7 @@ public class TestAcidTableSetup {
           .setTableName(tblName)
           .setTableParams(params)
           .setCols(type.getFields())
-          .build();
+          .build(conf);
       client.createTable(t);
       fail("Expected exception");
     } catch (MetaException e) {
@@ -139,7 +139,7 @@ public class TestAcidTableSetup {
           .setTableParams(params)
           .setCols(type.getFields())
           .setBucketCols(bucketCols)
-          .build();
+          .build(conf);
       client.createTable(t);
       fail("Expected exception");
     } catch (MetaException e) {
@@ -158,7 +158,7 @@ public class TestAcidTableSetup {
         .setBucketCols(bucketCols)
         .setInputFormat("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat")
         .setOutputFormat("org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat")
-        .build();
+        .build(conf);
     client.createTable(t);
     assertTrue("CREATE TABLE should succeed",
         "true".equals(t.getParameters().get(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL)));
@@ -188,7 +188,7 @@ public class TestAcidTableSetup {
           .setTableName(tblName)
           .setCols(type.getFields())
           .setInputFormat("org.apache.hadoop.mapred.FileInputFormat")
-          .build();
+          .build(conf);
       client.createTable(t);
       params.put("transactional", "true");
       t.setParameters(params);
@@ -210,7 +210,7 @@ public class TestAcidTableSetup {
         .setBucketCols(bucketCols)
         .setInputFormat("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat")
         .setOutputFormat("org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat")
-        .build();
+        .build(conf);
     client.createTable(t);
     params.put("transactional", "true");
     t.setParameters(params);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreMetrics.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreMetrics.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreMetrics.java
index 368d517..3f7135c 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreMetrics.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreMetrics.java
@@ -62,7 +62,7 @@ public class TestMetaStoreMetrics {
     driver.run("show databases");
 
     //one call by init, one called here.
-    Assert.assertEquals(2, Metrics.getRegistry().getTimers().get("api_get_all_databases").getCount());
+    Assert.assertEquals(2, Metrics.getRegistry().getTimers().get("api_get_databases").getCount());
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
index 0940938..7f1b662 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
@@ -82,6 +82,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertEquals;
@@ -3075,11 +3076,11 @@ public class TestReplicationScenarios {
     try {
       List<SQLPrimaryKey> pks = metaStoreClientMirror.getPrimaryKeys(new PrimaryKeysRequest(dbName+ "_dupe" , "tbl1"));
       assertEquals(pks.size(), 2);
-      List<SQLUniqueConstraint> uks = metaStoreClientMirror.getUniqueConstraints(new UniqueConstraintsRequest(dbName+ "_dupe" , "tbl3"));
+      List<SQLUniqueConstraint> uks = metaStoreClientMirror.getUniqueConstraints(new UniqueConstraintsRequest(DEFAULT_CATALOG_NAME, dbName+ "_dupe" , "tbl3"));
       assertEquals(uks.size(), 1);
       List<SQLForeignKey> fks = metaStoreClientMirror.getForeignKeys(new ForeignKeysRequest(null, null, dbName+ "_dupe" , "tbl2"));
       assertEquals(fks.size(), 2);
-      List<SQLNotNullConstraint> nns = metaStoreClientMirror.getNotNullConstraints(new NotNullConstraintsRequest(dbName+ "_dupe" , "tbl3"));
+      List<SQLNotNullConstraint> nns = metaStoreClientMirror.getNotNullConstraints(new NotNullConstraintsRequest(DEFAULT_CATALOG_NAME, dbName+ "_dupe" , "tbl3"));
       assertEquals(nns.size(), 1);
     } catch (TException te) {
       assertNull(te);
@@ -3104,13 +3105,13 @@ public class TestReplicationScenarios {
       List<SQLPrimaryKey> pks = metaStoreClientMirror.getPrimaryKeys(new PrimaryKeysRequest(dbName+ "_dupe" , "tbl4"));
       assertEquals(pks.size(), 2);
       pkName = pks.get(0).getPk_name();
-      List<SQLUniqueConstraint> uks = metaStoreClientMirror.getUniqueConstraints(new UniqueConstraintsRequest(dbName+ "_dupe" , "tbl6"));
+      List<SQLUniqueConstraint> uks = metaStoreClientMirror.getUniqueConstraints(new UniqueConstraintsRequest(DEFAULT_CATALOG_NAME, dbName+ "_dupe" , "tbl6"));
       assertEquals(uks.size(), 1);
       ukName = uks.get(0).getUk_name();
       List<SQLForeignKey> fks = metaStoreClientMirror.getForeignKeys(new ForeignKeysRequest(null, null, dbName+ "_dupe" , "tbl5"));
       assertEquals(fks.size(), 2);
       fkName = fks.get(0).getFk_name();
-      List<SQLNotNullConstraint> nns = metaStoreClientMirror.getNotNullConstraints(new NotNullConstraintsRequest(dbName+ "_dupe" , "tbl6"));
+      List<SQLNotNullConstraint> nns = metaStoreClientMirror.getNotNullConstraints(new NotNullConstraintsRequest(DEFAULT_CATALOG_NAME, dbName+ "_dupe" , "tbl6"));
       assertEquals(nns.size(), 1);
       nnName = nns.get(0).getNn_name();
 
@@ -3133,11 +3134,11 @@ public class TestReplicationScenarios {
     try {
       List<SQLPrimaryKey> pks = metaStoreClientMirror.getPrimaryKeys(new PrimaryKeysRequest(dbName+ "_dupe" , "tbl4"));
       assertTrue(pks.isEmpty());
-      List<SQLUniqueConstraint> uks = metaStoreClientMirror.getUniqueConstraints(new UniqueConstraintsRequest(dbName+ "_dupe" , "tbl4"));
+      List<SQLUniqueConstraint> uks = metaStoreClientMirror.getUniqueConstraints(new UniqueConstraintsRequest(DEFAULT_CATALOG_NAME, dbName+ "_dupe" , "tbl4"));
       assertTrue(uks.isEmpty());
       List<SQLForeignKey> fks = metaStoreClientMirror.getForeignKeys(new ForeignKeysRequest(null, null, dbName+ "_dupe" , "tbl5"));
       assertTrue(fks.isEmpty());
-      List<SQLNotNullConstraint> nns = metaStoreClientMirror.getNotNullConstraints(new NotNullConstraintsRequest(dbName+ "_dupe" , "tbl6"));
+      List<SQLNotNullConstraint> nns = metaStoreClientMirror.getNotNullConstraints(new NotNullConstraintsRequest(DEFAULT_CATALOG_NAME, dbName+ "_dupe" , "tbl6"));
       assertTrue(nns.isEmpty());
     } catch (TException te) {
       assertNull(te);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/metastore/scripts/upgrade/derby/053-HIVE-18755.derby.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/derby/053-HIVE-18755.derby.sql b/metastore/scripts/upgrade/derby/053-HIVE-18755.derby.sql
new file mode 100644
index 0000000..776ef14
--- /dev/null
+++ b/metastore/scripts/upgrade/derby/053-HIVE-18755.derby.sql
@@ -0,0 +1,54 @@
+
+CREATE TABLE "APP"."CTLGS" (
+    "CTLG_ID" BIGINT NOT NULL,
+    "NAME" VARCHAR(256) UNIQUE,
+    "DESC" VARCHAR(4000),
+    "LOCATION_URI" VARCHAR(4000) NOT NULL);
+
+ALTER TABLE "APP"."CTLGS" ADD CONSTRAINT "CTLGS_PK" PRIMARY KEY ("CTLG_ID");
+
+-- Insert a default value.  The location is TBD.  Hive will fix this when it starts
+INSERT INTO "APP"."CTLGS" VALUES (1, 'hive', 'Default catalog for Hive', 'TBD');
+
+-- Drop the unique index on DBS
+DROP INDEX "APP"."UNIQUE_DATABASE";
+
+-- Add the new column to the DBS table, can't put in the not null constraint yet
+ALTER TABLE "APP"."DBS" ADD COLUMN "CTLG_NAME" VARCHAR(256);
+
+-- Update all records in the DBS table to point to the Hive catalog
+UPDATE "APP"."DBS" 
+  SET "CTLG_NAME" = 'hive';
+
+-- Add the not null constraint
+ALTER TABLE "APP"."DBS" ALTER COLUMN "CTLG_NAME" NOT NULL;
+
+-- Put back the unique index 
+CREATE UNIQUE INDEX "APP"."UNIQUE_DATABASE" ON "APP"."DBS" ("NAME", "CTLG_NAME");
+
+-- Add the foreign key
+ALTER TABLE "APP"."DBS" ADD CONSTRAINT "DBS_FK1" FOREIGN KEY ("CTLG_NAME") REFERENCES "APP"."CTLGS" ("NAME") ON DELETE NO ACTION ON UPDATE NO ACTION;
+
+-- Add columns to table stats and part stats
+ALTER TABLE "APP"."TAB_COL_STATS" ADD COLUMN "CAT_NAME" VARCHAR(256);
+ALTER TABLE "APP"."PART_COL_STATS" ADD COLUMN "CAT_NAME" VARCHAR(256);
+
+-- Set the existing column names to Hive
+UPDATE "APP"."TAB_COL_STATS"
+  SET "CAT_NAME" = 'hive';
+UPDATE "APP"."PART_COL_STATS"
+  SET "CAT_NAME" = 'hive';
+
+-- Add the not null constraint
+ALTER TABLE "APP"."TAB_COL_STATS" ALTER COLUMN "CAT_NAME" NOT NULL;
+ALTER TABLE "APP"."PART_COL_STATS" ALTER COLUMN "CAT_NAME" NOT NULL;
+
+-- Rebuild the index for Part col stats.  No such index for table stats, which seems weird
+DROP INDEX "APP"."PCS_STATS_IDX";
+CREATE INDEX "APP"."PCS_STATS_IDX" ON "APP"."PART_COL_STATS" ("CAT_NAME", "DB_NAME","TABLE_NAME","COLUMN_NAME","PARTITION_NAME");
+
+-- Add column to partition events
+ALTER TABLE "APP"."PARTITION_EVENTS" ADD COLUMN "CAT_NAME" VARCHAR(256);
+
+-- Add column to notification log
+ALTER TABLE "APP"."NOTIFICATION_LOG" ADD COLUMN "CAT_NAME" VARCHAR(256);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/metastore/scripts/upgrade/derby/upgrade-2.3.0-to-3.0.0.derby.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/derby/upgrade-2.3.0-to-3.0.0.derby.sql b/metastore/scripts/upgrade/derby/upgrade-2.3.0-to-3.0.0.derby.sql
index 1e4dd99..1a3c00a 100644
--- a/metastore/scripts/upgrade/derby/upgrade-2.3.0-to-3.0.0.derby.sql
+++ b/metastore/scripts/upgrade/derby/upgrade-2.3.0-to-3.0.0.derby.sql
@@ -10,5 +10,6 @@ RUN '049-HIVE-18489.derby.sql';
 RUN '050-HIVE-18192.derby.sql';
 RUN '051-HIVE-18675.derby.sql';
 RUN '052-HIVE-18965.derby.sql';
+RUN '053-HIVE-18755.derby.sql';
 
 UPDATE "APP".VERSION SET SCHEMA_VERSION='3.0.0', VERSION_COMMENT='Hive release version 3.0.0' where VER_ID=1;

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
index 01cadea..429c296 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hive.ql.exec;
 
 import static org.apache.commons.lang.StringUtils.join;
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
 import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_STORAGE;
 
 import java.io.BufferedWriter;
@@ -5028,8 +5029,8 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       // We set the signature for the view if it is a materialized view
       if (tbl.isMaterializedView()) {
         CreationMetadata cm =
-            new CreationMetadata(tbl.getDbName(), tbl.getTableName(),
-                ImmutableSet.copyOf(crtView.getTablesUsed()));
+            new CreationMetadata(MetaStoreUtils.getDefaultCatalog(conf), tbl.getDbName(),
+                tbl.getTableName(), ImmutableSet.copyOf(crtView.getTablesUsed()));
         cm.setValidTxnList(conf.get(ValidTxnList.VALID_TXNS_KEY));
         tbl.getTTable().setCreationMetadata(cm);
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/ql/src/java/org/apache/hadoop/hive/ql/exec/MaterializedViewTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/MaterializedViewTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/MaterializedViewTask.java
index de120af..50fc4e0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/MaterializedViewTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/MaterializedViewTask.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.ql.exec;
 import com.google.common.collect.ImmutableSet;
 import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.metastore.api.CreationMetadata;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.DriverContext;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -31,6 +32,8 @@ import org.apache.hadoop.hive.ql.plan.api.StageType;
 
 import java.io.Serializable;
 
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
+
 /**
  * This task does some work related to materialized views. In particular, it adds
  * or removes the materialized view from the registry if needed, or registers new
@@ -63,7 +66,8 @@ public class MaterializedViewTask extends Task<MaterializedViewDesc> implements
         Hive db = Hive.get(conf);
         Table mvTable = db.getTable(getWork().getViewName());
         CreationMetadata cm =
-            new CreationMetadata(mvTable.getDbName(), mvTable.getTableName(),
+            new CreationMetadata(MetaStoreUtils.getDefaultCatalog(conf), mvTable.getDbName(),
+                mvTable.getTableName(),
                 ImmutableSet.copyOf(mvTable.getCreationMetadata().getTablesUsed()));
         cm.setValidTxnList(conf.get(ValidTxnList.VALID_TXNS_KEY));
         db.updateCreationMetadata(mvTable.getDbName(), mvTable.getTableName(), cm);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
index d76c8b6..eed37a1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
@@ -25,7 +25,9 @@ import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import static org.apache.hadoop.hive.conf.Constants.MATERIALIZED_VIEW_REWRITING_TIME_WINDOW;
+
 import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_STORAGE;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
 import static org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_FORMAT;
 import static org.apache.hadoop.hive.serde.serdeConstants.STRING_TYPE_NAME;
 
@@ -4491,7 +4493,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
 
   public List<SQLUniqueConstraint> getUniqueConstraintList(String dbName, String tblName) throws HiveException, NoSuchObjectException {
     try {
-      return getMSC().getUniqueConstraints(new UniqueConstraintsRequest(dbName, tblName));
+      return getMSC().getUniqueConstraints(new UniqueConstraintsRequest(getDefaultCatalog(conf), dbName, tblName));
     } catch (NoSuchObjectException e) {
       throw e;
     } catch (Exception e) {
@@ -4501,7 +4503,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
 
   public List<SQLNotNullConstraint> getNotNullConstraintList(String dbName, String tblName) throws HiveException, NoSuchObjectException {
     try {
-      return getMSC().getNotNullConstraints(new NotNullConstraintsRequest(dbName, tblName));
+      return getMSC().getNotNullConstraints(new NotNullConstraintsRequest(getDefaultCatalog(conf), dbName, tblName));
     } catch (NoSuchObjectException e) {
       throw e;
     } catch (Exception e) {
@@ -4511,7 +4513,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
 
   public List<SQLDefaultConstraint> getDefaultConstraintList(String dbName, String tblName) throws HiveException, NoSuchObjectException {
     try {
-      return getMSC().getDefaultConstraints(new DefaultConstraintsRequest(dbName, tblName));
+      return getMSC().getDefaultConstraints(new DefaultConstraintsRequest(getDefaultCatalog(conf), dbName, tblName));
     } catch (NoSuchObjectException e) {
       throw e;
     } catch (Exception e) {
@@ -4521,7 +4523,8 @@ private void constructOneLBLocationMap(FileStatus fSta,
 
   public List<SQLCheckConstraint> getCheckConstraintList(String dbName, String tblName) throws HiveException, NoSuchObjectException {
     try {
-      return getMSC().getCheckConstraints(new CheckConstraintsRequest(dbName, tblName));
+      return getMSC().getCheckConstraints(new CheckConstraintsRequest(getDefaultCatalog(conf),
+          dbName, tblName));
     } catch (NoSuchObjectException e) {
       throw e;
     } catch (Exception e) {
@@ -4635,7 +4638,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
       throws HiveException {
     try {
       List<SQLUniqueConstraint> uniqueConstraints = getMSC().getUniqueConstraints(
-              new UniqueConstraintsRequest(dbName, tblName));
+              new UniqueConstraintsRequest(getDefaultCatalog(conf), dbName, tblName));
       if (onlyReliable && uniqueConstraints != null && !uniqueConstraints.isEmpty()) {
         uniqueConstraints = uniqueConstraints.stream()
           .filter(uk -> uk.isRely_cstr())
@@ -4683,7 +4686,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
       throws HiveException {
     try {
       List<SQLNotNullConstraint> notNullConstraints = getMSC().getNotNullConstraints(
-              new NotNullConstraintsRequest(dbName, tblName));
+              new NotNullConstraintsRequest(getDefaultCatalog(conf), dbName, tblName));
       if (notNullConstraints != null && !notNullConstraints.isEmpty()) {
         notNullConstraints = notNullConstraints.stream()
           .filter(nnc -> nnc.isEnable_cstr())
@@ -4707,7 +4710,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
       throws HiveException {
     try {
       List<SQLCheckConstraint> checkConstraints = getMSC().getCheckConstraints(
-          new CheckConstraintsRequest(dbName, tblName));
+          new CheckConstraintsRequest(getDefaultCatalog(conf), dbName, tblName));
       if (checkConstraints != null && !checkConstraints.isEmpty()) {
         checkConstraints = checkConstraints.stream()
             .filter(nnc -> nnc.isEnable_cstr())
@@ -4730,7 +4733,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
       throws HiveException {
     try {
       List<SQLDefaultConstraint> defaultConstraints = getMSC().getDefaultConstraints(
-          new DefaultConstraintsRequest(dbName, tblName));
+          new DefaultConstraintsRequest(getDefaultCatalog(conf), dbName, tblName));
       if (defaultConstraints != null && !defaultConstraints.isEmpty()) {
         defaultConstraints = defaultConstraints.stream()
             .filter(nnc -> nnc.isEnable_cstr())
@@ -4746,7 +4749,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
       throws HiveException {
     try {
       List<SQLNotNullConstraint> notNullConstraints = getMSC().getNotNullConstraints(
-              new NotNullConstraintsRequest(dbName, tblName));
+              new NotNullConstraintsRequest(getDefaultCatalog(conf), dbName, tblName));
       if (onlyReliable && notNullConstraints != null && !notNullConstraints.isEmpty()) {
         notNullConstraints = notNullConstraints.stream()
           .filter(nnc -> nnc.isRely_cstr())
@@ -4762,7 +4765,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
       throws HiveException {
     try {
       List<SQLDefaultConstraint> defaultConstraints = getMSC().getDefaultConstraints(
-          new DefaultConstraintsRequest(dbName, tblName));
+          new DefaultConstraintsRequest(getDefaultCatalog(conf), dbName, tblName));
       if (defaultConstraints != null && !defaultConstraints.isEmpty()) {
         defaultConstraints = defaultConstraints.stream()
             .collect(Collectors.toList());
@@ -4777,7 +4780,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
       throws HiveException {
     try {
       List<SQLCheckConstraint> checkConstraints = getMSC().getCheckConstraints(
-          new CheckConstraintsRequest(dbName, tblName));
+          new CheckConstraintsRequest(getDefaultCatalog(conf), dbName, tblName));
       if (checkConstraints != null && !checkConstraints.isEmpty()) {
         checkConstraints = checkConstraints.stream()
             .collect(Collectors.toList());


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetCatalogResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetCatalogResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetCatalogResponse.java
new file mode 100644
index 0000000..096f5ef
--- /dev/null
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetCatalogResponse.java
@@ -0,0 +1,400 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class GetCatalogResponse implements org.apache.thrift.TBase<GetCatalogResponse, GetCatalogResponse._Fields>, java.io.Serializable, Cloneable, Comparable<GetCatalogResponse> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetCatalogResponse");
+
+  private static final org.apache.thrift.protocol.TField CATALOG_FIELD_DESC = new org.apache.thrift.protocol.TField("catalog", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new GetCatalogResponseStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new GetCatalogResponseTupleSchemeFactory());
+  }
+
+  private Catalog catalog; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    CATALOG((short)1, "catalog");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // CATALOG
+          return CATALOG;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.CATALOG, new org.apache.thrift.meta_data.FieldMetaData("catalog", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, Catalog.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetCatalogResponse.class, metaDataMap);
+  }
+
+  public GetCatalogResponse() {
+  }
+
+  public GetCatalogResponse(
+    Catalog catalog)
+  {
+    this();
+    this.catalog = catalog;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public GetCatalogResponse(GetCatalogResponse other) {
+    if (other.isSetCatalog()) {
+      this.catalog = new Catalog(other.catalog);
+    }
+  }
+
+  public GetCatalogResponse deepCopy() {
+    return new GetCatalogResponse(this);
+  }
+
+  @Override
+  public void clear() {
+    this.catalog = null;
+  }
+
+  public Catalog getCatalog() {
+    return this.catalog;
+  }
+
+  public void setCatalog(Catalog catalog) {
+    this.catalog = catalog;
+  }
+
+  public void unsetCatalog() {
+    this.catalog = null;
+  }
+
+  /** Returns true if field catalog is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatalog() {
+    return this.catalog != null;
+  }
+
+  public void setCatalogIsSet(boolean value) {
+    if (!value) {
+      this.catalog = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case CATALOG:
+      if (value == null) {
+        unsetCatalog();
+      } else {
+        setCatalog((Catalog)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case CATALOG:
+      return getCatalog();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case CATALOG:
+      return isSetCatalog();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof GetCatalogResponse)
+      return this.equals((GetCatalogResponse)that);
+    return false;
+  }
+
+  public boolean equals(GetCatalogResponse that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_catalog = true && this.isSetCatalog();
+    boolean that_present_catalog = true && that.isSetCatalog();
+    if (this_present_catalog || that_present_catalog) {
+      if (!(this_present_catalog && that_present_catalog))
+        return false;
+      if (!this.catalog.equals(that.catalog))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_catalog = true && (isSetCatalog());
+    list.add(present_catalog);
+    if (present_catalog)
+      list.add(catalog);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(GetCatalogResponse other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetCatalog()).compareTo(other.isSetCatalog());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatalog()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catalog, other.catalog);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("GetCatalogResponse(");
+    boolean first = true;
+
+    sb.append("catalog:");
+    if (this.catalog == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.catalog);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+    if (catalog != null) {
+      catalog.validate();
+    }
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class GetCatalogResponseStandardSchemeFactory implements SchemeFactory {
+    public GetCatalogResponseStandardScheme getScheme() {
+      return new GetCatalogResponseStandardScheme();
+    }
+  }
+
+  private static class GetCatalogResponseStandardScheme extends StandardScheme<GetCatalogResponse> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, GetCatalogResponse struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // CATALOG
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.catalog = new Catalog();
+              struct.catalog.read(iprot);
+              struct.setCatalogIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, GetCatalogResponse struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.catalog != null) {
+        oprot.writeFieldBegin(CATALOG_FIELD_DESC);
+        struct.catalog.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class GetCatalogResponseTupleSchemeFactory implements SchemeFactory {
+    public GetCatalogResponseTupleScheme getScheme() {
+      return new GetCatalogResponseTupleScheme();
+    }
+  }
+
+  private static class GetCatalogResponseTupleScheme extends TupleScheme<GetCatalogResponse> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, GetCatalogResponse struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.isSetCatalog()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.isSetCatalog()) {
+        struct.catalog.write(oprot);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, GetCatalogResponse struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.catalog = new Catalog();
+        struct.catalog.read(iprot);
+        struct.setCatalogIsSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetCatalogsResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetCatalogsResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetCatalogsResponse.java
new file mode 100644
index 0000000..aafd528
--- /dev/null
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetCatalogsResponse.java
@@ -0,0 +1,444 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class GetCatalogsResponse implements org.apache.thrift.TBase<GetCatalogsResponse, GetCatalogsResponse._Fields>, java.io.Serializable, Cloneable, Comparable<GetCatalogsResponse> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetCatalogsResponse");
+
+  private static final org.apache.thrift.protocol.TField NAMES_FIELD_DESC = new org.apache.thrift.protocol.TField("names", org.apache.thrift.protocol.TType.LIST, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new GetCatalogsResponseStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new GetCatalogsResponseTupleSchemeFactory());
+  }
+
+  private List<String> names; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    NAMES((short)1, "names");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // NAMES
+          return NAMES;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.NAMES, new org.apache.thrift.meta_data.FieldMetaData("names", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetCatalogsResponse.class, metaDataMap);
+  }
+
+  public GetCatalogsResponse() {
+  }
+
+  public GetCatalogsResponse(
+    List<String> names)
+  {
+    this();
+    this.names = names;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public GetCatalogsResponse(GetCatalogsResponse other) {
+    if (other.isSetNames()) {
+      List<String> __this__names = new ArrayList<String>(other.names);
+      this.names = __this__names;
+    }
+  }
+
+  public GetCatalogsResponse deepCopy() {
+    return new GetCatalogsResponse(this);
+  }
+
+  @Override
+  public void clear() {
+    this.names = null;
+  }
+
+  public int getNamesSize() {
+    return (this.names == null) ? 0 : this.names.size();
+  }
+
+  public java.util.Iterator<String> getNamesIterator() {
+    return (this.names == null) ? null : this.names.iterator();
+  }
+
+  public void addToNames(String elem) {
+    if (this.names == null) {
+      this.names = new ArrayList<String>();
+    }
+    this.names.add(elem);
+  }
+
+  public List<String> getNames() {
+    return this.names;
+  }
+
+  public void setNames(List<String> names) {
+    this.names = names;
+  }
+
+  public void unsetNames() {
+    this.names = null;
+  }
+
+  /** Returns true if field names is set (has been assigned a value) and false otherwise */
+  public boolean isSetNames() {
+    return this.names != null;
+  }
+
+  public void setNamesIsSet(boolean value) {
+    if (!value) {
+      this.names = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case NAMES:
+      if (value == null) {
+        unsetNames();
+      } else {
+        setNames((List<String>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case NAMES:
+      return getNames();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case NAMES:
+      return isSetNames();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof GetCatalogsResponse)
+      return this.equals((GetCatalogsResponse)that);
+    return false;
+  }
+
+  public boolean equals(GetCatalogsResponse that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_names = true && this.isSetNames();
+    boolean that_present_names = true && that.isSetNames();
+    if (this_present_names || that_present_names) {
+      if (!(this_present_names && that_present_names))
+        return false;
+      if (!this.names.equals(that.names))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_names = true && (isSetNames());
+    list.add(present_names);
+    if (present_names)
+      list.add(names);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(GetCatalogsResponse other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetNames()).compareTo(other.isSetNames());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetNames()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.names, other.names);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("GetCatalogsResponse(");
+    boolean first = true;
+
+    sb.append("names:");
+    if (this.names == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.names);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class GetCatalogsResponseStandardSchemeFactory implements SchemeFactory {
+    public GetCatalogsResponseStandardScheme getScheme() {
+      return new GetCatalogsResponseStandardScheme();
+    }
+  }
+
+  private static class GetCatalogsResponseStandardScheme extends StandardScheme<GetCatalogsResponse> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, GetCatalogsResponse struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // NAMES
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list94 = iprot.readListBegin();
+                struct.names = new ArrayList<String>(_list94.size);
+                String _elem95;
+                for (int _i96 = 0; _i96 < _list94.size; ++_i96)
+                {
+                  _elem95 = iprot.readString();
+                  struct.names.add(_elem95);
+                }
+                iprot.readListEnd();
+              }
+              struct.setNamesIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, GetCatalogsResponse struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.names != null) {
+        oprot.writeFieldBegin(NAMES_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.names.size()));
+          for (String _iter97 : struct.names)
+          {
+            oprot.writeString(_iter97);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class GetCatalogsResponseTupleSchemeFactory implements SchemeFactory {
+    public GetCatalogsResponseTupleScheme getScheme() {
+      return new GetCatalogsResponseTupleScheme();
+    }
+  }
+
+  private static class GetCatalogsResponseTupleScheme extends TupleScheme<GetCatalogsResponse> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, GetCatalogsResponse struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.isSetNames()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.isSetNames()) {
+        {
+          oprot.writeI32(struct.names.size());
+          for (String _iter98 : struct.names)
+          {
+            oprot.writeString(_iter98);
+          }
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, GetCatalogsResponse struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        {
+          org.apache.thrift.protocol.TList _list99 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.names = new ArrayList<String>(_list99.size);
+          String _elem100;
+          for (int _i101 = 0; _i101 < _list99.size; ++_i101)
+          {
+            _elem100 = iprot.readString();
+            struct.names.add(_elem100);
+          }
+        }
+        struct.setNamesIsSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java
index f5f1eb3..b64dea4 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java
@@ -619,13 +619,13 @@ import org.slf4j.LoggerFactory;
           case 1: // FILE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list726 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list726.size);
-                long _elem727;
-                for (int _i728 = 0; _i728 < _list726.size; ++_i728)
+                org.apache.thrift.protocol.TList _list734 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list734.size);
+                long _elem735;
+                for (int _i736 = 0; _i736 < _list734.size; ++_i736)
                 {
-                  _elem727 = iprot.readI64();
-                  struct.fileIds.add(_elem727);
+                  _elem735 = iprot.readI64();
+                  struct.fileIds.add(_elem735);
                 }
                 iprot.readListEnd();
               }
@@ -675,9 +675,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FILE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.fileIds.size()));
-          for (long _iter729 : struct.fileIds)
+          for (long _iter737 : struct.fileIds)
           {
-            oprot.writeI64(_iter729);
+            oprot.writeI64(_iter737);
           }
           oprot.writeListEnd();
         }
@@ -719,9 +719,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter730 : struct.fileIds)
+        for (long _iter738 : struct.fileIds)
         {
-          oprot.writeI64(_iter730);
+          oprot.writeI64(_iter738);
         }
       }
       oprot.writeBinary(struct.expr);
@@ -745,13 +745,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataByExprRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list731 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list731.size);
-        long _elem732;
-        for (int _i733 = 0; _i733 < _list731.size; ++_i733)
+        org.apache.thrift.protocol.TList _list739 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list739.size);
+        long _elem740;
+        for (int _i741 = 0; _i741 < _list739.size; ++_i741)
         {
-          _elem732 = iprot.readI64();
-          struct.fileIds.add(_elem732);
+          _elem740 = iprot.readI64();
+          struct.fileIds.add(_elem740);
         }
       }
       struct.setFileIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java
index 370ab66..a01a366 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java
@@ -444,16 +444,16 @@ import org.slf4j.LoggerFactory;
           case 1: // METADATA
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map716 = iprot.readMapBegin();
-                struct.metadata = new HashMap<Long,MetadataPpdResult>(2*_map716.size);
-                long _key717;
-                MetadataPpdResult _val718;
-                for (int _i719 = 0; _i719 < _map716.size; ++_i719)
+                org.apache.thrift.protocol.TMap _map724 = iprot.readMapBegin();
+                struct.metadata = new HashMap<Long,MetadataPpdResult>(2*_map724.size);
+                long _key725;
+                MetadataPpdResult _val726;
+                for (int _i727 = 0; _i727 < _map724.size; ++_i727)
                 {
-                  _key717 = iprot.readI64();
-                  _val718 = new MetadataPpdResult();
-                  _val718.read(iprot);
-                  struct.metadata.put(_key717, _val718);
+                  _key725 = iprot.readI64();
+                  _val726 = new MetadataPpdResult();
+                  _val726.read(iprot);
+                  struct.metadata.put(_key725, _val726);
                 }
                 iprot.readMapEnd();
               }
@@ -487,10 +487,10 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(METADATA_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRUCT, struct.metadata.size()));
-          for (Map.Entry<Long, MetadataPpdResult> _iter720 : struct.metadata.entrySet())
+          for (Map.Entry<Long, MetadataPpdResult> _iter728 : struct.metadata.entrySet())
           {
-            oprot.writeI64(_iter720.getKey());
-            _iter720.getValue().write(oprot);
+            oprot.writeI64(_iter728.getKey());
+            _iter728.getValue().write(oprot);
           }
           oprot.writeMapEnd();
         }
@@ -518,10 +518,10 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.metadata.size());
-        for (Map.Entry<Long, MetadataPpdResult> _iter721 : struct.metadata.entrySet())
+        for (Map.Entry<Long, MetadataPpdResult> _iter729 : struct.metadata.entrySet())
         {
-          oprot.writeI64(_iter721.getKey());
-          _iter721.getValue().write(oprot);
+          oprot.writeI64(_iter729.getKey());
+          _iter729.getValue().write(oprot);
         }
       }
       oprot.writeBool(struct.isSupported);
@@ -531,16 +531,16 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataByExprResult struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TMap _map722 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.metadata = new HashMap<Long,MetadataPpdResult>(2*_map722.size);
-        long _key723;
-        MetadataPpdResult _val724;
-        for (int _i725 = 0; _i725 < _map722.size; ++_i725)
+        org.apache.thrift.protocol.TMap _map730 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.metadata = new HashMap<Long,MetadataPpdResult>(2*_map730.size);
+        long _key731;
+        MetadataPpdResult _val732;
+        for (int _i733 = 0; _i733 < _map730.size; ++_i733)
         {
-          _key723 = iprot.readI64();
-          _val724 = new MetadataPpdResult();
-          _val724.read(iprot);
-          struct.metadata.put(_key723, _val724);
+          _key731 = iprot.readI64();
+          _val732 = new MetadataPpdResult();
+          _val732.read(iprot);
+          struct.metadata.put(_key731, _val732);
         }
       }
       struct.setMetadataIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java
index c74c2b0..4541cf4 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java
@@ -351,13 +351,13 @@ import org.slf4j.LoggerFactory;
           case 1: // FILE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list744 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list744.size);
-                long _elem745;
-                for (int _i746 = 0; _i746 < _list744.size; ++_i746)
+                org.apache.thrift.protocol.TList _list752 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list752.size);
+                long _elem753;
+                for (int _i754 = 0; _i754 < _list752.size; ++_i754)
                 {
-                  _elem745 = iprot.readI64();
-                  struct.fileIds.add(_elem745);
+                  _elem753 = iprot.readI64();
+                  struct.fileIds.add(_elem753);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FILE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.fileIds.size()));
-          for (long _iter747 : struct.fileIds)
+          for (long _iter755 : struct.fileIds)
           {
-            oprot.writeI64(_iter747);
+            oprot.writeI64(_iter755);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter748 : struct.fileIds)
+        for (long _iter756 : struct.fileIds)
         {
-          oprot.writeI64(_iter748);
+          oprot.writeI64(_iter756);
         }
       }
     }
@@ -421,13 +421,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list749 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list749.size);
-        long _elem750;
-        for (int _i751 = 0; _i751 < _list749.size; ++_i751)
+        org.apache.thrift.protocol.TList _list757 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list757.size);
+        long _elem758;
+        for (int _i759 = 0; _i759 < _list757.size; ++_i759)
         {
-          _elem750 = iprot.readI64();
-          struct.fileIds.add(_elem750);
+          _elem758 = iprot.readI64();
+          struct.fileIds.add(_elem758);
         }
       }
       struct.setFileIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java
index 6431b6d..3efb371 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java
@@ -433,15 +433,15 @@ import org.slf4j.LoggerFactory;
           case 1: // METADATA
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map734 = iprot.readMapBegin();
-                struct.metadata = new HashMap<Long,ByteBuffer>(2*_map734.size);
-                long _key735;
-                ByteBuffer _val736;
-                for (int _i737 = 0; _i737 < _map734.size; ++_i737)
+                org.apache.thrift.protocol.TMap _map742 = iprot.readMapBegin();
+                struct.metadata = new HashMap<Long,ByteBuffer>(2*_map742.size);
+                long _key743;
+                ByteBuffer _val744;
+                for (int _i745 = 0; _i745 < _map742.size; ++_i745)
                 {
-                  _key735 = iprot.readI64();
-                  _val736 = iprot.readBinary();
-                  struct.metadata.put(_key735, _val736);
+                  _key743 = iprot.readI64();
+                  _val744 = iprot.readBinary();
+                  struct.metadata.put(_key743, _val744);
                 }
                 iprot.readMapEnd();
               }
@@ -475,10 +475,10 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(METADATA_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRING, struct.metadata.size()));
-          for (Map.Entry<Long, ByteBuffer> _iter738 : struct.metadata.entrySet())
+          for (Map.Entry<Long, ByteBuffer> _iter746 : struct.metadata.entrySet())
           {
-            oprot.writeI64(_iter738.getKey());
-            oprot.writeBinary(_iter738.getValue());
+            oprot.writeI64(_iter746.getKey());
+            oprot.writeBinary(_iter746.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -506,10 +506,10 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.metadata.size());
-        for (Map.Entry<Long, ByteBuffer> _iter739 : struct.metadata.entrySet())
+        for (Map.Entry<Long, ByteBuffer> _iter747 : struct.metadata.entrySet())
         {
-          oprot.writeI64(_iter739.getKey());
-          oprot.writeBinary(_iter739.getValue());
+          oprot.writeI64(_iter747.getKey());
+          oprot.writeBinary(_iter747.getValue());
         }
       }
       oprot.writeBool(struct.isSupported);
@@ -519,15 +519,15 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataResult struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TMap _map740 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.metadata = new HashMap<Long,ByteBuffer>(2*_map740.size);
-        long _key741;
-        ByteBuffer _val742;
-        for (int _i743 = 0; _i743 < _map740.size; ++_i743)
+        org.apache.thrift.protocol.TMap _map748 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.metadata = new HashMap<Long,ByteBuffer>(2*_map748.size);
+        long _key749;
+        ByteBuffer _val750;
+        for (int _i751 = 0; _i751 < _map748.size; ++_i751)
         {
-          _key741 = iprot.readI64();
-          _val742 = iprot.readBinary();
-          struct.metadata.put(_key741, _val742);
+          _key749 = iprot.readI64();
+          _val750 = iprot.readBinary();
+          struct.metadata.put(_key749, _val750);
         }
       }
       struct.setMetadataIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsInfoResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsInfoResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsInfoResponse.java
index a9061ab..56f239e 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsInfoResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsInfoResponse.java
@@ -447,14 +447,14 @@ import org.slf4j.LoggerFactory;
           case 2: // OPEN_TXNS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list546 = iprot.readListBegin();
-                struct.open_txns = new ArrayList<TxnInfo>(_list546.size);
-                TxnInfo _elem547;
-                for (int _i548 = 0; _i548 < _list546.size; ++_i548)
+                org.apache.thrift.protocol.TList _list554 = iprot.readListBegin();
+                struct.open_txns = new ArrayList<TxnInfo>(_list554.size);
+                TxnInfo _elem555;
+                for (int _i556 = 0; _i556 < _list554.size; ++_i556)
                 {
-                  _elem547 = new TxnInfo();
-                  _elem547.read(iprot);
-                  struct.open_txns.add(_elem547);
+                  _elem555 = new TxnInfo();
+                  _elem555.read(iprot);
+                  struct.open_txns.add(_elem555);
                 }
                 iprot.readListEnd();
               }
@@ -483,9 +483,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(OPEN_TXNS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.open_txns.size()));
-          for (TxnInfo _iter549 : struct.open_txns)
+          for (TxnInfo _iter557 : struct.open_txns)
           {
-            _iter549.write(oprot);
+            _iter557.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -511,9 +511,9 @@ import org.slf4j.LoggerFactory;
       oprot.writeI64(struct.txn_high_water_mark);
       {
         oprot.writeI32(struct.open_txns.size());
-        for (TxnInfo _iter550 : struct.open_txns)
+        for (TxnInfo _iter558 : struct.open_txns)
         {
-          _iter550.write(oprot);
+          _iter558.write(oprot);
         }
       }
     }
@@ -524,14 +524,14 @@ import org.slf4j.LoggerFactory;
       struct.txn_high_water_mark = iprot.readI64();
       struct.setTxn_high_water_markIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list551 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.open_txns = new ArrayList<TxnInfo>(_list551.size);
-        TxnInfo _elem552;
-        for (int _i553 = 0; _i553 < _list551.size; ++_i553)
+        org.apache.thrift.protocol.TList _list559 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.open_txns = new ArrayList<TxnInfo>(_list559.size);
+        TxnInfo _elem560;
+        for (int _i561 = 0; _i561 < _list559.size; ++_i561)
         {
-          _elem552 = new TxnInfo();
-          _elem552.read(iprot);
-          struct.open_txns.add(_elem552);
+          _elem560 = new TxnInfo();
+          _elem560.read(iprot);
+          struct.open_txns.add(_elem560);
         }
       }
       struct.setOpen_txnsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsResponse.java
index 12a125f..9688297 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsResponse.java
@@ -615,13 +615,13 @@ import org.slf4j.LoggerFactory;
           case 2: // OPEN_TXNS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list554 = iprot.readListBegin();
-                struct.open_txns = new ArrayList<Long>(_list554.size);
-                long _elem555;
-                for (int _i556 = 0; _i556 < _list554.size; ++_i556)
+                org.apache.thrift.protocol.TList _list562 = iprot.readListBegin();
+                struct.open_txns = new ArrayList<Long>(_list562.size);
+                long _elem563;
+                for (int _i564 = 0; _i564 < _list562.size; ++_i564)
                 {
-                  _elem555 = iprot.readI64();
-                  struct.open_txns.add(_elem555);
+                  _elem563 = iprot.readI64();
+                  struct.open_txns.add(_elem563);
                 }
                 iprot.readListEnd();
               }
@@ -666,9 +666,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(OPEN_TXNS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.open_txns.size()));
-          for (long _iter557 : struct.open_txns)
+          for (long _iter565 : struct.open_txns)
           {
-            oprot.writeI64(_iter557);
+            oprot.writeI64(_iter565);
           }
           oprot.writeListEnd();
         }
@@ -704,9 +704,9 @@ import org.slf4j.LoggerFactory;
       oprot.writeI64(struct.txn_high_water_mark);
       {
         oprot.writeI32(struct.open_txns.size());
-        for (long _iter558 : struct.open_txns)
+        for (long _iter566 : struct.open_txns)
         {
-          oprot.writeI64(_iter558);
+          oprot.writeI64(_iter566);
         }
       }
       oprot.writeBinary(struct.abortedBits);
@@ -726,13 +726,13 @@ import org.slf4j.LoggerFactory;
       struct.txn_high_water_mark = iprot.readI64();
       struct.setTxn_high_water_markIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list559 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.open_txns = new ArrayList<Long>(_list559.size);
-        long _elem560;
-        for (int _i561 = 0; _i561 < _list559.size; ++_i561)
+        org.apache.thrift.protocol.TList _list567 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.open_txns = new ArrayList<Long>(_list567.size);
+        long _elem568;
+        for (int _i569 = 0; _i569 < _list567.size; ++_i569)
         {
-          _elem560 = iprot.readI64();
-          struct.open_txns.add(_elem560);
+          _elem568 = iprot.readI64();
+          struct.open_txns.add(_elem568);
         }
       }
       struct.setOpen_txnsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTableRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTableRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTableRequest.java
index 708bf90..3c88d8f 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTableRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTableRequest.java
@@ -41,6 +41,7 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbName", org.apache.thrift.protocol.TType.STRING, (short)1);
   private static final org.apache.thrift.protocol.TField TBL_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tblName", org.apache.thrift.protocol.TType.STRING, (short)2);
   private static final org.apache.thrift.protocol.TField CAPABILITIES_FIELD_DESC = new org.apache.thrift.protocol.TField("capabilities", org.apache.thrift.protocol.TType.STRUCT, (short)3);
+  private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)4);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -51,12 +52,14 @@ import org.slf4j.LoggerFactory;
   private String dbName; // required
   private String tblName; // required
   private ClientCapabilities capabilities; // optional
+  private String catName; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
     DB_NAME((short)1, "dbName"),
     TBL_NAME((short)2, "tblName"),
-    CAPABILITIES((short)3, "capabilities");
+    CAPABILITIES((short)3, "capabilities"),
+    CAT_NAME((short)4, "catName");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -77,6 +80,8 @@ import org.slf4j.LoggerFactory;
           return TBL_NAME;
         case 3: // CAPABILITIES
           return CAPABILITIES;
+        case 4: // CAT_NAME
+          return CAT_NAME;
         default:
           return null;
       }
@@ -117,7 +122,7 @@ import org.slf4j.LoggerFactory;
   }
 
   // isset id assignments
-  private static final _Fields optionals[] = {_Fields.CAPABILITIES};
+  private static final _Fields optionals[] = {_Fields.CAPABILITIES,_Fields.CAT_NAME};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -127,6 +132,8 @@ import org.slf4j.LoggerFactory;
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.CAPABILITIES, new org.apache.thrift.meta_data.FieldMetaData("capabilities", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ClientCapabilities.class)));
+    tmpMap.put(_Fields.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetTableRequest.class, metaDataMap);
   }
@@ -156,6 +163,9 @@ import org.slf4j.LoggerFactory;
     if (other.isSetCapabilities()) {
       this.capabilities = new ClientCapabilities(other.capabilities);
     }
+    if (other.isSetCatName()) {
+      this.catName = other.catName;
+    }
   }
 
   public GetTableRequest deepCopy() {
@@ -167,6 +177,7 @@ import org.slf4j.LoggerFactory;
     this.dbName = null;
     this.tblName = null;
     this.capabilities = null;
+    this.catName = null;
   }
 
   public String getDbName() {
@@ -238,6 +249,29 @@ import org.slf4j.LoggerFactory;
     }
   }
 
+  public String getCatName() {
+    return this.catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
+  public void unsetCatName() {
+    this.catName = null;
+  }
+
+  /** Returns true if field catName is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatName() {
+    return this.catName != null;
+  }
+
+  public void setCatNameIsSet(boolean value) {
+    if (!value) {
+      this.catName = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case DB_NAME:
@@ -264,6 +298,14 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case CAT_NAME:
+      if (value == null) {
+        unsetCatName();
+      } else {
+        setCatName((String)value);
+      }
+      break;
+
     }
   }
 
@@ -278,6 +320,9 @@ import org.slf4j.LoggerFactory;
     case CAPABILITIES:
       return getCapabilities();
 
+    case CAT_NAME:
+      return getCatName();
+
     }
     throw new IllegalStateException();
   }
@@ -295,6 +340,8 @@ import org.slf4j.LoggerFactory;
       return isSetTblName();
     case CAPABILITIES:
       return isSetCapabilities();
+    case CAT_NAME:
+      return isSetCatName();
     }
     throw new IllegalStateException();
   }
@@ -339,6 +386,15 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_catName = true && this.isSetCatName();
+    boolean that_present_catName = true && that.isSetCatName();
+    if (this_present_catName || that_present_catName) {
+      if (!(this_present_catName && that_present_catName))
+        return false;
+      if (!this.catName.equals(that.catName))
+        return false;
+    }
+
     return true;
   }
 
@@ -361,6 +417,11 @@ import org.slf4j.LoggerFactory;
     if (present_capabilities)
       list.add(capabilities);
 
+    boolean present_catName = true && (isSetCatName());
+    list.add(present_catName);
+    if (present_catName)
+      list.add(catName);
+
     return list.hashCode();
   }
 
@@ -402,6 +463,16 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetCatName()).compareTo(other.isSetCatName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catName, other.catName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -447,6 +518,16 @@ import org.slf4j.LoggerFactory;
       }
       first = false;
     }
+    if (isSetCatName()) {
+      if (!first) sb.append(", ");
+      sb.append("catName:");
+      if (this.catName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.catName);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -526,6 +607,14 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 4: // CAT_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.catName = iprot.readString();
+              struct.setCatNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -556,6 +645,13 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldEnd();
         }
       }
+      if (struct.catName != null) {
+        if (struct.isSetCatName()) {
+          oprot.writeFieldBegin(CAT_NAME_FIELD_DESC);
+          oprot.writeString(struct.catName);
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -579,10 +675,16 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetCapabilities()) {
         optionals.set(0);
       }
-      oprot.writeBitSet(optionals, 1);
+      if (struct.isSetCatName()) {
+        optionals.set(1);
+      }
+      oprot.writeBitSet(optionals, 2);
       if (struct.isSetCapabilities()) {
         struct.capabilities.write(oprot);
       }
+      if (struct.isSetCatName()) {
+        oprot.writeString(struct.catName);
+      }
     }
 
     @Override
@@ -592,12 +694,16 @@ import org.slf4j.LoggerFactory;
       struct.setDbNameIsSet(true);
       struct.tblName = iprot.readString();
       struct.setTblNameIsSet(true);
-      BitSet incoming = iprot.readBitSet(1);
+      BitSet incoming = iprot.readBitSet(2);
       if (incoming.get(0)) {
         struct.capabilities = new ClientCapabilities();
         struct.capabilities.read(iprot);
         struct.setCapabilitiesIsSet(true);
       }
+      if (incoming.get(1)) {
+        struct.catName = iprot.readString();
+        struct.setCatNameIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java
index 7b9e6c5..1c9fba8 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java
@@ -41,6 +41,7 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbName", org.apache.thrift.protocol.TType.STRING, (short)1);
   private static final org.apache.thrift.protocol.TField TBL_NAMES_FIELD_DESC = new org.apache.thrift.protocol.TField("tblNames", org.apache.thrift.protocol.TType.LIST, (short)2);
   private static final org.apache.thrift.protocol.TField CAPABILITIES_FIELD_DESC = new org.apache.thrift.protocol.TField("capabilities", org.apache.thrift.protocol.TType.STRUCT, (short)3);
+  private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)4);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -51,12 +52,14 @@ import org.slf4j.LoggerFactory;
   private String dbName; // required
   private List<String> tblNames; // optional
   private ClientCapabilities capabilities; // optional
+  private String catName; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
     DB_NAME((short)1, "dbName"),
     TBL_NAMES((short)2, "tblNames"),
-    CAPABILITIES((short)3, "capabilities");
+    CAPABILITIES((short)3, "capabilities"),
+    CAT_NAME((short)4, "catName");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -77,6 +80,8 @@ import org.slf4j.LoggerFactory;
           return TBL_NAMES;
         case 3: // CAPABILITIES
           return CAPABILITIES;
+        case 4: // CAT_NAME
+          return CAT_NAME;
         default:
           return null;
       }
@@ -117,7 +122,7 @@ import org.slf4j.LoggerFactory;
   }
 
   // isset id assignments
-  private static final _Fields optionals[] = {_Fields.TBL_NAMES,_Fields.CAPABILITIES};
+  private static final _Fields optionals[] = {_Fields.TBL_NAMES,_Fields.CAPABILITIES,_Fields.CAT_NAME};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -128,6 +133,8 @@ import org.slf4j.LoggerFactory;
             new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
     tmpMap.put(_Fields.CAPABILITIES, new org.apache.thrift.meta_data.FieldMetaData("capabilities", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ClientCapabilities.class)));
+    tmpMap.put(_Fields.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetTablesRequest.class, metaDataMap);
   }
@@ -156,6 +163,9 @@ import org.slf4j.LoggerFactory;
     if (other.isSetCapabilities()) {
       this.capabilities = new ClientCapabilities(other.capabilities);
     }
+    if (other.isSetCatName()) {
+      this.catName = other.catName;
+    }
   }
 
   public GetTablesRequest deepCopy() {
@@ -167,6 +177,7 @@ import org.slf4j.LoggerFactory;
     this.dbName = null;
     this.tblNames = null;
     this.capabilities = null;
+    this.catName = null;
   }
 
   public String getDbName() {
@@ -253,6 +264,29 @@ import org.slf4j.LoggerFactory;
     }
   }
 
+  public String getCatName() {
+    return this.catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
+  public void unsetCatName() {
+    this.catName = null;
+  }
+
+  /** Returns true if field catName is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatName() {
+    return this.catName != null;
+  }
+
+  public void setCatNameIsSet(boolean value) {
+    if (!value) {
+      this.catName = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case DB_NAME:
@@ -279,6 +313,14 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case CAT_NAME:
+      if (value == null) {
+        unsetCatName();
+      } else {
+        setCatName((String)value);
+      }
+      break;
+
     }
   }
 
@@ -293,6 +335,9 @@ import org.slf4j.LoggerFactory;
     case CAPABILITIES:
       return getCapabilities();
 
+    case CAT_NAME:
+      return getCatName();
+
     }
     throw new IllegalStateException();
   }
@@ -310,6 +355,8 @@ import org.slf4j.LoggerFactory;
       return isSetTblNames();
     case CAPABILITIES:
       return isSetCapabilities();
+    case CAT_NAME:
+      return isSetCatName();
     }
     throw new IllegalStateException();
   }
@@ -354,6 +401,15 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_catName = true && this.isSetCatName();
+    boolean that_present_catName = true && that.isSetCatName();
+    if (this_present_catName || that_present_catName) {
+      if (!(this_present_catName && that_present_catName))
+        return false;
+      if (!this.catName.equals(that.catName))
+        return false;
+    }
+
     return true;
   }
 
@@ -376,6 +432,11 @@ import org.slf4j.LoggerFactory;
     if (present_capabilities)
       list.add(capabilities);
 
+    boolean present_catName = true && (isSetCatName());
+    list.add(present_catName);
+    if (present_catName)
+      list.add(catName);
+
     return list.hashCode();
   }
 
@@ -417,6 +478,16 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetCatName()).compareTo(other.isSetCatName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catName, other.catName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -464,6 +535,16 @@ import org.slf4j.LoggerFactory;
       }
       first = false;
     }
+    if (isSetCatName()) {
+      if (!first) sb.append(", ");
+      sb.append("catName:");
+      if (this.catName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.catName);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -525,13 +606,13 @@ import org.slf4j.LoggerFactory;
           case 2: // TBL_NAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list792 = iprot.readListBegin();
-                struct.tblNames = new ArrayList<String>(_list792.size);
-                String _elem793;
-                for (int _i794 = 0; _i794 < _list792.size; ++_i794)
+                org.apache.thrift.protocol.TList _list800 = iprot.readListBegin();
+                struct.tblNames = new ArrayList<String>(_list800.size);
+                String _elem801;
+                for (int _i802 = 0; _i802 < _list800.size; ++_i802)
                 {
-                  _elem793 = iprot.readString();
-                  struct.tblNames.add(_elem793);
+                  _elem801 = iprot.readString();
+                  struct.tblNames.add(_elem801);
                 }
                 iprot.readListEnd();
               }
@@ -549,6 +630,14 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 4: // CAT_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.catName = iprot.readString();
+              struct.setCatNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -572,9 +661,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TBL_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tblNames.size()));
-            for (String _iter795 : struct.tblNames)
+            for (String _iter803 : struct.tblNames)
             {
-              oprot.writeString(_iter795);
+              oprot.writeString(_iter803);
             }
             oprot.writeListEnd();
           }
@@ -588,6 +677,13 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldEnd();
         }
       }
+      if (struct.catName != null) {
+        if (struct.isSetCatName()) {
+          oprot.writeFieldBegin(CAT_NAME_FIELD_DESC);
+          oprot.writeString(struct.catName);
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -613,19 +709,25 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetCapabilities()) {
         optionals.set(1);
       }
-      oprot.writeBitSet(optionals, 2);
+      if (struct.isSetCatName()) {
+        optionals.set(2);
+      }
+      oprot.writeBitSet(optionals, 3);
       if (struct.isSetTblNames()) {
         {
           oprot.writeI32(struct.tblNames.size());
-          for (String _iter796 : struct.tblNames)
+          for (String _iter804 : struct.tblNames)
           {
-            oprot.writeString(_iter796);
+            oprot.writeString(_iter804);
           }
         }
       }
       if (struct.isSetCapabilities()) {
         struct.capabilities.write(oprot);
       }
+      if (struct.isSetCatName()) {
+        oprot.writeString(struct.catName);
+      }
     }
 
     @Override
@@ -633,16 +735,16 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol iprot = (TTupleProtocol) prot;
       struct.dbName = iprot.readString();
       struct.setDbNameIsSet(true);
-      BitSet incoming = iprot.readBitSet(2);
+      BitSet incoming = iprot.readBitSet(3);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list797 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.tblNames = new ArrayList<String>(_list797.size);
-          String _elem798;
-          for (int _i799 = 0; _i799 < _list797.size; ++_i799)
+          org.apache.thrift.protocol.TList _list805 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.tblNames = new ArrayList<String>(_list805.size);
+          String _elem806;
+          for (int _i807 = 0; _i807 < _list805.size; ++_i807)
           {
-            _elem798 = iprot.readString();
-            struct.tblNames.add(_elem798);
+            _elem806 = iprot.readString();
+            struct.tblNames.add(_elem806);
           }
         }
         struct.setTblNamesIsSet(true);
@@ -652,6 +754,10 @@ import org.slf4j.LoggerFactory;
         struct.capabilities.read(iprot);
         struct.setCapabilitiesIsSet(true);
       }
+      if (incoming.get(2)) {
+        struct.catName = iprot.readString();
+        struct.setCatNameIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java
index 3ad5104..c020773 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // TABLES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list800 = iprot.readListBegin();
-                struct.tables = new ArrayList<Table>(_list800.size);
-                Table _elem801;
-                for (int _i802 = 0; _i802 < _list800.size; ++_i802)
+                org.apache.thrift.protocol.TList _list808 = iprot.readListBegin();
+                struct.tables = new ArrayList<Table>(_list808.size);
+                Table _elem809;
+                for (int _i810 = 0; _i810 < _list808.size; ++_i810)
                 {
-                  _elem801 = new Table();
-                  _elem801.read(iprot);
-                  struct.tables.add(_elem801);
+                  _elem809 = new Table();
+                  _elem809.read(iprot);
+                  struct.tables.add(_elem809);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TABLES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.tables.size()));
-          for (Table _iter803 : struct.tables)
+          for (Table _iter811 : struct.tables)
           {
-            _iter803.write(oprot);
+            _iter811.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.tables.size());
-        for (Table _iter804 : struct.tables)
+        for (Table _iter812 : struct.tables)
         {
-          _iter804.write(oprot);
+          _iter812.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetTablesResult struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list805 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.tables = new ArrayList<Table>(_list805.size);
-        Table _elem806;
-        for (int _i807 = 0; _i807 < _list805.size; ++_i807)
+        org.apache.thrift.protocol.TList _list813 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.tables = new ArrayList<Table>(_list813.size);
+        Table _elem814;
+        for (int _i815 = 0; _i815 < _list813.size; ++_i815)
         {
-          _elem806 = new Table();
-          _elem806.read(iprot);
-          struct.tables.add(_elem806);
+          _elem814 = new Table();
+          _elem814.read(iprot);
+          struct.tables.add(_elem814);
         }
       }
       struct.setTablesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsRequest.java
index f3db7ba..68256c7 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsRequest.java
@@ -436,13 +436,13 @@ import org.slf4j.LoggerFactory;
           case 1: // FULL_TABLE_NAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list578 = iprot.readListBegin();
-                struct.fullTableNames = new ArrayList<String>(_list578.size);
-                String _elem579;
-                for (int _i580 = 0; _i580 < _list578.size; ++_i580)
+                org.apache.thrift.protocol.TList _list586 = iprot.readListBegin();
+                struct.fullTableNames = new ArrayList<String>(_list586.size);
+                String _elem587;
+                for (int _i588 = 0; _i588 < _list586.size; ++_i588)
                 {
-                  _elem579 = iprot.readString();
-                  struct.fullTableNames.add(_elem579);
+                  _elem587 = iprot.readString();
+                  struct.fullTableNames.add(_elem587);
                 }
                 iprot.readListEnd();
               }
@@ -476,9 +476,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FULL_TABLE_NAMES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.fullTableNames.size()));
-          for (String _iter581 : struct.fullTableNames)
+          for (String _iter589 : struct.fullTableNames)
           {
-            oprot.writeString(_iter581);
+            oprot.writeString(_iter589);
           }
           oprot.writeListEnd();
         }
@@ -508,9 +508,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fullTableNames.size());
-        for (String _iter582 : struct.fullTableNames)
+        for (String _iter590 : struct.fullTableNames)
         {
-          oprot.writeString(_iter582);
+          oprot.writeString(_iter590);
         }
       }
       oprot.writeString(struct.validTxnList);
@@ -520,13 +520,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetValidWriteIdsRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list583 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.fullTableNames = new ArrayList<String>(_list583.size);
-        String _elem584;
-        for (int _i585 = 0; _i585 < _list583.size; ++_i585)
+        org.apache.thrift.protocol.TList _list591 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.fullTableNames = new ArrayList<String>(_list591.size);
+        String _elem592;
+        for (int _i593 = 0; _i593 < _list591.size; ++_i593)
         {
-          _elem584 = iprot.readString();
-          struct.fullTableNames.add(_elem584);
+          _elem592 = iprot.readString();
+          struct.fullTableNames.add(_elem592);
         }
       }
       struct.setFullTableNamesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsResponse.java
index e0b0dca..5512fb4 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsResponse.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // TBL_VALID_WRITE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list594 = iprot.readListBegin();
-                struct.tblValidWriteIds = new ArrayList<TableValidWriteIds>(_list594.size);
-                TableValidWriteIds _elem595;
-                for (int _i596 = 0; _i596 < _list594.size; ++_i596)
+                org.apache.thrift.protocol.TList _list602 = iprot.readListBegin();
+                struct.tblValidWriteIds = new ArrayList<TableValidWriteIds>(_list602.size);
+                TableValidWriteIds _elem603;
+                for (int _i604 = 0; _i604 < _list602.size; ++_i604)
                 {
-                  _elem595 = new TableValidWriteIds();
-                  _elem595.read(iprot);
-                  struct.tblValidWriteIds.add(_elem595);
+                  _elem603 = new TableValidWriteIds();
+                  _elem603.read(iprot);
+                  struct.tblValidWriteIds.add(_elem603);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TBL_VALID_WRITE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.tblValidWriteIds.size()));
-          for (TableValidWriteIds _iter597 : struct.tblValidWriteIds)
+          for (TableValidWriteIds _iter605 : struct.tblValidWriteIds)
           {
-            _iter597.write(oprot);
+            _iter605.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.tblValidWriteIds.size());
-        for (TableValidWriteIds _iter598 : struct.tblValidWriteIds)
+        for (TableValidWriteIds _iter606 : struct.tblValidWriteIds)
         {
-          _iter598.write(oprot);
+          _iter606.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetValidWriteIdsResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list599 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.tblValidWriteIds = new ArrayList<TableValidWriteIds>(_list599.size);
-        TableValidWriteIds _elem600;
-        for (int _i601 = 0; _i601 < _list599.size; ++_i601)
+        org.apache.thrift.protocol.TList _list607 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.tblValidWriteIds = new ArrayList<TableValidWriteIds>(_list607.size);
+        TableValidWriteIds _elem608;
+        for (int _i609 = 0; _i609 < _list607.size; ++_i609)
         {
-          _elem600 = new TableValidWriteIds();
-          _elem600.read(iprot);
-          struct.tblValidWriteIds.add(_elem600);
+          _elem608 = new TableValidWriteIds();
+          _elem608.read(iprot);
+          struct.tblValidWriteIds.add(_elem608);
         }
       }
       struct.setTblValidWriteIdsIsSet(true);


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
index e824f4a..751eec4 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
@@ -46,6 +46,14 @@ import org.slf4j.LoggerFactory;
 
     public void setMetaConf(String key, String value) throws MetaException, org.apache.thrift.TException;
 
+    public void create_catalog(CreateCatalogRequest catalog) throws AlreadyExistsException, InvalidObjectException, MetaException, org.apache.thrift.TException;
+
+    public GetCatalogResponse get_catalog(GetCatalogRequest catName) throws NoSuchObjectException, MetaException, org.apache.thrift.TException;
+
+    public GetCatalogsResponse get_catalogs() throws MetaException, org.apache.thrift.TException;
+
+    public void drop_catalog(DropCatalogRequest catName) throws NoSuchObjectException, InvalidOperationException, MetaException, org.apache.thrift.TException;
+
     public void create_database(Database database) throws AlreadyExistsException, InvalidObjectException, MetaException, org.apache.thrift.TException;
 
     public Database get_database(String name) throws NoSuchObjectException, MetaException, org.apache.thrift.TException;
@@ -120,7 +128,7 @@ import org.slf4j.LoggerFactory;
 
     public Map<String,Materialization> get_materialization_invalidation_info(String dbname, List<String> tbl_names) throws MetaException, InvalidOperationException, UnknownDBException, org.apache.thrift.TException;
 
-    public void update_creation_metadata(String dbname, String tbl_name, CreationMetadata creation_metadata) throws MetaException, InvalidOperationException, UnknownDBException, org.apache.thrift.TException;
+    public void update_creation_metadata(String catName, String dbname, String tbl_name, CreationMetadata creation_metadata) throws MetaException, InvalidOperationException, UnknownDBException, org.apache.thrift.TException;
 
     public List<String> get_table_names_by_filter(String dbname, String filter, short max_tables) throws MetaException, InvalidOperationException, UnknownDBException, org.apache.thrift.TException;
 
@@ -442,6 +450,14 @@ import org.slf4j.LoggerFactory;
 
     public void setMetaConf(String key, String value, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
+    public void create_catalog(CreateCatalogRequest catalog, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void get_catalog(GetCatalogRequest catName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void get_catalogs(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void drop_catalog(DropCatalogRequest catName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
     public void create_database(Database database, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
     public void get_database(String name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
@@ -516,7 +532,7 @@ import org.slf4j.LoggerFactory;
 
     public void get_materialization_invalidation_info(String dbname, List<String> tbl_names, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
-    public void update_creation_metadata(String dbname, String tbl_name, CreationMetadata creation_metadata, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+    public void update_creation_metadata(String catName, String dbname, String tbl_name, CreationMetadata creation_metadata, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
     public void get_table_names_by_filter(String dbname, String filter, short max_tables, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
@@ -902,6 +918,118 @@ import org.slf4j.LoggerFactory;
       return;
     }
 
+    public void create_catalog(CreateCatalogRequest catalog) throws AlreadyExistsException, InvalidObjectException, MetaException, org.apache.thrift.TException
+    {
+      send_create_catalog(catalog);
+      recv_create_catalog();
+    }
+
+    public void send_create_catalog(CreateCatalogRequest catalog) throws org.apache.thrift.TException
+    {
+      create_catalog_args args = new create_catalog_args();
+      args.setCatalog(catalog);
+      sendBase("create_catalog", args);
+    }
+
+    public void recv_create_catalog() throws AlreadyExistsException, InvalidObjectException, MetaException, org.apache.thrift.TException
+    {
+      create_catalog_result result = new create_catalog_result();
+      receiveBase(result, "create_catalog");
+      if (result.o1 != null) {
+        throw result.o1;
+      }
+      if (result.o2 != null) {
+        throw result.o2;
+      }
+      if (result.o3 != null) {
+        throw result.o3;
+      }
+      return;
+    }
+
+    public GetCatalogResponse get_catalog(GetCatalogRequest catName) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
+    {
+      send_get_catalog(catName);
+      return recv_get_catalog();
+    }
+
+    public void send_get_catalog(GetCatalogRequest catName) throws org.apache.thrift.TException
+    {
+      get_catalog_args args = new get_catalog_args();
+      args.setCatName(catName);
+      sendBase("get_catalog", args);
+    }
+
+    public GetCatalogResponse recv_get_catalog() throws NoSuchObjectException, MetaException, org.apache.thrift.TException
+    {
+      get_catalog_result result = new get_catalog_result();
+      receiveBase(result, "get_catalog");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.o1 != null) {
+        throw result.o1;
+      }
+      if (result.o2 != null) {
+        throw result.o2;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_catalog failed: unknown result");
+    }
+
+    public GetCatalogsResponse get_catalogs() throws MetaException, org.apache.thrift.TException
+    {
+      send_get_catalogs();
+      return recv_get_catalogs();
+    }
+
+    public void send_get_catalogs() throws org.apache.thrift.TException
+    {
+      get_catalogs_args args = new get_catalogs_args();
+      sendBase("get_catalogs", args);
+    }
+
+    public GetCatalogsResponse recv_get_catalogs() throws MetaException, org.apache.thrift.TException
+    {
+      get_catalogs_result result = new get_catalogs_result();
+      receiveBase(result, "get_catalogs");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.o1 != null) {
+        throw result.o1;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_catalogs failed: unknown result");
+    }
+
+    public void drop_catalog(DropCatalogRequest catName) throws NoSuchObjectException, InvalidOperationException, MetaException, org.apache.thrift.TException
+    {
+      send_drop_catalog(catName);
+      recv_drop_catalog();
+    }
+
+    public void send_drop_catalog(DropCatalogRequest catName) throws org.apache.thrift.TException
+    {
+      drop_catalog_args args = new drop_catalog_args();
+      args.setCatName(catName);
+      sendBase("drop_catalog", args);
+    }
+
+    public void recv_drop_catalog() throws NoSuchObjectException, InvalidOperationException, MetaException, org.apache.thrift.TException
+    {
+      drop_catalog_result result = new drop_catalog_result();
+      receiveBase(result, "drop_catalog");
+      if (result.o1 != null) {
+        throw result.o1;
+      }
+      if (result.o2 != null) {
+        throw result.o2;
+      }
+      if (result.o3 != null) {
+        throw result.o3;
+      }
+      return;
+    }
+
     public void create_database(Database database) throws AlreadyExistsException, InvalidObjectException, MetaException, org.apache.thrift.TException
     {
       send_create_database(database);
@@ -1969,15 +2097,16 @@ import org.slf4j.LoggerFactory;
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_materialization_invalidation_info failed: unknown result");
     }
 
-    public void update_creation_metadata(String dbname, String tbl_name, CreationMetadata creation_metadata) throws MetaException, InvalidOperationException, UnknownDBException, org.apache.thrift.TException
+    public void update_creation_metadata(String catName, String dbname, String tbl_name, CreationMetadata creation_metadata) throws MetaException, InvalidOperationException, UnknownDBException, org.apache.thrift.TException
     {
-      send_update_creation_metadata(dbname, tbl_name, creation_metadata);
+      send_update_creation_metadata(catName, dbname, tbl_name, creation_metadata);
       recv_update_creation_metadata();
     }
 
-    public void send_update_creation_metadata(String dbname, String tbl_name, CreationMetadata creation_metadata) throws org.apache.thrift.TException
+    public void send_update_creation_metadata(String catName, String dbname, String tbl_name, CreationMetadata creation_metadata) throws org.apache.thrift.TException
     {
       update_creation_metadata_args args = new update_creation_metadata_args();
+      args.setCatName(catName);
       args.setDbname(dbname);
       args.setTbl_name(tbl_name);
       args.setCreation_metadata(creation_metadata);
@@ -6590,6 +6719,131 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    public void create_catalog(CreateCatalogRequest catalog, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      create_catalog_call method_call = new create_catalog_call(catalog, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class create_catalog_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private CreateCatalogRequest catalog;
+      public create_catalog_call(CreateCatalogRequest catalog, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.catalog = catalog;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("create_catalog", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        create_catalog_args args = new create_catalog_args();
+        args.setCatalog(catalog);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws AlreadyExistsException, InvalidObjectException, MetaException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_create_catalog();
+      }
+    }
+
+    public void get_catalog(GetCatalogRequest catName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      get_catalog_call method_call = new get_catalog_call(catName, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_catalog_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private GetCatalogRequest catName;
+      public get_catalog_call(GetCatalogRequest catName, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.catName = catName;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("get_catalog", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        get_catalog_args args = new get_catalog_args();
+        args.setCatName(catName);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public GetCatalogResponse getResult() throws NoSuchObjectException, MetaException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_get_catalog();
+      }
+    }
+
+    public void get_catalogs(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      get_catalogs_call method_call = new get_catalogs_call(resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_catalogs_call extends org.apache.thrift.async.TAsyncMethodCall {
+      public get_catalogs_call(org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("get_catalogs", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        get_catalogs_args args = new get_catalogs_args();
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public GetCatalogsResponse getResult() throws MetaException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_get_catalogs();
+      }
+    }
+
+    public void drop_catalog(DropCatalogRequest catName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      drop_catalog_call method_call = new drop_catalog_call(catName, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class drop_catalog_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private DropCatalogRequest catName;
+      public drop_catalog_call(DropCatalogRequest catName, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.catName = catName;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("drop_catalog", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        drop_catalog_args args = new drop_catalog_args();
+        args.setCatName(catName);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws NoSuchObjectException, InvalidOperationException, MetaException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_drop_catalog();
+      }
+    }
+
     public void create_database(Database database, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
       create_database_call method_call = new create_database_call(database, resultHandler, this, ___protocolFactory, ___transport);
@@ -7864,19 +8118,21 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public void update_creation_metadata(String dbname, String tbl_name, CreationMetadata creation_metadata, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+    public void update_creation_metadata(String catName, String dbname, String tbl_name, CreationMetadata creation_metadata, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      update_creation_metadata_call method_call = new update_creation_metadata_call(dbname, tbl_name, creation_metadata, resultHandler, this, ___protocolFactory, ___transport);
+      update_creation_metadata_call method_call = new update_creation_metadata_call(catName, dbname, tbl_name, creation_metadata, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
     @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class update_creation_metadata_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String catName;
       private String dbname;
       private String tbl_name;
       private CreationMetadata creation_metadata;
-      public update_creation_metadata_call(String dbname, String tbl_name, CreationMetadata creation_metadata, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public update_creation_metadata_call(String catName, String dbname, String tbl_name, CreationMetadata creation_metadata, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
+        this.catName = catName;
         this.dbname = dbname;
         this.tbl_name = tbl_name;
         this.creation_metadata = creation_metadata;
@@ -7885,6 +8141,7 @@ import org.slf4j.LoggerFactory;
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
         prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("update_creation_metadata", org.apache.thrift.protocol.TMessageType.CALL, 0));
         update_creation_metadata_args args = new update_creation_metadata_args();
+        args.setCatName(catName);
         args.setDbname(dbname);
         args.setTbl_name(tbl_name);
         args.setCreation_metadata(creation_metadata);
@@ -13284,6 +13541,10 @@ import org.slf4j.LoggerFactory;
     private static <I extends Iface> Map<String,  org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> getProcessMap(Map<String,  org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> processMap) {
       processMap.put("getMetaConf", new getMetaConf());
       processMap.put("setMetaConf", new setMetaConf());
+      processMap.put("create_catalog", new create_catalog());
+      processMap.put("get_catalog", new get_catalog());
+      processMap.put("get_catalogs", new get_catalogs());
+      processMap.put("drop_catalog", new drop_catalog());
       processMap.put("create_database", new create_database());
       processMap.put("get_database", new get_database());
       processMap.put("drop_database", new drop_database());
@@ -13529,6 +13790,112 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class create_catalog<I extends Iface> extends org.apache.thrift.ProcessFunction<I, create_catalog_args> {
+      public create_catalog() {
+        super("create_catalog");
+      }
+
+      public create_catalog_args getEmptyArgsInstance() {
+        return new create_catalog_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public create_catalog_result getResult(I iface, create_catalog_args args) throws org.apache.thrift.TException {
+        create_catalog_result result = new create_catalog_result();
+        try {
+          iface.create_catalog(args.catalog);
+        } catch (AlreadyExistsException o1) {
+          result.o1 = o1;
+        } catch (InvalidObjectException o2) {
+          result.o2 = o2;
+        } catch (MetaException o3) {
+          result.o3 = o3;
+        }
+        return result;
+      }
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_catalog<I extends Iface> extends org.apache.thrift.ProcessFunction<I, get_catalog_args> {
+      public get_catalog() {
+        super("get_catalog");
+      }
+
+      public get_catalog_args getEmptyArgsInstance() {
+        return new get_catalog_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public get_catalog_result getResult(I iface, get_catalog_args args) throws org.apache.thrift.TException {
+        get_catalog_result result = new get_catalog_result();
+        try {
+          result.success = iface.get_catalog(args.catName);
+        } catch (NoSuchObjectException o1) {
+          result.o1 = o1;
+        } catch (MetaException o2) {
+          result.o2 = o2;
+        }
+        return result;
+      }
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_catalogs<I extends Iface> extends org.apache.thrift.ProcessFunction<I, get_catalogs_args> {
+      public get_catalogs() {
+        super("get_catalogs");
+      }
+
+      public get_catalogs_args getEmptyArgsInstance() {
+        return new get_catalogs_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public get_catalogs_result getResult(I iface, get_catalogs_args args) throws org.apache.thrift.TException {
+        get_catalogs_result result = new get_catalogs_result();
+        try {
+          result.success = iface.get_catalogs();
+        } catch (MetaException o1) {
+          result.o1 = o1;
+        }
+        return result;
+      }
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class drop_catalog<I extends Iface> extends org.apache.thrift.ProcessFunction<I, drop_catalog_args> {
+      public drop_catalog() {
+        super("drop_catalog");
+      }
+
+      public drop_catalog_args getEmptyArgsInstance() {
+        return new drop_catalog_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public drop_catalog_result getResult(I iface, drop_catalog_args args) throws org.apache.thrift.TException {
+        drop_catalog_result result = new drop_catalog_result();
+        try {
+          iface.drop_catalog(args.catName);
+        } catch (NoSuchObjectException o1) {
+          result.o1 = o1;
+        } catch (InvalidOperationException o2) {
+          result.o2 = o2;
+        } catch (MetaException o3) {
+          result.o3 = o3;
+        }
+        return result;
+      }
+    }
+
     @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class create_database<I extends Iface> extends org.apache.thrift.ProcessFunction<I, create_database_args> {
       public create_database() {
         super("create_database");
@@ -14515,7 +14882,7 @@ import org.slf4j.LoggerFactory;
       public update_creation_metadata_result getResult(I iface, update_creation_metadata_args args) throws org.apache.thrift.TException {
         update_creation_metadata_result result = new update_creation_metadata_result();
         try {
-          iface.update_creation_metadata(args.dbname, args.tbl_name, args.creation_metadata);
+          iface.update_creation_metadata(args.catName, args.dbname, args.tbl_name, args.creation_metadata);
         } catch (MetaException o1) {
           result.o1 = o1;
         } catch (InvalidOperationException o2) {
@@ -18539,6 +18906,10 @@ import org.slf4j.LoggerFactory;
     private static <I extends AsyncIface> Map<String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase,?>> getProcessMap(Map<String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase, ?>> processMap) {
       processMap.put("getMetaConf", new getMetaConf());
       processMap.put("setMetaConf", new setMetaConf());
+      processMap.put("create_catalog", new create_catalog());
+      processMap.put("get_catalog", new get_catalog());
+      processMap.put("get_catalogs", new get_catalogs());
+      processMap.put("drop_catalog", new drop_catalog());
       processMap.put("create_database", new create_database());
       processMap.put("get_database", new get_database());
       processMap.put("drop_database", new drop_database());
@@ -18849,20 +19220,20 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class create_database<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, create_database_args, Void> {
-      public create_database() {
-        super("create_database");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class create_catalog<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, create_catalog_args, Void> {
+      public create_catalog() {
+        super("create_catalog");
       }
 
-      public create_database_args getEmptyArgsInstance() {
-        return new create_database_args();
+      public create_catalog_args getEmptyArgsInstance() {
+        return new create_catalog_args();
       }
 
       public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
         return new AsyncMethodCallback<Void>() { 
           public void onComplete(Void o) {
-            create_database_result result = new create_database_result();
+            create_catalog_result result = new create_catalog_result();
             try {
               fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
               return;
@@ -18874,7 +19245,7 @@ import org.slf4j.LoggerFactory;
           public void onError(Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TBase msg;
-            create_database_result result = new create_database_result();
+            create_catalog_result result = new create_catalog_result();
             if (e instanceof AlreadyExistsException) {
                         result.o1 = (AlreadyExistsException) e;
                         result.setO1IsSet(true);
@@ -18910,25 +19281,25 @@ import org.slf4j.LoggerFactory;
         return false;
       }
 
-      public void start(I iface, create_database_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
-        iface.create_database(args.database,resultHandler);
+      public void start(I iface, create_catalog_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
+        iface.create_catalog(args.catalog,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_database<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_database_args, Database> {
-      public get_database() {
-        super("get_database");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_catalog<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_catalog_args, GetCatalogResponse> {
+      public get_catalog() {
+        super("get_catalog");
       }
 
-      public get_database_args getEmptyArgsInstance() {
-        return new get_database_args();
+      public get_catalog_args getEmptyArgsInstance() {
+        return new get_catalog_args();
       }
 
-      public AsyncMethodCallback<Database> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+      public AsyncMethodCallback<GetCatalogResponse> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new AsyncMethodCallback<Database>() { 
-          public void onComplete(Database o) {
-            get_database_result result = new get_database_result();
+        return new AsyncMethodCallback<GetCatalogResponse>() { 
+          public void onComplete(GetCatalogResponse o) {
+            get_catalog_result result = new get_catalog_result();
             result.success = o;
             try {
               fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -18941,7 +19312,258 @@ import org.slf4j.LoggerFactory;
           public void onError(Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TBase msg;
-            get_database_result result = new get_database_result();
+            get_catalog_result result = new get_catalog_result();
+            if (e instanceof NoSuchObjectException) {
+                        result.o1 = (NoSuchObjectException) e;
+                        result.setO1IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof MetaException) {
+                        result.o2 = (MetaException) e;
+                        result.setO2IsSet(true);
+                        msg = result;
+            }
+             else 
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, get_catalog_args args, org.apache.thrift.async.AsyncMethodCallback<GetCatalogResponse> resultHandler) throws TException {
+        iface.get_catalog(args.catName,resultHandler);
+      }
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_catalogs<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_catalogs_args, GetCatalogsResponse> {
+      public get_catalogs() {
+        super("get_catalogs");
+      }
+
+      public get_catalogs_args getEmptyArgsInstance() {
+        return new get_catalogs_args();
+      }
+
+      public AsyncMethodCallback<GetCatalogsResponse> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<GetCatalogsResponse>() { 
+          public void onComplete(GetCatalogsResponse o) {
+            get_catalogs_result result = new get_catalogs_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            get_catalogs_result result = new get_catalogs_result();
+            if (e instanceof MetaException) {
+                        result.o1 = (MetaException) e;
+                        result.setO1IsSet(true);
+                        msg = result;
+            }
+             else 
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, get_catalogs_args args, org.apache.thrift.async.AsyncMethodCallback<GetCatalogsResponse> resultHandler) throws TException {
+        iface.get_catalogs(resultHandler);
+      }
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class drop_catalog<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, drop_catalog_args, Void> {
+      public drop_catalog() {
+        super("drop_catalog");
+      }
+
+      public drop_catalog_args getEmptyArgsInstance() {
+        return new drop_catalog_args();
+      }
+
+      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            drop_catalog_result result = new drop_catalog_result();
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            drop_catalog_result result = new drop_catalog_result();
+            if (e instanceof NoSuchObjectException) {
+                        result.o1 = (NoSuchObjectException) e;
+                        result.setO1IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof InvalidOperationException) {
+                        result.o2 = (InvalidOperationException) e;
+                        result.setO2IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof MetaException) {
+                        result.o3 = (MetaException) e;
+                        result.setO3IsSet(true);
+                        msg = result;
+            }
+             else 
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, drop_catalog_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
+        iface.drop_catalog(args.catName,resultHandler);
+      }
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class create_database<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, create_database_args, Void> {
+      public create_database() {
+        super("create_database");
+      }
+
+      public create_database_args getEmptyArgsInstance() {
+        return new create_database_args();
+      }
+
+      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            create_database_result result = new create_database_result();
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            create_database_result result = new create_database_result();
+            if (e instanceof AlreadyExistsException) {
+                        result.o1 = (AlreadyExistsException) e;
+                        result.setO1IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof InvalidObjectException) {
+                        result.o2 = (InvalidObjectException) e;
+                        result.setO2IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof MetaException) {
+                        result.o3 = (MetaException) e;
+                        result.setO3IsSet(true);
+                        msg = result;
+            }
+             else 
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, create_database_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
+        iface.create_database(args.database,resultHandler);
+      }
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_database<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_database_args, Database> {
+      public get_database() {
+        super("get_database");
+      }
+
+      public get_database_args getEmptyArgsInstance() {
+        return new get_database_args();
+      }
+
+      public AsyncMethodCallback<Database> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<Database>() { 
+          public void onComplete(Database o) {
+            get_database_result result = new get_database_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            get_database_result result = new get_database_result();
             if (e instanceof NoSuchObjectException) {
                         result.o1 = (NoSuchObjectException) e;
                         result.setO1IsSet(true);
@@ -21210,7 +21832,7 @@ import org.slf4j.LoggerFactory;
       }
 
       public void start(I iface, update_creation_metadata_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
-        iface.update_creation_metadata(args.dbname, args.tbl_name, args.creation_metadata,resultHandler);
+        iface.update_creation_metadata(args.catName, args.dbname, args.tbl_name, args.creation_metadata,resultHandler);
       }
     }
 
@@ -31439,7 +32061,3412 @@ import org.slf4j.LoggerFactory;
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("getMetaConf_result(");
+      StringBuilder sb = new StringBuilder("getMetaConf_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("o1:");
+      if (this.o1 == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.o1);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class getMetaConf_resultStandardSchemeFactory implements SchemeFactory {
+      public getMetaConf_resultStandardScheme getScheme() {
+        return new getMetaConf_resultStandardScheme();
+      }
+    }
+
+    private static class getMetaConf_resultStandardScheme extends StandardScheme<getMetaConf_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getMetaConf_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.success = iprot.readString();
+                struct.setSuccessIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 1: // O1
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.o1 = new MetaException();
+                struct.o1.read(iprot);
+                struct.setO1IsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, getMetaConf_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.success != null) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          oprot.writeString(struct.success);
+          oprot.writeFieldEnd();
+        }
+        if (struct.o1 != null) {
+          oprot.writeFieldBegin(O1_FIELD_DESC);
+          struct.o1.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getMetaConf_resultTupleSchemeFactory implements SchemeFactory {
+      public getMetaConf_resultTupleScheme getScheme() {
+        return new getMetaConf_resultTupleScheme();
+      }
+    }
+
+    private static class getMetaConf_resultTupleScheme extends TupleScheme<getMetaConf_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getMetaConf_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetSuccess()) {
+          optionals.set(0);
+        }
+        if (struct.isSetO1()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetSuccess()) {
+          oprot.writeString(struct.success);
+        }
+        if (struct.isSetO1()) {
+          struct.o1.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getMetaConf_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(2);
+        if (incoming.get(0)) {
+          struct.success = iprot.readString();
+          struct.setSuccessIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.o1 = new MetaException();
+          struct.o1.read(iprot);
+          struct.setO1IsSet(true);
+        }
+      }
+    }
+
+  }
+
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class setMetaConf_args implements org.apache.thrift.TBase<setMetaConf_args, setMetaConf_args._Fields>, java.io.Serializable, Cloneable, Comparable<setMetaConf_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("setMetaConf_args");
+
+    private static final org.apache.thrift.protocol.TField KEY_FIELD_DESC = new org.apache.thrift.protocol.TField("key", org.apache.thrift.protocol.TType.STRING, (short)1);
+    private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.STRING, (short)2);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new setMetaConf_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new setMetaConf_argsTupleSchemeFactory());
+    }
+
+    private String key; // required
+    private String value; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      KEY((short)1, "key"),
+      VALUE((short)2, "value");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // KEY
+            return KEY;
+          case 2: // VALUE
+            return VALUE;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.KEY, new org.apache.thrift.meta_data.FieldMetaData("key", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(setMetaConf_args.class, metaDataMap);
+    }
+
+    public setMetaConf_args() {
+    }
+
+    public setMetaConf_args(
+      String key,
+      String value)
+    {
+      this();
+      this.key = key;
+      this.value = value;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public setMetaConf_args(setMetaConf_args other) {
+      if (other.isSetKey()) {
+        this.key = other.key;
+      }
+      if (other.isSetValue()) {
+        this.value = other.value;
+      }
+    }
+
+    public setMetaConf_args deepCopy() {
+      return new setMetaConf_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.key = null;
+      this.value = null;
+    }
+
+    public String getKey() {
+      return this.key;
+    }
+
+    public void setKey(String key) {
+      this.key = key;
+    }
+
+    public void unsetKey() {
+      this.key = null;
+    }
+
+    /** Returns true if field key is set (has been assigned a value) and false otherwise */
+    public boolean isSetKey() {
+      return this.key != null;
+    }
+
+    public void setKeyIsSet(boolean value) {
+      if (!value) {
+        this.key = null;
+      }
+    }
+
+    public String getValue() {
+      return this.value;
+    }
+
+    public void setValue(String value) {
+      this.value = value;
+    }
+
+    public void unsetValue() {
+      this.value = null;
+    }
+
+    /** Returns true if field value is set (has been assigned a value) and false otherwise */
+    public boolean isSetValue() {
+      return this.value != null;
+    }
+
+    public void setValueIsSet(boolean value) {
+      if (!value) {
+        this.value = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case KEY:
+        if (value == null) {
+          unsetKey();
+        } else {
+          setKey((String)value);
+        }
+        break;
+
+      case VALUE:
+        if (value == null) {
+          unsetValue();
+        } else {
+          setValue((String)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case KEY:
+        return getKey();
+
+      case VALUE:
+        return getValue();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case KEY:
+        return isSetKey();
+      case VALUE:
+        return isSetValue();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof setMetaConf_args)
+        return this.equals((setMetaConf_args)that);
+      return false;
+    }
+
+    public boolean equals(setMetaConf_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_key = true && this.isSetKey();
+      boolean that_present_key = true && that.isSetKey();
+      if (this_present_key || that_present_key) {
+        if (!(this_present_key && that_present_key))
+          return false;
+        if (!this.key.equals(that.key))
+          return false;
+      }
+
+      boolean this_present_value = true && this.isSetValue();
+      boolean that_present_value = true && that.isSetValue();
+      if (this_present_value || that_present_value) {
+        if (!(this_present_value && that_present_value))
+          return false;
+        if (!this.value.equals(that.value))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_key = true && (isSetKey());
+      list.add(present_key);
+      if (present_key)
+        list.add(key);
+
+      boolean present_value = true && (isSetValue());
+      list.add(present_value);
+      if (present_value)
+        list.add(value);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(setMetaConf_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(isSetKey()).compareTo(other.isSetKey());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetKey()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.key, other.key);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetValue()).compareTo(other.isSetValue());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetValue()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, other.value);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("setMetaConf_args(");
+      boolean first = true;
+
+      sb.append("key:");
+      if (this.key == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.key);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("value:");
+      if (this.value == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.value);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class setMetaConf_argsStandardSchemeFactory implements SchemeFactory {
+      public setMetaConf_argsStandardScheme getScheme() {
+        return new setMetaConf_argsStandardScheme();
+      }
+    }
+
+    private static class setMetaConf_argsStandardScheme extends StandardScheme<setMetaConf_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, setMetaConf_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // KEY
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.key = iprot.readString();
+                struct.setKeyIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // VALUE
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.value = iprot.readString();
+                struct.setValueIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, setMetaConf_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.key != null) {
+          oprot.writeFieldBegin(KEY_FIELD_DESC);
+          oprot.writeString(struct.key);
+          oprot.writeFieldEnd();
+        }
+        if (struct.value != null) {
+          oprot.writeFieldBegin(VALUE_FIELD_DESC);
+          oprot.writeString(struct.value);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class setMetaConf_argsTupleSchemeFactory implements SchemeFactory {
+      public setMetaConf_argsTupleScheme getScheme() {
+        return new setMetaConf_argsTupleScheme();
+      }
+    }
+
+    private static class setMetaConf_argsTupleScheme extends TupleScheme<setMetaConf_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, setMetaConf_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetKey()) {
+          optionals.set(0);
+        }
+        if (struct.isSetValue()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetKey()) {
+          oprot.writeString(struct.key);
+        }
+        if (struct.isSetValue()) {
+          oprot.writeString(struct.value);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, setMetaConf_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(2);
+        if (incoming.get(0)) {
+          struct.key = iprot.readString();
+          struct.setKeyIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.value = iprot.readString();
+          struct.setValueIsSet(true);
+        }
+      }
+    }
+
+  }
+
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class setMetaConf_result implements org.apache.thrift.TBase<setMetaConf_result, setMetaConf_result._Fields>, java.io.Serializable, Cloneable, Comparable<setMetaConf_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("setMetaConf_result");
+
+    private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new setMetaConf_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new setMetaConf_resultTupleSchemeFactory());
+    }
+
+    private MetaException o1; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      O1((short)1, "o1");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // O1
+            return O1;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(setMetaConf_result.class, metaDataMap);
+    }
+
+    public setMetaConf_result() {
+    }
+
+    public setMetaConf_result(
+      MetaException o1)
+    {
+      this();
+      this.o1 = o1;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public setMetaConf_result(setMetaConf_result other) {
+      if (other.isSetO1()) {
+        this.o1 = new MetaException(other.o1);
+      }
+    }
+
+    public setMetaConf_result deepCopy() {
+      return new setMetaConf_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.o1 = null;
+    }
+
+    public MetaException getO1() {
+      return this.o1;
+    }
+
+    public void setO1(MetaException o1) {
+      this.o1 = o1;
+    }
+
+    public void unsetO1() {
+      this.o1 = null;
+    }
+
+    /** Returns true if field o1 is set (has been assigned a value) and false otherwise */
+    public boolean isSetO1() {
+      return this.o1 != null;
+    }
+
+    public void setO1IsSet(boolean value) {
+      if (!value) {
+        this.o1 = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case O1:
+        if (value == null) {
+          unsetO1();
+        } else {
+          setO1((MetaException)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case O1:
+        return getO1();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case O1:
+        return isSetO1();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof setMetaConf_result)
+        return this.equals((setMetaConf_result)that);
+      return false;
+    }
+
+    public boolean equals(setMetaConf_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_o1 = true && this.isSetO1();
+      boolean that_present_o1 = true && that.isSetO1();
+      if (this_present_o1 || that_present_o1) {
+        if (!(this_present_o1 && that_present_o1))
+          return false;
+        if (!this.o1.equals(that.o1))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_o1 = true && (isSetO1());
+      list.add(present_o1);
+      if (present_o1)
+        list.add(o1);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(setMetaConf_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(isSetO1()).compareTo(other.isSetO1());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetO1()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o1, other.o1);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+      }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("setMetaConf_result(");
+      boolean first = true;
+
+      sb.append("o1:");
+      if (this.o1 == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.o1);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class setMetaConf_resultStandardSchemeFactory implements SchemeFactory {
+      public setMetaConf_resultStandardScheme getScheme() {
+        return new setMetaConf_resultStandardScheme();
+      }
+    }
+
+    private static class setMetaConf_resultStandardScheme extends StandardScheme<setMetaConf_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, setMetaConf_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // O1
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.o1 = new MetaException();
+                struct.o1.read(iprot);
+                struct.setO1IsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, setMetaConf_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.o1 != null) {
+          oprot.writeFieldBegin(O1_FIELD_DESC);
+          struct.o1.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class setMetaConf_resultTupleSchemeFactory implements SchemeFactory {
+      public setMetaConf_resultTupleScheme getScheme() {
+        return new setMetaConf_resultTupleScheme();
+      }
+    }
+
+    private static class setMetaConf_resultTupleScheme extends TupleScheme<setMetaConf_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, setMetaConf_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetO1()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetO1()) {
+          struct.o1.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, setMetaConf_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.o1 = new MetaException();
+          struct.o1.read(iprot);
+          struct.setO1IsSet(true);
+        }
+      }
+    }
+
+  }
+
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class create_catalog_args implements org.apache.thrift.TBase<create_catalog_args, create_catalog_args._Fields>, java.io.Serializable, Cloneable, Comparable<create_catalog_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("create_catalog_args");
+
+    private static final org.apache.thrift.protocol.TField CATALOG_FIELD_DESC = new org.apache.thrift.protocol.TField("catalog", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new create_catalog_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new create_catalog_argsTupleSchemeFactory());
+    }
+
+    private CreateCatalogRequest catalog; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      CATALOG((short)1, "catalog");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // CATALOG
+            return CATALOG;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.CATALOG, new org.apache.thrift.meta_data.FieldMetaData("catalog", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, CreateCatalogRequest.class)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(create_catalog_args.class, metaDataMap);
+    }
+
+    public create_catalog_args() {
+    }
+
+    public create_catalog_args(
+      CreateCatalogRequest catalog)
+    {
+      this();
+      this.catalog = catalog;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public create_catalog_args(create_catalog_args other) {
+      if (other.isSetCatalog()) {
+        this.catalog = new CreateCatalogRequest(other.catalog);
+      }
+    }
+
+    public create_catalog_args deepCopy() {
+      return new create_catalog_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.catalog = null;
+    }
+
+    public CreateCatalogRequest getCatalog() {
+      return this.catalog;
+    }
+
+    public void setCatalog(CreateCatalogRequest catalog) {
+      this.catalog = catalog;
+    }
+
+    public void unsetCatalog() {
+      this.catalog = null;
+    }
+
+    /** Returns true if field catalog is set (has been assigned a value) and false otherwise */
+    public boolean isSetCatalog() {
+      return this.catalog != null;
+    }
+
+    public void setCatalogIsSet(boolean value) {
+      if (!value) {
+        this.catalog = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case CATALOG:
+        if (value == null) {
+          unsetCatalog();
+        } else {
+          setCatalog((CreateCatalogRequest)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case CATALOG:
+        return getCatalog();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case CATALOG:
+        return isSetCatalog();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof create_catalog_args)
+        return this.equals((create_catalog_args)that);
+      return false;
+    }
+
+    public boolean equals(create_catalog_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_catalog = true && this.isSetCatalog();
+      boolean that_present_catalog = true && that.isSetCatalog();
+      if (this_present_catalog || that_present_catalog) {
+        if (!(this_present_catalog && that_present_catalog))
+          return false;
+        if (!this.catalog.equals(that.catalog))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_catalog = true && (isSetCatalog());
+      list.add(present_catalog);
+      if (present_catalog)
+        list.add(catalog);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(create_catalog_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(isSetCatalog()).compareTo(other.isSetCatalog());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetCatalog()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catalog, other.catalog);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("create_catalog_args(");
+      boolean first = true;
+
+      sb.append("catalog:");
+      if (this.catalog == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.catalog);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (catalog != null) {
+        catalog.validate();
+      }
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class create_catalog_argsStandardSchemeFactory implements SchemeFactory {
+      public create_catalog_argsStandardScheme getScheme() {
+        return new create_catalog_argsStandardScheme();
+      }
+    }
+
+    private static class create_catalog_argsStandardScheme extends StandardScheme<create_catalog_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, create_catalog_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // CATALOG
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.catalog = new CreateCatalogRequest();
+                struct.catalog.read(iprot);
+                struct.setCatalogIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, create_catalog_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.catalog != null) {
+          oprot.writeFieldBegin(CATALOG_FIELD_DESC);
+          struct.catalog.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class create_catalog_argsTupleSchemeFactory implements SchemeFactory {
+      public create_catalog_argsTupleScheme getScheme() {
+        return new create_catalog_argsTupleScheme();
+      }
+    }
+
+    private static class create_catalog_argsTupleScheme extends TupleScheme<create_catalog_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, create_catalog_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetCatalog()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetCatalog()) {
+          struct.catalog.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, create_catalog_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.catalog = new CreateCatalogRequest();
+          struct.catalog.read(iprot);
+          struct.setCatalogIsSet(true);
+        }
+      }
+    }
+
+  }
+
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class create_catalog_result implements org.apache.thrift.TBase<create_catalog_result, create_catalog_result._Fields>, java.io.Serializable, Cloneable, Comparable<create_catalog_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("create_catalog_result");
+
+    private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+    private static final org.apache.thrift.protocol.TField O2_FIELD_DESC = new org.apache.thrift.protocol.TField("o2", org.apache.thrift.protocol

<TRUNCATED>

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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLDefaultConstraint.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLDefaultConstraint.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLDefaultConstraint.java
index 185b77e..7b8a257 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLDefaultConstraint.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLDefaultConstraint.java
@@ -38,14 +38,15 @@ import org.slf4j.LoggerFactory;
 @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class SQLDefaultConstraint implements org.apache.thrift.TBase<SQLDefaultConstraint, SQLDefaultConstraint._Fields>, java.io.Serializable, Cloneable, Comparable<SQLDefaultConstraint> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SQLDefaultConstraint");
 
-  private static final org.apache.thrift.protocol.TField TABLE_DB_FIELD_DESC = new org.apache.thrift.protocol.TField("table_db", org.apache.thrift.protocol.TType.STRING, (short)1);
-  private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("table_name", org.apache.thrift.protocol.TType.STRING, (short)2);
-  private static final org.apache.thrift.protocol.TField COLUMN_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("column_name", org.apache.thrift.protocol.TType.STRING, (short)3);
-  private static final org.apache.thrift.protocol.TField DEFAULT_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("default_value", org.apache.thrift.protocol.TType.STRING, (short)4);
-  private static final org.apache.thrift.protocol.TField DC_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dc_name", org.apache.thrift.protocol.TType.STRING, (short)5);
-  private static final org.apache.thrift.protocol.TField ENABLE_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("enable_cstr", org.apache.thrift.protocol.TType.BOOL, (short)6);
-  private static final org.apache.thrift.protocol.TField VALIDATE_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("validate_cstr", org.apache.thrift.protocol.TType.BOOL, (short)7);
-  private static final org.apache.thrift.protocol.TField RELY_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("rely_cstr", org.apache.thrift.protocol.TType.BOOL, (short)8);
+  private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField TABLE_DB_FIELD_DESC = new org.apache.thrift.protocol.TField("table_db", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("table_name", org.apache.thrift.protocol.TType.STRING, (short)3);
+  private static final org.apache.thrift.protocol.TField COLUMN_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("column_name", org.apache.thrift.protocol.TType.STRING, (short)4);
+  private static final org.apache.thrift.protocol.TField DEFAULT_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("default_value", org.apache.thrift.protocol.TType.STRING, (short)5);
+  private static final org.apache.thrift.protocol.TField DC_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dc_name", org.apache.thrift.protocol.TType.STRING, (short)6);
+  private static final org.apache.thrift.protocol.TField ENABLE_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("enable_cstr", org.apache.thrift.protocol.TType.BOOL, (short)7);
+  private static final org.apache.thrift.protocol.TField VALIDATE_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("validate_cstr", org.apache.thrift.protocol.TType.BOOL, (short)8);
+  private static final org.apache.thrift.protocol.TField RELY_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("rely_cstr", org.apache.thrift.protocol.TType.BOOL, (short)9);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -53,6 +54,7 @@ import org.slf4j.LoggerFactory;
     schemes.put(TupleScheme.class, new SQLDefaultConstraintTupleSchemeFactory());
   }
 
+  private String catName; // required
   private String table_db; // required
   private String table_name; // required
   private String column_name; // required
@@ -64,14 +66,15 @@ import org.slf4j.LoggerFactory;
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-    TABLE_DB((short)1, "table_db"),
-    TABLE_NAME((short)2, "table_name"),
-    COLUMN_NAME((short)3, "column_name"),
-    DEFAULT_VALUE((short)4, "default_value"),
-    DC_NAME((short)5, "dc_name"),
-    ENABLE_CSTR((short)6, "enable_cstr"),
-    VALIDATE_CSTR((short)7, "validate_cstr"),
-    RELY_CSTR((short)8, "rely_cstr");
+    CAT_NAME((short)1, "catName"),
+    TABLE_DB((short)2, "table_db"),
+    TABLE_NAME((short)3, "table_name"),
+    COLUMN_NAME((short)4, "column_name"),
+    DEFAULT_VALUE((short)5, "default_value"),
+    DC_NAME((short)6, "dc_name"),
+    ENABLE_CSTR((short)7, "enable_cstr"),
+    VALIDATE_CSTR((short)8, "validate_cstr"),
+    RELY_CSTR((short)9, "rely_cstr");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -86,21 +89,23 @@ import org.slf4j.LoggerFactory;
      */
     public static _Fields findByThriftId(int fieldId) {
       switch(fieldId) {
-        case 1: // TABLE_DB
+        case 1: // CAT_NAME
+          return CAT_NAME;
+        case 2: // TABLE_DB
           return TABLE_DB;
-        case 2: // TABLE_NAME
+        case 3: // TABLE_NAME
           return TABLE_NAME;
-        case 3: // COLUMN_NAME
+        case 4: // COLUMN_NAME
           return COLUMN_NAME;
-        case 4: // DEFAULT_VALUE
+        case 5: // DEFAULT_VALUE
           return DEFAULT_VALUE;
-        case 5: // DC_NAME
+        case 6: // DC_NAME
           return DC_NAME;
-        case 6: // ENABLE_CSTR
+        case 7: // ENABLE_CSTR
           return ENABLE_CSTR;
-        case 7: // VALIDATE_CSTR
+        case 8: // VALIDATE_CSTR
           return VALIDATE_CSTR;
-        case 8: // RELY_CSTR
+        case 9: // RELY_CSTR
           return RELY_CSTR;
         default:
           return null;
@@ -149,6 +154,8 @@ import org.slf4j.LoggerFactory;
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.TABLE_DB, new org.apache.thrift.meta_data.FieldMetaData("table_db", org.apache.thrift.TFieldRequirementType.DEFAULT, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("table_name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -173,6 +180,7 @@ import org.slf4j.LoggerFactory;
   }
 
   public SQLDefaultConstraint(
+    String catName,
     String table_db,
     String table_name,
     String column_name,
@@ -183,6 +191,7 @@ import org.slf4j.LoggerFactory;
     boolean rely_cstr)
   {
     this();
+    this.catName = catName;
     this.table_db = table_db;
     this.table_name = table_name;
     this.column_name = column_name;
@@ -201,6 +210,9 @@ import org.slf4j.LoggerFactory;
    */
   public SQLDefaultConstraint(SQLDefaultConstraint other) {
     __isset_bitfield = other.__isset_bitfield;
+    if (other.isSetCatName()) {
+      this.catName = other.catName;
+    }
     if (other.isSetTable_db()) {
       this.table_db = other.table_db;
     }
@@ -227,6 +239,7 @@ import org.slf4j.LoggerFactory;
 
   @Override
   public void clear() {
+    this.catName = null;
     this.table_db = null;
     this.table_name = null;
     this.column_name = null;
@@ -240,6 +253,29 @@ import org.slf4j.LoggerFactory;
     this.rely_cstr = false;
   }
 
+  public String getCatName() {
+    return this.catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
+  public void unsetCatName() {
+    this.catName = null;
+  }
+
+  /** Returns true if field catName is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatName() {
+    return this.catName != null;
+  }
+
+  public void setCatNameIsSet(boolean value) {
+    if (!value) {
+      this.catName = null;
+    }
+  }
+
   public String getTable_db() {
     return this.table_db;
   }
@@ -423,6 +459,14 @@ import org.slf4j.LoggerFactory;
 
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
+    case CAT_NAME:
+      if (value == null) {
+        unsetCatName();
+      } else {
+        setCatName((String)value);
+      }
+      break;
+
     case TABLE_DB:
       if (value == null) {
         unsetTable_db();
@@ -492,6 +536,9 @@ import org.slf4j.LoggerFactory;
 
   public Object getFieldValue(_Fields field) {
     switch (field) {
+    case CAT_NAME:
+      return getCatName();
+
     case TABLE_DB:
       return getTable_db();
 
@@ -527,6 +574,8 @@ import org.slf4j.LoggerFactory;
     }
 
     switch (field) {
+    case CAT_NAME:
+      return isSetCatName();
     case TABLE_DB:
       return isSetTable_db();
     case TABLE_NAME:
@@ -560,6 +609,15 @@ import org.slf4j.LoggerFactory;
     if (that == null)
       return false;
 
+    boolean this_present_catName = true && this.isSetCatName();
+    boolean that_present_catName = true && that.isSetCatName();
+    if (this_present_catName || that_present_catName) {
+      if (!(this_present_catName && that_present_catName))
+        return false;
+      if (!this.catName.equals(that.catName))
+        return false;
+    }
+
     boolean this_present_table_db = true && this.isSetTable_db();
     boolean that_present_table_db = true && that.isSetTable_db();
     if (this_present_table_db || that_present_table_db) {
@@ -639,6 +697,11 @@ import org.slf4j.LoggerFactory;
   public int hashCode() {
     List<Object> list = new ArrayList<Object>();
 
+    boolean present_catName = true && (isSetCatName());
+    list.add(present_catName);
+    if (present_catName)
+      list.add(catName);
+
     boolean present_table_db = true && (isSetTable_db());
     list.add(present_table_db);
     if (present_table_db)
@@ -690,6 +753,16 @@ import org.slf4j.LoggerFactory;
 
     int lastComparison = 0;
 
+    lastComparison = Boolean.valueOf(isSetCatName()).compareTo(other.isSetCatName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catName, other.catName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     lastComparison = Boolean.valueOf(isSetTable_db()).compareTo(other.isSetTable_db());
     if (lastComparison != 0) {
       return lastComparison;
@@ -790,6 +863,14 @@ import org.slf4j.LoggerFactory;
     StringBuilder sb = new StringBuilder("SQLDefaultConstraint(");
     boolean first = true;
 
+    sb.append("catName:");
+    if (this.catName == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.catName);
+    }
+    first = false;
+    if (!first) sb.append(", ");
     sb.append("table_db:");
     if (this.table_db == null) {
       sb.append("null");
@@ -886,7 +967,15 @@ import org.slf4j.LoggerFactory;
           break;
         }
         switch (schemeField.id) {
-          case 1: // TABLE_DB
+          case 1: // CAT_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.catName = iprot.readString();
+              struct.setCatNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // TABLE_DB
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.table_db = iprot.readString();
               struct.setTable_dbIsSet(true);
@@ -894,7 +983,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 2: // TABLE_NAME
+          case 3: // TABLE_NAME
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.table_name = iprot.readString();
               struct.setTable_nameIsSet(true);
@@ -902,7 +991,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 3: // COLUMN_NAME
+          case 4: // COLUMN_NAME
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.column_name = iprot.readString();
               struct.setColumn_nameIsSet(true);
@@ -910,7 +999,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 4: // DEFAULT_VALUE
+          case 5: // DEFAULT_VALUE
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.default_value = iprot.readString();
               struct.setDefault_valueIsSet(true);
@@ -918,7 +1007,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 5: // DC_NAME
+          case 6: // DC_NAME
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.dc_name = iprot.readString();
               struct.setDc_nameIsSet(true);
@@ -926,7 +1015,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 6: // ENABLE_CSTR
+          case 7: // ENABLE_CSTR
             if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
               struct.enable_cstr = iprot.readBool();
               struct.setEnable_cstrIsSet(true);
@@ -934,7 +1023,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 7: // VALIDATE_CSTR
+          case 8: // VALIDATE_CSTR
             if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
               struct.validate_cstr = iprot.readBool();
               struct.setValidate_cstrIsSet(true);
@@ -942,7 +1031,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 8: // RELY_CSTR
+          case 9: // RELY_CSTR
             if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
               struct.rely_cstr = iprot.readBool();
               struct.setRely_cstrIsSet(true);
@@ -963,6 +1052,11 @@ import org.slf4j.LoggerFactory;
       struct.validate();
 
       oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.catName != null) {
+        oprot.writeFieldBegin(CAT_NAME_FIELD_DESC);
+        oprot.writeString(struct.catName);
+        oprot.writeFieldEnd();
+      }
       if (struct.table_db != null) {
         oprot.writeFieldBegin(TABLE_DB_FIELD_DESC);
         oprot.writeString(struct.table_db);
@@ -1015,31 +1109,37 @@ import org.slf4j.LoggerFactory;
     public void write(org.apache.thrift.protocol.TProtocol prot, SQLDefaultConstraint struct) throws org.apache.thrift.TException {
       TTupleProtocol oprot = (TTupleProtocol) prot;
       BitSet optionals = new BitSet();
-      if (struct.isSetTable_db()) {
+      if (struct.isSetCatName()) {
         optionals.set(0);
       }
-      if (struct.isSetTable_name()) {
+      if (struct.isSetTable_db()) {
         optionals.set(1);
       }
-      if (struct.isSetColumn_name()) {
+      if (struct.isSetTable_name()) {
         optionals.set(2);
       }
-      if (struct.isSetDefault_value()) {
+      if (struct.isSetColumn_name()) {
         optionals.set(3);
       }
-      if (struct.isSetDc_name()) {
+      if (struct.isSetDefault_value()) {
         optionals.set(4);
       }
-      if (struct.isSetEnable_cstr()) {
+      if (struct.isSetDc_name()) {
         optionals.set(5);
       }
-      if (struct.isSetValidate_cstr()) {
+      if (struct.isSetEnable_cstr()) {
         optionals.set(6);
       }
-      if (struct.isSetRely_cstr()) {
+      if (struct.isSetValidate_cstr()) {
         optionals.set(7);
       }
-      oprot.writeBitSet(optionals, 8);
+      if (struct.isSetRely_cstr()) {
+        optionals.set(8);
+      }
+      oprot.writeBitSet(optionals, 9);
+      if (struct.isSetCatName()) {
+        oprot.writeString(struct.catName);
+      }
       if (struct.isSetTable_db()) {
         oprot.writeString(struct.table_db);
       }
@@ -1069,36 +1169,40 @@ import org.slf4j.LoggerFactory;
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, SQLDefaultConstraint struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
-      BitSet incoming = iprot.readBitSet(8);
+      BitSet incoming = iprot.readBitSet(9);
       if (incoming.get(0)) {
+        struct.catName = iprot.readString();
+        struct.setCatNameIsSet(true);
+      }
+      if (incoming.get(1)) {
         struct.table_db = iprot.readString();
         struct.setTable_dbIsSet(true);
       }
-      if (incoming.get(1)) {
+      if (incoming.get(2)) {
         struct.table_name = iprot.readString();
         struct.setTable_nameIsSet(true);
       }
-      if (incoming.get(2)) {
+      if (incoming.get(3)) {
         struct.column_name = iprot.readString();
         struct.setColumn_nameIsSet(true);
       }
-      if (incoming.get(3)) {
+      if (incoming.get(4)) {
         struct.default_value = iprot.readString();
         struct.setDefault_valueIsSet(true);
       }
-      if (incoming.get(4)) {
+      if (incoming.get(5)) {
         struct.dc_name = iprot.readString();
         struct.setDc_nameIsSet(true);
       }
-      if (incoming.get(5)) {
+      if (incoming.get(6)) {
         struct.enable_cstr = iprot.readBool();
         struct.setEnable_cstrIsSet(true);
       }
-      if (incoming.get(6)) {
+      if (incoming.get(7)) {
         struct.validate_cstr = iprot.readBool();
         struct.setValidate_cstrIsSet(true);
       }
-      if (incoming.get(7)) {
+      if (incoming.get(8)) {
         struct.rely_cstr = iprot.readBool();
         struct.setRely_cstrIsSet(true);
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLForeignKey.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLForeignKey.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLForeignKey.java
index 6cf6f31..a7cf241 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLForeignKey.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLForeignKey.java
@@ -52,6 +52,7 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField ENABLE_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("enable_cstr", org.apache.thrift.protocol.TType.BOOL, (short)12);
   private static final org.apache.thrift.protocol.TField VALIDATE_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("validate_cstr", org.apache.thrift.protocol.TType.BOOL, (short)13);
   private static final org.apache.thrift.protocol.TField RELY_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("rely_cstr", org.apache.thrift.protocol.TType.BOOL, (short)14);
+  private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)15);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -73,6 +74,7 @@ import org.slf4j.LoggerFactory;
   private boolean enable_cstr; // required
   private boolean validate_cstr; // required
   private boolean rely_cstr; // required
+  private String catName; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -89,7 +91,8 @@ import org.slf4j.LoggerFactory;
     PK_NAME((short)11, "pk_name"),
     ENABLE_CSTR((short)12, "enable_cstr"),
     VALIDATE_CSTR((short)13, "validate_cstr"),
-    RELY_CSTR((short)14, "rely_cstr");
+    RELY_CSTR((short)14, "rely_cstr"),
+    CAT_NAME((short)15, "catName");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -132,6 +135,8 @@ import org.slf4j.LoggerFactory;
           return VALIDATE_CSTR;
         case 14: // RELY_CSTR
           return RELY_CSTR;
+        case 15: // CAT_NAME
+          return CAT_NAME;
         default:
           return null;
       }
@@ -179,6 +184,7 @@ import org.slf4j.LoggerFactory;
   private static final int __VALIDATE_CSTR_ISSET_ID = 4;
   private static final int __RELY_CSTR_ISSET_ID = 5;
   private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.CAT_NAME};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -210,6 +216,8 @@ import org.slf4j.LoggerFactory;
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
     tmpMap.put(_Fields.RELY_CSTR, new org.apache.thrift.meta_data.FieldMetaData("rely_cstr", org.apache.thrift.TFieldRequirementType.DEFAULT, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SQLForeignKey.class, metaDataMap);
   }
@@ -291,6 +299,9 @@ import org.slf4j.LoggerFactory;
     this.enable_cstr = other.enable_cstr;
     this.validate_cstr = other.validate_cstr;
     this.rely_cstr = other.rely_cstr;
+    if (other.isSetCatName()) {
+      this.catName = other.catName;
+    }
   }
 
   public SQLForeignKey deepCopy() {
@@ -319,6 +330,7 @@ import org.slf4j.LoggerFactory;
     this.validate_cstr = false;
     setRely_cstrIsSet(false);
     this.rely_cstr = false;
+    this.catName = null;
   }
 
   public String getPktable_db() {
@@ -637,6 +649,29 @@ import org.slf4j.LoggerFactory;
     __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __RELY_CSTR_ISSET_ID, value);
   }
 
+  public String getCatName() {
+    return this.catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
+  public void unsetCatName() {
+    this.catName = null;
+  }
+
+  /** Returns true if field catName is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatName() {
+    return this.catName != null;
+  }
+
+  public void setCatNameIsSet(boolean value) {
+    if (!value) {
+      this.catName = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case PKTABLE_DB:
@@ -751,6 +786,14 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case CAT_NAME:
+      if (value == null) {
+        unsetCatName();
+      } else {
+        setCatName((String)value);
+      }
+      break;
+
     }
   }
 
@@ -798,6 +841,9 @@ import org.slf4j.LoggerFactory;
     case RELY_CSTR:
       return isRely_cstr();
 
+    case CAT_NAME:
+      return getCatName();
+
     }
     throw new IllegalStateException();
   }
@@ -837,6 +883,8 @@ import org.slf4j.LoggerFactory;
       return isSetValidate_cstr();
     case RELY_CSTR:
       return isSetRely_cstr();
+    case CAT_NAME:
+      return isSetCatName();
     }
     throw new IllegalStateException();
   }
@@ -980,6 +1028,15 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_catName = true && this.isSetCatName();
+    boolean that_present_catName = true && that.isSetCatName();
+    if (this_present_catName || that_present_catName) {
+      if (!(this_present_catName && that_present_catName))
+        return false;
+      if (!this.catName.equals(that.catName))
+        return false;
+    }
+
     return true;
   }
 
@@ -1057,6 +1114,11 @@ import org.slf4j.LoggerFactory;
     if (present_rely_cstr)
       list.add(rely_cstr);
 
+    boolean present_catName = true && (isSetCatName());
+    list.add(present_catName);
+    if (present_catName)
+      list.add(catName);
+
     return list.hashCode();
   }
 
@@ -1208,6 +1270,16 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetCatName()).compareTo(other.isSetCatName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catName, other.catName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -1315,6 +1387,16 @@ import org.slf4j.LoggerFactory;
     sb.append("rely_cstr:");
     sb.append(this.rely_cstr);
     first = false;
+    if (isSetCatName()) {
+      if (!first) sb.append(", ");
+      sb.append("catName:");
+      if (this.catName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.catName);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -1472,6 +1554,14 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 15: // CAT_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.catName = iprot.readString();
+              struct.setCatNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -1543,6 +1633,13 @@ import org.slf4j.LoggerFactory;
       oprot.writeFieldBegin(RELY_CSTR_FIELD_DESC);
       oprot.writeBool(struct.rely_cstr);
       oprot.writeFieldEnd();
+      if (struct.catName != null) {
+        if (struct.isSetCatName()) {
+          oprot.writeFieldBegin(CAT_NAME_FIELD_DESC);
+          oprot.writeString(struct.catName);
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -1603,7 +1700,10 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetRely_cstr()) {
         optionals.set(13);
       }
-      oprot.writeBitSet(optionals, 14);
+      if (struct.isSetCatName()) {
+        optionals.set(14);
+      }
+      oprot.writeBitSet(optionals, 15);
       if (struct.isSetPktable_db()) {
         oprot.writeString(struct.pktable_db);
       }
@@ -1646,12 +1746,15 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetRely_cstr()) {
         oprot.writeBool(struct.rely_cstr);
       }
+      if (struct.isSetCatName()) {
+        oprot.writeString(struct.catName);
+      }
     }
 
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, SQLForeignKey struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
-      BitSet incoming = iprot.readBitSet(14);
+      BitSet incoming = iprot.readBitSet(15);
       if (incoming.get(0)) {
         struct.pktable_db = iprot.readString();
         struct.setPktable_dbIsSet(true);
@@ -1708,6 +1811,10 @@ import org.slf4j.LoggerFactory;
         struct.rely_cstr = iprot.readBool();
         struct.setRely_cstrIsSet(true);
       }
+      if (incoming.get(14)) {
+        struct.catName = iprot.readString();
+        struct.setCatNameIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLNotNullConstraint.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLNotNullConstraint.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLNotNullConstraint.java
index cb0f295..97b9c1f 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLNotNullConstraint.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLNotNullConstraint.java
@@ -38,13 +38,14 @@ import org.slf4j.LoggerFactory;
 @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class SQLNotNullConstraint implements org.apache.thrift.TBase<SQLNotNullConstraint, SQLNotNullConstraint._Fields>, java.io.Serializable, Cloneable, Comparable<SQLNotNullConstraint> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SQLNotNullConstraint");
 
-  private static final org.apache.thrift.protocol.TField TABLE_DB_FIELD_DESC = new org.apache.thrift.protocol.TField("table_db", org.apache.thrift.protocol.TType.STRING, (short)1);
-  private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("table_name", org.apache.thrift.protocol.TType.STRING, (short)2);
-  private static final org.apache.thrift.protocol.TField COLUMN_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("column_name", org.apache.thrift.protocol.TType.STRING, (short)3);
-  private static final org.apache.thrift.protocol.TField NN_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("nn_name", org.apache.thrift.protocol.TType.STRING, (short)4);
-  private static final org.apache.thrift.protocol.TField ENABLE_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("enable_cstr", org.apache.thrift.protocol.TType.BOOL, (short)5);
-  private static final org.apache.thrift.protocol.TField VALIDATE_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("validate_cstr", org.apache.thrift.protocol.TType.BOOL, (short)6);
-  private static final org.apache.thrift.protocol.TField RELY_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("rely_cstr", org.apache.thrift.protocol.TType.BOOL, (short)7);
+  private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField TABLE_DB_FIELD_DESC = new org.apache.thrift.protocol.TField("table_db", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("table_name", org.apache.thrift.protocol.TType.STRING, (short)3);
+  private static final org.apache.thrift.protocol.TField COLUMN_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("column_name", org.apache.thrift.protocol.TType.STRING, (short)4);
+  private static final org.apache.thrift.protocol.TField NN_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("nn_name", org.apache.thrift.protocol.TType.STRING, (short)5);
+  private static final org.apache.thrift.protocol.TField ENABLE_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("enable_cstr", org.apache.thrift.protocol.TType.BOOL, (short)6);
+  private static final org.apache.thrift.protocol.TField VALIDATE_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("validate_cstr", org.apache.thrift.protocol.TType.BOOL, (short)7);
+  private static final org.apache.thrift.protocol.TField RELY_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("rely_cstr", org.apache.thrift.protocol.TType.BOOL, (short)8);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -52,6 +53,7 @@ import org.slf4j.LoggerFactory;
     schemes.put(TupleScheme.class, new SQLNotNullConstraintTupleSchemeFactory());
   }
 
+  private String catName; // required
   private String table_db; // required
   private String table_name; // required
   private String column_name; // required
@@ -62,13 +64,14 @@ import org.slf4j.LoggerFactory;
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-    TABLE_DB((short)1, "table_db"),
-    TABLE_NAME((short)2, "table_name"),
-    COLUMN_NAME((short)3, "column_name"),
-    NN_NAME((short)4, "nn_name"),
-    ENABLE_CSTR((short)5, "enable_cstr"),
-    VALIDATE_CSTR((short)6, "validate_cstr"),
-    RELY_CSTR((short)7, "rely_cstr");
+    CAT_NAME((short)1, "catName"),
+    TABLE_DB((short)2, "table_db"),
+    TABLE_NAME((short)3, "table_name"),
+    COLUMN_NAME((short)4, "column_name"),
+    NN_NAME((short)5, "nn_name"),
+    ENABLE_CSTR((short)6, "enable_cstr"),
+    VALIDATE_CSTR((short)7, "validate_cstr"),
+    RELY_CSTR((short)8, "rely_cstr");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -83,19 +86,21 @@ import org.slf4j.LoggerFactory;
      */
     public static _Fields findByThriftId(int fieldId) {
       switch(fieldId) {
-        case 1: // TABLE_DB
+        case 1: // CAT_NAME
+          return CAT_NAME;
+        case 2: // TABLE_DB
           return TABLE_DB;
-        case 2: // TABLE_NAME
+        case 3: // TABLE_NAME
           return TABLE_NAME;
-        case 3: // COLUMN_NAME
+        case 4: // COLUMN_NAME
           return COLUMN_NAME;
-        case 4: // NN_NAME
+        case 5: // NN_NAME
           return NN_NAME;
-        case 5: // ENABLE_CSTR
+        case 6: // ENABLE_CSTR
           return ENABLE_CSTR;
-        case 6: // VALIDATE_CSTR
+        case 7: // VALIDATE_CSTR
           return VALIDATE_CSTR;
-        case 7: // RELY_CSTR
+        case 8: // RELY_CSTR
           return RELY_CSTR;
         default:
           return null;
@@ -144,6 +149,8 @@ import org.slf4j.LoggerFactory;
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.TABLE_DB, new org.apache.thrift.meta_data.FieldMetaData("table_db", org.apache.thrift.TFieldRequirementType.DEFAULT, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("table_name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -166,6 +173,7 @@ import org.slf4j.LoggerFactory;
   }
 
   public SQLNotNullConstraint(
+    String catName,
     String table_db,
     String table_name,
     String column_name,
@@ -175,6 +183,7 @@ import org.slf4j.LoggerFactory;
     boolean rely_cstr)
   {
     this();
+    this.catName = catName;
     this.table_db = table_db;
     this.table_name = table_name;
     this.column_name = column_name;
@@ -192,6 +201,9 @@ import org.slf4j.LoggerFactory;
    */
   public SQLNotNullConstraint(SQLNotNullConstraint other) {
     __isset_bitfield = other.__isset_bitfield;
+    if (other.isSetCatName()) {
+      this.catName = other.catName;
+    }
     if (other.isSetTable_db()) {
       this.table_db = other.table_db;
     }
@@ -215,6 +227,7 @@ import org.slf4j.LoggerFactory;
 
   @Override
   public void clear() {
+    this.catName = null;
     this.table_db = null;
     this.table_name = null;
     this.column_name = null;
@@ -227,6 +240,29 @@ import org.slf4j.LoggerFactory;
     this.rely_cstr = false;
   }
 
+  public String getCatName() {
+    return this.catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
+  public void unsetCatName() {
+    this.catName = null;
+  }
+
+  /** Returns true if field catName is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatName() {
+    return this.catName != null;
+  }
+
+  public void setCatNameIsSet(boolean value) {
+    if (!value) {
+      this.catName = null;
+    }
+  }
+
   public String getTable_db() {
     return this.table_db;
   }
@@ -387,6 +423,14 @@ import org.slf4j.LoggerFactory;
 
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
+    case CAT_NAME:
+      if (value == null) {
+        unsetCatName();
+      } else {
+        setCatName((String)value);
+      }
+      break;
+
     case TABLE_DB:
       if (value == null) {
         unsetTable_db();
@@ -448,6 +492,9 @@ import org.slf4j.LoggerFactory;
 
   public Object getFieldValue(_Fields field) {
     switch (field) {
+    case CAT_NAME:
+      return getCatName();
+
     case TABLE_DB:
       return getTable_db();
 
@@ -480,6 +527,8 @@ import org.slf4j.LoggerFactory;
     }
 
     switch (field) {
+    case CAT_NAME:
+      return isSetCatName();
     case TABLE_DB:
       return isSetTable_db();
     case TABLE_NAME:
@@ -511,6 +560,15 @@ import org.slf4j.LoggerFactory;
     if (that == null)
       return false;
 
+    boolean this_present_catName = true && this.isSetCatName();
+    boolean that_present_catName = true && that.isSetCatName();
+    if (this_present_catName || that_present_catName) {
+      if (!(this_present_catName && that_present_catName))
+        return false;
+      if (!this.catName.equals(that.catName))
+        return false;
+    }
+
     boolean this_present_table_db = true && this.isSetTable_db();
     boolean that_present_table_db = true && that.isSetTable_db();
     if (this_present_table_db || that_present_table_db) {
@@ -581,6 +639,11 @@ import org.slf4j.LoggerFactory;
   public int hashCode() {
     List<Object> list = new ArrayList<Object>();
 
+    boolean present_catName = true && (isSetCatName());
+    list.add(present_catName);
+    if (present_catName)
+      list.add(catName);
+
     boolean present_table_db = true && (isSetTable_db());
     list.add(present_table_db);
     if (present_table_db)
@@ -627,6 +690,16 @@ import org.slf4j.LoggerFactory;
 
     int lastComparison = 0;
 
+    lastComparison = Boolean.valueOf(isSetCatName()).compareTo(other.isSetCatName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catName, other.catName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     lastComparison = Boolean.valueOf(isSetTable_db()).compareTo(other.isSetTable_db());
     if (lastComparison != 0) {
       return lastComparison;
@@ -717,6 +790,14 @@ import org.slf4j.LoggerFactory;
     StringBuilder sb = new StringBuilder("SQLNotNullConstraint(");
     boolean first = true;
 
+    sb.append("catName:");
+    if (this.catName == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.catName);
+    }
+    first = false;
+    if (!first) sb.append(", ");
     sb.append("table_db:");
     if (this.table_db == null) {
       sb.append("null");
@@ -805,7 +886,15 @@ import org.slf4j.LoggerFactory;
           break;
         }
         switch (schemeField.id) {
-          case 1: // TABLE_DB
+          case 1: // CAT_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.catName = iprot.readString();
+              struct.setCatNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // TABLE_DB
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.table_db = iprot.readString();
               struct.setTable_dbIsSet(true);
@@ -813,7 +902,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 2: // TABLE_NAME
+          case 3: // TABLE_NAME
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.table_name = iprot.readString();
               struct.setTable_nameIsSet(true);
@@ -821,7 +910,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 3: // COLUMN_NAME
+          case 4: // COLUMN_NAME
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.column_name = iprot.readString();
               struct.setColumn_nameIsSet(true);
@@ -829,7 +918,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 4: // NN_NAME
+          case 5: // NN_NAME
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.nn_name = iprot.readString();
               struct.setNn_nameIsSet(true);
@@ -837,7 +926,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 5: // ENABLE_CSTR
+          case 6: // ENABLE_CSTR
             if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
               struct.enable_cstr = iprot.readBool();
               struct.setEnable_cstrIsSet(true);
@@ -845,7 +934,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 6: // VALIDATE_CSTR
+          case 7: // VALIDATE_CSTR
             if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
               struct.validate_cstr = iprot.readBool();
               struct.setValidate_cstrIsSet(true);
@@ -853,7 +942,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 7: // RELY_CSTR
+          case 8: // RELY_CSTR
             if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
               struct.rely_cstr = iprot.readBool();
               struct.setRely_cstrIsSet(true);
@@ -874,6 +963,11 @@ import org.slf4j.LoggerFactory;
       struct.validate();
 
       oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.catName != null) {
+        oprot.writeFieldBegin(CAT_NAME_FIELD_DESC);
+        oprot.writeString(struct.catName);
+        oprot.writeFieldEnd();
+      }
       if (struct.table_db != null) {
         oprot.writeFieldBegin(TABLE_DB_FIELD_DESC);
         oprot.writeString(struct.table_db);
@@ -921,28 +1015,34 @@ import org.slf4j.LoggerFactory;
     public void write(org.apache.thrift.protocol.TProtocol prot, SQLNotNullConstraint struct) throws org.apache.thrift.TException {
       TTupleProtocol oprot = (TTupleProtocol) prot;
       BitSet optionals = new BitSet();
-      if (struct.isSetTable_db()) {
+      if (struct.isSetCatName()) {
         optionals.set(0);
       }
-      if (struct.isSetTable_name()) {
+      if (struct.isSetTable_db()) {
         optionals.set(1);
       }
-      if (struct.isSetColumn_name()) {
+      if (struct.isSetTable_name()) {
         optionals.set(2);
       }
-      if (struct.isSetNn_name()) {
+      if (struct.isSetColumn_name()) {
         optionals.set(3);
       }
-      if (struct.isSetEnable_cstr()) {
+      if (struct.isSetNn_name()) {
         optionals.set(4);
       }
-      if (struct.isSetValidate_cstr()) {
+      if (struct.isSetEnable_cstr()) {
         optionals.set(5);
       }
-      if (struct.isSetRely_cstr()) {
+      if (struct.isSetValidate_cstr()) {
         optionals.set(6);
       }
-      oprot.writeBitSet(optionals, 7);
+      if (struct.isSetRely_cstr()) {
+        optionals.set(7);
+      }
+      oprot.writeBitSet(optionals, 8);
+      if (struct.isSetCatName()) {
+        oprot.writeString(struct.catName);
+      }
       if (struct.isSetTable_db()) {
         oprot.writeString(struct.table_db);
       }
@@ -969,32 +1069,36 @@ import org.slf4j.LoggerFactory;
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, SQLNotNullConstraint struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
-      BitSet incoming = iprot.readBitSet(7);
+      BitSet incoming = iprot.readBitSet(8);
       if (incoming.get(0)) {
+        struct.catName = iprot.readString();
+        struct.setCatNameIsSet(true);
+      }
+      if (incoming.get(1)) {
         struct.table_db = iprot.readString();
         struct.setTable_dbIsSet(true);
       }
-      if (incoming.get(1)) {
+      if (incoming.get(2)) {
         struct.table_name = iprot.readString();
         struct.setTable_nameIsSet(true);
       }
-      if (incoming.get(2)) {
+      if (incoming.get(3)) {
         struct.column_name = iprot.readString();
         struct.setColumn_nameIsSet(true);
       }
-      if (incoming.get(3)) {
+      if (incoming.get(4)) {
         struct.nn_name = iprot.readString();
         struct.setNn_nameIsSet(true);
       }
-      if (incoming.get(4)) {
+      if (incoming.get(5)) {
         struct.enable_cstr = iprot.readBool();
         struct.setEnable_cstrIsSet(true);
       }
-      if (incoming.get(5)) {
+      if (incoming.get(6)) {
         struct.validate_cstr = iprot.readBool();
         struct.setValidate_cstrIsSet(true);
       }
-      if (incoming.get(6)) {
+      if (incoming.get(7)) {
         struct.rely_cstr = iprot.readBool();
         struct.setRely_cstrIsSet(true);
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLPrimaryKey.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLPrimaryKey.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLPrimaryKey.java
index 45484a2..b77316f 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLPrimaryKey.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLPrimaryKey.java
@@ -46,6 +46,7 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField ENABLE_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("enable_cstr", org.apache.thrift.protocol.TType.BOOL, (short)6);
   private static final org.apache.thrift.protocol.TField VALIDATE_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("validate_cstr", org.apache.thrift.protocol.TType.BOOL, (short)7);
   private static final org.apache.thrift.protocol.TField RELY_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("rely_cstr", org.apache.thrift.protocol.TType.BOOL, (short)8);
+  private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)9);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -61,6 +62,7 @@ import org.slf4j.LoggerFactory;
   private boolean enable_cstr; // required
   private boolean validate_cstr; // required
   private boolean rely_cstr; // required
+  private String catName; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -71,7 +73,8 @@ import org.slf4j.LoggerFactory;
     PK_NAME((short)5, "pk_name"),
     ENABLE_CSTR((short)6, "enable_cstr"),
     VALIDATE_CSTR((short)7, "validate_cstr"),
-    RELY_CSTR((short)8, "rely_cstr");
+    RELY_CSTR((short)8, "rely_cstr"),
+    CAT_NAME((short)9, "catName");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -102,6 +105,8 @@ import org.slf4j.LoggerFactory;
           return VALIDATE_CSTR;
         case 8: // RELY_CSTR
           return RELY_CSTR;
+        case 9: // CAT_NAME
+          return CAT_NAME;
         default:
           return null;
       }
@@ -147,6 +152,7 @@ import org.slf4j.LoggerFactory;
   private static final int __VALIDATE_CSTR_ISSET_ID = 2;
   private static final int __RELY_CSTR_ISSET_ID = 3;
   private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.CAT_NAME};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -166,6 +172,8 @@ import org.slf4j.LoggerFactory;
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
     tmpMap.put(_Fields.RELY_CSTR, new org.apache.thrift.meta_data.FieldMetaData("rely_cstr", org.apache.thrift.TFieldRequirementType.DEFAULT, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SQLPrimaryKey.class, metaDataMap);
   }
@@ -219,6 +227,9 @@ import org.slf4j.LoggerFactory;
     this.enable_cstr = other.enable_cstr;
     this.validate_cstr = other.validate_cstr;
     this.rely_cstr = other.rely_cstr;
+    if (other.isSetCatName()) {
+      this.catName = other.catName;
+    }
   }
 
   public SQLPrimaryKey deepCopy() {
@@ -239,6 +250,7 @@ import org.slf4j.LoggerFactory;
     this.validate_cstr = false;
     setRely_cstrIsSet(false);
     this.rely_cstr = false;
+    this.catName = null;
   }
 
   public String getTable_db() {
@@ -421,6 +433,29 @@ import org.slf4j.LoggerFactory;
     __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __RELY_CSTR_ISSET_ID, value);
   }
 
+  public String getCatName() {
+    return this.catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
+  public void unsetCatName() {
+    this.catName = null;
+  }
+
+  /** Returns true if field catName is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatName() {
+    return this.catName != null;
+  }
+
+  public void setCatNameIsSet(boolean value) {
+    if (!value) {
+      this.catName = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case TABLE_DB:
@@ -487,6 +522,14 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case CAT_NAME:
+      if (value == null) {
+        unsetCatName();
+      } else {
+        setCatName((String)value);
+      }
+      break;
+
     }
   }
 
@@ -516,6 +559,9 @@ import org.slf4j.LoggerFactory;
     case RELY_CSTR:
       return isRely_cstr();
 
+    case CAT_NAME:
+      return getCatName();
+
     }
     throw new IllegalStateException();
   }
@@ -543,6 +589,8 @@ import org.slf4j.LoggerFactory;
       return isSetValidate_cstr();
     case RELY_CSTR:
       return isSetRely_cstr();
+    case CAT_NAME:
+      return isSetCatName();
     }
     throw new IllegalStateException();
   }
@@ -632,6 +680,15 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_catName = true && this.isSetCatName();
+    boolean that_present_catName = true && that.isSetCatName();
+    if (this_present_catName || that_present_catName) {
+      if (!(this_present_catName && that_present_catName))
+        return false;
+      if (!this.catName.equals(that.catName))
+        return false;
+    }
+
     return true;
   }
 
@@ -679,6 +736,11 @@ import org.slf4j.LoggerFactory;
     if (present_rely_cstr)
       list.add(rely_cstr);
 
+    boolean present_catName = true && (isSetCatName());
+    list.add(present_catName);
+    if (present_catName)
+      list.add(catName);
+
     return list.hashCode();
   }
 
@@ -770,6 +832,16 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetCatName()).compareTo(other.isSetCatName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catName, other.catName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -837,6 +909,16 @@ import org.slf4j.LoggerFactory;
     sb.append("rely_cstr:");
     sb.append(this.rely_cstr);
     first = false;
+    if (isSetCatName()) {
+      if (!first) sb.append(", ");
+      sb.append("catName:");
+      if (this.catName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.catName);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -946,6 +1028,14 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 9: // CAT_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.catName = iprot.readString();
+              struct.setCatNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -991,6 +1081,13 @@ import org.slf4j.LoggerFactory;
       oprot.writeFieldBegin(RELY_CSTR_FIELD_DESC);
       oprot.writeBool(struct.rely_cstr);
       oprot.writeFieldEnd();
+      if (struct.catName != null) {
+        if (struct.isSetCatName()) {
+          oprot.writeFieldBegin(CAT_NAME_FIELD_DESC);
+          oprot.writeString(struct.catName);
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -1033,7 +1130,10 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetRely_cstr()) {
         optionals.set(7);
       }
-      oprot.writeBitSet(optionals, 8);
+      if (struct.isSetCatName()) {
+        optionals.set(8);
+      }
+      oprot.writeBitSet(optionals, 9);
       if (struct.isSetTable_db()) {
         oprot.writeString(struct.table_db);
       }
@@ -1058,12 +1158,15 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetRely_cstr()) {
         oprot.writeBool(struct.rely_cstr);
       }
+      if (struct.isSetCatName()) {
+        oprot.writeString(struct.catName);
+      }
     }
 
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, SQLPrimaryKey struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
-      BitSet incoming = iprot.readBitSet(8);
+      BitSet incoming = iprot.readBitSet(9);
       if (incoming.get(0)) {
         struct.table_db = iprot.readString();
         struct.setTable_dbIsSet(true);
@@ -1096,6 +1199,10 @@ import org.slf4j.LoggerFactory;
         struct.rely_cstr = iprot.readBool();
         struct.setRely_cstrIsSet(true);
       }
+      if (incoming.get(8)) {
+        struct.catName = iprot.readString();
+        struct.setCatNameIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLUniqueConstraint.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLUniqueConstraint.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLUniqueConstraint.java
index 493fded..a25a91c 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLUniqueConstraint.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SQLUniqueConstraint.java
@@ -38,14 +38,15 @@ import org.slf4j.LoggerFactory;
 @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class SQLUniqueConstraint implements org.apache.thrift.TBase<SQLUniqueConstraint, SQLUniqueConstraint._Fields>, java.io.Serializable, Cloneable, Comparable<SQLUniqueConstraint> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SQLUniqueConstraint");
 
-  private static final org.apache.thrift.protocol.TField TABLE_DB_FIELD_DESC = new org.apache.thrift.protocol.TField("table_db", org.apache.thrift.protocol.TType.STRING, (short)1);
-  private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("table_name", org.apache.thrift.protocol.TType.STRING, (short)2);
-  private static final org.apache.thrift.protocol.TField COLUMN_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("column_name", org.apache.thrift.protocol.TType.STRING, (short)3);
-  private static final org.apache.thrift.protocol.TField KEY_SEQ_FIELD_DESC = new org.apache.thrift.protocol.TField("key_seq", org.apache.thrift.protocol.TType.I32, (short)4);
-  private static final org.apache.thrift.protocol.TField UK_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("uk_name", org.apache.thrift.protocol.TType.STRING, (short)5);
-  private static final org.apache.thrift.protocol.TField ENABLE_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("enable_cstr", org.apache.thrift.protocol.TType.BOOL, (short)6);
-  private static final org.apache.thrift.protocol.TField VALIDATE_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("validate_cstr", org.apache.thrift.protocol.TType.BOOL, (short)7);
-  private static final org.apache.thrift.protocol.TField RELY_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("rely_cstr", org.apache.thrift.protocol.TType.BOOL, (short)8);
+  private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField TABLE_DB_FIELD_DESC = new org.apache.thrift.protocol.TField("table_db", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("table_name", org.apache.thrift.protocol.TType.STRING, (short)3);
+  private static final org.apache.thrift.protocol.TField COLUMN_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("column_name", org.apache.thrift.protocol.TType.STRING, (short)4);
+  private static final org.apache.thrift.protocol.TField KEY_SEQ_FIELD_DESC = new org.apache.thrift.protocol.TField("key_seq", org.apache.thrift.protocol.TType.I32, (short)5);
+  private static final org.apache.thrift.protocol.TField UK_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("uk_name", org.apache.thrift.protocol.TType.STRING, (short)6);
+  private static final org.apache.thrift.protocol.TField ENABLE_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("enable_cstr", org.apache.thrift.protocol.TType.BOOL, (short)7);
+  private static final org.apache.thrift.protocol.TField VALIDATE_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("validate_cstr", org.apache.thrift.protocol.TType.BOOL, (short)8);
+  private static final org.apache.thrift.protocol.TField RELY_CSTR_FIELD_DESC = new org.apache.thrift.protocol.TField("rely_cstr", org.apache.thrift.protocol.TType.BOOL, (short)9);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -53,6 +54,7 @@ import org.slf4j.LoggerFactory;
     schemes.put(TupleScheme.class, new SQLUniqueConstraintTupleSchemeFactory());
   }
 
+  private String catName; // required
   private String table_db; // required
   private String table_name; // required
   private String column_name; // required
@@ -64,14 +66,15 @@ import org.slf4j.LoggerFactory;
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-    TABLE_DB((short)1, "table_db"),
-    TABLE_NAME((short)2, "table_name"),
-    COLUMN_NAME((short)3, "column_name"),
-    KEY_SEQ((short)4, "key_seq"),
-    UK_NAME((short)5, "uk_name"),
-    ENABLE_CSTR((short)6, "enable_cstr"),
-    VALIDATE_CSTR((short)7, "validate_cstr"),
-    RELY_CSTR((short)8, "rely_cstr");
+    CAT_NAME((short)1, "catName"),
+    TABLE_DB((short)2, "table_db"),
+    TABLE_NAME((short)3, "table_name"),
+    COLUMN_NAME((short)4, "column_name"),
+    KEY_SEQ((short)5, "key_seq"),
+    UK_NAME((short)6, "uk_name"),
+    ENABLE_CSTR((short)7, "enable_cstr"),
+    VALIDATE_CSTR((short)8, "validate_cstr"),
+    RELY_CSTR((short)9, "rely_cstr");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -86,21 +89,23 @@ import org.slf4j.LoggerFactory;
      */
     public static _Fields findByThriftId(int fieldId) {
       switch(fieldId) {
-        case 1: // TABLE_DB
+        case 1: // CAT_NAME
+          return CAT_NAME;
+        case 2: // TABLE_DB
           return TABLE_DB;
-        case 2: // TABLE_NAME
+        case 3: // TABLE_NAME
           return TABLE_NAME;
-        case 3: // COLUMN_NAME
+        case 4: // COLUMN_NAME
           return COLUMN_NAME;
-        case 4: // KEY_SEQ
+        case 5: // KEY_SEQ
           return KEY_SEQ;
-        case 5: // UK_NAME
+        case 6: // UK_NAME
           return UK_NAME;
-        case 6: // ENABLE_CSTR
+        case 7: // ENABLE_CSTR
           return ENABLE_CSTR;
-        case 7: // VALIDATE_CSTR
+        case 8: // VALIDATE_CSTR
           return VALIDATE_CSTR;
-        case 8: // RELY_CSTR
+        case 9: // RELY_CSTR
           return RELY_CSTR;
         default:
           return null;
@@ -150,6 +155,8 @@ import org.slf4j.LoggerFactory;
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.TABLE_DB, new org.apache.thrift.meta_data.FieldMetaData("table_db", org.apache.thrift.TFieldRequirementType.DEFAULT, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("table_name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -174,6 +181,7 @@ import org.slf4j.LoggerFactory;
   }
 
   public SQLUniqueConstraint(
+    String catName,
     String table_db,
     String table_name,
     String column_name,
@@ -184,6 +192,7 @@ import org.slf4j.LoggerFactory;
     boolean rely_cstr)
   {
     this();
+    this.catName = catName;
     this.table_db = table_db;
     this.table_name = table_name;
     this.column_name = column_name;
@@ -203,6 +212,9 @@ import org.slf4j.LoggerFactory;
    */
   public SQLUniqueConstraint(SQLUniqueConstraint other) {
     __isset_bitfield = other.__isset_bitfield;
+    if (other.isSetCatName()) {
+      this.catName = other.catName;
+    }
     if (other.isSetTable_db()) {
       this.table_db = other.table_db;
     }
@@ -227,6 +239,7 @@ import org.slf4j.LoggerFactory;
 
   @Override
   public void clear() {
+    this.catName = null;
     this.table_db = null;
     this.table_name = null;
     this.column_name = null;
@@ -241,6 +254,29 @@ import org.slf4j.LoggerFactory;
     this.rely_cstr = false;
   }
 
+  public String getCatName() {
+    return this.catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
+  public void unsetCatName() {
+    this.catName = null;
+  }
+
+  /** Returns true if field catName is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatName() {
+    return this.catName != null;
+  }
+
+  public void setCatNameIsSet(boolean value) {
+    if (!value) {
+      this.catName = null;
+    }
+  }
+
   public String getTable_db() {
     return this.table_db;
   }
@@ -423,6 +459,14 @@ import org.slf4j.LoggerFactory;
 
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
+    case CAT_NAME:
+      if (value == null) {
+        unsetCatName();
+      } else {
+        setCatName((String)value);
+      }
+      break;
+
     case TABLE_DB:
       if (value == null) {
         unsetTable_db();
@@ -492,6 +536,9 @@ import org.slf4j.LoggerFactory;
 
   public Object getFieldValue(_Fields field) {
     switch (field) {
+    case CAT_NAME:
+      return getCatName();
+
     case TABLE_DB:
       return getTable_db();
 
@@ -527,6 +574,8 @@ import org.slf4j.LoggerFactory;
     }
 
     switch (field) {
+    case CAT_NAME:
+      return isSetCatName();
     case TABLE_DB:
       return isSetTable_db();
     case TABLE_NAME:
@@ -560,6 +609,15 @@ import org.slf4j.LoggerFactory;
     if (that == null)
       return false;
 
+    boolean this_present_catName = true && this.isSetCatName();
+    boolean that_present_catName = true && that.isSetCatName();
+    if (this_present_catName || that_present_catName) {
+      if (!(this_present_catName && that_present_catName))
+        return false;
+      if (!this.catName.equals(that.catName))
+        return false;
+    }
+
     boolean this_present_table_db = true && this.isSetTable_db();
     boolean that_present_table_db = true && that.isSetTable_db();
     if (this_present_table_db || that_present_table_db) {
@@ -639,6 +697,11 @@ import org.slf4j.LoggerFactory;
   public int hashCode() {
     List<Object> list = new ArrayList<Object>();
 
+    boolean present_catName = true && (isSetCatName());
+    list.add(present_catName);
+    if (present_catName)
+      list.add(catName);
+
     boolean present_table_db = true && (isSetTable_db());
     list.add(present_table_db);
     if (present_table_db)
@@ -690,6 +753,16 @@ import org.slf4j.LoggerFactory;
 
     int lastComparison = 0;
 
+    lastComparison = Boolean.valueOf(isSetCatName()).compareTo(other.isSetCatName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catName, other.catName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     lastComparison = Boolean.valueOf(isSetTable_db()).compareTo(other.isSetTable_db());
     if (lastComparison != 0) {
       return lastComparison;
@@ -790,6 +863,14 @@ import org.slf4j.LoggerFactory;
     StringBuilder sb = new StringBuilder("SQLUniqueConstraint(");
     boolean first = true;
 
+    sb.append("catName:");
+    if (this.catName == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.catName);
+    }
+    first = false;
+    if (!first) sb.append(", ");
     sb.append("table_db:");
     if (this.table_db == null) {
       sb.append("null");
@@ -882,7 +963,15 @@ import org.slf4j.LoggerFactory;
           break;
         }
         switch (schemeField.id) {
-          case 1: // TABLE_DB
+          case 1: // CAT_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.catName = iprot.readString();
+              struct.setCatNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // TABLE_DB
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.table_db = iprot.readString();
               struct.setTable_dbIsSet(true);
@@ -890,7 +979,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 2: // TABLE_NAME
+          case 3: // TABLE_NAME
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.table_name = iprot.readString();
               struct.setTable_nameIsSet(true);
@@ -898,7 +987,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 3: // COLUMN_NAME
+          case 4: // COLUMN_NAME
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.column_name = iprot.readString();
               struct.setColumn_nameIsSet(true);
@@ -906,7 +995,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 4: // KEY_SEQ
+          case 5: // KEY_SEQ
             if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
               struct.key_seq = iprot.readI32();
               struct.setKey_seqIsSet(true);
@@ -914,7 +1003,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 5: // UK_NAME
+          case 6: // UK_NAME
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.uk_name = iprot.readString();
               struct.setUk_nameIsSet(true);
@@ -922,7 +1011,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 6: // ENABLE_CSTR
+          case 7: // ENABLE_CSTR
             if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
               struct.enable_cstr = iprot.readBool();
               struct.setEnable_cstrIsSet(true);
@@ -930,7 +1019,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 7: // VALIDATE_CSTR
+          case 8: // VALIDATE_CSTR
             if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
               struct.validate_cstr = iprot.readBool();
               struct.setValidate_cstrIsSet(true);
@@ -938,7 +1027,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 8: // RELY_CSTR
+          case 9: // RELY_CSTR
             if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
               struct.rely_cstr = iprot.readBool();
               struct.setRely_cstrIsSet(true);
@@ -959,6 +1048,11 @@ import org.slf4j.LoggerFactory;
       struct.validate();
 
       oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.catName != null) {
+        oprot.writeFieldBegin(CAT_NAME_FIELD_DESC);
+        oprot.writeString(struct.catName);
+        oprot.writeFieldEnd();
+      }
       if (struct.table_db != null) {
         oprot.writeFieldBegin(TABLE_DB_FIELD_DESC);
         oprot.writeString(struct.table_db);
@@ -1009,31 +1103,37 @@ import org.slf4j.LoggerFactory;
     public void write(org.apache.thrift.protocol.TProtocol prot, SQLUniqueConstraint struct) throws org.apache.thrift.TException {
       TTupleProtocol oprot = (TTupleProtocol) prot;
       BitSet optionals = new BitSet();
-      if (struct.isSetTable_db()) {
+      if (struct.isSetCatName()) {
         optionals.set(0);
       }
-      if (struct.isSetTable_name()) {
+      if (struct.isSetTable_db()) {
         optionals.set(1);
       }
-      if (struct.isSetColumn_name()) {
+      if (struct.isSetTable_name()) {
         optionals.set(2);
       }
-      if (struct.isSetKey_seq()) {
+      if (struct.isSetColumn_name()) {
         optionals.set(3);
       }
-      if (struct.isSetUk_name()) {
+      if (struct.isSetKey_seq()) {
         optionals.set(4);
       }
-      if (struct.isSetEnable_cstr()) {
+      if (struct.isSetUk_name()) {
         optionals.set(5);
       }
-      if (struct.isSetValidate_cstr()) {
+      if (struct.isSetEnable_cstr()) {
         optionals.set(6);
       }
-      if (struct.isSetRely_cstr()) {
+      if (struct.isSetValidate_cstr()) {
         optionals.set(7);
       }
-      oprot.writeBitSet(optionals, 8);
+      if (struct.isSetRely_cstr()) {
+        optionals.set(8);
+      }
+      oprot.writeBitSet(optionals, 9);
+      if (struct.isSetCatName()) {
+        oprot.writeString(struct.catName);
+      }
       if (struct.isSetTable_db()) {
         oprot.writeString(struct.table_db);
       }
@@ -1063,36 +1163,40 @@ import org.slf4j.LoggerFactory;
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, SQLUniqueConstraint struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
-      BitSet incoming = iprot.readBitSet(8);
+      BitSet incoming = iprot.readBitSet(9);
       if (incoming.get(0)) {
+        struct.catName = iprot.readString();
+        struct.setCatNameIsSet(true);
+      }
+      if (incoming.get(1)) {
         struct.table_db = iprot.readString();
         struct.setTable_dbIsSet(true);
       }
-      if (incoming.get(1)) {
+      if (incoming.get(2)) {
         struct.table_name = iprot.readString();
         struct.setTable_nameIsSet(true);
       }
-      if (incoming.get(2)) {
+      if (incoming.get(3)) {
         struct.column_name = iprot.readString();
         struct.setColumn_nameIsSet(true);
       }
-      if (incoming.get(3)) {
+      if (incoming.get(4)) {
         struct.key_seq = iprot.readI32();
         struct.setKey_seqIsSet(true);
       }
-      if (incoming.get(4)) {
+      if (incoming.get(5)) {
         struct.uk_name = iprot.readString();
         struct.setUk_nameIsSet(true);
       }
-      if (incoming.get(5)) {
+      if (incoming.get(6)) {
         struct.enable_cstr = iprot.readBool();
         struct.setEnable_cstrIsSet(true);
       }
-      if (incoming.get(6)) {
+      if (incoming.get(7)) {
         struct.validate_cstr = iprot.readBool();
         struct.setValidate_cstrIsSet(true);
       }
-      if (incoming.get(7)) {
+      if (incoming.get(8)) {
         struct.rely_cstr = iprot.readBool();
         struct.setRely_cstrIsSet(true);
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Schema.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Schema.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Schema.java
index 3f39bd5..d49f1c4 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Schema.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Schema.java
@@ -445,14 +445,14 @@ import org.slf4j.LoggerFactory;
           case 1: // FIELD_SCHEMAS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list284 = iprot.readListBegin();
-                struct.fieldSchemas = new ArrayList<FieldSchema>(_list284.size);
-                FieldSchema _elem285;
-                for (int _i286 = 0; _i286 < _list284.size; ++_i286)
+                org.apache.thrift.protocol.TList _list292 = iprot.readListBegin();
+                struct.fieldSchemas = new ArrayList<FieldSchema>(_list292.size);
+                FieldSchema _elem293;
+                for (int _i294 = 0; _i294 < _list292.size; ++_i294)
                 {
-                  _elem285 = new FieldSchema();
-                  _elem285.read(iprot);
-                  struct.fieldSchemas.add(_elem285);
+                  _elem293 = new FieldSchema();
+                  _elem293.read(iprot);
+                  struct.fieldSchemas.add(_elem293);
                 }
                 iprot.readListEnd();
               }
@@ -464,15 +464,15 @@ import org.slf4j.LoggerFactory;
           case 2: // PROPERTIES
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map287 = iprot.readMapBegin();
-                struct.properties = new HashMap<String,String>(2*_map287.size);
-                String _key288;
-                String _val289;
-                for (int _i290 = 0; _i290 < _map287.size; ++_i290)
+                org.apache.thrift.protocol.TMap _map295 = iprot.readMapBegin();
+                struct.properties = new HashMap<String,String>(2*_map295.size);
+                String _key296;
+                String _val297;
+                for (int _i298 = 0; _i298 < _map295.size; ++_i298)
                 {
-                  _key288 = iprot.readString();
-                  _val289 = iprot.readString();
-                  struct.properties.put(_key288, _val289);
+                  _key296 = iprot.readString();
+                  _val297 = iprot.readString();
+                  struct.properties.put(_key296, _val297);
                 }
                 iprot.readMapEnd();
               }
@@ -498,9 +498,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FIELD_SCHEMAS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.fieldSchemas.size()));
-          for (FieldSchema _iter291 : struct.fieldSchemas)
+          for (FieldSchema _iter299 : struct.fieldSchemas)
           {
-            _iter291.write(oprot);
+            _iter299.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -510,10 +510,10 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PROPERTIES_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.properties.size()));
-          for (Map.Entry<String, String> _iter292 : struct.properties.entrySet())
+          for (Map.Entry<String, String> _iter300 : struct.properties.entrySet())
           {
-            oprot.writeString(_iter292.getKey());
-            oprot.writeString(_iter292.getValue());
+            oprot.writeString(_iter300.getKey());
+            oprot.writeString(_iter300.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -547,19 +547,19 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetFieldSchemas()) {
         {
           oprot.writeI32(struct.fieldSchemas.size());
-          for (FieldSchema _iter293 : struct.fieldSchemas)
+          for (FieldSchema _iter301 : struct.fieldSchemas)
           {
-            _iter293.write(oprot);
+            _iter301.write(oprot);
           }
         }
       }
       if (struct.isSetProperties()) {
         {
           oprot.writeI32(struct.properties.size());
-          for (Map.Entry<String, String> _iter294 : struct.properties.entrySet())
+          for (Map.Entry<String, String> _iter302 : struct.properties.entrySet())
           {
-            oprot.writeString(_iter294.getKey());
-            oprot.writeString(_iter294.getValue());
+            oprot.writeString(_iter302.getKey());
+            oprot.writeString(_iter302.getValue());
           }
         }
       }
@@ -571,29 +571,29 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(2);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list295 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.fieldSchemas = new ArrayList<FieldSchema>(_list295.size);
-          FieldSchema _elem296;
-          for (int _i297 = 0; _i297 < _list295.size; ++_i297)
+          org.apache.thrift.protocol.TList _list303 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.fieldSchemas = new ArrayList<FieldSchema>(_list303.size);
+          FieldSchema _elem304;
+          for (int _i305 = 0; _i305 < _list303.size; ++_i305)
           {
-            _elem296 = new FieldSchema();
-            _elem296.read(iprot);
-            struct.fieldSchemas.add(_elem296);
+            _elem304 = new FieldSchema();
+            _elem304.read(iprot);
+            struct.fieldSchemas.add(_elem304);
           }
         }
         struct.setFieldSchemasIsSet(true);
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TMap _map298 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.properties = new HashMap<String,String>(2*_map298.size);
-          String _key299;
-          String _val300;
-          for (int _i301 = 0; _i301 < _map298.size; ++_i301)
+          org.apache.thrift.protocol.TMap _map306 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.properties = new HashMap<String,String>(2*_map306.size);
+          String _key307;
+          String _val308;
+          for (int _i309 = 0; _i309 < _map306.size; ++_i309)
           {
-            _key299 = iprot.readString();
-            _val300 = iprot.readString();
-            struct.properties.put(_key299, _val300);
+            _key307 = iprot.readString();
+            _val308 = iprot.readString();
+            struct.properties.put(_key307, _val308);
           }
         }
         struct.setPropertiesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java
index 50efdbd..da919d7 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java
@@ -1119,14 +1119,14 @@ import org.slf4j.LoggerFactory;
           case 4: // COLS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list880 = iprot.readListBegin();
-                struct.cols = new ArrayList<FieldSchema>(_list880.size);
-                FieldSchema _elem881;
-                for (int _i882 = 0; _i882 < _list880.size; ++_i882)
+                org.apache.thrift.protocol.TList _list888 = iprot.readListBegin();
+                struct.cols = new ArrayList<FieldSchema>(_list888.size);
+                FieldSchema _elem889;
+                for (int _i890 = 0; _i890 < _list888.size; ++_i890)
                 {
-                  _elem881 = new FieldSchema();
-                  _elem881.read(iprot);
-                  struct.cols.add(_elem881);
+                  _elem889 = new FieldSchema();
+                  _elem889.read(iprot);
+                  struct.cols.add(_elem889);
                 }
                 iprot.readListEnd();
               }
@@ -1212,9 +1212,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(COLS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.cols.size()));
-          for (FieldSchema _iter883 : struct.cols)
+          for (FieldSchema _iter891 : struct.cols)
           {
-            _iter883.write(oprot);
+            _iter891.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -1323,9 +1323,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetCols()) {
         {
           oprot.writeI32(struct.cols.size());
-          for (FieldSchema _iter884 : struct.cols)
+          for (FieldSchema _iter892 : struct.cols)
           {
-            _iter884.write(oprot);
+            _iter892.write(oprot);
           }
         }
       }
@@ -1368,14 +1368,14 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(3)) {
         {
-          org.apache.thrift.protocol.TList _list885 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.cols = new ArrayList<FieldSchema>(_list885.size);
-          FieldSchema _elem886;
-          for (int _i887 = 0; _i887 < _list885.size; ++_i887)
+          org.apache.thrift.protocol.TList _list893 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.cols = new ArrayList<FieldSchema>(_list893.size);
+          FieldSchema _elem894;
+          for (int _i895 = 0; _i895 < _list893.size; ++_i895)
           {
-            _elem886 = new FieldSchema();
-            _elem886.read(iprot);
-            struct.cols.add(_elem886);
+            _elem894 = new FieldSchema();
+            _elem894.read(iprot);
+            struct.cols.add(_elem894);
           }
         }
         struct.setColsIsSet(true);


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb b/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
index aa93158..969f4ab 100644
--- a/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
+++ b/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
@@ -231,6 +231,7 @@ class SQLPrimaryKey
   ENABLE_CSTR = 6
   VALIDATE_CSTR = 7
   RELY_CSTR = 8
+  CATNAME = 9
 
   FIELDS = {
     TABLE_DB => {:type => ::Thrift::Types::STRING, :name => 'table_db'},
@@ -240,7 +241,8 @@ class SQLPrimaryKey
     PK_NAME => {:type => ::Thrift::Types::STRING, :name => 'pk_name'},
     ENABLE_CSTR => {:type => ::Thrift::Types::BOOL, :name => 'enable_cstr'},
     VALIDATE_CSTR => {:type => ::Thrift::Types::BOOL, :name => 'validate_cstr'},
-    RELY_CSTR => {:type => ::Thrift::Types::BOOL, :name => 'rely_cstr'}
+    RELY_CSTR => {:type => ::Thrift::Types::BOOL, :name => 'rely_cstr'},
+    CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName', :optional => true}
   }
 
   def struct_fields; FIELDS; end
@@ -267,6 +269,7 @@ class SQLForeignKey
   ENABLE_CSTR = 12
   VALIDATE_CSTR = 13
   RELY_CSTR = 14
+  CATNAME = 15
 
   FIELDS = {
     PKTABLE_DB => {:type => ::Thrift::Types::STRING, :name => 'pktable_db'},
@@ -282,7 +285,8 @@ class SQLForeignKey
     PK_NAME => {:type => ::Thrift::Types::STRING, :name => 'pk_name'},
     ENABLE_CSTR => {:type => ::Thrift::Types::BOOL, :name => 'enable_cstr'},
     VALIDATE_CSTR => {:type => ::Thrift::Types::BOOL, :name => 'validate_cstr'},
-    RELY_CSTR => {:type => ::Thrift::Types::BOOL, :name => 'rely_cstr'}
+    RELY_CSTR => {:type => ::Thrift::Types::BOOL, :name => 'rely_cstr'},
+    CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName', :optional => true}
   }
 
   def struct_fields; FIELDS; end
@@ -295,16 +299,18 @@ end
 
 class SQLUniqueConstraint
   include ::Thrift::Struct, ::Thrift::Struct_Union
-  TABLE_DB = 1
-  TABLE_NAME = 2
-  COLUMN_NAME = 3
-  KEY_SEQ = 4
-  UK_NAME = 5
-  ENABLE_CSTR = 6
-  VALIDATE_CSTR = 7
-  RELY_CSTR = 8
+  CATNAME = 1
+  TABLE_DB = 2
+  TABLE_NAME = 3
+  COLUMN_NAME = 4
+  KEY_SEQ = 5
+  UK_NAME = 6
+  ENABLE_CSTR = 7
+  VALIDATE_CSTR = 8
+  RELY_CSTR = 9
 
   FIELDS = {
+    CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName'},
     TABLE_DB => {:type => ::Thrift::Types::STRING, :name => 'table_db'},
     TABLE_NAME => {:type => ::Thrift::Types::STRING, :name => 'table_name'},
     COLUMN_NAME => {:type => ::Thrift::Types::STRING, :name => 'column_name'},
@@ -325,15 +331,17 @@ end
 
 class SQLNotNullConstraint
   include ::Thrift::Struct, ::Thrift::Struct_Union
-  TABLE_DB = 1
-  TABLE_NAME = 2
-  COLUMN_NAME = 3
-  NN_NAME = 4
-  ENABLE_CSTR = 5
-  VALIDATE_CSTR = 6
-  RELY_CSTR = 7
+  CATNAME = 1
+  TABLE_DB = 2
+  TABLE_NAME = 3
+  COLUMN_NAME = 4
+  NN_NAME = 5
+  ENABLE_CSTR = 6
+  VALIDATE_CSTR = 7
+  RELY_CSTR = 8
 
   FIELDS = {
+    CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName'},
     TABLE_DB => {:type => ::Thrift::Types::STRING, :name => 'table_db'},
     TABLE_NAME => {:type => ::Thrift::Types::STRING, :name => 'table_name'},
     COLUMN_NAME => {:type => ::Thrift::Types::STRING, :name => 'column_name'},
@@ -353,16 +361,18 @@ end
 
 class SQLDefaultConstraint
   include ::Thrift::Struct, ::Thrift::Struct_Union
-  TABLE_DB = 1
-  TABLE_NAME = 2
-  COLUMN_NAME = 3
-  DEFAULT_VALUE = 4
-  DC_NAME = 5
-  ENABLE_CSTR = 6
-  VALIDATE_CSTR = 7
-  RELY_CSTR = 8
+  CATNAME = 1
+  TABLE_DB = 2
+  TABLE_NAME = 3
+  COLUMN_NAME = 4
+  DEFAULT_VALUE = 5
+  DC_NAME = 6
+  ENABLE_CSTR = 7
+  VALIDATE_CSTR = 8
+  RELY_CSTR = 9
 
   FIELDS = {
+    CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName'},
     TABLE_DB => {:type => ::Thrift::Types::STRING, :name => 'table_db'},
     TABLE_NAME => {:type => ::Thrift::Types::STRING, :name => 'table_name'},
     COLUMN_NAME => {:type => ::Thrift::Types::STRING, :name => 'column_name'},
@@ -383,16 +393,18 @@ end
 
 class SQLCheckConstraint
   include ::Thrift::Struct, ::Thrift::Struct_Union
-  TABLE_DB = 1
-  TABLE_NAME = 2
-  COLUMN_NAME = 3
-  CHECK_EXPRESSION = 4
-  DC_NAME = 5
-  ENABLE_CSTR = 6
-  VALIDATE_CSTR = 7
-  RELY_CSTR = 8
+  CATNAME = 1
+  TABLE_DB = 2
+  TABLE_NAME = 3
+  COLUMN_NAME = 4
+  CHECK_EXPRESSION = 5
+  DC_NAME = 6
+  ENABLE_CSTR = 7
+  VALIDATE_CSTR = 8
+  RELY_CSTR = 9
 
   FIELDS = {
+    CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName'},
     TABLE_DB => {:type => ::Thrift::Types::STRING, :name => 'table_db'},
     TABLE_NAME => {:type => ::Thrift::Types::STRING, :name => 'table_name'},
     COLUMN_NAME => {:type => ::Thrift::Types::STRING, :name => 'column_name'},
@@ -440,13 +452,15 @@ class HiveObjectRef
   OBJECTNAME = 3
   PARTVALUES = 4
   COLUMNNAME = 5
+  CATNAME = 6
 
   FIELDS = {
     OBJECTTYPE => {:type => ::Thrift::Types::I32, :name => 'objectType', :enum_class => ::HiveObjectType},
     DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName'},
     OBJECTNAME => {:type => ::Thrift::Types::STRING, :name => 'objectName'},
     PARTVALUES => {:type => ::Thrift::Types::LIST, :name => 'partValues', :element => {:type => ::Thrift::Types::STRING}},
-    COLUMNNAME => {:type => ::Thrift::Types::STRING, :name => 'columnName'}
+    COLUMNNAME => {:type => ::Thrift::Types::STRING, :name => 'columnName'},
+    CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName', :optional => true}
   }
 
   def struct_fields; FIELDS; end
@@ -768,6 +782,106 @@ class GrantRevokeRoleResponse
   ::Thrift::Struct.generate_accessors self
 end
 
+class Catalog
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  NAME = 1
+  DESCRIPTION = 2
+  LOCATIONURI = 3
+
+  FIELDS = {
+    NAME => {:type => ::Thrift::Types::STRING, :name => 'name'},
+    DESCRIPTION => {:type => ::Thrift::Types::STRING, :name => 'description', :optional => true},
+    LOCATIONURI => {:type => ::Thrift::Types::STRING, :name => 'locationUri'}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class CreateCatalogRequest
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  CATALOG = 1
+
+  FIELDS = {
+    CATALOG => {:type => ::Thrift::Types::STRUCT, :name => 'catalog', :class => ::Catalog}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class GetCatalogRequest
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  NAME = 1
+
+  FIELDS = {
+    NAME => {:type => ::Thrift::Types::STRING, :name => 'name'}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class GetCatalogResponse
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  CATALOG = 1
+
+  FIELDS = {
+    CATALOG => {:type => ::Thrift::Types::STRUCT, :name => 'catalog', :class => ::Catalog}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class GetCatalogsResponse
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  NAMES = 1
+
+  FIELDS = {
+    NAMES => {:type => ::Thrift::Types::LIST, :name => 'names', :element => {:type => ::Thrift::Types::STRING}}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class DropCatalogRequest
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  NAME = 1
+
+  FIELDS = {
+    NAME => {:type => ::Thrift::Types::STRING, :name => 'name'}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
 class Database
   include ::Thrift::Struct, ::Thrift::Struct_Union
   NAME = 1
@@ -777,6 +891,7 @@ class Database
   PRIVILEGES = 5
   OWNERNAME = 6
   OWNERTYPE = 7
+  CATALOGNAME = 8
 
   FIELDS = {
     NAME => {:type => ::Thrift::Types::STRING, :name => 'name'},
@@ -785,7 +900,8 @@ class Database
     PARAMETERS => {:type => ::Thrift::Types::MAP, :name => 'parameters', :key => {:type => ::Thrift::Types::STRING}, :value => {:type => ::Thrift::Types::STRING}},
     PRIVILEGES => {:type => ::Thrift::Types::STRUCT, :name => 'privileges', :class => ::PrincipalPrivilegeSet, :optional => true},
     OWNERNAME => {:type => ::Thrift::Types::STRING, :name => 'ownerName', :optional => true},
-    OWNERTYPE => {:type => ::Thrift::Types::I32, :name => 'ownerType', :optional => true, :enum_class => ::PrincipalType}
+    OWNERTYPE => {:type => ::Thrift::Types::I32, :name => 'ownerType', :optional => true, :enum_class => ::PrincipalType},
+    CATALOGNAME => {:type => ::Thrift::Types::STRING, :name => 'catalogName', :optional => true}
   }
 
   def struct_fields; FIELDS; end
@@ -924,6 +1040,7 @@ class Table
   TEMPORARY = 14
   REWRITEENABLED = 15
   CREATIONMETADATA = 16
+  CATNAME = 17
 
   FIELDS = {
     TABLENAME => {:type => ::Thrift::Types::STRING, :name => 'tableName'},
@@ -941,7 +1058,8 @@ class Table
     PRIVILEGES => {:type => ::Thrift::Types::STRUCT, :name => 'privileges', :class => ::PrincipalPrivilegeSet, :optional => true},
     TEMPORARY => {:type => ::Thrift::Types::BOOL, :name => 'temporary', :default => false, :optional => true},
     REWRITEENABLED => {:type => ::Thrift::Types::BOOL, :name => 'rewriteEnabled', :optional => true},
-    CREATIONMETADATA => {:type => ::Thrift::Types::STRUCT, :name => 'creationMetadata', :class => ::CreationMetadata, :optional => true}
+    CREATIONMETADATA => {:type => ::Thrift::Types::STRUCT, :name => 'creationMetadata', :class => ::CreationMetadata, :optional => true},
+    CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName', :optional => true}
   }
 
   def struct_fields; FIELDS; end
@@ -962,6 +1080,7 @@ class Partition
   SD = 6
   PARAMETERS = 7
   PRIVILEGES = 8
+  CATNAME = 9
 
   FIELDS = {
     VALUES => {:type => ::Thrift::Types::LIST, :name => 'values', :element => {:type => ::Thrift::Types::STRING}},
@@ -971,7 +1090,8 @@ class Partition
     LASTACCESSTIME => {:type => ::Thrift::Types::I32, :name => 'lastAccessTime'},
     SD => {:type => ::Thrift::Types::STRUCT, :name => 'sd', :class => ::StorageDescriptor},
     PARAMETERS => {:type => ::Thrift::Types::MAP, :name => 'parameters', :key => {:type => ::Thrift::Types::STRING}, :value => {:type => ::Thrift::Types::STRING}},
-    PRIVILEGES => {:type => ::Thrift::Types::STRUCT, :name => 'privileges', :class => ::PrincipalPrivilegeSet, :optional => true}
+    PRIVILEGES => {:type => ::Thrift::Types::STRUCT, :name => 'privileges', :class => ::PrincipalPrivilegeSet, :optional => true},
+    CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName', :optional => true}
   }
 
   def struct_fields; FIELDS; end
@@ -1049,13 +1169,15 @@ class PartitionSpec
   ROOTPATH = 3
   SHAREDSDPARTITIONSPEC = 4
   PARTITIONLIST = 5
+  CATNAME = 6
 
   FIELDS = {
     DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName'},
     TABLENAME => {:type => ::Thrift::Types::STRING, :name => 'tableName'},
     ROOTPATH => {:type => ::Thrift::Types::STRING, :name => 'rootPath'},
     SHAREDSDPARTITIONSPEC => {:type => ::Thrift::Types::STRUCT, :name => 'sharedSDPartitionSpec', :class => ::PartitionSpecWithSharedSD, :optional => true},
-    PARTITIONLIST => {:type => ::Thrift::Types::STRUCT, :name => 'partitionList', :class => ::PartitionListComposingSpec, :optional => true}
+    PARTITIONLIST => {:type => ::Thrift::Types::STRUCT, :name => 'partitionList', :class => ::PartitionListComposingSpec, :optional => true},
+    CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName', :optional => true}
   }
 
   def struct_fields; FIELDS; end
@@ -1374,13 +1496,15 @@ class ColumnStatisticsDesc
   TABLENAME = 3
   PARTNAME = 4
   LASTANALYZED = 5
+  CATNAME = 6
 
   FIELDS = {
     ISTBLLEVEL => {:type => ::Thrift::Types::BOOL, :name => 'isTblLevel'},
     DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName'},
     TABLENAME => {:type => ::Thrift::Types::STRING, :name => 'tableName'},
     PARTNAME => {:type => ::Thrift::Types::STRING, :name => 'partName', :optional => true},
-    LASTANALYZED => {:type => ::Thrift::Types::I64, :name => 'lastAnalyzed', :optional => true}
+    LASTANALYZED => {:type => ::Thrift::Types::I64, :name => 'lastAnalyzed', :optional => true},
+    CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName', :optional => true}
   }
 
   def struct_fields; FIELDS; end
@@ -1491,10 +1615,12 @@ class PrimaryKeysRequest
   include ::Thrift::Struct, ::Thrift::Struct_Union
   DB_NAME = 1
   TBL_NAME = 2
+  CATNAME = 3
 
   FIELDS = {
     DB_NAME => {:type => ::Thrift::Types::STRING, :name => 'db_name'},
-    TBL_NAME => {:type => ::Thrift::Types::STRING, :name => 'tbl_name'}
+    TBL_NAME => {:type => ::Thrift::Types::STRING, :name => 'tbl_name'},
+    CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName', :optional => true}
   }
 
   def struct_fields; FIELDS; end
@@ -1530,12 +1656,14 @@ class ForeignKeysRequest
   PARENT_TBL_NAME = 2
   FOREIGN_DB_NAME = 3
   FOREIGN_TBL_NAME = 4
+  CATNAME = 5
 
   FIELDS = {
     PARENT_DB_NAME => {:type => ::Thrift::Types::STRING, :name => 'parent_db_name'},
     PARENT_TBL_NAME => {:type => ::Thrift::Types::STRING, :name => 'parent_tbl_name'},
     FOREIGN_DB_NAME => {:type => ::Thrift::Types::STRING, :name => 'foreign_db_name'},
-    FOREIGN_TBL_NAME => {:type => ::Thrift::Types::STRING, :name => 'foreign_tbl_name'}
+    FOREIGN_TBL_NAME => {:type => ::Thrift::Types::STRING, :name => 'foreign_tbl_name'},
+    CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName', :optional => true}
   }
 
   def struct_fields; FIELDS; end
@@ -1565,10 +1693,12 @@ end
 
 class UniqueConstraintsRequest
   include ::Thrift::Struct, ::Thrift::Struct_Union
-  DB_NAME = 1
-  TBL_NAME = 2
+  CATNAME = 1
+  DB_NAME = 2
+  TBL_NAME = 3
 
   FIELDS = {
+    CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName'},
     DB_NAME => {:type => ::Thrift::Types::STRING, :name => 'db_name'},
     TBL_NAME => {:type => ::Thrift::Types::STRING, :name => 'tbl_name'}
   }
@@ -1576,6 +1706,7 @@ class UniqueConstraintsRequest
   def struct_fields; FIELDS; end
 
   def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field catName is unset!') unless @catName
     raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field db_name is unset!') unless @db_name
     raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field tbl_name is unset!') unless @tbl_name
   end
@@ -1602,10 +1733,12 @@ end
 
 class NotNullConstraintsRequest
   include ::Thrift::Struct, ::Thrift::Struct_Union
-  DB_NAME = 1
-  TBL_NAME = 2
+  CATNAME = 1
+  DB_NAME = 2
+  TBL_NAME = 3
 
   FIELDS = {
+    CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName'},
     DB_NAME => {:type => ::Thrift::Types::STRING, :name => 'db_name'},
     TBL_NAME => {:type => ::Thrift::Types::STRING, :name => 'tbl_name'}
   }
@@ -1613,6 +1746,7 @@ class NotNullConstraintsRequest
   def struct_fields; FIELDS; end
 
   def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field catName is unset!') unless @catName
     raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field db_name is unset!') unless @db_name
     raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field tbl_name is unset!') unless @tbl_name
   end
@@ -1639,10 +1773,12 @@ end
 
 class DefaultConstraintsRequest
   include ::Thrift::Struct, ::Thrift::Struct_Union
-  DB_NAME = 1
-  TBL_NAME = 2
+  CATNAME = 1
+  DB_NAME = 2
+  TBL_NAME = 3
 
   FIELDS = {
+    CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName'},
     DB_NAME => {:type => ::Thrift::Types::STRING, :name => 'db_name'},
     TBL_NAME => {:type => ::Thrift::Types::STRING, :name => 'tbl_name'}
   }
@@ -1650,6 +1786,7 @@ class DefaultConstraintsRequest
   def struct_fields; FIELDS; end
 
   def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field catName is unset!') unless @catName
     raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field db_name is unset!') unless @db_name
     raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field tbl_name is unset!') unless @tbl_name
   end
@@ -1676,10 +1813,12 @@ end
 
 class CheckConstraintsRequest
   include ::Thrift::Struct, ::Thrift::Struct_Union
-  DB_NAME = 1
-  TBL_NAME = 2
+  CATNAME = 1
+  DB_NAME = 2
+  TBL_NAME = 3
 
   FIELDS = {
+    CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName'},
     DB_NAME => {:type => ::Thrift::Types::STRING, :name => 'db_name'},
     TBL_NAME => {:type => ::Thrift::Types::STRING, :name => 'tbl_name'}
   }
@@ -1687,6 +1826,7 @@ class CheckConstraintsRequest
   def struct_fields; FIELDS; end
 
   def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field catName is unset!') unless @catName
     raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field db_name is unset!') unless @db_name
     raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field tbl_name is unset!') unless @tbl_name
   end
@@ -1716,11 +1856,13 @@ class DropConstraintRequest
   DBNAME = 1
   TABLENAME = 2
   CONSTRAINTNAME = 3
+  CATNAME = 4
 
   FIELDS = {
     DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbname'},
     TABLENAME => {:type => ::Thrift::Types::STRING, :name => 'tablename'},
-    CONSTRAINTNAME => {:type => ::Thrift::Types::STRING, :name => 'constraintname'}
+    CONSTRAINTNAME => {:type => ::Thrift::Types::STRING, :name => 'constraintname'},
+    CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName', :optional => true}
   }
 
   def struct_fields; FIELDS; end
@@ -1863,13 +2005,15 @@ class PartitionsByExprRequest
   EXPR = 3
   DEFAULTPARTITIONNAME = 4
   MAXPARTS = 5
+  CATNAME = 6
 
   FIELDS = {
     DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName'},
     TBLNAME => {:type => ::Thrift::Types::STRING, :name => 'tblName'},
     EXPR => {:type => ::Thrift::Types::STRING, :name => 'expr', :binary => true},
     DEFAULTPARTITIONNAME => {:type => ::Thrift::Types::STRING, :name => 'defaultPartitionName', :optional => true},
-    MAXPARTS => {:type => ::Thrift::Types::I16, :name => 'maxParts', :default => -1, :optional => true}
+    MAXPARTS => {:type => ::Thrift::Types::I16, :name => 'maxParts', :default => -1, :optional => true},
+    CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName', :optional => true}
   }
 
   def struct_fields; FIELDS; end
@@ -1922,11 +2066,13 @@ class TableStatsRequest
   DBNAME = 1
   TBLNAME = 2
   COLNAMES = 3
+  CATNAME = 4
 
   FIELDS = {
     DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName'},
     TBLNAME => {:type => ::Thrift::Types::STRING, :name => 'tblName'},
-    COLNAMES => {:type => ::Thrift::Types::LIST, :name => 'colNames', :element => {:type => ::Thrift::Types::STRING}}
+    COLNAMES => {:type => ::Thrift::Types::LIST, :name => 'colNames', :element => {:type => ::Thrift::Types::STRING}},
+    CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName', :optional => true}
   }
 
   def struct_fields; FIELDS; end
@@ -1946,12 +2092,14 @@ class PartitionsStatsRequest
   TBLNAME = 2
   COLNAMES = 3
   PARTNAMES = 4
+  CATNAME = 5
 
   FIELDS = {
     DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName'},
     TBLNAME => {:type => ::Thrift::Types::STRING, :name => 'tblName'},
     COLNAMES => {:type => ::Thrift::Types::LIST, :name => 'colNames', :element => {:type => ::Thrift::Types::STRING}},
-    PARTNAMES => {:type => ::Thrift::Types::LIST, :name => 'partNames', :element => {:type => ::Thrift::Types::STRING}}
+    PARTNAMES => {:type => ::Thrift::Types::LIST, :name => 'partNames', :element => {:type => ::Thrift::Types::STRING}},
+    CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName', :optional => true}
   }
 
   def struct_fields; FIELDS; end
@@ -1989,13 +2137,15 @@ class AddPartitionsRequest
   PARTS = 3
   IFNOTEXISTS = 4
   NEEDRESULT = 5
+  CATNAME = 6
 
   FIELDS = {
     DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName'},
     TBLNAME => {:type => ::Thrift::Types::STRING, :name => 'tblName'},
     PARTS => {:type => ::Thrift::Types::LIST, :name => 'parts', :element => {:type => ::Thrift::Types::STRUCT, :class => ::Partition}},
     IFNOTEXISTS => {:type => ::Thrift::Types::BOOL, :name => 'ifNotExists'},
-    NEEDRESULT => {:type => ::Thrift::Types::BOOL, :name => 'needResult', :default => true, :optional => true}
+    NEEDRESULT => {:type => ::Thrift::Types::BOOL, :name => 'needResult', :default => true, :optional => true},
+    CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName', :optional => true}
   }
 
   def struct_fields; FIELDS; end
@@ -2084,6 +2234,7 @@ class DropPartitionsRequest
   IGNOREPROTECTION = 6
   ENVIRONMENTCONTEXT = 7
   NEEDRESULT = 8
+  CATNAME = 9
 
   FIELDS = {
     DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName'},
@@ -2093,7 +2244,8 @@ class DropPartitionsRequest
     IFEXISTS => {:type => ::Thrift::Types::BOOL, :name => 'ifExists', :default => true, :optional => true},
     IGNOREPROTECTION => {:type => ::Thrift::Types::BOOL, :name => 'ignoreProtection', :optional => true},
     ENVIRONMENTCONTEXT => {:type => ::Thrift::Types::STRUCT, :name => 'environmentContext', :class => ::EnvironmentContext, :optional => true},
-    NEEDRESULT => {:type => ::Thrift::Types::BOOL, :name => 'needResult', :default => true, :optional => true}
+    NEEDRESULT => {:type => ::Thrift::Types::BOOL, :name => 'needResult', :default => true, :optional => true},
+    CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName', :optional => true}
   }
 
   def struct_fields; FIELDS; end
@@ -2117,6 +2269,7 @@ class PartitionValuesRequest
   PARTITIONORDER = 6
   ASCENDING = 7
   MAXPARTS = 8
+  CATNAME = 9
 
   FIELDS = {
     DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName'},
@@ -2126,7 +2279,8 @@ class PartitionValuesRequest
     FILTER => {:type => ::Thrift::Types::STRING, :name => 'filter', :optional => true},
     PARTITIONORDER => {:type => ::Thrift::Types::LIST, :name => 'partitionOrder', :element => {:type => ::Thrift::Types::STRUCT, :class => ::FieldSchema}, :optional => true},
     ASCENDING => {:type => ::Thrift::Types::BOOL, :name => 'ascending', :default => true, :optional => true},
-    MAXPARTS => {:type => ::Thrift::Types::I64, :name => 'maxParts', :default => -1, :optional => true}
+    MAXPARTS => {:type => ::Thrift::Types::I64, :name => 'maxParts', :default => -1, :optional => true},
+    CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName', :optional => true}
   }
 
   def struct_fields; FIELDS; end
@@ -2205,6 +2359,7 @@ class Function
   CREATETIME = 6
   FUNCTIONTYPE = 7
   RESOURCEURIS = 8
+  CATNAME = 9
 
   FIELDS = {
     FUNCTIONNAME => {:type => ::Thrift::Types::STRING, :name => 'functionName'},
@@ -2214,7 +2369,8 @@ class Function
     OWNERTYPE => {:type => ::Thrift::Types::I32, :name => 'ownerType', :enum_class => ::PrincipalType},
     CREATETIME => {:type => ::Thrift::Types::I32, :name => 'createTime'},
     FUNCTIONTYPE => {:type => ::Thrift::Types::I32, :name => 'functionType', :enum_class => ::FunctionType},
-    RESOURCEURIS => {:type => ::Thrift::Types::LIST, :name => 'resourceUris', :element => {:type => ::Thrift::Types::STRUCT, :class => ::ResourceUri}}
+    RESOURCEURIS => {:type => ::Thrift::Types::LIST, :name => 'resourceUris', :element => {:type => ::Thrift::Types::STRUCT, :class => ::ResourceUri}},
+    CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName', :optional => true}
   }
 
   def struct_fields; FIELDS; end
@@ -3015,12 +3171,14 @@ end
 
 class CreationMetadata
   include ::Thrift::Struct, ::Thrift::Struct_Union
-  DBNAME = 1
-  TBLNAME = 2
-  TABLESUSED = 3
-  VALIDTXNLIST = 4
+  CATNAME = 1
+  DBNAME = 2
+  TBLNAME = 3
+  TABLESUSED = 4
+  VALIDTXNLIST = 5
 
   FIELDS = {
+    CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName'},
     DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName'},
     TBLNAME => {:type => ::Thrift::Types::STRING, :name => 'tblName'},
     TABLESUSED => {:type => ::Thrift::Types::SET, :name => 'tablesUsed', :element => {:type => ::Thrift::Types::STRING}},
@@ -3030,6 +3188,7 @@ class CreationMetadata
   def struct_fields; FIELDS; end
 
   def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field catName is unset!') unless @catName
     raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field dbName is unset!') unless @dbName
     raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field tblName is unset!') unless @tblName
     raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field tablesUsed is unset!') unless @tablesUsed
@@ -3066,6 +3225,7 @@ class NotificationEvent
   TABLENAME = 5
   MESSAGE = 6
   MESSAGEFORMAT = 7
+  CATNAME = 8
 
   FIELDS = {
     EVENTID => {:type => ::Thrift::Types::I64, :name => 'eventId'},
@@ -3074,7 +3234,8 @@ class NotificationEvent
     DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName', :optional => true},
     TABLENAME => {:type => ::Thrift::Types::STRING, :name => 'tableName', :optional => true},
     MESSAGE => {:type => ::Thrift::Types::STRING, :name => 'message'},
-    MESSAGEFORMAT => {:type => ::Thrift::Types::STRING, :name => 'messageFormat', :optional => true}
+    MESSAGEFORMAT => {:type => ::Thrift::Types::STRING, :name => 'messageFormat', :optional => true},
+    CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName', :optional => true}
   }
 
   def struct_fields; FIELDS; end
@@ -3127,10 +3288,12 @@ class NotificationEventsCountRequest
   include ::Thrift::Struct, ::Thrift::Struct_Union
   FROMEVENTID = 1
   DBNAME = 2
+  CATNAME = 3
 
   FIELDS = {
     FROMEVENTID => {:type => ::Thrift::Types::I64, :name => 'fromEventId'},
-    DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName'}
+    DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName'},
+    CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName', :optional => true}
   }
 
   def struct_fields; FIELDS; end
@@ -3211,13 +3374,15 @@ class FireEventRequest
   DBNAME = 3
   TABLENAME = 4
   PARTITIONVALS = 5
+  CATNAME = 6
 
   FIELDS = {
     SUCCESSFUL => {:type => ::Thrift::Types::BOOL, :name => 'successful'},
     DATA => {:type => ::Thrift::Types::STRUCT, :name => 'data', :class => ::FireEventRequestData},
     DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName', :optional => true},
     TABLENAME => {:type => ::Thrift::Types::STRING, :name => 'tableName', :optional => true},
-    PARTITIONVALS => {:type => ::Thrift::Types::LIST, :name => 'partitionVals', :element => {:type => ::Thrift::Types::STRING}, :optional => true}
+    PARTITIONVALS => {:type => ::Thrift::Types::LIST, :name => 'partitionVals', :element => {:type => ::Thrift::Types::STRING}, :optional => true},
+    CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName', :optional => true}
   }
 
   def struct_fields; FIELDS; end
@@ -3498,11 +3663,13 @@ class GetTableRequest
   DBNAME = 1
   TBLNAME = 2
   CAPABILITIES = 3
+  CATNAME = 4
 
   FIELDS = {
     DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName'},
     TBLNAME => {:type => ::Thrift::Types::STRING, :name => 'tblName'},
-    CAPABILITIES => {:type => ::Thrift::Types::STRUCT, :name => 'capabilities', :class => ::ClientCapabilities, :optional => true}
+    CAPABILITIES => {:type => ::Thrift::Types::STRUCT, :name => 'capabilities', :class => ::ClientCapabilities, :optional => true},
+    CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName', :optional => true}
   }
 
   def struct_fields; FIELDS; end
@@ -3537,11 +3704,13 @@ class GetTablesRequest
   DBNAME = 1
   TBLNAMES = 2
   CAPABILITIES = 3
+  CATNAME = 4
 
   FIELDS = {
     DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName'},
     TBLNAMES => {:type => ::Thrift::Types::LIST, :name => 'tblNames', :element => {:type => ::Thrift::Types::STRING}, :optional => true},
-    CAPABILITIES => {:type => ::Thrift::Types::STRUCT, :name => 'capabilities', :class => ::ClientCapabilities, :optional => true}
+    CAPABILITIES => {:type => ::Thrift::Types::STRUCT, :name => 'capabilities', :class => ::ClientCapabilities, :optional => true},
+    CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName', :optional => true}
   }
 
   def struct_fields; FIELDS; end
@@ -3611,12 +3780,14 @@ class TableMeta
   TABLENAME = 2
   TABLETYPE = 3
   COMMENTS = 4
+  CATNAME = 5
 
   FIELDS = {
     DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName'},
     TABLENAME => {:type => ::Thrift::Types::STRING, :name => 'tableName'},
     TABLETYPE => {:type => ::Thrift::Types::STRING, :name => 'tableType'},
-    COMMENTS => {:type => ::Thrift::Types::STRING, :name => 'comments', :optional => true}
+    COMMENTS => {:type => ::Thrift::Types::STRING, :name => 'comments', :optional => true},
+    CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName', :optional => true}
   }
 
   def struct_fields; FIELDS; end
@@ -4421,16 +4592,18 @@ class ISchema
   include ::Thrift::Struct, ::Thrift::Struct_Union
   SCHEMATYPE = 1
   NAME = 2
-  DBNAME = 3
-  COMPATIBILITY = 4
-  VALIDATIONLEVEL = 5
-  CANEVOLVE = 6
-  SCHEMAGROUP = 7
-  DESCRIPTION = 8
+  CATNAME = 3
+  DBNAME = 4
+  COMPATIBILITY = 5
+  VALIDATIONLEVEL = 6
+  CANEVOLVE = 7
+  SCHEMAGROUP = 8
+  DESCRIPTION = 9
 
   FIELDS = {
     SCHEMATYPE => {:type => ::Thrift::Types::I32, :name => 'schemaType', :enum_class => ::SchemaType},
     NAME => {:type => ::Thrift::Types::STRING, :name => 'name'},
+    CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName'},
     DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName'},
     COMPATIBILITY => {:type => ::Thrift::Types::I32, :name => 'compatibility', :enum_class => ::SchemaCompatibility},
     VALIDATIONLEVEL => {:type => ::Thrift::Types::I32, :name => 'validationLevel', :enum_class => ::SchemaValidation},
@@ -4458,10 +4631,12 @@ end
 
 class ISchemaName
   include ::Thrift::Struct, ::Thrift::Struct_Union
-  DBNAME = 1
-  SCHEMANAME = 2
+  CATNAME = 1
+  DBNAME = 2
+  SCHEMANAME = 3
 
   FIELDS = {
+    CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName'},
     DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName'},
     SCHEMANAME => {:type => ::Thrift::Types::STRING, :name => 'schemaName'}
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb b/standalone-metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
index 7a07b73..c103675 100644
--- a/standalone-metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
+++ b/standalone-metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
@@ -43,6 +43,73 @@ module ThriftHiveMetastore
       return
     end
 
+    def create_catalog(catalog)
+      send_create_catalog(catalog)
+      recv_create_catalog()
+    end
+
+    def send_create_catalog(catalog)
+      send_message('create_catalog', Create_catalog_args, :catalog => catalog)
+    end
+
+    def recv_create_catalog()
+      result = receive_message(Create_catalog_result)
+      raise result.o1 unless result.o1.nil?
+      raise result.o2 unless result.o2.nil?
+      raise result.o3 unless result.o3.nil?
+      return
+    end
+
+    def get_catalog(catName)
+      send_get_catalog(catName)
+      return recv_get_catalog()
+    end
+
+    def send_get_catalog(catName)
+      send_message('get_catalog', Get_catalog_args, :catName => catName)
+    end
+
+    def recv_get_catalog()
+      result = receive_message(Get_catalog_result)
+      return result.success unless result.success.nil?
+      raise result.o1 unless result.o1.nil?
+      raise result.o2 unless result.o2.nil?
+      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_catalog failed: unknown result')
+    end
+
+    def get_catalogs()
+      send_get_catalogs()
+      return recv_get_catalogs()
+    end
+
+    def send_get_catalogs()
+      send_message('get_catalogs', Get_catalogs_args)
+    end
+
+    def recv_get_catalogs()
+      result = receive_message(Get_catalogs_result)
+      return result.success unless result.success.nil?
+      raise result.o1 unless result.o1.nil?
+      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_catalogs failed: unknown result')
+    end
+
+    def drop_catalog(catName)
+      send_drop_catalog(catName)
+      recv_drop_catalog()
+    end
+
+    def send_drop_catalog(catName)
+      send_message('drop_catalog', Drop_catalog_args, :catName => catName)
+    end
+
+    def recv_drop_catalog()
+      result = receive_message(Drop_catalog_result)
+      raise result.o1 unless result.o1.nil?
+      raise result.o2 unless result.o2.nil?
+      raise result.o3 unless result.o3.nil?
+      return
+    end
+
     def create_database(database)
       send_create_database(database)
       recv_create_database()
@@ -660,13 +727,13 @@ module ThriftHiveMetastore
       raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_materialization_invalidation_info failed: unknown result')
     end
 
-    def update_creation_metadata(dbname, tbl_name, creation_metadata)
-      send_update_creation_metadata(dbname, tbl_name, creation_metadata)
+    def update_creation_metadata(catName, dbname, tbl_name, creation_metadata)
+      send_update_creation_metadata(catName, dbname, tbl_name, creation_metadata)
       recv_update_creation_metadata()
     end
 
-    def send_update_creation_metadata(dbname, tbl_name, creation_metadata)
-      send_message('update_creation_metadata', Update_creation_metadata_args, :dbname => dbname, :tbl_name => tbl_name, :creation_metadata => creation_metadata)
+    def send_update_creation_metadata(catName, dbname, tbl_name, creation_metadata)
+      send_message('update_creation_metadata', Update_creation_metadata_args, :catName => catName, :dbname => dbname, :tbl_name => tbl_name, :creation_metadata => creation_metadata)
     end
 
     def recv_update_creation_metadata()
@@ -3308,6 +3375,60 @@ module ThriftHiveMetastore
       write_result(result, oprot, 'setMetaConf', seqid)
     end
 
+    def process_create_catalog(seqid, iprot, oprot)
+      args = read_args(iprot, Create_catalog_args)
+      result = Create_catalog_result.new()
+      begin
+        @handler.create_catalog(args.catalog)
+      rescue ::AlreadyExistsException => o1
+        result.o1 = o1
+      rescue ::InvalidObjectException => o2
+        result.o2 = o2
+      rescue ::MetaException => o3
+        result.o3 = o3
+      end
+      write_result(result, oprot, 'create_catalog', seqid)
+    end
+
+    def process_get_catalog(seqid, iprot, oprot)
+      args = read_args(iprot, Get_catalog_args)
+      result = Get_catalog_result.new()
+      begin
+        result.success = @handler.get_catalog(args.catName)
+      rescue ::NoSuchObjectException => o1
+        result.o1 = o1
+      rescue ::MetaException => o2
+        result.o2 = o2
+      end
+      write_result(result, oprot, 'get_catalog', seqid)
+    end
+
+    def process_get_catalogs(seqid, iprot, oprot)
+      args = read_args(iprot, Get_catalogs_args)
+      result = Get_catalogs_result.new()
+      begin
+        result.success = @handler.get_catalogs()
+      rescue ::MetaException => o1
+        result.o1 = o1
+      end
+      write_result(result, oprot, 'get_catalogs', seqid)
+    end
+
+    def process_drop_catalog(seqid, iprot, oprot)
+      args = read_args(iprot, Drop_catalog_args)
+      result = Drop_catalog_result.new()
+      begin
+        @handler.drop_catalog(args.catName)
+      rescue ::NoSuchObjectException => o1
+        result.o1 = o1
+      rescue ::InvalidOperationException => o2
+        result.o2 = o2
+      rescue ::MetaException => o3
+        result.o3 = o3
+      end
+      write_result(result, oprot, 'drop_catalog', seqid)
+    end
+
     def process_create_database(seqid, iprot, oprot)
       args = read_args(iprot, Create_database_args)
       result = Create_database_result.new()
@@ -3799,7 +3920,7 @@ module ThriftHiveMetastore
       args = read_args(iprot, Update_creation_metadata_args)
       result = Update_creation_metadata_result.new()
       begin
-        @handler.update_creation_metadata(args.dbname, args.tbl_name, args.creation_metadata)
+        @handler.update_creation_metadata(args.catName, args.dbname, args.tbl_name, args.creation_metadata)
       rescue ::MetaException => o1
         result.o1 = o1
       rescue ::InvalidOperationException => o2
@@ -5826,6 +5947,147 @@ module ThriftHiveMetastore
     ::Thrift::Struct.generate_accessors self
   end
 
+  class Create_catalog_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    CATALOG = 1
+
+    FIELDS = {
+      CATALOG => {:type => ::Thrift::Types::STRUCT, :name => 'catalog', :class => ::CreateCatalogRequest}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Create_catalog_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    O1 = 1
+    O2 = 2
+    O3 = 3
+
+    FIELDS = {
+      O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::AlreadyExistsException},
+      O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::InvalidObjectException},
+      O3 => {:type => ::Thrift::Types::STRUCT, :name => 'o3', :class => ::MetaException}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Get_catalog_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    CATNAME = 1
+
+    FIELDS = {
+      CATNAME => {:type => ::Thrift::Types::STRUCT, :name => 'catName', :class => ::GetCatalogRequest}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Get_catalog_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SUCCESS = 0
+    O1 = 1
+    O2 = 2
+
+    FIELDS = {
+      SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::GetCatalogResponse},
+      O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::NoSuchObjectException},
+      O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::MetaException}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Get_catalogs_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+
+    FIELDS = {
+
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Get_catalogs_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SUCCESS = 0
+    O1 = 1
+
+    FIELDS = {
+      SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::GetCatalogsResponse},
+      O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::MetaException}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Drop_catalog_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    CATNAME = 1
+
+    FIELDS = {
+      CATNAME => {:type => ::Thrift::Types::STRUCT, :name => 'catName', :class => ::DropCatalogRequest}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Drop_catalog_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    O1 = 1
+    O2 = 2
+    O3 = 3
+
+    FIELDS = {
+      O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::NoSuchObjectException},
+      O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::InvalidOperationException},
+      O3 => {:type => ::Thrift::Types::STRUCT, :name => 'o3', :class => ::MetaException}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
   class Create_database_args
     include ::Thrift::Struct, ::Thrift::Struct_Union
     DATABASE = 1
@@ -7197,11 +7459,13 @@ module ThriftHiveMetastore
 
   class Update_creation_metadata_args
     include ::Thrift::Struct, ::Thrift::Struct_Union
-    DBNAME = 1
-    TBL_NAME = 2
-    CREATION_METADATA = 3
+    CATNAME = 1
+    DBNAME = 2
+    TBL_NAME = 3
+    CREATION_METADATA = 4
 
     FIELDS = {
+      CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName'},
       DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbname'},
       TBL_NAME => {:type => ::Thrift::Types::STRING, :name => 'tbl_name'},
       CREATION_METADATA => {:type => ::Thrift::Types::STRUCT, :name => 'creation_metadata', :class => ::CreationMetadata}

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/AggregateStatsCache.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/AggregateStatsCache.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/AggregateStatsCache.java
index bdac161..8e920bb 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/AggregateStatsCache.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/AggregateStatsCache.java
@@ -152,16 +152,16 @@ public class AggregateStatsCache {
    * Return aggregate stats for a column from the cache or null.
    * While reading from the nodelist for a key, we wait maxReaderWaitTime to acquire the lock,
    * failing which we return a cache miss (i.e. null)
-   *
-   * @param dbName
-   * @param tblName
-   * @param colName
-   * @param partNames
-   * @return
+   * @param catName catalog name
+   * @param dbName database name
+   * @param tblName table name
+   * @param colName column name
+   * @param partNames list of partition names
+   * @return aggregated col stats
    */
-  public AggrColStats get(String dbName, String tblName, String colName, List<String> partNames) {
+  public AggrColStats get(String catName, String dbName, String tblName, String colName, List<String> partNames) {
     // Cache key
-    Key key = new Key(dbName, tblName, colName);
+    Key key = new Key(catName, dbName, tblName, colName);
     AggrColStatsList candidateList = cacheStore.get(key);
     // No key, or no nodes in candidate list
     if ((candidateList == null) || (candidateList.nodes.size() == 0)) {
@@ -267,23 +267,23 @@ public class AggregateStatsCache {
    * Add a new node to the cache; may trigger the cleaner thread if the cache is near full capacity.
    * We'll however add the node even if we temporaily exceed maxCacheNodes, because the cleaner
    * will eventually create space from expired nodes or by removing LRU nodes.
-   *
-   * @param dbName
-   * @param tblName
-   * @param colName
+   * @param catName catalog name
+   * @param dbName database name
+   * @param tblName table name
+   * @param colName column name
    * @param numPartsCached
    * @param colStats
    * @param bloomFilter
    */
   // TODO: make add asynchronous: add shouldn't block the higher level calls
-  public void add(String dbName, String tblName, String colName, long numPartsCached,
+  public void add(String catName, String dbName, String tblName, String colName, long numPartsCached,
       ColumnStatisticsObj colStats, BloomFilter bloomFilter) {
     // If we have no space in the cache, run cleaner thread
     if (getCurrentNodes() / maxCacheNodes > maxFull) {
       spawnCleaner();
     }
     // Cache key
-    Key key = new Key(dbName, tblName, colName);
+    Key key = new Key(catName, dbName, tblName, colName);
     // Add new node to the cache
     AggrColStats node = new AggrColStats(numPartsCached, bloomFilter, colStats);
     AggrColStatsList nodeList;
@@ -463,15 +463,17 @@ public class AggregateStatsCache {
    * Key object for the stats cache hashtable
    */
   static class Key {
+    private final String catName;
     private final String dbName;
     private final String tblName;
     private final String colName;
 
-    Key(String db, String table, String col) {
+    Key(String cat, String db, String table, String col) {
       // Don't construct an illegal cache key
-      if ((db == null) || (table == null) || (col == null)) {
-        throw new IllegalArgumentException("dbName, tblName, colName can't be null");
+      if (cat == null || (db == null) || (table == null) || (col == null)) {
+        throw new IllegalArgumentException("catName, dbName, tblName, colName can't be null");
       }
+      catName = cat;
       dbName = db;
       tblName = table;
       colName = col;
@@ -483,18 +485,20 @@ public class AggregateStatsCache {
         return false;
       }
       Key that = (Key) other;
-      return dbName.equals(that.dbName) && tblName.equals(that.tblName)
-          && colName.equals(that.colName);
+      return catName.equals(that.catName) && dbName.equals(that.dbName) &&
+          tblName.equals(that.tblName) && colName.equals(that.colName);
     }
 
     @Override
     public int hashCode() {
-      return dbName.hashCode() * 31 + tblName.hashCode() * 31 + colName.hashCode();
+      return catName.hashCode() * 31 + dbName.hashCode() * 31 + tblName.hashCode() * 31 +
+          colName.hashCode();
     }
 
     @Override
     public String toString() {
-      return "database:" + dbName + ", table:" + tblName + ", column:" + colName;
+      return "catalog: " + catName + ", database:" + dbName + ", table:" + tblName + ", column:" +
+          colName;
     }
 
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/AlterHandler.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/AlterHandler.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/AlterHandler.java
index fc0b4d7..050dca9 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/AlterHandler.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/AlterHandler.java
@@ -35,7 +35,7 @@ public interface AlterHandler extends Configurable {
 
   /**
    * @deprecated As of release 2.2.0. Replaced by {@link #alterTable(RawStore, Warehouse, String,
-   * String, Table, EnvironmentContext, IHMSHandler)}
+   * String, String, Table, EnvironmentContext, IHMSHandler)}
    *
    * handles alter table, the changes could be cascaded to partitions if applicable
    *
@@ -43,6 +43,8 @@ public interface AlterHandler extends Configurable {
    *          object to get metadata
    * @param wh
    *          Hive Warehouse where table data is stored
+   * @param catName
+   *          catalog of the table being altered
    * @param dbname
    *          database of the table being altered
    * @param name
@@ -56,9 +58,11 @@ public interface AlterHandler extends Configurable {
    *           thrown if there is any other error
    */
   @Deprecated
-  void alterTable(RawStore msdb, Warehouse wh, String dbname,
+  default void alterTable(RawStore msdb, Warehouse wh, String catName, String dbname,
     String name, Table newTable, EnvironmentContext envContext)
-      throws InvalidOperationException, MetaException;
+      throws InvalidOperationException, MetaException {
+    alterTable(msdb, wh, catName, dbname, name, newTable, envContext, null);
+  }
 
   /**
    * handles alter table, the changes could be cascaded to partitions if applicable
@@ -67,6 +71,7 @@ public interface AlterHandler extends Configurable {
    *          object to get metadata
    * @param wh
    *          Hive Warehouse where table data is stored
+   * @param catName catalog of the table being altered
    * @param dbname
    *          database of the table being altered
    * @param name
@@ -81,7 +86,7 @@ public interface AlterHandler extends Configurable {
    * @throws MetaException
    *           thrown if there is any other error
    */
-  void alterTable(RawStore msdb, Warehouse wh, String dbname,
+  void alterTable(RawStore msdb, Warehouse wh, String catName, String dbname,
       String name, Table newTable, EnvironmentContext envContext,
       IHMSHandler handler) throws InvalidOperationException, MetaException;
 
@@ -119,7 +124,8 @@ public interface AlterHandler extends Configurable {
    *
    * @param msdb
    *          object to get metadata
-   * @param wh
+   * @param wh physical warehouse class
+   * @param catName catalog name
    * @param dbname
    *          database of the partition being altered
    * @param name
@@ -136,14 +142,15 @@ public interface AlterHandler extends Configurable {
    * @throws AlreadyExistsException
    * @throws MetaException
    */
-  Partition alterPartition(final RawStore msdb, Warehouse wh, final String dbname,
-    final String name, final List<String> part_vals, final Partition new_part, EnvironmentContext environmentContext,
-    IHMSHandler handler)
+  Partition alterPartition(final RawStore msdb, Warehouse wh, final String catName,
+                           final String dbname, final String name, final List<String> part_vals,
+                           final Partition new_part, EnvironmentContext environmentContext,
+                           IHMSHandler handler)
       throws InvalidOperationException, InvalidObjectException, AlreadyExistsException, MetaException;
 
   /**
-   * @deprecated As of release 2.2.0. Replaced by {@link #alterPartitions(RawStore, Warehouse, String,
-   * String, List, EnvironmentContext, IHMSHandler)}
+   * @deprecated As of release 3.0.0. Replaced by {@link #alterPartitions(RawStore, Warehouse, String,
+   * String, String, List, EnvironmentContext, IHMSHandler)}
    *
    * handles alter partitions
    *
@@ -188,7 +195,7 @@ public interface AlterHandler extends Configurable {
    * @throws AlreadyExistsException
    * @throws MetaException
    */
-  List<Partition> alterPartitions(final RawStore msdb, Warehouse wh,
+  List<Partition> alterPartitions(final RawStore msdb, Warehouse wh, final String catName,
     final String dbname, final String name, final List<Partition> new_parts,
     EnvironmentContext environmentContext,IHMSHandler handler)
       throws InvalidOperationException, InvalidObjectException, AlreadyExistsException, MetaException;

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/DefaultMetaStoreFilterHookImpl.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/DefaultMetaStoreFilterHookImpl.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/DefaultMetaStoreFilterHookImpl.java
index ca63333..4e1daba 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/DefaultMetaStoreFilterHookImpl.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/DefaultMetaStoreFilterHookImpl.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.PartitionSpec;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TableMeta;
 
 /**
  * Default no-op implementation of the MetaStoreFilterHook that returns the result as is
@@ -47,11 +48,17 @@ public class DefaultMetaStoreFilterHookImpl implements MetaStoreFilterHook {
   }
 
   @Override
-  public List<String> filterTableNames(String dbName, List<String> tableList) throws MetaException {
+  public List<String> filterTableNames(String catName, String dbName, List<String> tableList)
+      throws MetaException {
     return tableList;
   }
 
   @Override
+  public List<TableMeta> filterTableMetas(List<TableMeta> tableMetas) throws MetaException {
+    return tableMetas;
+  }
+
+  @Override
   public Table filterTable(Table table)  throws NoSuchObjectException {
     return table;
   }
@@ -78,7 +85,7 @@ public class DefaultMetaStoreFilterHookImpl implements MetaStoreFilterHook {
   }
 
   @Override
-  public List<String> filterPartitionNames(String dbName, String tblName,
+  public List<String> filterPartitionNames(String catName, String dbName, String tblName,
       List<String> partitionNames) throws MetaException {
     return partitionNames;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
index 04828e5..ed1b8c5 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
@@ -61,6 +61,10 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+import static org.apache.hadoop.hive.metastore.utils.StringUtils.normalizeIdentifier;
+
 /**
  * Hive specific implementation of alter
  */
@@ -85,16 +89,10 @@ public class HiveAlterHandler implements AlterHandler {
   }
 
   @Override
-  public void alterTable(RawStore msdb, Warehouse wh, String dbname,
-    String name, Table newt, EnvironmentContext environmentContext)
-      throws InvalidOperationException, MetaException {
-    alterTable(msdb, wh, dbname, name, newt, environmentContext, null);
-  }
-
-  @Override
-  public void alterTable(RawStore msdb, Warehouse wh, String dbname,
+  public void alterTable(RawStore msdb, Warehouse wh, String catName, String dbname,
       String name, Table newt, EnvironmentContext environmentContext,
       IHMSHandler handler) throws InvalidOperationException, MetaException {
+    catName = normalizeIdentifier(catName);
     name = name.toLowerCase();
     dbname = dbname.toLowerCase();
 
@@ -135,9 +133,15 @@ public class HiveAlterHandler implements AlterHandler {
       boolean isPartitionedTable = false;
       List<Partition> parts;
 
+      // Switching tables between catalogs is not allowed.
+      if (!catName.equalsIgnoreCase(newt.getCatName())) {
+        throw new InvalidOperationException("Tables cannot be moved between catalogs, old catalog" +
+            catName + ", new catalog " + newt.getCatName());
+      }
+
       // check if table with the new name already exists
       if (!newTblName.equals(name) || !newDbName.equals(dbname)) {
-        if (msdb.getTable(newDbName, newTblName) != null) {
+        if (msdb.getTable(catName, newDbName, newTblName) != null) {
           throw new InvalidOperationException("new table " + newDbName
               + "." + newTblName + " already exists");
         }
@@ -146,9 +150,10 @@ public class HiveAlterHandler implements AlterHandler {
 
       msdb.openTransaction();
       // get old table
-      oldt = msdb.getTable(dbname, name);
+      oldt = msdb.getTable(catName, dbname, name);
       if (oldt == null) {
-        throw new InvalidOperationException("table " + dbname + "." + name + " doesn't exist");
+        throw new InvalidOperationException("table " +
+            Warehouse.getCatalogQualifiedTableName(catName, dbname, name) + " doesn't exist");
       }
 
       if (oldt.getPartitionKeysSize() != 0) {
@@ -188,7 +193,7 @@ public class HiveAlterHandler implements AlterHandler {
           && (oldt.getSd().getLocation().compareTo(newt.getSd().getLocation()) == 0
             || StringUtils.isEmpty(newt.getSd().getLocation()))
           && !MetaStoreUtils.isExternalTable(oldt)) {
-        Database olddb = msdb.getDatabase(dbname);
+        Database olddb = msdb.getDatabase(catName, dbname);
         // if a table was created in a user specified location using the DDL like
         // create table tbl ... location ...., it should be treated like an external table
         // in the table rename, its data location should not be changed. We can check
@@ -204,7 +209,7 @@ public class HiveAlterHandler implements AlterHandler {
           srcFs = wh.getFs(srcPath);
 
           // get new location
-          Database db = msdb.getDatabase(newDbName);
+          Database db = msdb.getDatabase(catName, newDbName);
           Path databasePath = constructRenamedPath(wh.getDatabasePath(db), srcPath);
           destPath = new Path(databasePath, newTblName);
           destFs = wh.getFs(destPath);
@@ -222,8 +227,9 @@ public class HiveAlterHandler implements AlterHandler {
 
           try {
             if (destFs.exists(destPath)) {
-              throw new InvalidOperationException("New location for this table "
-                  + newDbName + "." + newTblName + " already exists : " + destPath);
+              throw new InvalidOperationException("New location for this table " +
+                  Warehouse.getCatalogQualifiedTableName(catName, newDbName, newTblName) +
+                      " already exists : " + destPath);
             }
             // check that src exists and also checks permissions necessary, rename src to dest
             if (srcFs.exists(srcPath) && wh.renameDir(srcPath, destPath, true)) {
@@ -242,7 +248,7 @@ public class HiveAlterHandler implements AlterHandler {
           String newTblLocPath = dataWasMoved ? destPath.toUri().getPath() : null;
 
           // also the location field in partition
-          parts = msdb.getPartitions(dbname, name, -1);
+          parts = msdb.getPartitions(catName, dbname, name, -1);
           Map<Partition, ColumnStatistics> columnStatsNeedUpdated = new HashMap<>();
           for (Partition part : parts) {
             String oldPartLoc = part.getSd().getLocation();
@@ -254,13 +260,13 @@ public class HiveAlterHandler implements AlterHandler {
             }
             part.setDbName(newDbName);
             part.setTableName(newTblName);
-            ColumnStatistics colStats = updateOrGetPartitionColumnStats(msdb, dbname, name,
+            ColumnStatistics colStats = updateOrGetPartitionColumnStats(msdb, catName, dbname, name,
                 part.getValues(), part.getSd().getCols(), oldt, part, null);
             if (colStats != null) {
               columnStatsNeedUpdated.put(part, colStats);
             }
           }
-          msdb.alterTable(dbname, name, newt);
+          msdb.alterTable(catName, dbname, name, newt);
           // alterPartition is only for changing the partition location in the table rename
           if (dataWasMoved) {
 
@@ -278,7 +284,7 @@ public class HiveAlterHandler implements AlterHandler {
               for (Partition part : partBatch) {
                 partValues.add(part.getValues());
               }
-              msdb.alterPartitions(newDbName, newTblName, partValues, partBatch);
+              msdb.alterPartitions(catName, newDbName, newTblName, partValues, partBatch);
             }
           }
 
@@ -295,7 +301,7 @@ public class HiveAlterHandler implements AlterHandler {
         // operations other than table rename
         if (MetaStoreUtils.requireCalStats(null, null, newt, environmentContext) &&
             !isPartitionedTable) {
-          Database db = msdb.getDatabase(newDbName);
+          Database db = msdb.getDatabase(catName, newDbName);
           // Update table stats. For partitioned table, we update stats in alterPartition()
           MetaStoreUtils.updateTableStatsFast(db, newt, wh, false, true, environmentContext, false);
         }
@@ -303,23 +309,23 @@ public class HiveAlterHandler implements AlterHandler {
         if (isPartitionedTable) {
           //Currently only column related changes can be cascaded in alter table
           if(!MetaStoreUtils.areSameColumns(oldt.getSd().getCols(), newt.getSd().getCols())) {
-            parts = msdb.getPartitions(dbname, name, -1);
+            parts = msdb.getPartitions(catName, dbname, name, -1);
             for (Partition part : parts) {
               Partition oldPart = new Partition(part);
               List<FieldSchema> oldCols = part.getSd().getCols();
               part.getSd().setCols(newt.getSd().getCols());
-              ColumnStatistics colStats = updateOrGetPartitionColumnStats(msdb, dbname, name,
+              ColumnStatistics colStats = updateOrGetPartitionColumnStats(msdb, catName, dbname, name,
                   part.getValues(), oldCols, oldt, part, null);
               assert(colStats == null);
               if (cascade) {
-                msdb.alterPartition(dbname, name, part.getValues(), part);
+                msdb.alterPartition(catName, dbname, name, part.getValues(), part);
               } else {
                 // update changed properties (stats)
                 oldPart.setParameters(part.getParameters());
-                msdb.alterPartition(dbname, name, part.getValues(), oldPart);
+                msdb.alterPartition(catName, dbname, name, part.getValues(), oldPart);
               }
             }
-            msdb.alterTable(dbname, name, newt);
+            msdb.alterTable(catName, dbname, name, newt);
           } else {
             LOG.warn("Alter table not cascaded to partitions.");
             alterTableUpdateTableColumnStats(msdb, oldt, newt);
@@ -345,7 +351,8 @@ public class HiveAlterHandler implements AlterHandler {
                   new CreateTableEvent(newt, true, handler),
                   environmentContext);
           if (isPartitionedTable) {
-            parts = msdb.getPartitions(newt.getDbName(), newt.getTableName(), -1);
+            String cName = newt.isSetCatName() ? newt.getCatName() : DEFAULT_CATALOG_NAME;
+            parts = msdb.getPartitions(cName, newt.getDbName(), newt.getTableName(), -1);
             MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
                     EventMessage.EventType.ADD_PARTITION,
                     new AddPartitionEvent(newt, parts, true, handler),
@@ -372,7 +379,8 @@ public class HiveAlterHandler implements AlterHandler {
               + " Check metastore logs for detailed stack." + e.getMessage());
     } finally {
       if (!success) {
-        LOG.error("Failed to alter table " + dbname + "." + name);
+        LOG.error("Failed to alter table " +
+            Warehouse.getCatalogQualifiedTableName(catName, dbname, name));
         msdb.rollbackTransaction();
         if (dataWasMoved) {
           try {
@@ -413,13 +421,15 @@ public class HiveAlterHandler implements AlterHandler {
     final String name, final List<String> part_vals, final Partition new_part,
     EnvironmentContext environmentContext)
       throws InvalidOperationException, InvalidObjectException, AlreadyExistsException, MetaException {
-    return alterPartition(msdb, wh, dbname, name, part_vals, new_part, environmentContext, null);
+    return alterPartition(msdb, wh, DEFAULT_CATALOG_NAME, dbname, name, part_vals, new_part,
+        environmentContext, null);
   }
 
   @Override
-  public Partition alterPartition(final RawStore msdb, Warehouse wh, final String dbname,
-    final String name, final List<String> part_vals, final Partition new_part,
-    EnvironmentContext environmentContext, IHMSHandler handler)
+  public Partition alterPartition(final RawStore msdb, Warehouse wh, final String catName,
+                                  final String dbname, final String name,
+                                  final List<String> part_vals, final Partition new_part,
+                                  EnvironmentContext environmentContext, IHMSHandler handler)
       throws InvalidOperationException, InvalidObjectException, AlreadyExistsException, MetaException {
     boolean success = false;
     Partition oldPart;
@@ -436,18 +446,17 @@ public class HiveAlterHandler implements AlterHandler {
           .currentTimeMillis() / 1000));
     }
 
-
     //alter partition
     if (part_vals == null || part_vals.size() == 0) {
       try {
         msdb.openTransaction();
 
-        Table tbl = msdb.getTable(dbname, name);
+        Table tbl = msdb.getTable(catName, dbname, name);
         if (tbl == null) {
           throw new InvalidObjectException(
               "Unable to alter partition because table or database does not exist.");
         }
-        oldPart = msdb.getPartition(dbname, name, new_part.getValues());
+        oldPart = msdb.getPartition(catName, dbname, name, new_part.getValues());
         if (MetaStoreUtils.requireCalStats(oldPart, new_part, tbl, environmentContext)) {
           // if stats are same, no need to update
           if (MetaStoreUtils.isFastStatsSame(oldPart, new_part)) {
@@ -460,10 +469,10 @@ public class HiveAlterHandler implements AlterHandler {
 
         // PartitionView does not have SD. We do not need update its column stats
         if (oldPart.getSd() != null) {
-          updateOrGetPartitionColumnStats(msdb, dbname, name, new_part.getValues(),
+          updateOrGetPartitionColumnStats(msdb, catName, dbname, name, new_part.getValues(),
               oldPart.getSd().getCols(), tbl, new_part, null);
         }
-        msdb.alterPartition(dbname, name, new_part.getValues(), new_part);
+        msdb.alterPartition(catName, dbname, name, new_part.getValues(), new_part);
         if (transactionalListeners != null && !transactionalListeners.isEmpty()) {
           MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
                                                 EventMessage.EventType.ALTER_PARTITION,
@@ -496,13 +505,13 @@ public class HiveAlterHandler implements AlterHandler {
     boolean dataWasMoved = false;
     try {
       msdb.openTransaction();
-      Table tbl = msdb.getTable(dbname, name);
+      Table tbl = msdb.getTable(DEFAULT_CATALOG_NAME, dbname, name);
       if (tbl == null) {
         throw new InvalidObjectException(
             "Unable to alter partition because table or database does not exist.");
       }
       try {
-        oldPart = msdb.getPartition(dbname, name, part_vals);
+        oldPart = msdb.getPartition(catName, dbname, name, part_vals);
       } catch (NoSuchObjectException e) {
         // this means there is no existing partition
         throw new InvalidObjectException(
@@ -511,7 +520,7 @@ public class HiveAlterHandler implements AlterHandler {
 
       Partition check_part;
       try {
-        check_part = msdb.getPartition(dbname, name, new_part.getValues());
+        check_part = msdb.getPartition(catName, dbname, name, new_part.getValues());
       } catch(NoSuchObjectException e) {
         // this means there is no existing partition
         check_part = null;
@@ -530,7 +539,7 @@ public class HiveAlterHandler implements AlterHandler {
         try {
           // if tbl location is available use it
           // else derive the tbl location from database location
-          destPath = wh.getPartitionPath(msdb.getDatabase(dbname), tbl, new_part.getValues());
+          destPath = wh.getPartitionPath(msdb.getDatabase(catName, dbname), tbl, new_part.getValues());
           destPath = constructRenamedPath(destPath, new Path(new_part.getSd().getLocation()));
         } catch (NoSuchObjectException e) {
           LOG.debug("Didn't find object in metastore ", e);
@@ -593,9 +602,9 @@ public class HiveAlterHandler implements AlterHandler {
       }
 
       String newPartName = Warehouse.makePartName(tbl.getPartitionKeys(), new_part.getValues());
-      ColumnStatistics cs = updateOrGetPartitionColumnStats(msdb, dbname, name, oldPart.getValues(),
+      ColumnStatistics cs = updateOrGetPartitionColumnStats(msdb, catName, dbname, name, oldPart.getValues(),
           oldPart.getSd().getCols(), tbl, new_part, null);
-      msdb.alterPartition(dbname, name, part_vals, new_part);
+      msdb.alterPartition(catName, dbname, name, part_vals, new_part);
       if (cs != null) {
         cs.getStatsDesc().setPartName(newPartName);
         try {
@@ -643,13 +652,15 @@ public class HiveAlterHandler implements AlterHandler {
     final String name, final List<Partition> new_parts,
     EnvironmentContext environmentContext)
       throws InvalidOperationException, InvalidObjectException, AlreadyExistsException, MetaException {
-    return alterPartitions(msdb, wh, dbname, name, new_parts, environmentContext, null);
+    return alterPartitions(msdb, wh, DEFAULT_CATALOG_NAME, dbname, name, new_parts,
+        environmentContext, null);
   }
 
   @Override
-  public List<Partition> alterPartitions(final RawStore msdb, Warehouse wh, final String dbname,
-    final String name, final List<Partition> new_parts, EnvironmentContext environmentContext,
-    IHMSHandler handler)
+  public List<Partition> alterPartitions(final RawStore msdb, Warehouse wh, final String catName,
+                                         final String dbname, final String name,
+                                         final List<Partition> new_parts,
+                                         EnvironmentContext environmentContext, IHMSHandler handler)
       throws InvalidOperationException, InvalidObjectException, AlreadyExistsException, MetaException {
     List<Partition> oldParts = new ArrayList<>();
     List<List<String>> partValsList = new ArrayList<>();
@@ -658,12 +669,11 @@ public class HiveAlterHandler implements AlterHandler {
       transactionalListeners = handler.getTransactionalListeners();
     }
 
-
     boolean success = false;
     try {
       msdb.openTransaction();
 
-      Table tbl = msdb.getTable(dbname, name);
+      Table tbl = msdb.getTable(catName, dbname, name);
       if (tbl == null) {
         throw new InvalidObjectException(
             "Unable to alter partitions because table or database does not exist.");
@@ -677,7 +687,7 @@ public class HiveAlterHandler implements AlterHandler {
               .currentTimeMillis() / 1000));
         }
 
-        Partition oldTmpPart = msdb.getPartition(dbname, name, tmpPart.getValues());
+        Partition oldTmpPart = msdb.getPartition(catName, dbname, name, tmpPart.getValues());
         oldParts.add(oldTmpPart);
         partValsList.add(tmpPart.getValues());
 
@@ -693,12 +703,12 @@ public class HiveAlterHandler implements AlterHandler {
 
         // PartitionView does not have SD and we do not need to update its column stats
         if (oldTmpPart.getSd() != null) {
-          updateOrGetPartitionColumnStats(msdb, dbname, name, oldTmpPart.getValues(),
+          updateOrGetPartitionColumnStats(msdb, catName, dbname, name, oldTmpPart.getValues(),
               oldTmpPart.getSd().getCols(), tbl, tmpPart, null);
         }
       }
 
-      msdb.alterPartitions(dbname, name, partValsList, new_parts);
+      msdb.alterPartitions(catName, dbname, name, partValsList, new_parts);
       Iterator<Partition> oldPartsIt = oldParts.iterator();
       for (Partition newPart : new_parts) {
         Partition oldPart;
@@ -768,10 +778,12 @@ public class HiveAlterHandler implements AlterHandler {
   @VisibleForTesting
   void alterTableUpdateTableColumnStats(RawStore msdb, Table oldTable, Table newTable)
       throws MetaException, InvalidObjectException {
+    String catName = normalizeIdentifier(oldTable.isSetCatName() ? oldTable.getCatName() :
+        getDefaultCatalog(conf));
     String dbName = oldTable.getDbName().toLowerCase();
-    String tableName = org.apache.hadoop.hive.metastore.utils.StringUtils.normalizeIdentifier(oldTable.getTableName());
+    String tableName = normalizeIdentifier(oldTable.getTableName());
     String newDbName = newTable.getDbName().toLowerCase();
-    String newTableName = org.apache.hadoop.hive.metastore.utils.StringUtils.normalizeIdentifier(newTable.getTableName());
+    String newTableName = normalizeIdentifier(newTable.getTableName());
 
     try {
       List<FieldSchema> oldCols = oldTable.getSd().getCols();
@@ -794,7 +806,7 @@ public class HiveAlterHandler implements AlterHandler {
           }
 
           // Collect column stats which need to be rewritten and remove old stats
-          colStats = msdb.getTableColumnStatistics(dbName, tableName, oldColNames);
+          colStats = msdb.getTableColumnStatistics(catName, dbName, tableName, oldColNames);
           if (colStats == null) {
             updateColumnStats = false;
           } else {
@@ -813,12 +825,12 @@ public class HiveAlterHandler implements AlterHandler {
 
                 if (found) {
                   if (!newDbName.equals(dbName) || !newTableName.equals(tableName)) {
-                    msdb.deleteTableColumnStatistics(dbName, tableName, statsObj.getColName());
+                    msdb.deleteTableColumnStatistics(catName, dbName, tableName, statsObj.getColName());
                     newStatsObjs.add(statsObj);
                     deletedCols.add(statsObj.getColName());
                   }
                 } else {
-                  msdb.deleteTableColumnStatistics(dbName, tableName, statsObj.getColName());
+                  msdb.deleteTableColumnStatistics(catName, dbName, tableName, statsObj.getColName());
                   deletedCols.add(statsObj.getColName());
                 }
               }
@@ -828,7 +840,7 @@ public class HiveAlterHandler implements AlterHandler {
         }
 
         // Change to new table and append stats for the new table
-        msdb.alterTable(dbName, tableName, newTable);
+        msdb.alterTable(catName, dbName, tableName, newTable);
         if (updateColumnStats && !newStatsObjs.isEmpty()) {
           ColumnStatisticsDesc statsDesc = colStats.getStatsDesc();
           statsDesc.setDbName(newDbName);
@@ -845,7 +857,7 @@ public class HiveAlterHandler implements AlterHandler {
   }
 
   private ColumnStatistics updateOrGetPartitionColumnStats(
-      RawStore msdb, String dbname, String tblname, List<String> partVals,
+      RawStore msdb, String catName, String dbname, String tblname, List<String> partVals,
       List<FieldSchema> oldCols, Table table, Partition part, List<FieldSchema> newCols)
           throws MetaException, InvalidObjectException {
     ColumnStatistics newPartsColStats = null;
@@ -868,7 +880,7 @@ public class HiveAlterHandler implements AlterHandler {
         oldColNames.add(oldCol.getName());
       }
       List<String> oldPartNames = Lists.newArrayList(oldPartName);
-      List<ColumnStatistics> partsColStats = msdb.getPartitionColumnStatistics(dbname, tblname,
+      List<ColumnStatistics> partsColStats = msdb.getPartitionColumnStatistics(catName, dbname, tblname,
           oldPartNames, oldColNames);
       assert (partsColStats.size() <= 1);
       for (ColumnStatistics partColStats : partsColStats) { //actually only at most one loop
@@ -886,12 +898,12 @@ public class HiveAlterHandler implements AlterHandler {
           }
           if (found) {
             if (rename) {
-              msdb.deletePartitionColumnStatistics(dbname, tblname, partColStats.getStatsDesc().getPartName(),
+              msdb.deletePartitionColumnStatistics(catName, dbname, tblname, partColStats.getStatsDesc().getPartName(),
                   partVals, statsObj.getColName());
               newStatsObjs.add(statsObj);
             }
           } else {
-            msdb.deletePartitionColumnStatistics(dbname, tblname, partColStats.getStatsDesc().getPartName(),
+            msdb.deletePartitionColumnStatistics(catName, dbname, tblname, partColStats.getStatsDesc().getPartName(),
                 partVals, statsObj.getColName());
             deletedCols.add(statsObj.getColName());
           }


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestCatalogs.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestCatalogs.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestCatalogs.java
new file mode 100644
index 0000000..02fb9eb
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestCatalogs.java
@@ -0,0 +1,217 @@
+package org.apache.hadoop.hive.metastore.client;
+
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
+import org.apache.hadoop.hive.metastore.Warehouse;
+import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
+import org.apache.hadoop.hive.metastore.api.Catalog;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.client.builder.CatalogBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
+import org.apache.hadoop.hive.metastore.minihms.AbstractMetaStoreService;
+import org.apache.thrift.TException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
+
+/**
+ * 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.
+ */
+@RunWith(Parameterized.class)
+@Category(MetastoreCheckinTest.class)
+public class TestCatalogs extends MetaStoreClientTest {
+  private static final Logger LOG = LoggerFactory.getLogger(TestCatalogs.class);
+  private final AbstractMetaStoreService metaStore;
+  private IMetaStoreClient client;
+
+  public TestCatalogs(String name, AbstractMetaStoreService metaStore) throws Exception {
+    this.metaStore = metaStore;
+    this.metaStore.start();
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    // Get new client
+    client = metaStore.getClient();
+
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    // Drop any left over catalogs
+    List<String> catalogs = client.getCatalogs();
+    for (String catName : catalogs) {
+      if (!catName.equalsIgnoreCase(Warehouse.DEFAULT_CATALOG_NAME)) {
+        // First drop any databases in catalog
+        List<String> databases = client.getAllDatabases(catName);
+        for (String db : databases) {
+          client.dropDatabase(catName, db, true, false, true);
+        }
+        client.dropCatalog(catName);
+      } else {
+        List<String> databases = client.getAllDatabases(catName);
+        for (String db : databases) {
+          if (!db.equalsIgnoreCase(DEFAULT_DATABASE_NAME)) {
+            client.dropDatabase(catName, db, true, false, true);
+          }
+        }
+
+      }
+    }
+    try {
+      if (client != null) {
+        client.close();
+      }
+    } finally {
+      client = null;
+    }
+  }
+
+  @Test
+  public void catalogOperations() throws TException {
+    String[] catNames = {"cat1", "cat2", "ADifferentName"};
+    String[] description = {"a description", "super descriptive", null};
+    String[] location = {MetaStoreTestUtils.getTestWarehouseDir("cat1"),
+                         MetaStoreTestUtils.getTestWarehouseDir("cat2"),
+                         MetaStoreTestUtils.getTestWarehouseDir("different")};
+
+    for (int i = 0; i < catNames.length; i++) {
+      Catalog cat = new CatalogBuilder()
+          .setName(catNames[i])
+          .setLocation(location[i])
+          .setDescription(description[i])
+          .build();
+      client.createCatalog(cat);
+      File dir = new File(cat.getLocationUri());
+      Assert.assertTrue(dir.exists() && dir.isDirectory());
+    }
+
+    for (int i = 0; i < catNames.length; i++) {
+      Catalog cat = client.getCatalog(catNames[i]);
+      Assert.assertTrue(catNames[i].equalsIgnoreCase(cat.getName()));
+      Assert.assertEquals(description[i], cat.getDescription());
+      Assert.assertEquals(location[i], cat.getLocationUri());
+      File dir = new File(cat.getLocationUri());
+      Assert.assertTrue(dir.exists() && dir.isDirectory());
+
+      // Make sure there's a default database associated with each catalog
+      Database db = client.getDatabase(catNames[i], DEFAULT_DATABASE_NAME);
+      Assert.assertEquals("file:" + cat.getLocationUri(), db.getLocationUri());
+    }
+
+    List<String> catalogs = client.getCatalogs();
+    Assert.assertEquals(4, catalogs.size());
+    catalogs.sort(Comparator.naturalOrder());
+    List<String> expected = new ArrayList<>(catNames.length + 1);
+    expected.add(Warehouse.DEFAULT_CATALOG_NAME);
+    expected.addAll(Arrays.asList(catNames));
+    expected.sort(Comparator.naturalOrder());
+    for (int i = 0; i < catalogs.size(); i++) {
+      Assert.assertTrue("Expected " + expected.get(i) + " actual " + catalogs.get(i),
+          catalogs.get(i).equalsIgnoreCase(expected.get(i)));
+    }
+
+    for (int i = 0; i < catNames.length; i++) {
+      client.dropCatalog(catNames[i]);
+      File dir = new File(location[i]);
+      Assert.assertFalse(dir.exists());
+    }
+
+    catalogs = client.getCatalogs();
+    Assert.assertEquals(1, catalogs.size());
+    Assert.assertTrue(catalogs.get(0).equalsIgnoreCase(Warehouse.DEFAULT_CATALOG_NAME));
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void getNonExistentCatalog() throws TException {
+    client.getCatalog("noSuchCatalog");
+  }
+
+  @Test(expected = MetaException.class)
+  @Ignore // TODO This test passes fine locally but fails on Linux, not sure why
+  public void createCatalogWithBadLocation() throws TException {
+    Catalog cat = new CatalogBuilder()
+        .setName("goodluck")
+        .setLocation("/nosuch/nosuch")
+        .build();
+    client.createCatalog(cat);
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void dropNonExistentCatalog() throws TException {
+    client.dropCatalog("noSuchCatalog");
+  }
+
+  @Test(expected = MetaException.class)
+  public void dropHiveCatalog() throws TException {
+    client.dropCatalog(Warehouse.DEFAULT_CATALOG_NAME);
+  }
+
+  @Test(expected = InvalidOperationException.class)
+  public void dropNonEmptyCatalog() throws TException {
+    String catName = "toBeDropped";
+    Catalog cat = new CatalogBuilder()
+        .setName(catName)
+        .setLocation(MetaStoreTestUtils.getTestWarehouseDir(catName))
+        .build();
+    client.createCatalog(cat);
+
+    String dbName = "dontDropMe";
+    new DatabaseBuilder()
+        .setName(dbName)
+        .setCatalogName(catName)
+        .create(client, metaStore.getConf());
+
+    client.dropCatalog(catName);
+  }
+
+  @Test(expected = InvalidOperationException.class)
+  public void dropCatalogWithNonEmptyDefaultDb() throws TException {
+    String catName = "toBeDropped2";
+    new CatalogBuilder()
+        .setName(catName)
+        .setLocation(MetaStoreTestUtils.getTestWarehouseDir(catName))
+        .create(client);
+
+    new TableBuilder()
+        .setTableName("not_droppable")
+        .setCatName(catName)
+        .addCol("cola1", "bigint")
+        .create(client, metaStore.getConf());
+
+    client.dropCatalog(catName);
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestDatabases.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestDatabases.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestDatabases.java
index f2d745e..24e3c56 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestDatabases.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestDatabases.java
@@ -20,8 +20,11 @@ package org.apache.hadoop.hive.metastore.client;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
+import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
+import org.apache.hadoop.hive.metastore.api.Catalog;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.Function;
 import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
@@ -30,10 +33,13 @@ import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.client.builder.CatalogBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.FunctionBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
 import org.apache.hadoop.hive.metastore.minihms.AbstractMetaStoreService;
+import org.apache.hadoop.hive.metastore.utils.SecurityUtils;
+import org.apache.thrift.TException;
 import org.apache.thrift.transport.TTransportException;
 import org.junit.After;
 import org.junit.Assert;
@@ -43,12 +49,20 @@ import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
+import java.io.File;
+import java.net.URI;
+import java.net.URISyntaxException;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
 
 /**
  * Test class for IMetaStoreClient API. Testing the Database related functions.
  */
+
 @RunWith(Parameterized.class)
 @Category(MetastoreCheckinTest.class)
 public class TestDatabases extends MetaStoreClientTest {
@@ -74,17 +88,16 @@ public class TestDatabases extends MetaStoreClientTest {
     }
 
     testDatabases[0] =
-        new DatabaseBuilder().setName("test_database_1").build();
+        new DatabaseBuilder().setName("test_database_1").create(client, metaStore.getConf());
     testDatabases[1] =
-        new DatabaseBuilder().setName("test_database_to_find_1").build();
+        new DatabaseBuilder().setName("test_database_to_find_1").create(client, metaStore.getConf());
     testDatabases[2] =
-        new DatabaseBuilder().setName("test_database_to_find_2").build();
+        new DatabaseBuilder().setName("test_database_to_find_2").create(client, metaStore.getConf());
     testDatabases[3] =
-        new DatabaseBuilder().setName("test_database_hidden_1").build();
+        new DatabaseBuilder().setName("test_database_hidden_1").create(client, metaStore.getConf());
 
     // Create the databases, and reload them from the MetaStore
-    for(int i=0; i < testDatabases.length; i++) {
-      client.createDatabase(testDatabases[i]);
+    for (int i=0; i < testDatabases.length; i++) {
       testDatabases[i] = client.getDatabase(testDatabases[i].getName());
     }
   }
@@ -102,7 +115,6 @@ public class TestDatabases extends MetaStoreClientTest {
 
   /**
    * This test creates and queries a database and then drops it. Good for testing the happy path.
-   * @throws Exception
    */
   @Test
   public void testCreateGetDeleteDatabase() throws Exception {
@@ -127,10 +139,10 @@ public class TestDatabases extends MetaStoreClientTest {
 
   @Test
   public void testCreateDatabaseDefaultValues() throws Exception {
-    Database database = new Database();
-    database.setName("dummy");
+    Database database = new DatabaseBuilder()
+        .setName("dummy")
+        .create(client, metaStore.getConf());
 
-    client.createDatabase(database);
     Database createdDatabase = client.getDatabase(database.getName());
 
     Assert.assertNull("Comparing description", createdDatabase.getDescription());
@@ -139,7 +151,8 @@ public class TestDatabases extends MetaStoreClientTest {
     Assert.assertEquals("Comparing parameters", new HashMap<String, String>(),
         createdDatabase.getParameters());
     Assert.assertNull("Comparing privileges", createdDatabase.getPrivileges());
-    Assert.assertNull("Comparing owner name", createdDatabase.getOwnerName());
+    Assert.assertEquals("Comparing owner name", SecurityUtils.getUser(),
+        createdDatabase.getOwnerName());
     Assert.assertEquals("Comparing owner type", PrincipalType.USER, createdDatabase.getOwnerType());
   }
 
@@ -280,7 +293,7 @@ public class TestDatabases extends MetaStoreClientTest {
   @Test
   public void testDropDatabaseDeleteData() throws Exception {
     Database database = testDatabases[0];
-    Path dataFile = new Path(database.getLocationUri().toString() + "/dataFile");
+    Path dataFile = new Path(database.getLocationUri() + "/dataFile");
     metaStore.createFile(dataFile, "100");
 
     // Do not delete the data
@@ -318,8 +331,7 @@ public class TestDatabases extends MetaStoreClientTest {
             .setDbName(database.getName())
             .setTableName("test_table")
             .addCol("test_col", "int")
-            .build();
-    client.createTable(testTable);
+            .create(client, metaStore.getConf());
 
     client.dropDatabase(database.getName(), true, true, false);
   }
@@ -332,8 +344,7 @@ public class TestDatabases extends MetaStoreClientTest {
             .setDbName(database.getName())
             .setTableName("test_table")
             .addCol("test_col", "int")
-            .build();
-    client.createTable(testTable);
+            .create(client, metaStore.getConf());
 
     client.dropDatabase(database.getName(), true, true, true);
     Assert.assertFalse("The directory should be removed",
@@ -349,9 +360,8 @@ public class TestDatabases extends MetaStoreClientTest {
             .setDbName(database.getName())
             .setName("test_function")
             .setClass("org.apache.hadoop.hive.ql.udf.generic.GenericUDFUpper")
-            .build();
+            .create(client, metaStore.getConf());
 
-    client.createFunction(testFunction);
 
     client.dropDatabase(database.getName(), true, true, false);
   }
@@ -365,16 +375,14 @@ public class TestDatabases extends MetaStoreClientTest {
             .setDbName(database.getName())
             .setName("test_function")
             .setClass("org.apache.hadoop.hive.ql.udf.generic.GenericUDFUpper")
-            .build();
+            .create(client, metaStore.getConf());
 
-    client.createFunction(testFunction);
 
     client.dropDatabase(database.getName(), true, true, true);
     Assert.assertFalse("The directory should be removed",
         metaStore.isPathExists(new Path(database.getLocationUri())));
   }
 
-
   @Test
   public void testGetAllDatabases() throws Exception {
     List<String> allDatabases = client.getAllDatabases();
@@ -446,7 +454,7 @@ public class TestDatabases extends MetaStoreClientTest {
             .setDescription("dummy description 2")
             .addParam("param_key_1", "param_value_1_2")
             .addParam("param_key_2_3", "param_value_2_3")
-            .build();
+            .build(metaStore.getConf());
 
     client.alterDatabase(originalDatabase.getName(), newDatabase);
     Database alteredDatabase = client.getDatabase(newDatabase.getName());
@@ -460,6 +468,7 @@ public class TestDatabases extends MetaStoreClientTest {
     Database originalDatabase = client.getDatabase(database.getName());
     Database newDatabase = new Database();
     newDatabase.setName("new_name");
+    newDatabase.setCatalogName(DEFAULT_CATALOG_NAME);
 
     client.alterDatabase(originalDatabase.getName(), newDatabase);
     // The name should not be changed, so reload the db with the original name
@@ -480,7 +489,9 @@ public class TestDatabases extends MetaStoreClientTest {
 
   @Test(expected = NoSuchObjectException.class)
   public void testAlterDatabaseNoSuchDatabase() throws Exception {
-    Database newDatabase = new DatabaseBuilder().setName("test_database_altered").build();
+    Database newDatabase = new DatabaseBuilder()
+        .setName("test_database_altered")
+        .build(metaStore.getConf());
 
     client.alterDatabase("no_such_database", newDatabase);
   }
@@ -505,6 +516,131 @@ public class TestDatabases extends MetaStoreClientTest {
     Assert.assertEquals("Comparing databases", newDatabase, alteredDatabase);
   }
 
+  @Test
+  public void databasesInCatalogs() throws TException, URISyntaxException {
+    String catName = "mycatalog";
+    Catalog cat = new CatalogBuilder()
+        .setName(catName)
+        .setLocation(MetaStoreTestUtils.getTestWarehouseDir(catName))
+        .build();
+    client.createCatalog(cat);
+
+    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])
+        .setCatalogName(catName)
+        .create(client, metaStore.getConf());
+
+    // 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])
+        .setCatalogName(catName)
+        .setLocation(db1Location)
+        .create(client, metaStore.getConf());
+
+    Database fetched = client.getDatabase(catName, dbNames[0]);
+    String expectedLocation = new File(cat.getLocationUri(), dbNames[0] + ".db").toURI().toString();
+    Assert.assertEquals(expectedLocation, fetched.getLocationUri() + "/");
+    String db0Location = new URI(fetched.getLocationUri()).getPath();
+    File dir = new File(db0Location);
+    Assert.assertTrue(dir.exists() && dir.isDirectory());
+
+    fetched = client.getDatabase(catName, 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());
+
+    Set<String> fetchedDbs = new HashSet<>(client.getAllDatabases(catName));
+    Assert.assertEquals(3, fetchedDbs.size());
+    for (String dbName : dbNames) Assert.assertTrue(fetchedDbs.contains(dbName));
+
+    fetchedDbs = new HashSet<>(client.getAllDatabases());
+    Assert.assertEquals(5, fetchedDbs.size());
+    Assert.assertTrue(fetchedDbs.contains(Warehouse.DEFAULT_DATABASE_NAME));
+
+    // Intentionally using the deprecated method to make sure it returns correct results.
+    fetchedDbs = new HashSet<>(client.getAllDatabases());
+    Assert.assertEquals(5, fetchedDbs.size());
+    Assert.assertTrue(fetchedDbs.contains(Warehouse.DEFAULT_DATABASE_NAME));
+
+    fetchedDbs = new HashSet<>(client.getDatabases(catName, "d*"));
+    Assert.assertEquals(3, fetchedDbs.size());
+    for (String dbName : dbNames) Assert.assertTrue(fetchedDbs.contains(dbName));
+
+    fetchedDbs = new HashSet<>(client.getDatabases("d*"));
+    Assert.assertEquals(1, fetchedDbs.size());
+    Assert.assertTrue(fetchedDbs.contains(Warehouse.DEFAULT_DATABASE_NAME));
+
+    // Intentionally using the deprecated method to make sure it returns correct results.
+    fetchedDbs = new HashSet<>(client.getDatabases("d*"));
+    Assert.assertEquals(1, fetchedDbs.size());
+    Assert.assertTrue(fetchedDbs.contains(Warehouse.DEFAULT_DATABASE_NAME));
+
+    fetchedDbs = new HashSet<>(client.getDatabases(catName, "*1"));
+    Assert.assertEquals(1, fetchedDbs.size());
+    Assert.assertTrue(fetchedDbs.contains(dbNames[0]));
+
+    fetchedDbs = new HashSet<>(client.getDatabases("*9"));
+    Assert.assertEquals(0, fetchedDbs.size());
+
+    // Intentionally using the deprecated method to make sure it returns correct results.
+    fetchedDbs = new HashSet<>(client.getDatabases("*9"));
+    Assert.assertEquals(0, fetchedDbs.size());
+
+    fetchedDbs = new HashSet<>(client.getDatabases(catName, "*x"));
+    Assert.assertEquals(0, fetchedDbs.size());
+
+    // Check that dropping database from wrong catalog fails
+    try {
+      client.dropDatabase(dbNames[0], true, false, false);
+      Assert.fail();
+    } catch (NoSuchObjectException e) {
+      // NOP
+    }
+
+    // Check that dropping database from wrong catalog fails
+    try {
+      // Intentionally using deprecated method
+      client.dropDatabase(dbNames[0], true, false, false);
+      Assert.fail();
+    } catch (NoSuchObjectException e) {
+      // NOP
+    }
+
+    // Drop them from the proper catalog
+    client.dropDatabase(catName, dbNames[0], true, false, false);
+    dir = new File(db0Location);
+    Assert.assertFalse(dir.exists());
+
+    client.dropDatabase(catName, dbNames[1], true, false, false);
+    dir = new File(db1Location);
+    Assert.assertFalse(dir.exists());
+
+    fetchedDbs = new HashSet<>(client.getAllDatabases(catName));
+    Assert.assertEquals(1, fetchedDbs.size());
+  }
+
+  @Test(expected = InvalidObjectException.class)
+  public void createDatabaseInNonExistentCatalog() throws TException {
+    Database db = new DatabaseBuilder()
+        .setName("doomed")
+        .setCatalogName("nosuch")
+        .create(client, metaStore.getConf());
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void fetchDatabaseInNonExistentCatalog() throws TException {
+    client.getDatabase("nosuch", Warehouse.DEFAULT_DATABASE_NAME);
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void dropDatabaseInNonExistentCatalog() throws TException {
+    client.dropDatabase("nosuch", Warehouse.DEFAULT_DATABASE_NAME, true, false, false);
+  }
+
   private Database getDatabaseWithAllParametersSet() throws Exception {
     return new DatabaseBuilder()
                .setName("dummy")
@@ -514,6 +650,6 @@ public class TestDatabases extends MetaStoreClientTest {
                .setDescription("dummy description")
                .addParam("param_key_1", "param_value_1")
                .addParam("param_key_2", "param_value_2")
-               .build();
+               .build(metaStore.getConf());
   }
 }

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

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestDropPartitions.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestDropPartitions.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestDropPartitions.java
index d2ba4be..9037001 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestDropPartitions.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestDropPartitions.java
@@ -18,25 +18,31 @@
 package org.apache.hadoop.hive.metastore.client;
 
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
 import org.apache.hadoop.hive.metastore.PartitionDropOptions;
 import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
+import org.apache.hadoop.hive.metastore.api.Catalog;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.client.builder.CatalogBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.PartitionBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.minihms.AbstractMetaStoreService;
+import org.apache.thrift.TException;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -48,6 +54,8 @@ import org.junit.runners.Parameterized;
 
 import com.google.common.collect.Lists;
 
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
+
 /**
  * Tests for dropping partitions.
  */
@@ -69,7 +77,7 @@ public class TestDropPartitions extends MetaStoreClientTest {
   public static void startMetaStores() {
     Map<MetastoreConf.ConfVars, String> msConf = new HashMap<MetastoreConf.ConfVars, String>();
     // Enable trash, so it can be tested
-    Map<String, String> extraConf = new HashMap<String, String>();
+    Map<String, String> extraConf = new HashMap<>();
     extraConf.put("fs.trash.checkpoint.interval", "30");  // FS_TRASH_CHECKPOINT_INTERVAL_KEY
     extraConf.put("fs.trash.interval", "30");             // FS_TRASH_INTERVAL_KEY (hadoop-2)
     startMetaStores(msConf, extraConf);
@@ -89,8 +97,7 @@ public class TestDropPartitions extends MetaStoreClientTest {
     metaStore.cleanWarehouseDirs();
     Database db = new DatabaseBuilder().
         setName(DB_NAME).
-        build();
-    client.createDatabase(db);
+        create(client, metaStore.getConf());
 
     // Create test tables with 3 partitions
     createTable(TABLE_NAME, getYearAndMonthPartCols(), null);
@@ -489,7 +496,71 @@ public class TestDropPartitions extends MetaStoreClientTest {
     client.dropPartition(DB_NAME, TABLE_NAME, "", true);
   }
 
-    // Helper methods
+  @Test
+  public void otherCatalog() throws TException {
+    String catName = "drop_partition_catalog";
+    Catalog cat = new CatalogBuilder()
+        .setName(catName)
+        .setLocation(MetaStoreTestUtils.getTestWarehouseDir(catName))
+        .build();
+    client.createCatalog(cat);
+
+    String dbName = "drop_partition_database_in_other_catalog";
+    Database db = new DatabaseBuilder()
+        .setName(dbName)
+        .setCatalogName(catName)
+        .create(client, metaStore.getConf());
+
+    String tableName = "table_in_other_catalog";
+    Table table = new TableBuilder()
+        .inDb(db)
+        .setTableName(tableName)
+        .addCol("id", "int")
+        .addCol("name", "string")
+        .addPartCol("partcol", "string")
+        .create(client, metaStore.getConf());
+
+    Partition[] parts = new Partition[2];
+    for (int i = 0; i < parts.length; i++) {
+      parts[i] = new PartitionBuilder()
+          .inTable(table)
+          .addValue("a" + i)
+          .build(metaStore.getConf());
+    }
+    client.add_partitions(Arrays.asList(parts));
+    List<Partition> fetched = client.listPartitions(catName, dbName, tableName, (short)-1);
+    Assert.assertEquals(parts.length, fetched.size());
+
+    Assert.assertTrue(client.dropPartition(catName, dbName, tableName,
+        Collections.singletonList("a0"), PartitionDropOptions.instance().ifExists(false)));
+    try {
+      client.getPartition(catName, dbName, tableName, Collections.singletonList("a0"));
+      Assert.fail();
+    } catch (NoSuchObjectException e) {
+      // NOP
+    }
+
+    Assert.assertTrue(client.dropPartition(catName, dbName, tableName, "partcol=a1", true));
+    try {
+      client.getPartition(catName, dbName, tableName, Collections.singletonList("a1"));
+      Assert.fail();
+    } catch (NoSuchObjectException e) {
+      // NOP
+    }
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void testDropPartitionBogusCatalog() throws Exception {
+    client.dropPartition("nosuch", DB_NAME, TABLE_NAME, Lists.newArrayList("2017"), false);
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void testDropPartitionByNameBogusCatalog() throws Exception {
+    client.dropPartition("nosuch", DB_NAME, TABLE_NAME, "year=2017", false);
+  }
+
+
+  // Helper methods
 
   private Table createTable(String tableName, List<FieldSchema> partCols,
       Map<String, String> tableParams) throws Exception {
@@ -501,36 +572,33 @@ public class TestDropPartitions extends MetaStoreClientTest {
         .setPartCols(partCols)
         .setLocation(metaStore.getWarehouseRoot() + "/" + tableName)
         .setTableParams(tableParams)
-        .build();
-    client.createTable(table);
+        .create(client, metaStore.getConf());
     return table;
   }
 
   private Partition createPartition(List<String> values,
       List<FieldSchema> partCols) throws Exception {
-    Partition partition = new PartitionBuilder()
+    new PartitionBuilder()
         .setDbName(DB_NAME)
         .setTableName(TABLE_NAME)
         .setValues(values)
         .setCols(partCols)
-        .build();
-    client.add_partition(partition);
-    partition = client.getPartition(DB_NAME, TABLE_NAME, values);
+        .addToTable(client, metaStore.getConf());
+    Partition partition = client.getPartition(DB_NAME, TABLE_NAME, values);
     return partition;
   }
 
   private Partition createPartition(String tableName, String location, List<String> values,
       List<FieldSchema> partCols, Map<String, String> partParams) throws Exception {
-    Partition partition = new PartitionBuilder()
+    new PartitionBuilder()
         .setDbName(DB_NAME)
         .setTableName(tableName)
         .setValues(values)
         .setCols(partCols)
         .setLocation(location)
         .setPartParams(partParams)
-        .build();
-    client.add_partition(partition);
-    partition = client.getPartition(DB_NAME, tableName, values);
+        .addToTable(client, metaStore.getConf());
+    Partition partition = client.getPartition(DB_NAME, tableName, values);
     return partition;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestExchangePartitions.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestExchangePartitions.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestExchangePartitions.java
index 5a7c71c..473b171 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestExchangePartitions.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestExchangePartitions.java
@@ -27,7 +27,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
-import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
@@ -1162,10 +1161,9 @@ public class TestExchangePartitions extends MetaStoreClientTest {
 
   // Helper methods
   private void createDB(String dbName) throws TException {
-    Database db = new DatabaseBuilder()
+    new DatabaseBuilder()
         .setName(dbName)
-        .build();
-    client.createDatabase(db);
+        .create(client, metaStore.getConf());
   }
 
   private Table createSourceTable() throws Exception {
@@ -1186,14 +1184,13 @@ public class TestExchangePartitions extends MetaStoreClientTest {
 
   private Table createTable(String dbName, String tableName, List<FieldSchema> partCols,
       List<FieldSchema> cols, String location) throws Exception {
-    Table table = new TableBuilder()
+    new TableBuilder()
         .setDbName(dbName)
         .setTableName(tableName)
         .setCols(cols)
         .setPartCols(partCols)
         .setLocation(location)
-        .build();
-    client.createTable(table);
+        .create(client, metaStore.getConf());
     return client.getTable(dbName, tableName);
   }
 
@@ -1244,7 +1241,7 @@ public class TestExchangePartitions extends MetaStoreClientTest {
         .addStorageDescriptorParam("test_exch_sd_param_key", "test_exch_sd_param_value")
         .setCols(getYearMonthAndDayPartCols())
         .setLocation(location)
-        .build();
+        .build(metaStore.getConf());
     return partition;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestForeignKey.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestForeignKey.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestForeignKey.java
new file mode 100644
index 0000000..d8192b1
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestForeignKey.java
@@ -0,0 +1,535 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.client;
+
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
+import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
+import org.apache.hadoop.hive.metastore.api.Catalog;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.ForeignKeysRequest;
+import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.client.builder.CatalogBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.SQLForeignKeyBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.SQLPrimaryKeyBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
+import org.apache.hadoop.hive.metastore.minihms.AbstractMetaStoreService;
+import org.apache.thrift.TApplicationException;
+import org.apache.thrift.TException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.List;
+
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
+
+@RunWith(Parameterized.class)
+@Category(MetastoreCheckinTest.class)
+public class TestForeignKey extends MetaStoreClientTest {
+  private static final String OTHER_DATABASE = "test_fk_other_database";
+  private static final String OTHER_CATALOG = "test_fk_other_catalog";
+  private static final String DATABASE_IN_OTHER_CATALOG = "test_fk_database_in_other_catalog";
+  private final AbstractMetaStoreService metaStore;
+  private IMetaStoreClient client;
+  private Table[] testTables = new Table[4];
+  private Database inOtherCatalog;
+
+  public TestForeignKey(String name, AbstractMetaStoreService metaStore) throws Exception {
+    this.metaStore = metaStore;
+    this.metaStore.start();
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    // Get new client
+    client = metaStore.getClient();
+
+    // Clean up the database
+    client.dropDatabase(OTHER_DATABASE, true, true, true);
+    // Drop every table in the default database
+    for(String tableName : client.getAllTables(DEFAULT_DATABASE_NAME)) {
+      client.dropTable(DEFAULT_DATABASE_NAME, tableName, true, true, true);
+    }
+
+    client.dropDatabase(OTHER_CATALOG, DATABASE_IN_OTHER_CATALOG, true, true, true);
+    try {
+      client.dropCatalog(OTHER_CATALOG);
+    } catch (NoSuchObjectException e) {
+      // NOP
+    }
+
+    // Clean up trash
+    metaStore.cleanWarehouseDirs();
+
+    new DatabaseBuilder().setName(OTHER_DATABASE).create(client, metaStore.getConf());
+
+    Catalog cat = new CatalogBuilder()
+        .setName(OTHER_CATALOG)
+        .setLocation(MetaStoreTestUtils.getTestWarehouseDir(OTHER_CATALOG))
+        .build();
+    client.createCatalog(cat);
+
+    // For this one don't specify a location to make sure it gets put in the catalog directory
+    inOtherCatalog = new DatabaseBuilder()
+        .setName(DATABASE_IN_OTHER_CATALOG)
+        .setCatalogName(OTHER_CATALOG)
+        .create(client, metaStore.getConf());
+
+    testTables[0] =
+        new TableBuilder()
+            .setTableName("test_table_1")
+            .addCol("col1", "int")
+            .addCol("col2", "varchar(32)")
+            .create(client, metaStore.getConf());
+
+    testTables[1] =
+        new TableBuilder()
+            .setDbName(OTHER_DATABASE)
+            .setTableName("test_table_2")
+            .addCol("col1", "int")
+            .addCol("col2", "varchar(32)")
+            .create(client, metaStore.getConf());
+
+    testTables[2] =
+        new TableBuilder()
+            .inDb(inOtherCatalog)
+            .setTableName("test_table_3")
+            .addCol("col1", "int")
+            .addCol("col2", "varchar(32)")
+            .create(client, metaStore.getConf());
+
+    testTables[3] =
+        new TableBuilder()
+            .inDb(inOtherCatalog)
+            .setTableName("test_table_4")
+            .addCol("col1", "int")
+            .addCol("col2", "varchar(32)")
+            .create(client, metaStore.getConf());
+
+    // Reload tables from the MetaStore
+    for(int i=0; i < testTables.length; i++) {
+      testTables[i] = client.getTable(testTables[i].getCatName(), testTables[i].getDbName(),
+          testTables[i].getTableName());
+    }
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    try {
+      if (client != null) {
+        client.close();
+      }
+    } finally {
+      client = null;
+    }
+  }
+
+  @Test
+  public void createGetDrop() throws TException {
+    Table parentTable = testTables[1];
+    Table table = testTables[0];
+    // Make sure get on a table with no key returns empty list
+    ForeignKeysRequest rqst =
+        new ForeignKeysRequest(parentTable.getDbName(), parentTable.getTableName(),
+            table.getDbName(), table.getTableName());
+    rqst.setCatName(table.getCatName());
+    List<SQLForeignKey> fetched = client.getForeignKeys(rqst);
+    Assert.assertTrue(fetched.isEmpty());
+
+    // Single column unnamed primary key in default catalog and database
+    List<SQLPrimaryKey> pk = new SQLPrimaryKeyBuilder()
+        .onTable(parentTable)
+        .addColumn("col1")
+        .build(metaStore.getConf());
+    client.addPrimaryKey(pk);
+
+    List<SQLForeignKey> fk = new SQLForeignKeyBuilder()
+        .fromPrimaryKey(pk)
+        .onTable(table)
+        .addColumn("col1")
+        .build(metaStore.getConf());
+    client.addForeignKey(fk);
+
+
+    rqst = new ForeignKeysRequest(parentTable.getDbName(), parentTable.getTableName(),
+        table.getDbName(), table.getTableName());
+    rqst.setCatName(table.getCatName());
+    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("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("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(table.getTableName() + "_to_" + parentTable.getTableName() +
+        "_foreign_key", 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.getCatName(), 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());
+
+    // Make sure I can add it back
+    client.addForeignKey(fk);
+  }
+
+  @Test
+  public void createGetDrop2Column() throws TException {
+    Table parentTable = testTables[1];
+    Table table = testTables[0];
+    String constraintName = "2colfk";
+
+    // Single column unnamed primary key in default catalog and database
+    List<SQLPrimaryKey> pk = new SQLPrimaryKeyBuilder()
+        .onTable(parentTable)
+        .addColumn("col1")
+        .addColumn("col2")
+        .build(metaStore.getConf());
+    client.addPrimaryKey(pk);
+
+    List<SQLForeignKey> fk = new SQLForeignKeyBuilder()
+        .fromPrimaryKey(pk)
+        .onTable(table)
+        .addColumn("col1")
+        .addColumn("col2")
+        .setConstraintName(constraintName)
+        .build(metaStore.getConf());
+    client.addForeignKey(fk);
+
+
+    ForeignKeysRequest rqst = new ForeignKeysRequest(parentTable.getDbName(),
+        parentTable.getTableName(), table.getDbName(), table.getTableName());
+    rqst.setCatName(table.getCatName());
+    List<SQLForeignKey> fetched = client.getForeignKeys(rqst);
+    Assert.assertEquals(2, fetched.size());
+    Assert.assertEquals(table.getDbName(), fetched.get(0).getFktable_db());
+    Assert.assertEquals(table.getTableName(), fetched.get(0).getFktable_name());
+    Assert.assertEquals("col1", fetched.get(0).getFkcolumn_name());
+    Assert.assertEquals("col2", fetched.get(1).getFkcolumn_name());
+    Assert.assertEquals(parentTable.getDbName(), fetched.get(0).getPktable_db());
+    Assert.assertEquals(parentTable.getTableName(), fetched.get(0).getPktable_name());
+    Assert.assertEquals("col1", fetched.get(0).getFkcolumn_name());
+    Assert.assertEquals("col2", fetched.get(1).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.getCatName(), 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());
+
+    // Make sure I can add it back
+    client.addForeignKey(fk);
+  }
+
+  @Test
+  public void inOtherCatalog() 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("col1")
+        .build(metaStore.getConf());
+    client.addPrimaryKey(pk);
+
+    List<SQLForeignKey> fk = new SQLForeignKeyBuilder()
+        .fromPrimaryKey(pk)
+        .onTable(table)
+        .addColumn("col1")
+        .setConstraintName(constraintName)
+        .build(metaStore.getConf());
+    client.addForeignKey(fk);
+
+
+    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("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("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.getCatName(), 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());
+
+    // Make sure I can add it back
+    client.addForeignKey(fk);
+  }
+
+  @Test
+  public void createTableWithConstraints() throws TException {
+    String constraintName = "ctwckk";
+    Table parentTable = testTables[0];
+    Table table = new TableBuilder()
+        .setTableName("table_with_constraints")
+        .setDbName(parentTable.getDbName())
+        .addCol("col1", "int")
+        .addCol("col2", "varchar(32)")
+        .build(metaStore.getConf());
+
+    List<SQLPrimaryKey> pk = new SQLPrimaryKeyBuilder()
+        .onTable(parentTable)
+        .addColumn("col1")
+        .build(metaStore.getConf());
+    client.addPrimaryKey(pk);
+
+    List<SQLForeignKey> fk = new SQLForeignKeyBuilder()
+        .fromPrimaryKey(pk)
+        .onTable(table)
+        .addColumn("col1")
+        .setConstraintName(constraintName)
+        .build(metaStore.getConf());
+
+    client.createTableWithConstraints(table, null, fk, null, null, null, null);
+
+    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("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("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());
+    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());
+  }
+
+  @Test
+  public void createTableWithConstraintsInOtherCatalog() throws TException {
+    String constraintName = "ctwcocfk";
+    Table parentTable = testTables[2];
+    Table table = new TableBuilder()
+        .setTableName("table_with_constraints")
+        .inDb(inOtherCatalog)
+        .addCol("col1", "int")
+        .addCol("col2", "varchar(32)")
+        .build(metaStore.getConf());
+
+    List<SQLPrimaryKey> pk = new SQLPrimaryKeyBuilder()
+        .onTable(parentTable)
+        .addColumn("col1")
+        .build(metaStore.getConf());
+    client.addPrimaryKey(pk);
+
+    List<SQLForeignKey> fk = new SQLForeignKeyBuilder()
+        .fromPrimaryKey(pk)
+        .onTable(table)
+        .addColumn("col1")
+        .setConstraintName(constraintName)
+        .build(metaStore.getConf());
+
+    client.createTableWithConstraints(table, null, fk, null, null, null, null);
+
+    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("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("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());
+    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());
+  }
+
+  @Test(expected = MetaException.class)
+  public void noSuchPk() throws TException {
+    List<SQLPrimaryKey> pk = new SQLPrimaryKeyBuilder()
+        .onTable(testTables[1])
+        .addColumn("col1")
+        .build(metaStore.getConf());
+    // Don't actually create the key
+    List<SQLForeignKey> fk = new SQLForeignKeyBuilder()
+        .onTable(testTables[0])
+        .fromPrimaryKey(pk)
+        .addColumn("col2")
+        .build(metaStore.getConf());
+    client.addForeignKey(fk);
+    Assert.fail();
+  }
+
+  @Test
+  public void addNoSuchTable() throws TException {
+    Table parentTable = testTables[0];
+
+    List<SQLPrimaryKey> pk = new SQLPrimaryKeyBuilder()
+        .onTable(parentTable)
+        .addColumn("col1")
+        .build(metaStore.getConf());
+    client.addPrimaryKey(pk);
+
+    try {
+      List<SQLForeignKey> fk = new SQLForeignKeyBuilder()
+          .setTableName("nosuch")
+          .fromPrimaryKey(pk)
+          .addColumn("col2")
+          .build(metaStore.getConf());
+      client.addForeignKey(fk);
+      Assert.fail();
+    } catch (InvalidObjectException |TApplicationException e) {
+      // NOP
+    }
+  }
+
+  @Test
+  public void addNoSuchDb() throws TException {
+    Table parentTable = testTables[0];
+
+    List<SQLPrimaryKey> pk = new SQLPrimaryKeyBuilder()
+        .onTable(parentTable)
+        .addColumn("col1")
+        .build(metaStore.getConf());
+    client.addPrimaryKey(pk);
+
+    try {
+      List<SQLForeignKey> fk = new SQLForeignKeyBuilder()
+          .setTableName(testTables[0].getTableName())
+          .setDbName("nosuch")
+          .fromPrimaryKey(pk)
+          .addColumn("col2")
+          .build(metaStore.getConf());
+      client.addForeignKey(fk);
+      Assert.fail();
+    } catch (InvalidObjectException |TApplicationException e) {
+      // NOP
+    }
+  }
+
+  @Test
+  public void addNoSuchCatalog() throws TException {
+    Table parentTable = testTables[0];
+
+    List<SQLPrimaryKey> pk = new SQLPrimaryKeyBuilder()
+        .onTable(parentTable)
+        .addColumn("col1")
+        .build(metaStore.getConf());
+    client.addPrimaryKey(pk);
+
+    try {
+      List<SQLForeignKey> fk = new SQLForeignKeyBuilder()
+          .setTableName(testTables[0].getTableName())
+          .setDbName(testTables[0].getDbName())
+          .setCatName("nosuch")
+          .fromPrimaryKey(pk)
+          .addColumn("col2")
+          .build(metaStore.getConf());
+      client.addForeignKey(fk);
+      Assert.fail();
+    } catch (InvalidObjectException |TApplicationException e) {
+      // NOP
+    }
+  }
+
+  @Test
+  public void foreignKeyAcrossCatalogs() throws TException {
+    Table parentTable = testTables[2];
+    Table table = testTables[0];
+
+    // Single column unnamed primary key in default catalog and database
+    List<SQLPrimaryKey> pk = new SQLPrimaryKeyBuilder()
+        .onTable(parentTable)
+        .addColumn("col1")
+        .build(metaStore.getConf());
+    client.addPrimaryKey(pk);
+
+    try {
+      List<SQLForeignKey> fk = new SQLForeignKeyBuilder()
+          .fromPrimaryKey(pk)
+          .onTable(table)
+          .addColumn("col1")
+          .build(metaStore.getConf());
+      client.addForeignKey(fk);
+      Assert.fail();
+    } catch (InvalidObjectException |TApplicationException e) {
+      // NOP
+    }
+  }
+
+}


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
new file mode 100644
index 0000000..c1c39bf
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
@@ -0,0 +1,3286 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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 static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
+
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Proxy;
+import java.net.InetAddress;
+import java.net.URI;
+import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.NoSuchElementException;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import javax.security.auth.login.LoginException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.common.ValidTxnWriteIdList;
+import org.apache.hadoop.hive.common.ValidWriteIdList;
+import org.apache.hadoop.hive.metastore.api.*;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
+import org.apache.hadoop.hive.metastore.hooks.URIResolverHook;
+import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
+import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.JavaUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.metastore.utils.ObjectPair;
+import org.apache.hadoop.hive.metastore.utils.SecurityUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.thrift.TApplicationException;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.transport.TFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+
+/**
+ * Hive Metastore Client.
+ * The public implementation of IMetaStoreClient. Methods not inherited from IMetaStoreClient
+ * are not public and can change. Hence this is marked as unstable.
+ * For users who require retry mechanism when the connection between metastore and client is
+ * broken, RetryingMetaStoreClient class should be used.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class HiveMetaStoreClientPreCatalog implements IMetaStoreClient, AutoCloseable {
+  /**
+   * Capabilities of the current client. If this client talks to a MetaStore server in a manner
+   * implying the usage of some expanded features that require client-side support that this client
+   * doesn't have (e.g. a getting a table of a new type), it will get back failures when the
+   * capability checking is enabled (the default).
+   */
+  public final static ClientCapabilities VERSION = new ClientCapabilities(
+      Lists.newArrayList(ClientCapability.INSERT_ONLY_TABLES));
+  // Test capability for tests.
+  public final static ClientCapabilities TEST_VERSION = new ClientCapabilities(
+      Lists.newArrayList(ClientCapability.INSERT_ONLY_TABLES, ClientCapability.TEST_CAPABILITY));
+
+  ThriftHiveMetastore.Iface client = null;
+  private TTransport transport = null;
+  private boolean isConnected = false;
+  private URI metastoreUris[];
+  private final HiveMetaHookLoader hookLoader;
+  protected final Configuration conf;  // Keep a copy of HiveConf so if Session conf changes, we may need to get a new HMS client.
+  protected boolean fastpath = false;
+  private String tokenStrForm;
+  private final boolean localMetaStore;
+  private final MetaStoreFilterHook filterHook;
+  private final URIResolverHook uriResolverHook;
+  private final int fileMetadataBatchSize;
+
+  private Map<String, String> currentMetaVars;
+
+  private static final AtomicInteger connCount = new AtomicInteger(0);
+
+  // for thrift connects
+  private int retries = 5;
+  private long retryDelaySeconds = 0;
+  private final ClientCapabilities version;
+
+  static final protected Logger LOG = LoggerFactory.getLogger(HiveMetaStoreClientPreCatalog.class);
+
+  public HiveMetaStoreClientPreCatalog(Configuration conf) throws MetaException {
+    this(conf, null, true);
+  }
+
+  public HiveMetaStoreClientPreCatalog(Configuration conf, HiveMetaHookLoader hookLoader) throws MetaException {
+    this(conf, hookLoader, true);
+  }
+
+  public HiveMetaStoreClientPreCatalog(Configuration conf, HiveMetaHookLoader hookLoader, Boolean allowEmbedded)
+    throws MetaException {
+
+    this.hookLoader = hookLoader;
+    if (conf == null) {
+      conf = MetastoreConf.newMetastoreConf();
+      this.conf = conf;
+    } else {
+      this.conf = new Configuration(conf);
+    }
+    version = MetastoreConf.getBoolVar(conf, ConfVars.HIVE_IN_TEST) ? TEST_VERSION : VERSION;
+    filterHook = loadFilterHooks();
+    uriResolverHook = loadUriResolverHook();
+    fileMetadataBatchSize = MetastoreConf.getIntVar(
+        conf, ConfVars.BATCH_RETRIEVE_OBJECTS_MAX);
+
+    String msUri = MetastoreConf.getVar(conf, ConfVars.THRIFT_URIS);
+    localMetaStore = MetastoreConf.isEmbeddedMetaStore(msUri);
+    if (localMetaStore) {
+      if (!allowEmbedded) {
+        throw new MetaException("Embedded metastore is not allowed here. Please configure "
+            + ConfVars.THRIFT_URIS.toString() + "; it is currently set to [" + msUri + "]");
+      }
+      // instantiate the metastore server handler directly instead of connecting
+      // through the network
+      client = HiveMetaStore.newRetryingHMSHandler("hive client", this.conf, true);
+      // Initialize materializations invalidation cache (only for local metastore)
+      MaterializationsInvalidationCache.get().init(conf, (IHMSHandler) client);
+      isConnected = true;
+      snapshotActiveConf();
+      return;
+    }
+
+    // get the number retries
+    retries = MetastoreConf.getIntVar(conf, ConfVars.THRIFT_CONNECTION_RETRIES);
+    retryDelaySeconds = MetastoreConf.getTimeVar(conf,
+        ConfVars.CLIENT_CONNECT_RETRY_DELAY, TimeUnit.SECONDS);
+
+    // user wants file store based configuration
+    if (MetastoreConf.getVar(conf, ConfVars.THRIFT_URIS) != null) {
+      resolveUris();
+    } else {
+      LOG.error("NOT getting uris from conf");
+      throw new MetaException("MetaStoreURIs not found in conf file");
+    }
+
+    //If HADOOP_PROXY_USER is set in env or property,
+    //then need to create metastore client that proxies as that user.
+    String HADOOP_PROXY_USER = "HADOOP_PROXY_USER";
+    String proxyUser = System.getenv(HADOOP_PROXY_USER);
+    if (proxyUser == null) {
+      proxyUser = System.getProperty(HADOOP_PROXY_USER);
+    }
+    //if HADOOP_PROXY_USER is set, create DelegationToken using real user
+    if(proxyUser != null) {
+      LOG.info(HADOOP_PROXY_USER + " is set. Using delegation "
+          + "token for HiveMetaStore connection.");
+      try {
+        UserGroupInformation.getLoginUser().getRealUser().doAs(
+            new PrivilegedExceptionAction<Void>() {
+              @Override
+              public Void run() throws Exception {
+                open();
+                return null;
+              }
+            });
+        String delegationTokenPropString = "DelegationTokenForHiveMetaStoreServer";
+        String delegationTokenStr = getDelegationToken(proxyUser, proxyUser);
+        SecurityUtils.setTokenStr(UserGroupInformation.getCurrentUser(), delegationTokenStr,
+            delegationTokenPropString);
+        MetastoreConf.setVar(this.conf, ConfVars.TOKEN_SIGNATURE, delegationTokenPropString);
+        close();
+      } catch (Exception e) {
+        LOG.error("Error while setting delegation token for " + proxyUser, e);
+        if(e instanceof MetaException) {
+          throw (MetaException)e;
+        } else {
+          throw new MetaException(e.getMessage());
+        }
+      }
+    }
+    // finally open the store
+    open();
+  }
+
+  private void resolveUris() throws MetaException {
+    String metastoreUrisString[] =  MetastoreConf.getVar(conf,
+            ConfVars.THRIFT_URIS).split(",");
+
+    List<URI> metastoreURIArray = new ArrayList<URI>();
+    try {
+      int i = 0;
+      for (String s : metastoreUrisString) {
+        URI tmpUri = new URI(s);
+        if (tmpUri.getScheme() == null) {
+          throw new IllegalArgumentException("URI: " + s
+                  + " does not have a scheme");
+        }
+        if (uriResolverHook != null) {
+          metastoreURIArray.addAll(uriResolverHook.resolveURI(tmpUri));
+        } else {
+          metastoreURIArray.add(new URI(
+                  tmpUri.getScheme(),
+                  tmpUri.getUserInfo(),
+                  HadoopThriftAuthBridge.getBridge().getCanonicalHostName(tmpUri.getHost()),
+                  tmpUri.getPort(),
+                  tmpUri.getPath(),
+                  tmpUri.getQuery(),
+                  tmpUri.getFragment()
+          ));
+        }
+      }
+      metastoreUris = new URI[metastoreURIArray.size()];
+      for (int j = 0; j < metastoreURIArray.size(); j++) {
+        metastoreUris[j] = metastoreURIArray.get(j);
+      }
+
+      if (MetastoreConf.getVar(conf, ConfVars.THRIFT_URI_SELECTION).equalsIgnoreCase("RANDOM")) {
+        List uriList = Arrays.asList(metastoreUris);
+        Collections.shuffle(uriList);
+        metastoreUris = (URI[]) uriList.toArray();
+      }
+    } catch (IllegalArgumentException e) {
+      throw (e);
+    } catch (Exception e) {
+      MetaStoreUtils.logAndThrowMetaException(e);
+    }
+  }
+
+
+  private MetaStoreFilterHook loadFilterHooks() throws IllegalStateException {
+    Class<? extends MetaStoreFilterHook> authProviderClass = MetastoreConf.
+        getClass(conf, ConfVars.FILTER_HOOK, DefaultMetaStoreFilterHookImpl.class,
+            MetaStoreFilterHook.class);
+    String msg = "Unable to create instance of " + authProviderClass.getName() + ": ";
+    try {
+      Constructor<? extends MetaStoreFilterHook> constructor =
+          authProviderClass.getConstructor(Configuration.class);
+      return constructor.newInstance(conf);
+    } catch (NoSuchMethodException | SecurityException | IllegalAccessException | InstantiationException | IllegalArgumentException | InvocationTargetException e) {
+      throw new IllegalStateException(msg + e.getMessage(), e);
+    }
+  }
+
+  //multiple clients may initialize the hook at the same time
+  synchronized private URIResolverHook loadUriResolverHook() throws IllegalStateException {
+
+    String uriResolverClassName =
+            MetastoreConf.getAsString(conf, ConfVars.URI_RESOLVER);
+    if (uriResolverClassName.equals("")) {
+      return null;
+    } else {
+      LOG.info("Loading uri resolver" + uriResolverClassName);
+      try {
+        Class<?> uriResolverClass = Class.forName(uriResolverClassName, true,
+                JavaUtils.getClassLoader());
+        return (URIResolverHook) ReflectionUtils.newInstance(uriResolverClass, null);
+      } catch (Exception e) {
+        LOG.error("Exception loading uri resolver hook" + e);
+        return null;
+      }
+    }
+  }
+
+  /**
+   * Swaps the first element of the metastoreUris array with a random element from the
+   * remainder of the array.
+   */
+  private void promoteRandomMetaStoreURI() {
+    if (metastoreUris.length <= 1) {
+      return;
+    }
+    Random rng = new Random();
+    int index = rng.nextInt(metastoreUris.length - 1) + 1;
+    URI tmp = metastoreUris[0];
+    metastoreUris[0] = metastoreUris[index];
+    metastoreUris[index] = tmp;
+  }
+
+  @VisibleForTesting
+  public TTransport getTTransport() {
+    return transport;
+  }
+
+  @Override
+  public boolean isLocalMetaStore() {
+    return localMetaStore;
+  }
+
+  @Override
+  public boolean isCompatibleWith(Configuration conf) {
+    // Make a copy of currentMetaVars, there is a race condition that
+	// currentMetaVars might be changed during the execution of the method
+    Map<String, String> currentMetaVarsCopy = currentMetaVars;
+    if (currentMetaVarsCopy == null) {
+      return false; // recreate
+    }
+    boolean compatible = true;
+    for (ConfVars oneVar : MetastoreConf.metaVars) {
+      // Since metaVars are all of different types, use string for comparison
+      String oldVar = currentMetaVarsCopy.get(oneVar.getVarname());
+      String newVar = MetastoreConf.getAsString(conf, oneVar);
+      if (oldVar == null ||
+          (oneVar.isCaseSensitive() ? !oldVar.equals(newVar) : !oldVar.equalsIgnoreCase(newVar))) {
+        LOG.info("Mestastore configuration " + oneVar.toString() +
+            " changed from " + oldVar + " to " + newVar);
+        compatible = false;
+      }
+    }
+    return compatible;
+  }
+
+  @Override
+  public void setHiveAddedJars(String addedJars) {
+    MetastoreConf.setVar(conf, ConfVars.ADDED_JARS, addedJars);
+  }
+
+  @Override
+  public void reconnect() throws MetaException {
+    if (localMetaStore) {
+      // For direct DB connections we don't yet support reestablishing connections.
+      throw new MetaException("For direct MetaStore DB connections, we don't support retries" +
+          " at the client level.");
+    } else {
+      close();
+
+      if (uriResolverHook != null) {
+        //for dynamic uris, re-lookup if there are new metastore locations
+        resolveUris();
+      }
+
+      if (MetastoreConf.getVar(conf, ConfVars.THRIFT_URI_SELECTION).equalsIgnoreCase("RANDOM")) {
+        // Swap the first element of the metastoreUris[] with a random element from the rest
+        // of the array. Rationale being that this method will generally be called when the default
+        // connection has died and the default connection is likely to be the first array element.
+        promoteRandomMetaStoreURI();
+      }
+      open();
+    }
+  }
+
+  /**
+   * @param dbname
+   * @param tbl_name
+   * @param new_tbl
+   * @throws InvalidOperationException
+   * @throws MetaException
+   * @throws TException
+   * @see
+   *   org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#alter_table(
+   *   java.lang.String, java.lang.String,
+   *   org.apache.hadoop.hive.metastore.api.Table)
+   */
+  @Override
+  public void alter_table(String dbname, String tbl_name, Table new_tbl)
+      throws InvalidOperationException, MetaException, TException {
+    alter_table_with_environmentContext(dbname, tbl_name, new_tbl, null);
+  }
+
+  @Override
+  public void alter_table(String defaultDatabaseName, String tblName, Table table,
+      boolean cascade) throws InvalidOperationException, MetaException, TException {
+    EnvironmentContext environmentContext = new EnvironmentContext();
+    if (cascade) {
+      environmentContext.putToProperties(StatsSetupConst.CASCADE, StatsSetupConst.TRUE);
+    }
+    alter_table_with_environmentContext(defaultDatabaseName, tblName, table, environmentContext);
+  }
+
+  @Override
+  public void alter_table_with_environmentContext(String dbname, String tbl_name, Table new_tbl,
+      EnvironmentContext envContext) throws InvalidOperationException, MetaException, TException {
+    client.alter_table_with_environment_context(dbname, tbl_name, new_tbl, envContext);
+  }
+
+  /**
+   * @param dbname
+   * @param name
+   * @param part_vals
+   * @param newPart
+   * @throws InvalidOperationException
+   * @throws MetaException
+   * @throws TException
+   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#rename_partition(
+   *      java.lang.String, java.lang.String, java.util.List, org.apache.hadoop.hive.metastore.api.Partition)
+   */
+  @Override
+  public void renamePartition(final String dbname, final String name, final List<String> part_vals, final Partition newPart)
+      throws InvalidOperationException, MetaException, TException {
+    client.rename_partition(dbname, name, part_vals, newPart);
+  }
+
+  private void open() throws MetaException {
+    isConnected = false;
+    TTransportException tte = null;
+    boolean useSSL = MetastoreConf.getBoolVar(conf, ConfVars.USE_SSL);
+    boolean useSasl = MetastoreConf.getBoolVar(conf, ConfVars.USE_THRIFT_SASL);
+    boolean useFramedTransport = MetastoreConf.getBoolVar(conf, ConfVars.USE_THRIFT_FRAMED_TRANSPORT);
+    boolean useCompactProtocol = MetastoreConf.getBoolVar(conf, ConfVars.USE_THRIFT_COMPACT_PROTOCOL);
+    int clientSocketTimeout = (int) MetastoreConf.getTimeVar(conf,
+        ConfVars.CLIENT_SOCKET_TIMEOUT, TimeUnit.MILLISECONDS);
+
+    for (int attempt = 0; !isConnected && attempt < retries; ++attempt) {
+      for (URI store : metastoreUris) {
+        LOG.info("Trying to connect to metastore with URI " + store);
+
+        try {
+          if (useSSL) {
+            try {
+              String trustStorePath = MetastoreConf.getVar(conf, ConfVars.SSL_TRUSTSTORE_PATH).trim();
+              if (trustStorePath.isEmpty()) {
+                throw new IllegalArgumentException(ConfVars.SSL_TRUSTSTORE_PATH.toString()
+                    + " Not configured for SSL connection");
+              }
+              String trustStorePassword =
+                  MetastoreConf.getPassword(conf, MetastoreConf.ConfVars.SSL_TRUSTSTORE_PASSWORD);
+
+              // Create an SSL socket and connect
+              transport = SecurityUtils.getSSLSocket(store.getHost(), store.getPort(), clientSocketTimeout,
+                  trustStorePath, trustStorePassword );
+              LOG.info("Opened an SSL connection to metastore, current connections: " + connCount.incrementAndGet());
+            } catch(IOException e) {
+              throw new IllegalArgumentException(e);
+            } catch(TTransportException e) {
+              tte = e;
+              throw new MetaException(e.toString());
+            }
+          } else {
+            transport = new TSocket(store.getHost(), store.getPort(), clientSocketTimeout);
+          }
+
+          if (useSasl) {
+            // Wrap thrift connection with SASL for secure connection.
+            try {
+              HadoopThriftAuthBridge.Client authBridge =
+                HadoopThriftAuthBridge.getBridge().createClient();
+
+              // check if we should use delegation tokens to authenticate
+              // the call below gets hold of the tokens if they are set up by hadoop
+              // this should happen on the map/reduce tasks if the client added the
+              // tokens into hadoop's credential store in the front end during job
+              // submission.
+              String tokenSig = MetastoreConf.getVar(conf, ConfVars.TOKEN_SIGNATURE);
+              // tokenSig could be null
+              tokenStrForm = SecurityUtils.getTokenStrForm(tokenSig);
+
+              if(tokenStrForm != null) {
+                LOG.info("HMSC::open(): Found delegation token. Creating DIGEST-based thrift connection.");
+                // authenticate using delegation tokens via the "DIGEST" mechanism
+                transport = authBridge.createClientTransport(null, store.getHost(),
+                    "DIGEST", tokenStrForm, transport,
+                        MetaStoreUtils.getMetaStoreSaslProperties(conf, useSSL));
+              } else {
+                LOG.info("HMSC::open(): Could not find delegation token. Creating KERBEROS-based thrift connection.");
+                String principalConfig =
+                    MetastoreConf.getVar(conf, ConfVars.KERBEROS_PRINCIPAL);
+                transport = authBridge.createClientTransport(
+                    principalConfig, store.getHost(), "KERBEROS", null,
+                    transport, MetaStoreUtils.getMetaStoreSaslProperties(conf, useSSL));
+              }
+            } catch (IOException ioe) {
+              LOG.error("Couldn't create client transport", ioe);
+              throw new MetaException(ioe.toString());
+            }
+          } else {
+            if (useFramedTransport) {
+              transport = new TFramedTransport(transport);
+            }
+          }
+
+          final TProtocol protocol;
+          if (useCompactProtocol) {
+            protocol = new TCompactProtocol(transport);
+          } else {
+            protocol = new TBinaryProtocol(transport);
+          }
+          client = new ThriftHiveMetastore.Client(protocol);
+          try {
+            if (!transport.isOpen()) {
+              transport.open();
+              LOG.info("Opened a connection to metastore, current connections: " + connCount.incrementAndGet());
+            }
+            isConnected = true;
+          } catch (TTransportException e) {
+            tte = e;
+            if (LOG.isDebugEnabled()) {
+              LOG.warn("Failed to connect to the MetaStore Server...", e);
+            } else {
+              // Don't print full exception trace if DEBUG is not on.
+              LOG.warn("Failed to connect to the MetaStore Server...");
+            }
+          }
+
+          if (isConnected && !useSasl && MetastoreConf.getBoolVar(conf, ConfVars.EXECUTE_SET_UGI)){
+            // Call set_ugi, only in unsecure mode.
+            try {
+              UserGroupInformation ugi = SecurityUtils.getUGI();
+              client.set_ugi(ugi.getUserName(), Arrays.asList(ugi.getGroupNames()));
+            } catch (LoginException e) {
+              LOG.warn("Failed to do login. set_ugi() is not successful, " +
+                       "Continuing without it.", e);
+            } catch (IOException e) {
+              LOG.warn("Failed to find ugi of client set_ugi() is not successful, " +
+                  "Continuing without it.", e);
+            } catch (TException e) {
+              LOG.warn("set_ugi() not successful, Likely cause: new client talking to old server. "
+                  + "Continuing without it.", e);
+            }
+          }
+        } catch (MetaException e) {
+          LOG.error("Unable to connect to metastore with URI " + store
+                    + " in attempt " + attempt, e);
+        }
+        if (isConnected) {
+          break;
+        }
+      }
+      // Wait before launching the next round of connection retries.
+      if (!isConnected && retryDelaySeconds > 0) {
+        try {
+          LOG.info("Waiting " + retryDelaySeconds + " seconds before next connection attempt.");
+          Thread.sleep(retryDelaySeconds * 1000);
+        } catch (InterruptedException ignore) {}
+      }
+    }
+
+    if (!isConnected) {
+      throw new MetaException("Could not connect to meta store using any of the URIs provided." +
+        " Most recent failure: " + StringUtils.stringifyException(tte));
+    }
+
+    snapshotActiveConf();
+
+    LOG.info("Connected to metastore.");
+  }
+
+  private void snapshotActiveConf() {
+    currentMetaVars = new HashMap<>(MetastoreConf.metaVars.length);
+    for (ConfVars oneVar : MetastoreConf.metaVars) {
+      currentMetaVars.put(oneVar.getVarname(), MetastoreConf.getAsString(conf, oneVar));
+    }
+  }
+
+  @Override
+  public String getTokenStrForm() throws IOException {
+    return tokenStrForm;
+   }
+
+  @Override
+  public void close() {
+    isConnected = false;
+    currentMetaVars = null;
+    try {
+      if (null != client) {
+        client.shutdown();
+      }
+    } catch (TException e) {
+      LOG.debug("Unable to shutdown metastore client. Will try closing transport directly.", e);
+    }
+    // Transport would have got closed via client.shutdown(), so we dont need this, but
+    // just in case, we make this call.
+    if ((transport != null) && transport.isOpen()) {
+      transport.close();
+      LOG.info("Closed a connection to metastore, current connections: " + connCount.decrementAndGet());
+    }
+  }
+
+  @Override
+  public void setMetaConf(String key, String value) throws TException {
+    client.setMetaConf(key, value);
+  }
+
+  @Override
+  public String getMetaConf(String key) throws TException {
+    return client.getMetaConf(key);
+  }
+
+  /**
+   * @param new_part
+   * @return the added partition
+   * @throws InvalidObjectException
+   * @throws AlreadyExistsException
+   * @throws MetaException
+   * @throws TException
+   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#add_partition(org.apache.hadoop.hive.metastore.api.Partition)
+   */
+  @Override
+  public Partition add_partition(Partition new_part) throws TException {
+    return add_partition(new_part, null);
+  }
+
+  public Partition add_partition(Partition new_part, EnvironmentContext envContext)
+      throws TException {
+    Partition p = client.add_partition_with_environment_context(new_part, envContext);
+    return fastpath ? p : deepCopy(p);
+  }
+
+  /**
+   * @param new_parts
+   * @throws InvalidObjectException
+   * @throws AlreadyExistsException
+   * @throws MetaException
+   * @throws TException
+   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#add_partitions(List)
+   */
+  @Override
+  public int add_partitions(List<Partition> new_parts) throws TException {
+    return client.add_partitions(new_parts);
+  }
+
+  @Override
+  public List<Partition> add_partitions(
+      List<Partition> parts, boolean ifNotExists, boolean needResults) throws TException {
+    if (parts.isEmpty()) {
+      return needResults ? new ArrayList<>() : null;
+    }
+    Partition part = parts.get(0);
+    AddPartitionsRequest req = new AddPartitionsRequest(
+        part.getDbName(), part.getTableName(), parts, ifNotExists);
+    req.setNeedResult(needResults);
+    AddPartitionsResult result = client.add_partitions_req(req);
+    return needResults ? filterHook.filterPartitions(result.getPartitions()) : null;
+  }
+
+  @Override
+  public int add_partitions_pspec(PartitionSpecProxy partitionSpec) throws TException {
+    return client.add_partitions_pspec(partitionSpec.toPartitionSpec());
+  }
+
+  /**
+   * @param table_name
+   * @param db_name
+   * @param part_vals
+   * @return the appended partition
+   * @throws InvalidObjectException
+   * @throws AlreadyExistsException
+   * @throws MetaException
+   * @throws TException
+   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#append_partition(java.lang.String,
+   *      java.lang.String, java.util.List)
+   */
+  @Override
+  public Partition appendPartition(String db_name, String table_name,
+      List<String> part_vals) throws TException {
+    return appendPartition(db_name, table_name, part_vals, null);
+  }
+
+  public Partition appendPartition(String db_name, String table_name, List<String> part_vals,
+      EnvironmentContext envContext) throws TException {
+    Partition p = client.append_partition_with_environment_context(db_name, table_name,
+        part_vals, envContext);
+    return fastpath ? p : deepCopy(p);
+  }
+
+  @Override
+  public Partition appendPartition(String dbName, String tableName, String partName)
+      throws TException {
+    return appendPartition(dbName, tableName, partName, (EnvironmentContext)null);
+  }
+
+  public Partition appendPartition(String dbName, String tableName, String partName,
+      EnvironmentContext envContext) throws TException {
+    Partition p = client.append_partition_by_name_with_environment_context(dbName, tableName,
+        partName, envContext);
+    return fastpath ? p : deepCopy(p);
+  }
+
+  /**
+   * Exchange the partition between two tables
+   * @param partitionSpecs partitions specs of the parent partition to be exchanged
+   * @param destDb the db of the destination table
+   * @param destinationTableName the destination table name
+   * @return new partition after exchanging
+   */
+  @Override
+  public Partition exchange_partition(Map<String, String> partitionSpecs,
+      String sourceDb, String sourceTable, String destDb,
+      String destinationTableName) throws MetaException,
+      NoSuchObjectException, InvalidObjectException, TException {
+    return client.exchange_partition(partitionSpecs, sourceDb, sourceTable,
+        destDb, destinationTableName);
+  }
+
+  /**
+   * Exchange the partitions between two tables
+   * @param partitionSpecs partitions specs of the parent partition to be exchanged
+   * @param destDb the db of the destination table
+   * @param destinationTableName the destination table name
+   * @return new partitions after exchanging
+   */
+  @Override
+  public List<Partition> exchange_partitions(Map<String, String> partitionSpecs,
+      String sourceDb, String sourceTable, String destDb,
+      String destinationTableName) throws MetaException,
+      NoSuchObjectException, InvalidObjectException, TException {
+    return client.exchange_partitions(partitionSpecs, sourceDb, sourceTable,
+        destDb, destinationTableName);
+  }
+
+  @Override
+  public void validatePartitionNameCharacters(List<String> partVals)
+      throws TException, MetaException {
+    client.partition_name_has_valid_characters(partVals, true);
+  }
+
+  /**
+   * Create a new Database
+   * @param db
+   * @throws AlreadyExistsException
+   * @throws InvalidObjectException
+   * @throws MetaException
+   * @throws TException
+   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#create_database(Database)
+   */
+  @Override
+  public void createDatabase(Database db)
+      throws AlreadyExistsException, InvalidObjectException, MetaException, TException {
+    client.create_database(db);
+  }
+
+  /**
+   * @param tbl
+   * @throws MetaException
+   * @throws NoSuchObjectException
+   * @throws TException
+   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#create_table(org.apache.hadoop.hive.metastore.api.Table)
+   */
+  @Override
+  public void createTable(Table tbl) throws AlreadyExistsException,
+      InvalidObjectException, MetaException, NoSuchObjectException, TException {
+    createTable(tbl, null);
+  }
+
+  public void createTable(Table tbl, EnvironmentContext envContext) throws AlreadyExistsException,
+      InvalidObjectException, MetaException, NoSuchObjectException, TException {
+    HiveMetaHook hook = getHook(tbl);
+    if (hook != null) {
+      hook.preCreateTable(tbl);
+    }
+    boolean success = false;
+    try {
+      // Subclasses can override this step (for example, for temporary tables)
+      create_table_with_environment_context(tbl, envContext);
+      if (hook != null) {
+        hook.commitCreateTable(tbl);
+      }
+      success = true;
+    }
+    finally {
+      if (!success && (hook != null)) {
+        try {
+          hook.rollbackCreateTable(tbl);
+        } catch (Exception e){
+          LOG.error("Create rollback failed with", e);
+        }
+      }
+    }
+  }
+
+  @Override
+  public void createTableWithConstraints(Table tbl,
+      List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys,
+      List<SQLUniqueConstraint> uniqueConstraints,
+      List<SQLNotNullConstraint> notNullConstraints,
+      List<SQLDefaultConstraint> defaultConstraints,
+      List<SQLCheckConstraint> checkConstraints)
+        throws AlreadyExistsException, InvalidObjectException,
+        MetaException, NoSuchObjectException, TException {
+    HiveMetaHook hook = getHook(tbl);
+    if (hook != null) {
+      hook.preCreateTable(tbl);
+    }
+    boolean success = false;
+    try {
+      // Subclasses can override this step (for example, for temporary tables)
+      client.create_table_with_constraints(tbl, primaryKeys, foreignKeys,
+          uniqueConstraints, notNullConstraints, defaultConstraints, checkConstraints);
+      if (hook != null) {
+        hook.commitCreateTable(tbl);
+      }
+      success = true;
+    } finally {
+      if (!success && (hook != null)) {
+        hook.rollbackCreateTable(tbl);
+      }
+    }
+  }
+
+  @Override
+  public void dropConstraint(String dbName, String tableName, String constraintName) throws
+    NoSuchObjectException, MetaException, TException {
+    client.drop_constraint(new DropConstraintRequest(dbName, tableName, constraintName));
+  }
+
+  @Override
+  public void addPrimaryKey(List<SQLPrimaryKey> primaryKeyCols) throws
+    NoSuchObjectException, MetaException, TException {
+    client.add_primary_key(new AddPrimaryKeyRequest(primaryKeyCols));
+  }
+
+  @Override
+  public void addForeignKey(List<SQLForeignKey> foreignKeyCols) throws
+    NoSuchObjectException, MetaException, TException {
+    client.add_foreign_key(new AddForeignKeyRequest(foreignKeyCols));
+  }
+
+  @Override
+  public void addUniqueConstraint(List<SQLUniqueConstraint> uniqueConstraintCols) throws
+    NoSuchObjectException, MetaException, TException {
+    client.add_unique_constraint(new AddUniqueConstraintRequest(uniqueConstraintCols));
+  }
+
+  @Override
+  public void addNotNullConstraint(List<SQLNotNullConstraint> notNullConstraintCols) throws
+    NoSuchObjectException, MetaException, TException {
+    client.add_not_null_constraint(new AddNotNullConstraintRequest(notNullConstraintCols));
+  }
+
+  @Override
+  public void addDefaultConstraint(List<SQLDefaultConstraint> defaultConstraints) throws
+      NoSuchObjectException, MetaException, TException {
+    client.add_default_constraint(new AddDefaultConstraintRequest(defaultConstraints));
+  }
+
+  @Override
+  public void addCheckConstraint(List<SQLCheckConstraint> checkConstraints) throws MetaException,
+      NoSuchObjectException, TException {
+    client.add_check_constraint(new AddCheckConstraintRequest(checkConstraints));
+  }
+
+  /**
+   * @param type
+   * @return true or false
+   * @throws AlreadyExistsException
+   * @throws InvalidObjectException
+   * @throws MetaException
+   * @throws TException
+   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#create_type(org.apache.hadoop.hive.metastore.api.Type)
+   */
+  public boolean createType(Type type) throws AlreadyExistsException,
+      InvalidObjectException, MetaException, TException {
+    return client.create_type(type);
+  }
+
+  /**
+   * @param name
+   * @throws NoSuchObjectException
+   * @throws InvalidOperationException
+   * @throws MetaException
+   * @throws TException
+   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#drop_database(java.lang.String, boolean, boolean)
+   */
+  @Override
+  public void dropDatabase(String name)
+      throws NoSuchObjectException, InvalidOperationException, MetaException, TException {
+    dropDatabase(name, true, false, false);
+  }
+
+  @Override
+  public void dropDatabase(String name, boolean deleteData, boolean ignoreUnknownDb)
+      throws NoSuchObjectException, InvalidOperationException, MetaException, TException {
+    dropDatabase(name, deleteData, ignoreUnknownDb, false);
+  }
+
+  @Override
+  public void dropDatabase(String name, boolean deleteData, boolean ignoreUnknownDb, boolean cascade)
+      throws NoSuchObjectException, InvalidOperationException, MetaException, TException {
+    try {
+      getDatabase(name);
+    } catch (NoSuchObjectException e) {
+      if (!ignoreUnknownDb) {
+        throw e;
+      }
+      return;
+    }
+
+    if (cascade) {
+       List<String> tableList = getAllTables(name);
+       for (String table : tableList) {
+         try {
+           // Subclasses can override this step (for example, for temporary tables)
+           dropTable(name, table, deleteData, true);
+         } catch (UnsupportedOperationException e) {
+           // Ignore Index tables, those will be dropped with parent tables
+         }
+        }
+    }
+    client.drop_database(name, deleteData, cascade);
+  }
+
+  /**
+   * @param tbl_name
+   * @param db_name
+   * @param part_vals
+   * @return true or false
+   * @throws NoSuchObjectException
+   * @throws MetaException
+   * @throws TException
+   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#drop_partition(java.lang.String,
+   *      java.lang.String, java.util.List, boolean)
+   */
+  public boolean dropPartition(String db_name, String tbl_name,
+      List<String> part_vals) throws NoSuchObjectException, MetaException,
+      TException {
+    return dropPartition(db_name, tbl_name, part_vals, true, null);
+  }
+
+  public boolean dropPartition(String db_name, String tbl_name, List<String> part_vals,
+      EnvironmentContext env_context) throws NoSuchObjectException, MetaException, TException {
+    return dropPartition(db_name, tbl_name, part_vals, true, env_context);
+  }
+
+  @Override
+  public boolean dropPartition(String dbName, String tableName, String partName, boolean deleteData)
+      throws NoSuchObjectException, MetaException, TException {
+    return dropPartition(dbName, tableName, partName, deleteData, null);
+  }
+
+  private static EnvironmentContext getEnvironmentContextWithIfPurgeSet() {
+    Map<String, String> warehouseOptions = new HashMap<>();
+    warehouseOptions.put("ifPurge", "TRUE");
+    return new EnvironmentContext(warehouseOptions);
+  }
+
+  /*
+  public boolean dropPartition(String dbName, String tableName, String partName, boolean deleteData, boolean ifPurge)
+      throws NoSuchObjectException, MetaException, TException {
+
+    return dropPartition(dbName, tableName, partName, deleteData,
+                         ifPurge? getEnvironmentContextWithIfPurgeSet() : null);
+  }
+  */
+
+  public boolean dropPartition(String dbName, String tableName, String partName, boolean deleteData,
+      EnvironmentContext envContext) throws NoSuchObjectException, MetaException, TException {
+    return client.drop_partition_by_name_with_environment_context(dbName, tableName, partName,
+        deleteData, envContext);
+  }
+
+  /**
+   * @param db_name
+   * @param tbl_name
+   * @param part_vals
+   * @param deleteData
+   *          delete the underlying data or just delete the table in metadata
+   * @return true or false
+   * @throws NoSuchObjectException
+   * @throws MetaException
+   * @throws TException
+   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#drop_partition(java.lang.String,
+   *      java.lang.String, java.util.List, boolean)
+   */
+  @Override
+  public boolean dropPartition(String db_name, String tbl_name,
+      List<String> part_vals, boolean deleteData) throws NoSuchObjectException,
+      MetaException, TException {
+    return dropPartition(db_name, tbl_name, part_vals, deleteData, null);
+  }
+
+  @Override
+  public boolean dropPartition(String db_name, String tbl_name,
+      List<String> part_vals, PartitionDropOptions options) throws TException {
+    return dropPartition(db_name, tbl_name, part_vals, options.deleteData,
+                         options.purgeData? getEnvironmentContextWithIfPurgeSet() : null);
+  }
+
+  public boolean dropPartition(String db_name, String tbl_name, List<String> part_vals,
+      boolean deleteData, EnvironmentContext envContext) throws NoSuchObjectException,
+      MetaException, TException {
+    return client.drop_partition_with_environment_context(db_name, tbl_name, part_vals, deleteData,
+        envContext);
+  }
+
+  @Override
+  public List<Partition> dropPartitions(String dbName, String tblName,
+                                        List<ObjectPair<Integer, byte[]>> partExprs, PartitionDropOptions options)
+      throws TException {
+    RequestPartsSpec rps = new RequestPartsSpec();
+    List<DropPartitionsExpr> exprs = new ArrayList<>(partExprs.size());
+    for (ObjectPair<Integer, byte[]> partExpr : partExprs) {
+      DropPartitionsExpr dpe = new DropPartitionsExpr();
+      dpe.setExpr(partExpr.getSecond());
+      dpe.setPartArchiveLevel(partExpr.getFirst());
+      exprs.add(dpe);
+    }
+    rps.setExprs(exprs);
+    DropPartitionsRequest req = new DropPartitionsRequest(dbName, tblName, rps);
+    req.setDeleteData(options.deleteData);
+    req.setNeedResult(options.returnResults);
+    req.setIfExists(options.ifExists);
+    if (options.purgeData) {
+      LOG.info("Dropped partitions will be purged!");
+      req.setEnvironmentContext(getEnvironmentContextWithIfPurgeSet());
+    }
+    return client.drop_partitions_req(req).getPartitions();
+  }
+
+  @Override
+  public List<Partition> dropPartitions(String dbName, String tblName,
+      List<ObjectPair<Integer, byte[]>> partExprs, boolean deleteData,
+      boolean ifExists, boolean needResult) throws NoSuchObjectException, MetaException, TException {
+
+    return dropPartitions(dbName, tblName, partExprs,
+                          PartitionDropOptions.instance()
+                                              .deleteData(deleteData)
+                                              .ifExists(ifExists)
+                                              .returnResults(needResult));
+
+  }
+
+  @Override
+  public List<Partition> dropPartitions(String dbName, String tblName,
+      List<ObjectPair<Integer, byte[]>> partExprs, boolean deleteData,
+      boolean ifExists) throws NoSuchObjectException, MetaException, TException {
+    // By default, we need the results from dropPartitions();
+    return dropPartitions(dbName, tblName, partExprs,
+                          PartitionDropOptions.instance()
+                                              .deleteData(deleteData)
+                                              .ifExists(ifExists));
+  }
+
+  /**
+   * {@inheritDoc}
+   * @see #dropTable(String, String, boolean, boolean, EnvironmentContext)
+   */
+  @Override
+  public void dropTable(String dbname, String name, boolean deleteData,
+      boolean ignoreUnknownTab) throws MetaException, TException,
+      NoSuchObjectException, UnsupportedOperationException {
+    dropTable(dbname, name, deleteData, ignoreUnknownTab, null);
+  }
+
+  /**
+   * Drop the table and choose whether to save the data in the trash.
+   * @param ifPurge completely purge the table (skipping trash) while removing
+   *                data from warehouse
+   * @see #dropTable(String, String, boolean, boolean, EnvironmentContext)
+   */
+  @Override
+  public void dropTable(String dbname, String name, boolean deleteData,
+      boolean ignoreUnknownTab, boolean ifPurge)
+      throws MetaException, TException, NoSuchObjectException, UnsupportedOperationException {
+    //build new environmentContext with ifPurge;
+    EnvironmentContext envContext = null;
+    if(ifPurge){
+      Map<String, String> warehouseOptions;
+      warehouseOptions = new HashMap<>();
+      warehouseOptions.put("ifPurge", "TRUE");
+      envContext = new EnvironmentContext(warehouseOptions);
+    }
+    dropTable(dbname, name, deleteData, ignoreUnknownTab, envContext);
+  }
+
+  /**
+   * @see #dropTable(String, String, boolean, boolean, EnvironmentContext)
+   */
+  @Override
+  public void dropTable(String dbname, String name)
+      throws NoSuchObjectException, MetaException, TException {
+    dropTable(dbname, name, true, true, null);
+  }
+
+  /**
+   * Drop the table and choose whether to: delete the underlying table data;
+   * throw if the table doesn't exist; save the data in the trash.
+   *
+   * @param dbname
+   * @param name
+   * @param deleteData
+   *          delete the underlying data or just delete the table in metadata
+   * @param ignoreUnknownTab
+   *          don't throw if the requested table doesn't exist
+   * @param envContext
+   *          for communicating with thrift
+   * @throws MetaException
+   *           could not drop table properly
+   * @throws NoSuchObjectException
+   *           the table wasn't found
+   * @throws TException
+   *           a thrift communication error occurred
+   * @throws UnsupportedOperationException
+   *           dropping an index table is not allowed
+   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#drop_table(java.lang.String,
+   *      java.lang.String, boolean)
+   */
+  public void dropTable(String dbname, String name, boolean deleteData,
+      boolean ignoreUnknownTab, EnvironmentContext envContext) throws MetaException, TException,
+      NoSuchObjectException, UnsupportedOperationException {
+    Table tbl;
+    try {
+      tbl = getTable(dbname, name);
+    } catch (NoSuchObjectException e) {
+      if (!ignoreUnknownTab) {
+        throw e;
+      }
+      return;
+    }
+    HiveMetaHook hook = getHook(tbl);
+    if (hook != null) {
+      hook.preDropTable(tbl);
+    }
+    boolean success = false;
+    try {
+      drop_table_with_environment_context(dbname, name, deleteData, envContext);
+      if (hook != null) {
+        hook.commitDropTable(tbl, deleteData || (envContext != null && "TRUE".equals(envContext.getProperties().get("ifPurge"))));
+      }
+      success=true;
+    } catch (NoSuchObjectException e) {
+      if (!ignoreUnknownTab) {
+        throw e;
+      }
+    } finally {
+      if (!success && (hook != null)) {
+        hook.rollbackDropTable(tbl);
+      }
+    }
+  }
+
+  /**
+   * Truncate the table/partitions in the DEFAULT database.
+   * @param dbName
+   *          The db to which the table to be truncate belongs to
+   * @param tableName
+   *          The table to truncate
+   * @param partNames
+   *          List of partitions to truncate. NULL will truncate the whole table/all partitions
+   * @throws MetaException
+   * @throws TException
+   *           Could not truncate table properly.
+   */
+  @Override
+  public void truncateTable(String dbName, String tableName, List<String> partNames) throws MetaException, TException {
+    client.truncate_table(dbName, tableName, partNames);
+  }
+
+  /**
+   * Recycles the files recursively from the input path to the cmroot directory either by copying or moving it.
+   *
+   * @param request Inputs for path of the data files to be recycled to cmroot and
+   *                isPurge flag when set to true files which needs to be recycled are not moved to Trash
+   * @return Response which is currently void
+   */
+  @Override
+  public CmRecycleResponse recycleDirToCmPath(CmRecycleRequest request) throws MetaException, TException {
+    return client.cm_recycle(request);
+  }
+
+  /**
+   * @param type
+   * @return true if the type is dropped
+   * @throws MetaException
+   * @throws TException
+   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#drop_type(java.lang.String)
+   */
+  public boolean dropType(String type) throws NoSuchObjectException, MetaException, TException {
+    return client.drop_type(type);
+  }
+
+  /**
+   * @param name
+   * @return map of types
+   * @throws MetaException
+   * @throws TException
+   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#get_type_all(java.lang.String)
+   */
+  public Map<String, Type> getTypeAll(String name) throws MetaException,
+      TException {
+    Map<String, Type> result = null;
+    Map<String, Type> fromClient = client.get_type_all(name);
+    if (fromClient != null) {
+      result = new LinkedHashMap<>();
+      for (String key : fromClient.keySet()) {
+        result.put(key, deepCopy(fromClient.get(key)));
+      }
+    }
+    return result;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public List<String> getDatabases(String databasePattern)
+    throws MetaException {
+    try {
+      return filterHook.filterDatabases(client.get_databases(databasePattern));
+    } catch (Exception e) {
+      MetaStoreUtils.logAndThrowMetaException(e);
+    }
+    return null;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public List<String> getAllDatabases() throws MetaException {
+    try {
+      return filterHook.filterDatabases(client.get_all_databases());
+    } catch (Exception e) {
+      MetaStoreUtils.logAndThrowMetaException(e);
+    }
+    return null;
+  }
+
+  /**
+   * @param tbl_name
+   * @param db_name
+   * @param max_parts
+   * @return list of partitions
+   * @throws NoSuchObjectException
+   * @throws MetaException
+   * @throws TException
+   */
+  @Override
+  public List<Partition> listPartitions(String db_name, String tbl_name,
+      short max_parts) throws NoSuchObjectException, MetaException, TException {
+    List<Partition> parts = client.get_partitions(db_name, tbl_name, max_parts);
+    return fastpath ? parts : deepCopyPartitions(filterHook.filterPartitions(parts));
+  }
+
+  @Override
+  public PartitionSpecProxy listPartitionSpecs(String dbName, String tableName, int maxParts) throws TException {
+    return PartitionSpecProxy.Factory.get(filterHook.filterPartitionSpecs(
+        client.get_partitions_pspec(dbName, tableName, maxParts)));
+  }
+
+  @Override
+  public List<Partition> listPartitions(String db_name, String tbl_name,
+      List<String> part_vals, short max_parts)
+      throws NoSuchObjectException, MetaException, TException {
+    List<Partition> parts = client.get_partitions_ps(db_name, tbl_name, part_vals, max_parts);
+    return fastpath ? parts : deepCopyPartitions(filterHook.filterPartitions(parts));
+  }
+
+  @Override
+  public List<Partition> listPartitionsWithAuthInfo(String db_name,
+      String tbl_name, short max_parts, String user_name, List<String> group_names)
+       throws NoSuchObjectException, MetaException, TException {
+    List<Partition> parts = client.get_partitions_with_auth(db_name, tbl_name, max_parts,
+        user_name, group_names);
+    return fastpath ? parts :deepCopyPartitions(filterHook.filterPartitions(parts));
+  }
+
+  @Override
+  public List<Partition> listPartitionsWithAuthInfo(String db_name,
+      String tbl_name, List<String> part_vals, short max_parts,
+      String user_name, List<String> group_names) throws NoSuchObjectException,
+      MetaException, TException {
+    List<Partition> parts = client.get_partitions_ps_with_auth(db_name,
+        tbl_name, part_vals, max_parts, user_name, group_names);
+    return fastpath ? parts : deepCopyPartitions(filterHook.filterPartitions(parts));
+  }
+
+  /**
+   * Get list of partitions matching specified filter
+   * @param db_name the database name
+   * @param tbl_name the table name
+   * @param filter the filter string,
+   *    for example "part1 = \"p1_abc\" and part2 &lt;= "\p2_test\"". Filtering can
+   *    be done only on string partition keys.
+   * @param max_parts the maximum number of partitions to return,
+   *    all partitions are returned if -1 is passed
+   * @return list of partitions
+   * @throws MetaException
+   * @throws NoSuchObjectException
+   * @throws TException
+   */
+  @Override
+  public List<Partition> listPartitionsByFilter(String db_name, String tbl_name,
+      String filter, short max_parts) throws MetaException,
+         NoSuchObjectException, TException {
+    List<Partition> parts = client.get_partitions_by_filter(db_name, tbl_name, filter, max_parts);
+    return fastpath ? parts :deepCopyPartitions(filterHook.filterPartitions(parts));
+  }
+
+  @Override
+  public PartitionSpecProxy listPartitionSpecsByFilter(String db_name, String tbl_name,
+                                                       String filter, int max_parts) throws MetaException,
+         NoSuchObjectException, TException {
+    return PartitionSpecProxy.Factory.get(filterHook.filterPartitionSpecs(
+        client.get_part_specs_by_filter(db_name, tbl_name, filter, max_parts)));
+  }
+
+  @Override
+  public boolean listPartitionsByExpr(String db_name, String tbl_name, byte[] expr,
+      String default_partition_name, short max_parts, List<Partition> result)
+          throws TException {
+    assert result != null;
+    PartitionsByExprRequest req = new PartitionsByExprRequest(
+        db_name, tbl_name, ByteBuffer.wrap(expr));
+    if (default_partition_name != null) {
+      req.setDefaultPartitionName(default_partition_name);
+    }
+    if (max_parts >= 0) {
+      req.setMaxParts(max_parts);
+    }
+    PartitionsByExprResult r;
+    try {
+      r = client.get_partitions_by_expr(req);
+    } catch (TApplicationException te) {
+      // TODO: backward compat for Hive <= 0.12. Can be removed later.
+      if (te.getType() != TApplicationException.UNKNOWN_METHOD
+          && te.getType() != TApplicationException.WRONG_METHOD_NAME) {
+        throw te;
+      }
+      throw new IncompatibleMetastoreException(
+          "Metastore doesn't support listPartitionsByExpr: " + te.getMessage());
+    }
+    if (fastpath) {
+      result.addAll(r.getPartitions());
+    } else {
+      r.setPartitions(filterHook.filterPartitions(r.getPartitions()));
+      // TODO: in these methods, do we really need to deepcopy?
+      deepCopyPartitions(r.getPartitions(), result);
+    }
+    return !r.isSetHasUnknownPartitions() || r.isHasUnknownPartitions(); // Assume the worst.
+  }
+
+  /**
+   * @param name
+   * @return the database
+   * @throws NoSuchObjectException
+   * @throws MetaException
+   * @throws TException
+   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#get_database(java.lang.String)
+   */
+  @Override
+  public Database getDatabase(String name) throws NoSuchObjectException,
+      MetaException, TException {
+    Database d = client.get_database(name);
+    return fastpath ? d :deepCopy(filterHook.filterDatabase(d));
+  }
+
+  /**
+   * @param tbl_name
+   * @param db_name
+   * @param part_vals
+   * @return the partition
+   * @throws MetaException
+   * @throws TException
+   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#get_partition(java.lang.String,
+   *      java.lang.String, java.util.List)
+   */
+  @Override
+  public Partition getPartition(String db_name, String tbl_name,
+      List<String> part_vals) throws NoSuchObjectException, MetaException, TException {
+    Partition p = client.get_partition(db_name, tbl_name, part_vals);
+    return fastpath ? p : deepCopy(filterHook.filterPartition(p));
+  }
+
+  @Override
+  public List<Partition> getPartitionsByNames(String db_name, String tbl_name,
+      List<String> part_names) throws NoSuchObjectException, MetaException, TException {
+    List<Partition> parts = client.get_partitions_by_names(db_name, tbl_name, part_names);
+    return fastpath ? parts : deepCopyPartitions(filterHook.filterPartitions(parts));
+  }
+
+  @Override
+  public PartitionValuesResponse listPartitionValues(PartitionValuesRequest request)
+      throws MetaException, TException, NoSuchObjectException {
+    return client.get_partition_values(request);
+  }
+
+  @Override
+  public Partition getPartitionWithAuthInfo(String db_name, String tbl_name,
+      List<String> part_vals, String user_name, List<String> group_names)
+      throws MetaException, UnknownTableException, NoSuchObjectException,
+      TException {
+    Partition p = client.get_partition_with_auth(db_name, tbl_name, part_vals, user_name,
+        group_names);
+    return fastpath ? p : deepCopy(filterHook.filterPartition(p));
+  }
+
+  /**
+   * @param name
+   * @param dbname
+   * @return the table
+   * @throws NoSuchObjectException
+   * @throws MetaException
+   * @throws TException
+   * @throws NoSuchObjectException
+   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#get_table(java.lang.String,
+   *      java.lang.String)
+   */
+  @Override
+  public Table getTable(String dbname, String name) throws MetaException,
+      TException, NoSuchObjectException {
+    GetTableRequest req = new GetTableRequest(dbname, name);
+    req.setCapabilities(version);
+    Table t = client.get_table_req(req).getTable();
+    return fastpath ? t : deepCopy(filterHook.filterTable(t));
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public List<Table> getTableObjectsByName(String dbName, List<String> tableNames)
+      throws MetaException, InvalidOperationException, UnknownDBException, TException {
+    GetTablesRequest req = new GetTablesRequest(dbName);
+    req.setTblNames(tableNames);
+    req.setCapabilities(version);
+    List<Table> tabs = client.get_table_objects_by_name_req(req).getTables();
+    return fastpath ? tabs : deepCopyTables(filterHook.filterTables(tabs));
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public Map<String, Materialization> getMaterializationsInvalidationInfo(String dbName, List<String> viewNames)
+      throws MetaException, InvalidOperationException, UnknownDBException, TException {
+    return client.get_materialization_invalidation_info(
+        dbName, filterHook.filterTableNames(null, dbName, viewNames));
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public void updateCreationMetadata(String dbName, String tableName, CreationMetadata cm)
+      throws MetaException, InvalidOperationException, UnknownDBException, TException {
+    client.update_creation_metadata(null, dbName, tableName, cm);
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public List<String> listTableNamesByFilter(String dbName, String filter, short maxTables)
+      throws MetaException, TException, InvalidOperationException, UnknownDBException {
+    return filterHook.filterTableNames(null, dbName,
+        client.get_table_names_by_filter(dbName, filter, maxTables));
+  }
+
+  /**
+   * @param name
+   * @return the type
+   * @throws MetaException
+   * @throws TException
+   * @throws NoSuchObjectException
+   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#get_type(java.lang.String)
+   */
+  public Type getType(String name) throws NoSuchObjectException, MetaException, TException {
+    return deepCopy(client.get_type(name));
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public List<String> getTables(String dbname, String tablePattern) throws MetaException {
+    try {
+      return filterHook.filterTableNames(null, dbname, client.get_tables(dbname, tablePattern));
+    } catch (Exception e) {
+      MetaStoreUtils.logAndThrowMetaException(e);
+    }
+    return null;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public List<String> getTables(String dbname, String tablePattern, TableType tableType) throws MetaException {
+    try {
+      return filterHook.filterTableNames(null, dbname,
+          client.get_tables_by_type(dbname, tablePattern, tableType.toString()));
+    } catch (Exception e) {
+      MetaStoreUtils.logAndThrowMetaException(e);
+    }
+    return null;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public List<String> getMaterializedViewsForRewriting(String dbname) throws MetaException {
+    try {
+      return filterHook.filterTableNames(null, dbname, client.get_materialized_views_for_rewriting(dbname));
+    } catch (Exception e) {
+      MetaStoreUtils.logAndThrowMetaException(e);
+    }
+    return null;
+  }
+
+  @Override
+  public List<TableMeta> getTableMeta(String dbPatterns, String tablePatterns, List<String> tableTypes)
+      throws MetaException {
+    try {
+      return filterNames(client.get_table_meta(dbPatterns, tablePatterns, tableTypes));
+    } catch (Exception e) {
+      MetaStoreUtils.logAndThrowMetaException(e);
+    }
+    return null;
+  }
+
+  private List<TableMeta> filterNames(List<TableMeta> metas) throws MetaException {
+    Map<String, TableMeta> sources = new LinkedHashMap<>();
+    Map<String, List<String>> dbTables = new LinkedHashMap<>();
+    for (TableMeta meta : metas) {
+      sources.put(meta.getDbName() + "." + meta.getTableName(), meta);
+      List<String> tables = dbTables.get(meta.getDbName());
+      if (tables == null) {
+        dbTables.put(meta.getDbName(), tables = new ArrayList<>());
+      }
+      tables.add(meta.getTableName());
+    }
+    List<TableMeta> filtered = new ArrayList<>();
+    for (Map.Entry<String, List<String>> entry : dbTables.entrySet()) {
+      for (String table : filterHook.filterTableNames(null, entry.getKey(), entry.getValue())) {
+        filtered.add(sources.get(entry.getKey() + "." + table));
+      }
+    }
+    return filtered;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public List<String> getAllTables(String dbname) throws MetaException {
+    try {
+      return filterHook.filterTableNames(null, dbname, client.get_all_tables(dbname));
+    } catch (Exception e) {
+      MetaStoreUtils.logAndThrowMetaException(e);
+    }
+    return null;
+  }
+
+  @Override
+  public boolean tableExists(String databaseName, String tableName) throws MetaException,
+      TException, UnknownDBException {
+    try {
+      GetTableRequest req = new GetTableRequest(databaseName, tableName);
+      req.setCapabilities(version);
+      return filterHook.filterTable(client.get_table_req(req).getTable()) != null;
+    } catch (NoSuchObjectException e) {
+      return false;
+    }
+  }
+
+  @Override
+  public List<String> listPartitionNames(String dbName, String tblName,
+      short max) throws NoSuchObjectException, MetaException, TException {
+    return filterHook.filterPartitionNames(null, dbName, tblName,
+        client.get_partition_names(dbName, tblName, max));
+  }
+
+  @Override
+  public List<String> listPartitionNames(String db_name, String tbl_name,
+      List<String> part_vals, short max_parts)
+      throws MetaException, TException, NoSuchObjectException {
+    return filterHook.filterPartitionNames(null, db_name, tbl_name,
+        client.get_partition_names_ps(db_name, tbl_name, part_vals, max_parts));
+  }
+
+  /**
+   * Get number of partitions matching specified filter
+   * @param db_name the database name
+   * @param tbl_name the table name
+   * @param filter the filter string,
+   *    for example "part1 = \"p1_abc\" and part2 &lt;= "\p2_test\"". Filtering can
+   *    be done only on string partition keys.
+   * @return number of partitions
+   * @throws MetaException
+   * @throws NoSuchObjectException
+   * @throws TException
+   */
+  @Override
+  public int getNumPartitionsByFilter(String db_name, String tbl_name,
+                                      String filter) throws MetaException,
+          NoSuchObjectException, TException {
+    return client.get_num_partitions_by_filter(db_name, tbl_name, filter);
+  }
+
+  @Override
+  public void alter_partition(String dbName, String tblName, Partition newPart)
+      throws InvalidOperationException, MetaException, TException {
+    client.alter_partition_with_environment_context(dbName, tblName, newPart, null);
+  }
+
+  @Override
+  public void alter_partition(String dbName, String tblName, Partition newPart, EnvironmentContext environmentContext)
+      throws InvalidOperationException, MetaException, TException {
+    client.alter_partition_with_environment_context(dbName, tblName, newPart, environmentContext);
+  }
+
+  @Override
+  public void alter_partitions(String dbName, String tblName, List<Partition> newParts)
+      throws InvalidOperationException, MetaException, TException {
+    client.alter_partitions_with_environment_context(dbName, tblName, newParts, null);
+  }
+
+  @Override
+  public void alter_partitions(String dbName, String tblName, List<Partition> newParts, EnvironmentContext environmentContext)
+  throws InvalidOperationException, MetaException, TException {
+    client.alter_partitions_with_environment_context(dbName, tblName, newParts, environmentContext);
+  }
+
+  @Override
+  public void alterDatabase(String dbName, Database db)
+      throws MetaException, NoSuchObjectException, TException {
+    client.alter_database(dbName, db);
+  }
+  /**
+   * @param db
+   * @param tableName
+   * @throws UnknownTableException
+   * @throws UnknownDBException
+   * @throws MetaException
+   * @throws TException
+   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#get_fields(java.lang.String,
+   *      java.lang.String)
+   */
+  @Override
+  public List<FieldSchema> getFields(String db, String tableName)
+      throws MetaException, TException, UnknownTableException,
+      UnknownDBException {
+    List<FieldSchema> fields = client.get_fields(db, tableName);
+    return fastpath ? fields : deepCopyFieldSchemas(fields);
+  }
+
+  @Override
+  public List<SQLPrimaryKey> getPrimaryKeys(PrimaryKeysRequest req)
+    throws MetaException, NoSuchObjectException, TException {
+    return client.get_primary_keys(req).getPrimaryKeys();
+  }
+
+  @Override
+  public List<SQLForeignKey> getForeignKeys(ForeignKeysRequest req) throws MetaException,
+    NoSuchObjectException, TException {
+    return client.get_foreign_keys(req).getForeignKeys();
+  }
+
+  @Override
+  public List<SQLUniqueConstraint> getUniqueConstraints(UniqueConstraintsRequest req)
+    throws MetaException, NoSuchObjectException, TException {
+    return client.get_unique_constraints(req).getUniqueConstraints();
+  }
+
+  @Override
+  public List<SQLNotNullConstraint> getNotNullConstraints(NotNullConstraintsRequest req)
+    throws MetaException, NoSuchObjectException, TException {
+    return client.get_not_null_constraints(req).getNotNullConstraints();
+  }
+
+  @Override
+  public List<SQLDefaultConstraint> getDefaultConstraints(DefaultConstraintsRequest req)
+      throws MetaException, NoSuchObjectException, TException {
+    return client.get_default_constraints(req).getDefaultConstraints();
+  }
+
+  @Override
+  public List<SQLCheckConstraint> getCheckConstraints(CheckConstraintsRequest request) throws
+      MetaException, NoSuchObjectException, TException {
+    return client.get_check_constraints(request).getCheckConstraints();
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  @Deprecated
+  //use setPartitionColumnStatistics instead
+  public boolean updateTableColumnStatistics(ColumnStatistics statsObj)
+    throws NoSuchObjectException, InvalidObjectException, MetaException, TException,
+    InvalidInputException{
+    return client.update_table_column_statistics(statsObj);
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  @Deprecated
+  //use setPartitionColumnStatistics instead
+  public boolean updatePartitionColumnStatistics(ColumnStatistics statsObj)
+    throws NoSuchObjectException, InvalidObjectException, MetaException, TException,
+    InvalidInputException{
+    return client.update_partition_column_statistics(statsObj);
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public boolean setPartitionColumnStatistics(SetPartitionsStatsRequest request)
+    throws NoSuchObjectException, InvalidObjectException, MetaException, TException,
+    InvalidInputException{
+    return client.set_aggr_stats_for(request);
+  }
+
+  @Override
+  public void flushCache() {
+    try {
+      client.flushCache();
+    } catch (TException e) {
+      // Not much we can do about it honestly
+      LOG.warn("Got error flushing the cache", e);
+    }
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public List<ColumnStatisticsObj> getTableColumnStatistics(String dbName, String tableName,
+      List<String> colNames) throws NoSuchObjectException, MetaException, TException,
+      InvalidInputException, InvalidObjectException {
+    return client.get_table_statistics_req(
+        new TableStatsRequest(dbName, tableName, colNames)).getTableStats();
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public Map<String, List<ColumnStatisticsObj>> getPartitionColumnStatistics(
+      String dbName, String tableName, List<String> partNames, List<String> colNames)
+          throws NoSuchObjectException, MetaException, TException {
+    return client.get_partitions_statistics_req(
+        new PartitionsStatsRequest(dbName, tableName, colNames, partNames)).getPartStats();
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public boolean deletePartitionColumnStatistics(String dbName, String tableName, String partName,
+    String colName) throws NoSuchObjectException, InvalidObjectException, MetaException,
+    TException, InvalidInputException
+  {
+    return client.delete_partition_column_statistics(dbName, tableName, partName, colName);
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public boolean deleteTableColumnStatistics(String dbName, String tableName, String colName)
+    throws NoSuchObjectException, InvalidObjectException, MetaException, TException,
+    InvalidInputException
+  {
+    return client.delete_table_column_statistics(dbName, tableName, colName);
+  }
+
+  /**
+   * @param db
+   * @param tableName
+   * @throws UnknownTableException
+   * @throws UnknownDBException
+   * @throws MetaException
+   * @throws TException
+   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#get_schema(java.lang.String,
+   *      java.lang.String)
+   */
+  @Override
+  public List<FieldSchema> getSchema(String db, String tableName)
+      throws MetaException, TException, UnknownTableException,
+      UnknownDBException {
+      EnvironmentContext envCxt = null;
+      String addedJars = MetastoreConf.getVar(conf, ConfVars.ADDED_JARS);
+      if(org.apache.commons.lang.StringUtils.isNotBlank(addedJars)) {
+         Map<String, String> props = new HashMap<String, String>();
+         props.put("hive.added.jars.path", addedJars);
+         envCxt = new EnvironmentContext(props);
+       }
+
+    List<FieldSchema> fields = client.get_schema_with_environment_context(db, tableName, envCxt);
+    return fastpath ? fields : deepCopyFieldSchemas(fields);
+  }
+
+  @Override
+  public String getConfigValue(String name, String defaultValue)
+      throws TException, ConfigValSecurityException {
+    return client.get_config_value(name, defaultValue);
+  }
+
+  @Override
+  public Partition getPartition(String db, String tableName, String partName)
+      throws MetaException, TException, UnknownTableException, NoSuchObjectException {
+    Partition p = client.get_partition_by_name(db, tableName, partName);
+    return fastpath ? p : deepCopy(filterHook.filterPartition(p));
+  }
+
+  public Partition appendPartitionByName(String dbName, String tableName, String partName)
+      throws InvalidObjectException, AlreadyExistsException, MetaException, TException {
+    return appendPartitionByName(dbName, tableName, partName, null);
+  }
+
+  public Partition appendPartitionByName(String dbName, String tableName, String partName,
+      EnvironmentContext envContext) throws InvalidObjectException, AlreadyExistsException,
+      MetaException, TException {
+    Partition p = client.append_partition_by_name_with_environment_context(dbName, tableName,
+        partName, envContext);
+    return fastpath ? p : deepCopy(p);
+  }
+
+  public boolean dropPartitionByName(String dbName, String tableName, String partName,
+      boolean deleteData) throws NoSuchObjectException, MetaException, TException {
+    return dropPartitionByName(dbName, tableName, partName, deleteData, null);
+  }
+
+  public boolean dropPartitionByName(String dbName, String tableName, String partName,
+      boolean deleteData, EnvironmentContext envContext) throws NoSuchObjectException,
+      MetaException, TException {
+    return client.drop_partition_by_name_with_environment_context(dbName, tableName, partName,
+        deleteData, envContext);
+  }
+
+  private HiveMetaHook getHook(Table tbl) throws MetaException {
+    if (hookLoader == null) {
+      return null;
+    }
+    return hookLoader.getHook(tbl);
+  }
+
+  @Override
+  public List<String> partitionNameToVals(String name) throws MetaException, TException {
+    return client.partition_name_to_vals(name);
+  }
+
+  @Override
+  public Map<String, String> partitionNameToSpec(String name) throws MetaException, TException{
+    return client.partition_name_to_spec(name);
+  }
+
+  /**
+   * @param partition
+   * @return
+   */
+  private Partition deepCopy(Partition partition) {
+    Partition copy = null;
+    if (partition != null) {
+      copy = new Partition(partition);
+    }
+    return copy;
+  }
+
+  private Database deepCopy(Database database) {
+    Database copy = null;
+    if (database != null) {
+      copy = new Database(database);
+    }
+    return copy;
+  }
+
+  protected Table deepCopy(Table table) {
+    Table copy = null;
+    if (table != null) {
+      copy = new Table(table);
+    }
+    return copy;
+  }
+
+  private Type deepCopy(Type type) {
+    Type copy = null;
+    if (type != null) {
+      copy = new Type(type);
+    }
+    return copy;
+  }
+
+  private FieldSchema deepCopy(FieldSchema schema) {
+    FieldSchema copy = null;
+    if (schema != null) {
+      copy = new FieldSchema(schema);
+    }
+    return copy;
+  }
+
+  private Function deepCopy(Function func) {
+    Function copy = null;
+    if (func != null) {
+      copy = new Function(func);
+    }
+    return copy;
+  }
+
+  protected PrincipalPrivilegeSet deepCopy(PrincipalPrivilegeSet pps) {
+    PrincipalPrivilegeSet copy = null;
+    if (pps != null) {
+      copy = new PrincipalPrivilegeSet(pps);
+    }
+    return copy;
+  }
+
+  private List<Partition> deepCopyPartitions(List<Partition> partitions) {
+    return deepCopyPartitions(partitions, null);
+  }
+
+  private List<Partition> deepCopyPartitions(
+      Collection<Partition> src, List<Partition> dest) {
+    if (src == null) {
+      return dest;
+    }
+    if (dest == null) {
+      dest = new ArrayList<Partition>(src.size());
+    }
+    for (Partition part : src) {
+      dest.add(deepCopy(part));
+    }
+    return dest;
+  }
+
+  private List<Table> deepCopyTables(List<Table> tables) {
+    List<Table> copy = null;
+    if (tables != null) {
+      copy = new ArrayList<Table>();
+      for (Table tab : tables) {
+        copy.add(deepCopy(tab));
+      }
+    }
+    return copy;
+  }
+
+  protected List<FieldSchema> deepCopyFieldSchemas(List<FieldSchema> schemas) {
+    List<FieldSchema> copy = null;
+    if (schemas != null) {
+      copy = new ArrayList<FieldSchema>();
+      for (FieldSchema schema : schemas) {
+        copy.add(deepCopy(schema));
+      }
+    }
+    return copy;
+  }
+
+  @Override
+  public boolean grant_role(String roleName, String userName,
+      PrincipalType principalType, String grantor, PrincipalType grantorType,
+      boolean grantOption) throws MetaException, TException {
+    GrantRevokeRoleRequest req = new GrantRevokeRoleRequest();
+    req.setRequestType(GrantRevokeType.GRANT);
+    req.setRoleName(roleName);
+    req.setPrincipalName(userName);
+    req.setPrincipalType(principalType);
+    req.setGrantor(grantor);
+    req.setGrantorType(grantorType);
+    req.setGrantOption(grantOption);
+    GrantRevokeRoleResponse res = client.grant_revoke_role(req);
+    if (!res.isSetSuccess()) {
+      throw new MetaException("GrantRevokeResponse missing success field");
+    }
+    return res.isSuccess();
+  }
+
+  @Override
+  public boolean create_role(Role role)
+      throws MetaException, TException {
+    return client.create_role(role);
+  }
+
+  @Override
+  public boolean drop_role(String roleName) throws MetaException, TException {
+    return client.drop_role(roleName);
+  }
+
+  @Override
+  public List<Role> list_roles(String principalName,
+      PrincipalType principalType) throws MetaException, TException {
+    return client.list_roles(principalName, principalType);
+  }
+
+  @Override
+  public List<String> listRoleNames() throws MetaException, TException {
+    return client.get_role_names();
+  }
+
+  @Override
+  public GetPrincipalsInRoleResponse get_principals_in_role(GetPrincipalsInRoleRequest req)
+      throws MetaException, TException {
+    return client.get_principals_in_role(req);
+  }
+
+  @Override
+  public GetRoleGrantsForPrincipalResponse get_role_grants_for_principal(
+      GetRoleGrantsForPrincipalRequest getRolePrincReq) throws MetaException, TException {
+    return client.get_role_grants_for_principal(getRolePrincReq);
+  }
+
+  @Override
+  public boolean grant_privileges(PrivilegeBag privileges)
+      throws MetaException, TException {
+    GrantRevokePrivilegeRequest req = new GrantRevokePrivilegeRequest();
+    req.setRequestType(GrantRevokeType.GRANT);
+    req.setPrivileges(privileges);
+    GrantRevokePrivilegeResponse res = client.grant_revoke_privileges(req);
+    if (!res.isSetSuccess()) {
+      throw new MetaException("GrantRevokePrivilegeResponse missing success field");
+    }
+    return res.isSuccess();
+  }
+
+  @Override
+  public boolean revoke_role(String roleName, String userName,
+      PrincipalType principalType, boolean grantOption) throws MetaException, TException {
+    GrantRevokeRoleRequest req = new GrantRevokeRoleRequest();
+    req.setRequestType(GrantRevokeType.REVOKE);
+    req.setRoleName(roleName);
+    req.setPrincipalName(userName);
+    req.setPrincipalType(principalType);
+    req.setGrantOption(grantOption);
+    GrantRevokeRoleResponse res = client.grant_revoke_role(req);
+    if (!res.isSetSuccess()) {
+      throw new MetaException("GrantRevokeResponse missing success field");
+    }
+    return res.isSuccess();
+  }
+
+  @Override
+  public boolean revoke_privileges(PrivilegeBag privileges, boolean grantOption) throws MetaException,
+      TException {
+    GrantRevokePrivilegeRequest req = new GrantRevokePrivilegeRequest();
+    req.setRequestType(GrantRevokeType.REVOKE);
+    req.setPrivileges(privileges);
+    req.setRevokeGrantOption(grantOption);
+    GrantRevokePrivilegeResponse res = client.grant_revoke_privileges(req);
+    if (!res.isSetSuccess()) {
+      throw new MetaException("GrantRevokePrivilegeResponse missing success field");
+    }
+    return res.isSuccess();
+  }
+
+  @Override
+  public PrincipalPrivilegeSet get_privilege_set(HiveObjectRef hiveObject,
+      String userName, List<String> groupNames) throws MetaException,
+      TException {
+    return client.get_privilege_set(hiveObject, userName, groupNames);
+  }
+
+  @Override
+  public List<HiveObjectPrivilege> list_privileges(String principalName,
+      PrincipalType principalType, HiveObjectRef hiveObject)
+      throws MetaException, TException {
+    return client.list_privileges(principalName, principalType, hiveObject);
+  }
+
+  public String getDelegationToken(String renewerKerberosPrincipalName) throws
+  MetaException, TException, IOException {
+    //a convenience method that makes the intended owner for the delegation
+    //token request the current user
+    String owner = SecurityUtils.getUser();
+    return getDelegationToken(owner, renewerKerberosPrincipalName);
+  }
+
+  @Override
+  public String getDelegationToken(String owner, String renewerKerberosPrincipalName) throws
+  MetaException, TException {
+    // This is expected to be a no-op, so we will return null when we use local metastore.
+    if (localMetaStore) {
+      return null;
+    }
+    return client.get_delegation_token(owner, renewerKerberosPrincipalName);
+  }
+
+  @Override
+  public long renewDelegationToken(String tokenStrForm) throws MetaException, TException {
+    if (localMetaStore) {
+      return 0;
+    }
+    return client.renew_delegation_token(tokenStrForm);
+
+  }
+
+  @Override
+  public void cancelDelegationToken(String tokenStrForm) throws MetaException, TException {
+    if (localMetaStore) {
+      return;
+    }
+    client.cancel_delegation_token(tokenStrForm);
+  }
+
+  @Override
+  public boolean addToken(String tokenIdentifier, String delegationToken) throws TException {
+     return client.add_token(tokenIdentifier, delegationToken);
+  }
+
+  @Override
+  public boolean removeToken(String tokenIdentifier) throws TException {
+    return client.remove_token(tokenIdentifier);
+  }
+
+  @Override
+  public String getToken(String tokenIdentifier) throws TException {
+    return client.get_token(tokenIdentifier);
+  }
+
+  @Override
+  public List<String> getAllTokenIdentifiers() throws TException {
+    return client.get_all_token_identifiers();
+  }
+
+  @Override
+  public int addMasterKey(String key) throws MetaException, TException {
+    return client.add_master_key(key);
+  }
+
+  @Override
+  public void updateMasterKey(Integer seqNo, String key)
+      throws NoSuchObjectException, MetaException, TException {
+    client.update_master_key(seqNo, key);
+  }
+
+  @Override
+  public boolean removeMasterKey(Integer keySeq) throws TException {
+    return client.remove_master_key(keySeq);
+  }
+
+  @Override
+  public String[] getMasterKeys() throws TException {
+    List<String> keyList = client.get_master_keys();
+    return keyList.toArray(new String[keyList.size()]);
+  }
+
+  @Override
+  public ValidTxnList getValidTxns() throws TException {
+    return TxnUtils.createValidReadTxnList(client.get_open_txns(), 0);
+  }
+
+  @Override
+  public ValidTxnList getValidTxns(long currentTxn) throws TException {
+    return TxnUtils.createValidReadTxnList(client.get_open_txns(), currentTxn);
+  }
+
+  @Override
+  public ValidWriteIdList getValidWriteIds(String fullTableName) throws TException {
+    GetValidWriteIdsRequest rqst = new GetValidWriteIdsRequest(Collections.singletonList(fullTableName), null);
+    GetValidWriteIdsResponse validWriteIds = client.get_valid_write_ids(rqst);
+    return TxnUtils.createValidReaderWriteIdList(validWriteIds.getTblValidWriteIds().get(0));
+  }
+
+  @Override
+  public ValidTxnWriteIdList getValidWriteIds(Long currentTxnId, List<String> tablesList, String validTxnList)
+          throws TException {
+    GetValidWriteIdsRequest rqst = new GetValidWriteIdsRequest(tablesList, validTxnList);
+    return TxnUtils.createValidTxnWriteIdList(currentTxnId, client.get_valid_write_ids(rqst));
+  }
+
+  @Override
+  public long openTxn(String user) throws TException {
+    OpenTxnsResponse txns = openTxns(user, 1);
+    return txns.getTxn_ids().get(0);
+  }
+
+  @Override
+  public OpenTxnsResponse openTxns(String user, int numTxns) throws TException {
+    String hostname = null;
+    try {
+      hostname = InetAddress.getLocalHost().getHostName();
+    } catch (UnknownHostException e) {
+      LOG.error("Unable to resolve my host name " + e.getMessage());
+      throw new RuntimeException(e);
+    }
+    return client.open_txns(new OpenTxnRequest(numTxns, user, hostname));
+  }
+
+  @Override
+  public void rollbackTxn(long txnid) throws NoSuchTxnException, TException {
+    client.abort_txn(new AbortTxnRequest(txnid));
+  }
+
+  @Override
+  public void commitTxn(long txnid)
+      throws NoSuchTxnException, TxnAbortedException, TException {
+    client.commit_txn(new CommitTxnRequest(txnid));
+  }
+
+  @Override
+  public GetOpenTxnsInfoResponse showTxns() throws TException {
+    return client.get_open_txns_info();
+  }
+
+  @Override
+  public void abortTxns(List<Long> txnids) throws NoSuchTxnException, TException {
+    client.abort_txns(new AbortTxnsRequest(txnids));
+  }
+
+  @Override
+  public long allocateTableWriteId(long txnId, String dbName, String tableName) throws TException {
+    return allocateTableWriteIdsBatch(Collections.singletonList(txnId), dbName, tableName).get(0).getWriteId();
+  }
+
+  @Override
+  public List<TxnToWriteId> allocateTableWriteIdsBatch(List<Long> txnIds, String dbName, String tableName)
+          throws TException {
+    AllocateTableWriteIdsRequest rqst = new AllocateTableWriteIdsRequest(txnIds, dbName, tableName);
+    AllocateTableWriteIdsResponse writeIds = client.allocate_table_write_ids(rqst);
+    return writeIds.getTxnToWriteIds();
+  }
+
+  @Override
+  public LockResponse lock(LockRequest request)
+      throws NoSuchTxnException, TxnAbortedException, TException {
+    return client.lock(request);
+  }
+
+  @Override
+  public LockResponse checkLock(long lockid)
+      throws NoSuchTxnException, TxnAbortedException, NoSuchLockException,
+      TException {
+    return client.check_lock(new CheckLockRequest(lockid));
+  }
+
+  @Override
+  public void unlock(long lockid)
+      throws NoSuchLockException, TxnOpenException, TException {
+    client.unlock(new UnlockRequest(lockid));
+  }
+
+  @Override
+  @Deprecated
+  public ShowLocksResponse showLocks() throws TException {
+    return client.show_locks(new ShowLocksRequest());
+  }
+
+  @Override
+  public ShowLocksResponse showLocks(ShowLocksRequest request) throws TException {
+    return client.show_locks(request);
+  }
+
+  @Override
+  public void heartbeat(long txnid, long lockid)
+      throws NoSuchLockException, NoSuchTxnException, TxnAbortedException,
+      TException {
+    HeartbeatRequest hb = new HeartbeatRequest();
+    hb.setLockid(lockid);
+    hb.setTxnid(txnid);
+    client.heartbeat(hb);
+  }
+
+  @Override
+  public HeartbeatTxnRangeResponse heartbeatTxnRange(long min, long max)
+    throws NoSuchTxnException, TxnAbortedException, TException {
+    HeartbeatTxnRangeRequest rqst = new HeartbeatTxnRangeRequest(min, max);
+    return client.heartbeat_txn_range(rqst);
+  }
+
+  @Override
+  @Deprecated
+  public void compact(String dbname, String tableName, String partitionName,  CompactionType type)
+      throws TException {
+    CompactionRequest cr = new CompactionRequest();
+    if (dbname == null) {
+      cr.setDbname(DEFAULT_DATABASE_NAME);
+    } else {
+      cr.setDbname(dbname);
+    }
+    cr.setTablename(tableName);
+    if (partitionName != null) {
+      cr.setPartitionname(partitionName);
+    }
+    cr.setType(type);
+    client.compact(cr);
+  }
+  @Deprecated
+  @Override
+  public void compact(String dbname, String tableName, String partitionName, CompactionType type,
+                      Map<String, String> tblproperties) throws TException {
+    compact2(dbname, tableName, partitionName, type, tblproperties);
+  }
+
+  @Override
+  public CompactionResponse compact2(String dbname, String tableName, String partitionName, CompactionType type,
+                      Map<String, String> tblproperties) throws TException {
+    CompactionRequest cr = new CompactionRequest();
+    if (dbname == null) {
+      cr.setDbname(DEFAULT_DATABASE_NAME);
+    } else {
+      cr.setDbname(dbname);
+    }
+    cr.setTablename(tableName);
+    if (partitionName != null) {
+      cr.setPartitionname(partitionName);
+    }
+    cr.setType(type);
+    cr.setProperties(tblproperties);
+    return client.compact2(cr);
+  }
+  @Override
+  public ShowCompactResponse showCompactions() throws TException {
+    return client.show_compact(new ShowCompactRequest());
+  }
+
+  @Deprecated
+  @Override
+  public void addDynamicPartitions(long txnId, long writeId, String dbName, String tableName,
+                                   List<String> partNames) throws TException {
+    client.add_dynamic_partitions(new AddDynamicPartitions(txnId, writeId, dbName, tableName, partNames));
+  }
+  @Override
+  public void addDynamicPartitions(long txnId, long writeId, String dbName, String tableName,
+                                   List<String> partNames, DataOperationType operationType) throws TException {
+    AddDynamicPartitions adp = new AddDynamicPartitions(txnId, writeId, dbName, tableName, partNames);
+    adp.setOperationType(operationType);
+    client.add_dynamic_partitions(adp);
+  }
+
+  @Override
+  public void insertTable(Table table, boolean overwrite) throws MetaException {
+    boolean failed = true;
+    HiveMetaHook hook = getHook(table);
+    if (hook == null || !(hook instanceof DefaultHiveMetaHook)) {
+      return;
+    }
+    DefaultHiveMetaHook hiveMetaHook = (DefaultHiveMetaHook) hook;
+    try {
+      hiveMetaHook.commitInsertTable(table, overwrite);
+      failed = false;
+    }
+    finally {
+  

<TRUNCATED>

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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
index fb2f4dc..c5bc23e 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
@@ -453,13 +453,13 @@ import org.slf4j.LoggerFactory;
           case 1: // ABORTED
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set634 = iprot.readSetBegin();
-                struct.aborted = new HashSet<Long>(2*_set634.size);
-                long _elem635;
-                for (int _i636 = 0; _i636 < _set634.size; ++_i636)
+                org.apache.thrift.protocol.TSet _set642 = iprot.readSetBegin();
+                struct.aborted = new HashSet<Long>(2*_set642.size);
+                long _elem643;
+                for (int _i644 = 0; _i644 < _set642.size; ++_i644)
                 {
-                  _elem635 = iprot.readI64();
-                  struct.aborted.add(_elem635);
+                  _elem643 = iprot.readI64();
+                  struct.aborted.add(_elem643);
                 }
                 iprot.readSetEnd();
               }
@@ -471,13 +471,13 @@ import org.slf4j.LoggerFactory;
           case 2: // NOSUCH
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set637 = iprot.readSetBegin();
-                struct.nosuch = new HashSet<Long>(2*_set637.size);
-                long _elem638;
-                for (int _i639 = 0; _i639 < _set637.size; ++_i639)
+                org.apache.thrift.protocol.TSet _set645 = iprot.readSetBegin();
+                struct.nosuch = new HashSet<Long>(2*_set645.size);
+                long _elem646;
+                for (int _i647 = 0; _i647 < _set645.size; ++_i647)
                 {
-                  _elem638 = iprot.readI64();
-                  struct.nosuch.add(_elem638);
+                  _elem646 = iprot.readI64();
+                  struct.nosuch.add(_elem646);
                 }
                 iprot.readSetEnd();
               }
@@ -503,9 +503,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(ABORTED_FIELD_DESC);
         {
           oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, struct.aborted.size()));
-          for (long _iter640 : struct.aborted)
+          for (long _iter648 : struct.aborted)
           {
-            oprot.writeI64(_iter640);
+            oprot.writeI64(_iter648);
           }
           oprot.writeSetEnd();
         }
@@ -515,9 +515,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(NOSUCH_FIELD_DESC);
         {
           oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, struct.nosuch.size()));
-          for (long _iter641 : struct.nosuch)
+          for (long _iter649 : struct.nosuch)
           {
-            oprot.writeI64(_iter641);
+            oprot.writeI64(_iter649);
           }
           oprot.writeSetEnd();
         }
@@ -542,16 +542,16 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.aborted.size());
-        for (long _iter642 : struct.aborted)
+        for (long _iter650 : struct.aborted)
         {
-          oprot.writeI64(_iter642);
+          oprot.writeI64(_iter650);
         }
       }
       {
         oprot.writeI32(struct.nosuch.size());
-        for (long _iter643 : struct.nosuch)
+        for (long _iter651 : struct.nosuch)
         {
-          oprot.writeI64(_iter643);
+          oprot.writeI64(_iter651);
         }
       }
     }
@@ -560,24 +560,24 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, HeartbeatTxnRangeResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TSet _set644 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.aborted = new HashSet<Long>(2*_set644.size);
-        long _elem645;
-        for (int _i646 = 0; _i646 < _set644.size; ++_i646)
+        org.apache.thrift.protocol.TSet _set652 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.aborted = new HashSet<Long>(2*_set652.size);
+        long _elem653;
+        for (int _i654 = 0; _i654 < _set652.size; ++_i654)
         {
-          _elem645 = iprot.readI64();
-          struct.aborted.add(_elem645);
+          _elem653 = iprot.readI64();
+          struct.aborted.add(_elem653);
         }
       }
       struct.setAbortedIsSet(true);
       {
-        org.apache.thrift.protocol.TSet _set647 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.nosuch = new HashSet<Long>(2*_set647.size);
-        long _elem648;
-        for (int _i649 = 0; _i649 < _set647.size; ++_i649)
+        org.apache.thrift.protocol.TSet _set655 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.nosuch = new HashSet<Long>(2*_set655.size);
+        long _elem656;
+        for (int _i657 = 0; _i657 < _set655.size; ++_i657)
         {
-          _elem648 = iprot.readI64();
-          struct.nosuch.add(_elem648);
+          _elem656 = iprot.readI64();
+          struct.nosuch.add(_elem656);
         }
       }
       struct.setNosuchIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HiveObjectRef.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HiveObjectRef.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HiveObjectRef.java
index 6d13d60..c37ce58 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HiveObjectRef.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HiveObjectRef.java
@@ -43,6 +43,7 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField OBJECT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("objectName", org.apache.thrift.protocol.TType.STRING, (short)3);
   private static final org.apache.thrift.protocol.TField PART_VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("partValues", org.apache.thrift.protocol.TType.LIST, (short)4);
   private static final org.apache.thrift.protocol.TField COLUMN_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("columnName", org.apache.thrift.protocol.TType.STRING, (short)5);
+  private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)6);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -55,6 +56,7 @@ import org.slf4j.LoggerFactory;
   private String objectName; // required
   private List<String> partValues; // required
   private String columnName; // required
+  private String catName; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -66,7 +68,8 @@ import org.slf4j.LoggerFactory;
     DB_NAME((short)2, "dbName"),
     OBJECT_NAME((short)3, "objectName"),
     PART_VALUES((short)4, "partValues"),
-    COLUMN_NAME((short)5, "columnName");
+    COLUMN_NAME((short)5, "columnName"),
+    CAT_NAME((short)6, "catName");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -91,6 +94,8 @@ import org.slf4j.LoggerFactory;
           return PART_VALUES;
         case 5: // COLUMN_NAME
           return COLUMN_NAME;
+        case 6: // CAT_NAME
+          return CAT_NAME;
         default:
           return null;
       }
@@ -131,6 +136,7 @@ import org.slf4j.LoggerFactory;
   }
 
   // isset id assignments
+  private static final _Fields optionals[] = {_Fields.CAT_NAME};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -145,6 +151,8 @@ import org.slf4j.LoggerFactory;
             new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
     tmpMap.put(_Fields.COLUMN_NAME, new org.apache.thrift.meta_data.FieldMetaData("columnName", org.apache.thrift.TFieldRequirementType.DEFAULT, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(HiveObjectRef.class, metaDataMap);
   }
@@ -187,6 +195,9 @@ import org.slf4j.LoggerFactory;
     if (other.isSetColumnName()) {
       this.columnName = other.columnName;
     }
+    if (other.isSetCatName()) {
+      this.catName = other.catName;
+    }
   }
 
   public HiveObjectRef deepCopy() {
@@ -200,6 +211,7 @@ import org.slf4j.LoggerFactory;
     this.objectName = null;
     this.partValues = null;
     this.columnName = null;
+    this.catName = null;
   }
 
   /**
@@ -340,6 +352,29 @@ import org.slf4j.LoggerFactory;
     }
   }
 
+  public String getCatName() {
+    return this.catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
+  public void unsetCatName() {
+    this.catName = null;
+  }
+
+  /** Returns true if field catName is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatName() {
+    return this.catName != null;
+  }
+
+  public void setCatNameIsSet(boolean value) {
+    if (!value) {
+      this.catName = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case OBJECT_TYPE:
@@ -382,6 +417,14 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case CAT_NAME:
+      if (value == null) {
+        unsetCatName();
+      } else {
+        setCatName((String)value);
+      }
+      break;
+
     }
   }
 
@@ -402,6 +445,9 @@ import org.slf4j.LoggerFactory;
     case COLUMN_NAME:
       return getColumnName();
 
+    case CAT_NAME:
+      return getCatName();
+
     }
     throw new IllegalStateException();
   }
@@ -423,6 +469,8 @@ import org.slf4j.LoggerFactory;
       return isSetPartValues();
     case COLUMN_NAME:
       return isSetColumnName();
+    case CAT_NAME:
+      return isSetCatName();
     }
     throw new IllegalStateException();
   }
@@ -485,6 +533,15 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_catName = true && this.isSetCatName();
+    boolean that_present_catName = true && that.isSetCatName();
+    if (this_present_catName || that_present_catName) {
+      if (!(this_present_catName && that_present_catName))
+        return false;
+      if (!this.catName.equals(that.catName))
+        return false;
+    }
+
     return true;
   }
 
@@ -517,6 +574,11 @@ import org.slf4j.LoggerFactory;
     if (present_columnName)
       list.add(columnName);
 
+    boolean present_catName = true && (isSetCatName());
+    list.add(present_catName);
+    if (present_catName)
+      list.add(catName);
+
     return list.hashCode();
   }
 
@@ -578,6 +640,16 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetCatName()).compareTo(other.isSetCatName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catName, other.catName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -637,6 +709,16 @@ import org.slf4j.LoggerFactory;
       sb.append(this.columnName);
     }
     first = false;
+    if (isSetCatName()) {
+      if (!first) sb.append(", ");
+      sb.append("catName:");
+      if (this.catName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.catName);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -730,6 +812,14 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 6: // CAT_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.catName = iprot.readString();
+              struct.setCatNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -775,6 +865,13 @@ import org.slf4j.LoggerFactory;
         oprot.writeString(struct.columnName);
         oprot.writeFieldEnd();
       }
+      if (struct.catName != null) {
+        if (struct.isSetCatName()) {
+          oprot.writeFieldBegin(CAT_NAME_FIELD_DESC);
+          oprot.writeString(struct.catName);
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -808,7 +905,10 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetColumnName()) {
         optionals.set(4);
       }
-      oprot.writeBitSet(optionals, 5);
+      if (struct.isSetCatName()) {
+        optionals.set(5);
+      }
+      oprot.writeBitSet(optionals, 6);
       if (struct.isSetObjectType()) {
         oprot.writeI32(struct.objectType.getValue());
       }
@@ -830,12 +930,15 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetColumnName()) {
         oprot.writeString(struct.columnName);
       }
+      if (struct.isSetCatName()) {
+        oprot.writeString(struct.catName);
+      }
     }
 
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, HiveObjectRef struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
-      BitSet incoming = iprot.readBitSet(5);
+      BitSet incoming = iprot.readBitSet(6);
       if (incoming.get(0)) {
         struct.objectType = org.apache.hadoop.hive.metastore.api.HiveObjectType.findByValue(iprot.readI32());
         struct.setObjectTypeIsSet(true);
@@ -865,6 +968,10 @@ import org.slf4j.LoggerFactory;
         struct.columnName = iprot.readString();
         struct.setColumnNameIsSet(true);
       }
+      if (incoming.get(5)) {
+        struct.catName = iprot.readString();
+        struct.setCatNameIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ISchema.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ISchema.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ISchema.java
index 92d8b52..285f402 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ISchema.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ISchema.java
@@ -40,12 +40,13 @@ import org.slf4j.LoggerFactory;
 
   private static final org.apache.thrift.protocol.TField SCHEMA_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("schemaType", org.apache.thrift.protocol.TType.I32, (short)1);
   private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)2);
-  private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbName", org.apache.thrift.protocol.TType.STRING, (short)3);
-  private static final org.apache.thrift.protocol.TField COMPATIBILITY_FIELD_DESC = new org.apache.thrift.protocol.TField("compatibility", org.apache.thrift.protocol.TType.I32, (short)4);
-  private static final org.apache.thrift.protocol.TField VALIDATION_LEVEL_FIELD_DESC = new org.apache.thrift.protocol.TField("validationLevel", org.apache.thrift.protocol.TType.I32, (short)5);
-  private static final org.apache.thrift.protocol.TField CAN_EVOLVE_FIELD_DESC = new org.apache.thrift.protocol.TField("canEvolve", org.apache.thrift.protocol.TType.BOOL, (short)6);
-  private static final org.apache.thrift.protocol.TField SCHEMA_GROUP_FIELD_DESC = new org.apache.thrift.protocol.TField("schemaGroup", org.apache.thrift.protocol.TType.STRING, (short)7);
-  private static final org.apache.thrift.protocol.TField DESCRIPTION_FIELD_DESC = new org.apache.thrift.protocol.TField("description", org.apache.thrift.protocol.TType.STRING, (short)8);
+  private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)3);
+  private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbName", org.apache.thrift.protocol.TType.STRING, (short)4);
+  private static final org.apache.thrift.protocol.TField COMPATIBILITY_FIELD_DESC = new org.apache.thrift.protocol.TField("compatibility", org.apache.thrift.protocol.TType.I32, (short)5);
+  private static final org.apache.thrift.protocol.TField VALIDATION_LEVEL_FIELD_DESC = new org.apache.thrift.protocol.TField("validationLevel", org.apache.thrift.protocol.TType.I32, (short)6);
+  private static final org.apache.thrift.protocol.TField CAN_EVOLVE_FIELD_DESC = new org.apache.thrift.protocol.TField("canEvolve", org.apache.thrift.protocol.TType.BOOL, (short)7);
+  private static final org.apache.thrift.protocol.TField SCHEMA_GROUP_FIELD_DESC = new org.apache.thrift.protocol.TField("schemaGroup", org.apache.thrift.protocol.TType.STRING, (short)8);
+  private static final org.apache.thrift.protocol.TField DESCRIPTION_FIELD_DESC = new org.apache.thrift.protocol.TField("description", org.apache.thrift.protocol.TType.STRING, (short)9);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -55,6 +56,7 @@ import org.slf4j.LoggerFactory;
 
   private SchemaType schemaType; // required
   private String name; // required
+  private String catName; // required
   private String dbName; // required
   private SchemaCompatibility compatibility; // required
   private SchemaValidation validationLevel; // required
@@ -70,20 +72,21 @@ import org.slf4j.LoggerFactory;
      */
     SCHEMA_TYPE((short)1, "schemaType"),
     NAME((short)2, "name"),
-    DB_NAME((short)3, "dbName"),
+    CAT_NAME((short)3, "catName"),
+    DB_NAME((short)4, "dbName"),
     /**
      * 
      * @see SchemaCompatibility
      */
-    COMPATIBILITY((short)4, "compatibility"),
+    COMPATIBILITY((short)5, "compatibility"),
     /**
      * 
      * @see SchemaValidation
      */
-    VALIDATION_LEVEL((short)5, "validationLevel"),
-    CAN_EVOLVE((short)6, "canEvolve"),
-    SCHEMA_GROUP((short)7, "schemaGroup"),
-    DESCRIPTION((short)8, "description");
+    VALIDATION_LEVEL((short)6, "validationLevel"),
+    CAN_EVOLVE((short)7, "canEvolve"),
+    SCHEMA_GROUP((short)8, "schemaGroup"),
+    DESCRIPTION((short)9, "description");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -102,17 +105,19 @@ import org.slf4j.LoggerFactory;
           return SCHEMA_TYPE;
         case 2: // NAME
           return NAME;
-        case 3: // DB_NAME
+        case 3: // CAT_NAME
+          return CAT_NAME;
+        case 4: // DB_NAME
           return DB_NAME;
-        case 4: // COMPATIBILITY
+        case 5: // COMPATIBILITY
           return COMPATIBILITY;
-        case 5: // VALIDATION_LEVEL
+        case 6: // VALIDATION_LEVEL
           return VALIDATION_LEVEL;
-        case 6: // CAN_EVOLVE
+        case 7: // CAN_EVOLVE
           return CAN_EVOLVE;
-        case 7: // SCHEMA_GROUP
+        case 8: // SCHEMA_GROUP
           return SCHEMA_GROUP;
-        case 8: // DESCRIPTION
+        case 9: // DESCRIPTION
           return DESCRIPTION;
         default:
           return null;
@@ -164,6 +169,8 @@ import org.slf4j.LoggerFactory;
         new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, SchemaType.class)));
     tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.DB_NAME, new org.apache.thrift.meta_data.FieldMetaData("dbName", org.apache.thrift.TFieldRequirementType.DEFAULT, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.COMPATIBILITY, new org.apache.thrift.meta_data.FieldMetaData("compatibility", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -186,6 +193,7 @@ import org.slf4j.LoggerFactory;
   public ISchema(
     SchemaType schemaType,
     String name,
+    String catName,
     String dbName,
     SchemaCompatibility compatibility,
     SchemaValidation validationLevel,
@@ -194,6 +202,7 @@ import org.slf4j.LoggerFactory;
     this();
     this.schemaType = schemaType;
     this.name = name;
+    this.catName = catName;
     this.dbName = dbName;
     this.compatibility = compatibility;
     this.validationLevel = validationLevel;
@@ -212,6 +221,9 @@ import org.slf4j.LoggerFactory;
     if (other.isSetName()) {
       this.name = other.name;
     }
+    if (other.isSetCatName()) {
+      this.catName = other.catName;
+    }
     if (other.isSetDbName()) {
       this.dbName = other.dbName;
     }
@@ -238,6 +250,7 @@ import org.slf4j.LoggerFactory;
   public void clear() {
     this.schemaType = null;
     this.name = null;
+    this.catName = null;
     this.dbName = null;
     this.compatibility = null;
     this.validationLevel = null;
@@ -301,6 +314,29 @@ import org.slf4j.LoggerFactory;
     }
   }
 
+  public String getCatName() {
+    return this.catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
+  public void unsetCatName() {
+    this.catName = null;
+  }
+
+  /** Returns true if field catName is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatName() {
+    return this.catName != null;
+  }
+
+  public void setCatNameIsSet(boolean value) {
+    if (!value) {
+      this.catName = null;
+    }
+  }
+
   public String getDbName() {
     return this.dbName;
   }
@@ -472,6 +508,14 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case CAT_NAME:
+      if (value == null) {
+        unsetCatName();
+      } else {
+        setCatName((String)value);
+      }
+      break;
+
     case DB_NAME:
       if (value == null) {
         unsetDbName();
@@ -531,6 +575,9 @@ import org.slf4j.LoggerFactory;
     case NAME:
       return getName();
 
+    case CAT_NAME:
+      return getCatName();
+
     case DB_NAME:
       return getDbName();
 
@@ -564,6 +611,8 @@ import org.slf4j.LoggerFactory;
       return isSetSchemaType();
     case NAME:
       return isSetName();
+    case CAT_NAME:
+      return isSetCatName();
     case DB_NAME:
       return isSetDbName();
     case COMPATIBILITY:
@@ -611,6 +660,15 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_catName = true && this.isSetCatName();
+    boolean that_present_catName = true && that.isSetCatName();
+    if (this_present_catName || that_present_catName) {
+      if (!(this_present_catName && that_present_catName))
+        return false;
+      if (!this.catName.equals(that.catName))
+        return false;
+    }
+
     boolean this_present_dbName = true && this.isSetDbName();
     boolean that_present_dbName = true && that.isSetDbName();
     if (this_present_dbName || that_present_dbName) {
@@ -682,6 +740,11 @@ import org.slf4j.LoggerFactory;
     if (present_name)
       list.add(name);
 
+    boolean present_catName = true && (isSetCatName());
+    list.add(present_catName);
+    if (present_catName)
+      list.add(catName);
+
     boolean present_dbName = true && (isSetDbName());
     list.add(present_dbName);
     if (present_dbName)
@@ -743,6 +806,16 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetCatName()).compareTo(other.isSetCatName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catName, other.catName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     lastComparison = Boolean.valueOf(isSetDbName()).compareTo(other.isSetDbName());
     if (lastComparison != 0) {
       return lastComparison;
@@ -839,6 +912,14 @@ import org.slf4j.LoggerFactory;
     }
     first = false;
     if (!first) sb.append(", ");
+    sb.append("catName:");
+    if (this.catName == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.catName);
+    }
+    first = false;
+    if (!first) sb.append(", ");
     sb.append("dbName:");
     if (this.dbName == null) {
       sb.append("null");
@@ -947,7 +1028,15 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 3: // DB_NAME
+          case 3: // CAT_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.catName = iprot.readString();
+              struct.setCatNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // DB_NAME
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.dbName = iprot.readString();
               struct.setDbNameIsSet(true);
@@ -955,7 +1044,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 4: // COMPATIBILITY
+          case 5: // COMPATIBILITY
             if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
               struct.compatibility = org.apache.hadoop.hive.metastore.api.SchemaCompatibility.findByValue(iprot.readI32());
               struct.setCompatibilityIsSet(true);
@@ -963,7 +1052,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 5: // VALIDATION_LEVEL
+          case 6: // VALIDATION_LEVEL
             if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
               struct.validationLevel = org.apache.hadoop.hive.metastore.api.SchemaValidation.findByValue(iprot.readI32());
               struct.setValidationLevelIsSet(true);
@@ -971,7 +1060,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 6: // CAN_EVOLVE
+          case 7: // CAN_EVOLVE
             if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
               struct.canEvolve = iprot.readBool();
               struct.setCanEvolveIsSet(true);
@@ -979,7 +1068,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 7: // SCHEMA_GROUP
+          case 8: // SCHEMA_GROUP
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.schemaGroup = iprot.readString();
               struct.setSchemaGroupIsSet(true);
@@ -987,7 +1076,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 8: // DESCRIPTION
+          case 9: // DESCRIPTION
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.description = iprot.readString();
               struct.setDescriptionIsSet(true);
@@ -1018,6 +1107,11 @@ import org.slf4j.LoggerFactory;
         oprot.writeString(struct.name);
         oprot.writeFieldEnd();
       }
+      if (struct.catName != null) {
+        oprot.writeFieldBegin(CAT_NAME_FIELD_DESC);
+        oprot.writeString(struct.catName);
+        oprot.writeFieldEnd();
+      }
       if (struct.dbName != null) {
         oprot.writeFieldBegin(DB_NAME_FIELD_DESC);
         oprot.writeString(struct.dbName);
@@ -1074,31 +1168,37 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetName()) {
         optionals.set(1);
       }
-      if (struct.isSetDbName()) {
+      if (struct.isSetCatName()) {
         optionals.set(2);
       }
-      if (struct.isSetCompatibility()) {
+      if (struct.isSetDbName()) {
         optionals.set(3);
       }
-      if (struct.isSetValidationLevel()) {
+      if (struct.isSetCompatibility()) {
         optionals.set(4);
       }
-      if (struct.isSetCanEvolve()) {
+      if (struct.isSetValidationLevel()) {
         optionals.set(5);
       }
-      if (struct.isSetSchemaGroup()) {
+      if (struct.isSetCanEvolve()) {
         optionals.set(6);
       }
-      if (struct.isSetDescription()) {
+      if (struct.isSetSchemaGroup()) {
         optionals.set(7);
       }
-      oprot.writeBitSet(optionals, 8);
+      if (struct.isSetDescription()) {
+        optionals.set(8);
+      }
+      oprot.writeBitSet(optionals, 9);
       if (struct.isSetSchemaType()) {
         oprot.writeI32(struct.schemaType.getValue());
       }
       if (struct.isSetName()) {
         oprot.writeString(struct.name);
       }
+      if (struct.isSetCatName()) {
+        oprot.writeString(struct.catName);
+      }
       if (struct.isSetDbName()) {
         oprot.writeString(struct.dbName);
       }
@@ -1122,7 +1222,7 @@ import org.slf4j.LoggerFactory;
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, ISchema struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
-      BitSet incoming = iprot.readBitSet(8);
+      BitSet incoming = iprot.readBitSet(9);
       if (incoming.get(0)) {
         struct.schemaType = org.apache.hadoop.hive.metastore.api.SchemaType.findByValue(iprot.readI32());
         struct.setSchemaTypeIsSet(true);
@@ -1132,26 +1232,30 @@ import org.slf4j.LoggerFactory;
         struct.setNameIsSet(true);
       }
       if (incoming.get(2)) {
+        struct.catName = iprot.readString();
+        struct.setCatNameIsSet(true);
+      }
+      if (incoming.get(3)) {
         struct.dbName = iprot.readString();
         struct.setDbNameIsSet(true);
       }
-      if (incoming.get(3)) {
+      if (incoming.get(4)) {
         struct.compatibility = org.apache.hadoop.hive.metastore.api.SchemaCompatibility.findByValue(iprot.readI32());
         struct.setCompatibilityIsSet(true);
       }
-      if (incoming.get(4)) {
+      if (incoming.get(5)) {
         struct.validationLevel = org.apache.hadoop.hive.metastore.api.SchemaValidation.findByValue(iprot.readI32());
         struct.setValidationLevelIsSet(true);
       }
-      if (incoming.get(5)) {
+      if (incoming.get(6)) {
         struct.canEvolve = iprot.readBool();
         struct.setCanEvolveIsSet(true);
       }
-      if (incoming.get(6)) {
+      if (incoming.get(7)) {
         struct.schemaGroup = iprot.readString();
         struct.setSchemaGroupIsSet(true);
       }
-      if (incoming.get(7)) {
+      if (incoming.get(8)) {
         struct.description = iprot.readString();
         struct.setDescriptionIsSet(true);
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ISchemaName.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ISchemaName.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ISchemaName.java
index ad25051..6f0e052 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ISchemaName.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ISchemaName.java
@@ -38,8 +38,9 @@ import org.slf4j.LoggerFactory;
 @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class ISchemaName implements org.apache.thrift.TBase<ISchemaName, ISchemaName._Fields>, java.io.Serializable, Cloneable, Comparable<ISchemaName> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ISchemaName");
 
-  private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbName", org.apache.thrift.protocol.TType.STRING, (short)1);
-  private static final org.apache.thrift.protocol.TField SCHEMA_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("schemaName", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbName", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField SCHEMA_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("schemaName", org.apache.thrift.protocol.TType.STRING, (short)3);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -47,13 +48,15 @@ import org.slf4j.LoggerFactory;
     schemes.put(TupleScheme.class, new ISchemaNameTupleSchemeFactory());
   }
 
+  private String catName; // required
   private String dbName; // required
   private String schemaName; // required
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-    DB_NAME((short)1, "dbName"),
-    SCHEMA_NAME((short)2, "schemaName");
+    CAT_NAME((short)1, "catName"),
+    DB_NAME((short)2, "dbName"),
+    SCHEMA_NAME((short)3, "schemaName");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -68,9 +71,11 @@ import org.slf4j.LoggerFactory;
      */
     public static _Fields findByThriftId(int fieldId) {
       switch(fieldId) {
-        case 1: // DB_NAME
+        case 1: // CAT_NAME
+          return CAT_NAME;
+        case 2: // DB_NAME
           return DB_NAME;
-        case 2: // SCHEMA_NAME
+        case 3: // SCHEMA_NAME
           return SCHEMA_NAME;
         default:
           return null;
@@ -115,6 +120,8 @@ import org.slf4j.LoggerFactory;
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.DB_NAME, new org.apache.thrift.meta_data.FieldMetaData("dbName", org.apache.thrift.TFieldRequirementType.DEFAULT, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.SCHEMA_NAME, new org.apache.thrift.meta_data.FieldMetaData("schemaName", org.apache.thrift.TFieldRequirementType.DEFAULT, 
@@ -127,10 +134,12 @@ import org.slf4j.LoggerFactory;
   }
 
   public ISchemaName(
+    String catName,
     String dbName,
     String schemaName)
   {
     this();
+    this.catName = catName;
     this.dbName = dbName;
     this.schemaName = schemaName;
   }
@@ -139,6 +148,9 @@ import org.slf4j.LoggerFactory;
    * Performs a deep copy on <i>other</i>.
    */
   public ISchemaName(ISchemaName other) {
+    if (other.isSetCatName()) {
+      this.catName = other.catName;
+    }
     if (other.isSetDbName()) {
       this.dbName = other.dbName;
     }
@@ -153,10 +165,34 @@ import org.slf4j.LoggerFactory;
 
   @Override
   public void clear() {
+    this.catName = null;
     this.dbName = null;
     this.schemaName = null;
   }
 
+  public String getCatName() {
+    return this.catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
+  public void unsetCatName() {
+    this.catName = null;
+  }
+
+  /** Returns true if field catName is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatName() {
+    return this.catName != null;
+  }
+
+  public void setCatNameIsSet(boolean value) {
+    if (!value) {
+      this.catName = null;
+    }
+  }
+
   public String getDbName() {
     return this.dbName;
   }
@@ -205,6 +241,14 @@ import org.slf4j.LoggerFactory;
 
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
+    case CAT_NAME:
+      if (value == null) {
+        unsetCatName();
+      } else {
+        setCatName((String)value);
+      }
+      break;
+
     case DB_NAME:
       if (value == null) {
         unsetDbName();
@@ -226,6 +270,9 @@ import org.slf4j.LoggerFactory;
 
   public Object getFieldValue(_Fields field) {
     switch (field) {
+    case CAT_NAME:
+      return getCatName();
+
     case DB_NAME:
       return getDbName();
 
@@ -243,6 +290,8 @@ import org.slf4j.LoggerFactory;
     }
 
     switch (field) {
+    case CAT_NAME:
+      return isSetCatName();
     case DB_NAME:
       return isSetDbName();
     case SCHEMA_NAME:
@@ -264,6 +313,15 @@ import org.slf4j.LoggerFactory;
     if (that == null)
       return false;
 
+    boolean this_present_catName = true && this.isSetCatName();
+    boolean that_present_catName = true && that.isSetCatName();
+    if (this_present_catName || that_present_catName) {
+      if (!(this_present_catName && that_present_catName))
+        return false;
+      if (!this.catName.equals(that.catName))
+        return false;
+    }
+
     boolean this_present_dbName = true && this.isSetDbName();
     boolean that_present_dbName = true && that.isSetDbName();
     if (this_present_dbName || that_present_dbName) {
@@ -289,6 +347,11 @@ import org.slf4j.LoggerFactory;
   public int hashCode() {
     List<Object> list = new ArrayList<Object>();
 
+    boolean present_catName = true && (isSetCatName());
+    list.add(present_catName);
+    if (present_catName)
+      list.add(catName);
+
     boolean present_dbName = true && (isSetDbName());
     list.add(present_dbName);
     if (present_dbName)
@@ -310,6 +373,16 @@ import org.slf4j.LoggerFactory;
 
     int lastComparison = 0;
 
+    lastComparison = Boolean.valueOf(isSetCatName()).compareTo(other.isSetCatName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catName, other.catName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     lastComparison = Boolean.valueOf(isSetDbName()).compareTo(other.isSetDbName());
     if (lastComparison != 0) {
       return lastComparison;
@@ -350,6 +423,14 @@ import org.slf4j.LoggerFactory;
     StringBuilder sb = new StringBuilder("ISchemaName(");
     boolean first = true;
 
+    sb.append("catName:");
+    if (this.catName == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.catName);
+    }
+    first = false;
+    if (!first) sb.append(", ");
     sb.append("dbName:");
     if (this.dbName == null) {
       sb.append("null");
@@ -408,7 +489,15 @@ import org.slf4j.LoggerFactory;
           break;
         }
         switch (schemeField.id) {
-          case 1: // DB_NAME
+          case 1: // CAT_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.catName = iprot.readString();
+              struct.setCatNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // DB_NAME
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.dbName = iprot.readString();
               struct.setDbNameIsSet(true);
@@ -416,7 +505,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 2: // SCHEMA_NAME
+          case 3: // SCHEMA_NAME
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.schemaName = iprot.readString();
               struct.setSchemaNameIsSet(true);
@@ -437,6 +526,11 @@ import org.slf4j.LoggerFactory;
       struct.validate();
 
       oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.catName != null) {
+        oprot.writeFieldBegin(CAT_NAME_FIELD_DESC);
+        oprot.writeString(struct.catName);
+        oprot.writeFieldEnd();
+      }
       if (struct.dbName != null) {
         oprot.writeFieldBegin(DB_NAME_FIELD_DESC);
         oprot.writeString(struct.dbName);
@@ -465,13 +559,19 @@ import org.slf4j.LoggerFactory;
     public void write(org.apache.thrift.protocol.TProtocol prot, ISchemaName struct) throws org.apache.thrift.TException {
       TTupleProtocol oprot = (TTupleProtocol) prot;
       BitSet optionals = new BitSet();
-      if (struct.isSetDbName()) {
+      if (struct.isSetCatName()) {
         optionals.set(0);
       }
-      if (struct.isSetSchemaName()) {
+      if (struct.isSetDbName()) {
         optionals.set(1);
       }
-      oprot.writeBitSet(optionals, 2);
+      if (struct.isSetSchemaName()) {
+        optionals.set(2);
+      }
+      oprot.writeBitSet(optionals, 3);
+      if (struct.isSetCatName()) {
+        oprot.writeString(struct.catName);
+      }
       if (struct.isSetDbName()) {
         oprot.writeString(struct.dbName);
       }
@@ -483,12 +583,16 @@ import org.slf4j.LoggerFactory;
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, ISchemaName struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
-      BitSet incoming = iprot.readBitSet(2);
+      BitSet incoming = iprot.readBitSet(3);
       if (incoming.get(0)) {
+        struct.catName = iprot.readString();
+        struct.setCatNameIsSet(true);
+      }
+      if (incoming.get(1)) {
         struct.dbName = iprot.readString();
         struct.setDbNameIsSet(true);
       }
-      if (incoming.get(1)) {
+      if (incoming.get(2)) {
         struct.schemaName = iprot.readString();
         struct.setSchemaNameIsSet(true);
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
index d1cdb4b..8a33611 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
@@ -538,13 +538,13 @@ import org.slf4j.LoggerFactory;
           case 2: // FILES_ADDED
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list692 = iprot.readListBegin();
-                struct.filesAdded = new ArrayList<String>(_list692.size);
-                String _elem693;
-                for (int _i694 = 0; _i694 < _list692.size; ++_i694)
+                org.apache.thrift.protocol.TList _list700 = iprot.readListBegin();
+                struct.filesAdded = new ArrayList<String>(_list700.size);
+                String _elem701;
+                for (int _i702 = 0; _i702 < _list700.size; ++_i702)
                 {
-                  _elem693 = iprot.readString();
-                  struct.filesAdded.add(_elem693);
+                  _elem701 = iprot.readString();
+                  struct.filesAdded.add(_elem701);
                 }
                 iprot.readListEnd();
               }
@@ -556,13 +556,13 @@ import org.slf4j.LoggerFactory;
           case 3: // FILES_ADDED_CHECKSUM
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list695 = iprot.readListBegin();
-                struct.filesAddedChecksum = new ArrayList<String>(_list695.size);
-                String _elem696;
-                for (int _i697 = 0; _i697 < _list695.size; ++_i697)
+                org.apache.thrift.protocol.TList _list703 = iprot.readListBegin();
+                struct.filesAddedChecksum = new ArrayList<String>(_list703.size);
+                String _elem704;
+                for (int _i705 = 0; _i705 < _list703.size; ++_i705)
                 {
-                  _elem696 = iprot.readString();
-                  struct.filesAddedChecksum.add(_elem696);
+                  _elem704 = iprot.readString();
+                  struct.filesAddedChecksum.add(_elem704);
                 }
                 iprot.readListEnd();
               }
@@ -593,9 +593,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FILES_ADDED_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.filesAdded.size()));
-          for (String _iter698 : struct.filesAdded)
+          for (String _iter706 : struct.filesAdded)
           {
-            oprot.writeString(_iter698);
+            oprot.writeString(_iter706);
           }
           oprot.writeListEnd();
         }
@@ -606,9 +606,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(FILES_ADDED_CHECKSUM_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.filesAddedChecksum.size()));
-            for (String _iter699 : struct.filesAddedChecksum)
+            for (String _iter707 : struct.filesAddedChecksum)
             {
-              oprot.writeString(_iter699);
+              oprot.writeString(_iter707);
             }
             oprot.writeListEnd();
           }
@@ -634,9 +634,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.filesAdded.size());
-        for (String _iter700 : struct.filesAdded)
+        for (String _iter708 : struct.filesAdded)
         {
-          oprot.writeString(_iter700);
+          oprot.writeString(_iter708);
         }
       }
       BitSet optionals = new BitSet();
@@ -653,9 +653,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetFilesAddedChecksum()) {
         {
           oprot.writeI32(struct.filesAddedChecksum.size());
-          for (String _iter701 : struct.filesAddedChecksum)
+          for (String _iter709 : struct.filesAddedChecksum)
           {
-            oprot.writeString(_iter701);
+            oprot.writeString(_iter709);
           }
         }
       }
@@ -665,13 +665,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, InsertEventRequestData struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list702 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.filesAdded = new ArrayList<String>(_list702.size);
-        String _elem703;
-        for (int _i704 = 0; _i704 < _list702.size; ++_i704)
+        org.apache.thrift.protocol.TList _list710 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.filesAdded = new ArrayList<String>(_list710.size);
+        String _elem711;
+        for (int _i712 = 0; _i712 < _list710.size; ++_i712)
         {
-          _elem703 = iprot.readString();
-          struct.filesAdded.add(_elem703);
+          _elem711 = iprot.readString();
+          struct.filesAdded.add(_elem711);
         }
       }
       struct.setFilesAddedIsSet(true);
@@ -682,13 +682,13 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TList _list705 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.filesAddedChecksum = new ArrayList<String>(_list705.size);
-          String _elem706;
-          for (int _i707 = 0; _i707 < _list705.size; ++_i707)
+          org.apache.thrift.protocol.TList _list713 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.filesAddedChecksum = new ArrayList<String>(_list713.size);
+          String _elem714;
+          for (int _i715 = 0; _i715 < _list713.size; ++_i715)
           {
-            _elem706 = iprot.readString();
-            struct.filesAddedChecksum.add(_elem706);
+            _elem714 = iprot.readString();
+            struct.filesAddedChecksum.add(_elem714);
           }
         }
         struct.setFilesAddedChecksumIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
index 722619f..6f03ea9 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
@@ -689,14 +689,14 @@ import org.slf4j.LoggerFactory;
           case 1: // COMPONENT
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list618 = iprot.readListBegin();
-                struct.component = new ArrayList<LockComponent>(_list618.size);
-                LockComponent _elem619;
-                for (int _i620 = 0; _i620 < _list618.size; ++_i620)
+                org.apache.thrift.protocol.TList _list626 = iprot.readListBegin();
+                struct.component = new ArrayList<LockComponent>(_list626.size);
+                LockComponent _elem627;
+                for (int _i628 = 0; _i628 < _list626.size; ++_i628)
                 {
-                  _elem619 = new LockComponent();
-                  _elem619.read(iprot);
-                  struct.component.add(_elem619);
+                  _elem627 = new LockComponent();
+                  _elem627.read(iprot);
+                  struct.component.add(_elem627);
                 }
                 iprot.readListEnd();
               }
@@ -754,9 +754,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(COMPONENT_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.component.size()));
-          for (LockComponent _iter621 : struct.component)
+          for (LockComponent _iter629 : struct.component)
           {
-            _iter621.write(oprot);
+            _iter629.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -803,9 +803,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.component.size());
-        for (LockComponent _iter622 : struct.component)
+        for (LockComponent _iter630 : struct.component)
         {
-          _iter622.write(oprot);
+          _iter630.write(oprot);
         }
       }
       oprot.writeString(struct.user);
@@ -830,14 +830,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, LockRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list623 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.component = new ArrayList<LockComponent>(_list623.size);
-        LockComponent _elem624;
-        for (int _i625 = 0; _i625 < _list623.size; ++_i625)
+        org.apache.thrift.protocol.TList _list631 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.component = new ArrayList<LockComponent>(_list631.size);
+        LockComponent _elem632;
+        for (int _i633 = 0; _i633 < _list631.size; ++_i633)
         {
-          _elem624 = new LockComponent();
-          _elem624.read(iprot);
-          struct.component.add(_elem624);
+          _elem632 = new LockComponent();
+          _elem632.read(iprot);
+          struct.component.add(_elem632);
         }
       }
       struct.setComponentIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Materialization.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Materialization.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Materialization.java
index fec35d5..faee4be 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Materialization.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Materialization.java
@@ -518,13 +518,13 @@ import org.slf4j.LoggerFactory;
           case 1: // TABLES_USED
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set808 = iprot.readSetBegin();
-                struct.tablesUsed = new HashSet<String>(2*_set808.size);
-                String _elem809;
-                for (int _i810 = 0; _i810 < _set808.size; ++_i810)
+                org.apache.thrift.protocol.TSet _set816 = iprot.readSetBegin();
+                struct.tablesUsed = new HashSet<String>(2*_set816.size);
+                String _elem817;
+                for (int _i818 = 0; _i818 < _set816.size; ++_i818)
                 {
-                  _elem809 = iprot.readString();
-                  struct.tablesUsed.add(_elem809);
+                  _elem817 = iprot.readString();
+                  struct.tablesUsed.add(_elem817);
                 }
                 iprot.readSetEnd();
               }
@@ -566,9 +566,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TABLES_USED_FIELD_DESC);
         {
           oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, struct.tablesUsed.size()));
-          for (String _iter811 : struct.tablesUsed)
+          for (String _iter819 : struct.tablesUsed)
           {
-            oprot.writeString(_iter811);
+            oprot.writeString(_iter819);
           }
           oprot.writeSetEnd();
         }
@@ -603,9 +603,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.tablesUsed.size());
-        for (String _iter812 : struct.tablesUsed)
+        for (String _iter820 : struct.tablesUsed)
         {
-          oprot.writeString(_iter812);
+          oprot.writeString(_iter820);
         }
       }
       oprot.writeI64(struct.invalidationTime);
@@ -623,13 +623,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, Materialization struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TSet _set813 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.tablesUsed = new HashSet<String>(2*_set813.size);
-        String _elem814;
-        for (int _i815 = 0; _i815 < _set813.size; ++_i815)
+        org.apache.thrift.protocol.TSet _set821 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.tablesUsed = new HashSet<String>(2*_set821.size);
+        String _elem822;
+        for (int _i823 = 0; _i823 < _set821.size; ++_i823)
         {
-          _elem814 = iprot.readString();
-          struct.tablesUsed.add(_elem814);
+          _elem822 = iprot.readString();
+          struct.tablesUsed.add(_elem822);
         }
       }
       struct.setTablesUsedIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotNullConstraintsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotNullConstraintsRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotNullConstraintsRequest.java
index da5d72b..803dc20 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotNullConstraintsRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotNullConstraintsRequest.java
@@ -38,8 +38,9 @@ import org.slf4j.LoggerFactory;
 @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class NotNullConstraintsRequest implements org.apache.thrift.TBase<NotNullConstraintsRequest, NotNullConstraintsRequest._Fields>, java.io.Serializable, Cloneable, Comparable<NotNullConstraintsRequest> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NotNullConstraintsRequest");
 
-  private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("db_name", org.apache.thrift.protocol.TType.STRING, (short)1);
-  private static final org.apache.thrift.protocol.TField TBL_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tbl_name", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("db_name", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField TBL_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tbl_name", org.apache.thrift.protocol.TType.STRING, (short)3);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -47,13 +48,15 @@ import org.slf4j.LoggerFactory;
     schemes.put(TupleScheme.class, new NotNullConstraintsRequestTupleSchemeFactory());
   }
 
+  private String catName; // required
   private String db_name; // required
   private String tbl_name; // required
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-    DB_NAME((short)1, "db_name"),
-    TBL_NAME((short)2, "tbl_name");
+    CAT_NAME((short)1, "catName"),
+    DB_NAME((short)2, "db_name"),
+    TBL_NAME((short)3, "tbl_name");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -68,9 +71,11 @@ import org.slf4j.LoggerFactory;
      */
     public static _Fields findByThriftId(int fieldId) {
       switch(fieldId) {
-        case 1: // DB_NAME
+        case 1: // CAT_NAME
+          return CAT_NAME;
+        case 2: // DB_NAME
           return DB_NAME;
-        case 2: // TBL_NAME
+        case 3: // TBL_NAME
           return TBL_NAME;
         default:
           return null;
@@ -115,6 +120,8 @@ import org.slf4j.LoggerFactory;
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.DB_NAME, new org.apache.thrift.meta_data.FieldMetaData("db_name", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.TBL_NAME, new org.apache.thrift.meta_data.FieldMetaData("tbl_name", org.apache.thrift.TFieldRequirementType.REQUIRED, 
@@ -127,10 +134,12 @@ import org.slf4j.LoggerFactory;
   }
 
   public NotNullConstraintsRequest(
+    String catName,
     String db_name,
     String tbl_name)
   {
     this();
+    this.catName = catName;
     this.db_name = db_name;
     this.tbl_name = tbl_name;
   }
@@ -139,6 +148,9 @@ import org.slf4j.LoggerFactory;
    * Performs a deep copy on <i>other</i>.
    */
   public NotNullConstraintsRequest(NotNullConstraintsRequest other) {
+    if (other.isSetCatName()) {
+      this.catName = other.catName;
+    }
     if (other.isSetDb_name()) {
       this.db_name = other.db_name;
     }
@@ -153,10 +165,34 @@ import org.slf4j.LoggerFactory;
 
   @Override
   public void clear() {
+    this.catName = null;
     this.db_name = null;
     this.tbl_name = null;
   }
 
+  public String getCatName() {
+    return this.catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
+  public void unsetCatName() {
+    this.catName = null;
+  }
+
+  /** Returns true if field catName is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatName() {
+    return this.catName != null;
+  }
+
+  public void setCatNameIsSet(boolean value) {
+    if (!value) {
+      this.catName = null;
+    }
+  }
+
   public String getDb_name() {
     return this.db_name;
   }
@@ -205,6 +241,14 @@ import org.slf4j.LoggerFactory;
 
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
+    case CAT_NAME:
+      if (value == null) {
+        unsetCatName();
+      } else {
+        setCatName((String)value);
+      }
+      break;
+
     case DB_NAME:
       if (value == null) {
         unsetDb_name();
@@ -226,6 +270,9 @@ import org.slf4j.LoggerFactory;
 
   public Object getFieldValue(_Fields field) {
     switch (field) {
+    case CAT_NAME:
+      return getCatName();
+
     case DB_NAME:
       return getDb_name();
 
@@ -243,6 +290,8 @@ import org.slf4j.LoggerFactory;
     }
 
     switch (field) {
+    case CAT_NAME:
+      return isSetCatName();
     case DB_NAME:
       return isSetDb_name();
     case TBL_NAME:
@@ -264,6 +313,15 @@ import org.slf4j.LoggerFactory;
     if (that == null)
       return false;
 
+    boolean this_present_catName = true && this.isSetCatName();
+    boolean that_present_catName = true && that.isSetCatName();
+    if (this_present_catName || that_present_catName) {
+      if (!(this_present_catName && that_present_catName))
+        return false;
+      if (!this.catName.equals(that.catName))
+        return false;
+    }
+
     boolean this_present_db_name = true && this.isSetDb_name();
     boolean that_present_db_name = true && that.isSetDb_name();
     if (this_present_db_name || that_present_db_name) {
@@ -289,6 +347,11 @@ import org.slf4j.LoggerFactory;
   public int hashCode() {
     List<Object> list = new ArrayList<Object>();
 
+    boolean present_catName = true && (isSetCatName());
+    list.add(present_catName);
+    if (present_catName)
+      list.add(catName);
+
     boolean present_db_name = true && (isSetDb_name());
     list.add(present_db_name);
     if (present_db_name)
@@ -310,6 +373,16 @@ import org.slf4j.LoggerFactory;
 
     int lastComparison = 0;
 
+    lastComparison = Boolean.valueOf(isSetCatName()).compareTo(other.isSetCatName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catName, other.catName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     lastComparison = Boolean.valueOf(isSetDb_name()).compareTo(other.isSetDb_name());
     if (lastComparison != 0) {
       return lastComparison;
@@ -350,6 +423,14 @@ import org.slf4j.LoggerFactory;
     StringBuilder sb = new StringBuilder("NotNullConstraintsRequest(");
     boolean first = true;
 
+    sb.append("catName:");
+    if (this.catName == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.catName);
+    }
+    first = false;
+    if (!first) sb.append(", ");
     sb.append("db_name:");
     if (this.db_name == null) {
       sb.append("null");
@@ -371,6 +452,10 @@ import org.slf4j.LoggerFactory;
 
   public void validate() throws org.apache.thrift.TException {
     // check for required fields
+    if (!isSetCatName()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'catName' is unset! Struct:" + toString());
+    }
+
     if (!isSetDb_name()) {
       throw new org.apache.thrift.protocol.TProtocolException("Required field 'db_name' is unset! Struct:" + toString());
     }
@@ -416,7 +501,15 @@ import org.slf4j.LoggerFactory;
           break;
         }
         switch (schemeField.id) {
-          case 1: // DB_NAME
+          case 1: // CAT_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.catName = iprot.readString();
+              struct.setCatNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // DB_NAME
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.db_name = iprot.readString();
               struct.setDb_nameIsSet(true);
@@ -424,7 +517,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 2: // TBL_NAME
+          case 3: // TBL_NAME
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.tbl_name = iprot.readString();
               struct.setTbl_nameIsSet(true);
@@ -445,6 +538,11 @@ import org.slf4j.LoggerFactory;
       struct.validate();
 
       oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.catName != null) {
+        oprot.writeFieldBegin(CAT_NAME_FIELD_DESC);
+        oprot.writeString(struct.catName);
+        oprot.writeFieldEnd();
+      }
       if (struct.db_name != null) {
         oprot.writeFieldBegin(DB_NAME_FIELD_DESC);
         oprot.writeString(struct.db_name);
@@ -472,6 +570,7 @@ import org.slf4j.LoggerFactory;
     @Override
     public void write(org.apache.thrift.protocol.TProtocol prot, NotNullConstraintsRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.catName);
       oprot.writeString(struct.db_name);
       oprot.writeString(struct.tbl_name);
     }
@@ -479,6 +578,8 @@ import org.slf4j.LoggerFactory;
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, NotNullConstraintsRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.catName = iprot.readString();
+      struct.setCatNameIsSet(true);
       struct.db_name = iprot.readString();
       struct.setDb_nameIsSet(true);
       struct.tbl_name = iprot.readString();

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotNullConstraintsResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotNullConstraintsResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotNullConstraintsResponse.java
index a049468..002ca13 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotNullConstraintsResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotNullConstraintsResponse.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // NOT_NULL_CONSTRAINTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list336 = iprot.readListBegin();
-                struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list336.size);
-                SQLNotNullConstraint _elem337;
-                for (int _i338 = 0; _i338 < _list336.size; ++_i338)
+                org.apache.thrift.protocol.TList _list344 = iprot.readListBegin();
+                struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list344.size);
+                SQLNotNullConstraint _elem345;
+                for (int _i346 = 0; _i346 < _list344.size; ++_i346)
                 {
-                  _elem337 = new SQLNotNullConstraint();
-                  _elem337.read(iprot);
-                  struct.notNullConstraints.add(_elem337);
+                  _elem345 = new SQLNotNullConstraint();
+                  _elem345.read(iprot);
+                  struct.notNullConstraints.add(_elem345);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(NOT_NULL_CONSTRAINTS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.notNullConstraints.size()));
-          for (SQLNotNullConstraint _iter339 : struct.notNullConstraints)
+          for (SQLNotNullConstraint _iter347 : struct.notNullConstraints)
           {
-            _iter339.write(oprot);
+            _iter347.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.notNullConstraints.size());
-        for (SQLNotNullConstraint _iter340 : struct.notNullConstraints)
+        for (SQLNotNullConstraint _iter348 : struct.notNullConstraints)
         {
-          _iter340.write(oprot);
+          _iter348.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, NotNullConstraintsResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list341 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list341.size);
-        SQLNotNullConstraint _elem342;
-        for (int _i343 = 0; _i343 < _list341.size; ++_i343)
+        org.apache.thrift.protocol.TList _list349 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list349.size);
+        SQLNotNullConstraint _elem350;
+        for (int _i351 = 0; _i351 < _list349.size; ++_i351)
         {
-          _elem342 = new SQLNotNullConstraint();
-          _elem342.read(iprot);
-          struct.notNullConstraints.add(_elem342);
+          _elem350 = new SQLNotNullConstraint();
+          _elem350.read(iprot);
+          struct.notNullConstraints.add(_elem350);
         }
       }
       struct.setNotNullConstraintsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEvent.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEvent.java
index 49ede82..e0e1cd4 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEvent.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEvent.java
@@ -45,6 +45,7 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)5);
   private static final org.apache.thrift.protocol.TField MESSAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("message", org.apache.thrift.protocol.TType.STRING, (short)6);
   private static final org.apache.thrift.protocol.TField MESSAGE_FORMAT_FIELD_DESC = new org.apache.thrift.protocol.TField("messageFormat", org.apache.thrift.protocol.TType.STRING, (short)7);
+  private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)8);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -59,6 +60,7 @@ import org.slf4j.LoggerFactory;
   private String tableName; // optional
   private String message; // required
   private String messageFormat; // optional
+  private String catName; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -68,7 +70,8 @@ import org.slf4j.LoggerFactory;
     DB_NAME((short)4, "dbName"),
     TABLE_NAME((short)5, "tableName"),
     MESSAGE((short)6, "message"),
-    MESSAGE_FORMAT((short)7, "messageFormat");
+    MESSAGE_FORMAT((short)7, "messageFormat"),
+    CAT_NAME((short)8, "catName");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -97,6 +100,8 @@ import org.slf4j.LoggerFactory;
           return MESSAGE;
         case 7: // MESSAGE_FORMAT
           return MESSAGE_FORMAT;
+        case 8: // CAT_NAME
+          return CAT_NAME;
         default:
           return null;
       }
@@ -140,7 +145,7 @@ import org.slf4j.LoggerFactory;
   private static final int __EVENTID_ISSET_ID = 0;
   private static final int __EVENTTIME_ISSET_ID = 1;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.DB_NAME,_Fields.TABLE_NAME,_Fields.MESSAGE_FORMAT};
+  private static final _Fields optionals[] = {_Fields.DB_NAME,_Fields.TABLE_NAME,_Fields.MESSAGE_FORMAT,_Fields.CAT_NAME};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -158,6 +163,8 @@ import org.slf4j.LoggerFactory;
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.MESSAGE_FORMAT, new org.apache.thrift.meta_data.FieldMetaData("messageFormat", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(NotificationEvent.class, metaDataMap);
   }
@@ -202,6 +209,9 @@ import org.slf4j.LoggerFactory;
     if (other.isSetMessageFormat()) {
       this.messageFormat = other.messageFormat;
     }
+    if (other.isSetCatName()) {
+      this.catName = other.catName;
+    }
   }
 
   public NotificationEvent deepCopy() {
@@ -219,6 +229,7 @@ import org.slf4j.LoggerFactory;
     this.tableName = null;
     this.message = null;
     this.messageFormat = null;
+    this.catName = null;
   }
 
   public long getEventId() {
@@ -380,6 +391,29 @@ import org.slf4j.LoggerFactory;
     }
   }
 
+  public String getCatName() {
+    return this.catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
+  public void unsetCatName() {
+    this.catName = null;
+  }
+
+  /** Returns true if field catName is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatName() {
+    return this.catName != null;
+  }
+
+  public void setCatNameIsSet(boolean value) {
+    if (!value) {
+      this.catName = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case EVENT_ID:
@@ -438,6 +472,14 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case CAT_NAME:
+      if (value == null) {
+        unsetCatName();
+      } else {
+        setCatName((String)value);
+      }
+      break;
+
     }
   }
 
@@ -464,6 +506,9 @@ import org.slf4j.LoggerFactory;
     case MESSAGE_FORMAT:
       return getMessageFormat();
 
+    case CAT_NAME:
+      return getCatName();
+
     }
     throw new IllegalStateException();
   }
@@ -489,6 +534,8 @@ import org.slf4j.LoggerFactory;
       return isSetMessage();
     case MESSAGE_FORMAT:
       return isSetMessageFormat();
+    case CAT_NAME:
+      return isSetCatName();
     }
     throw new IllegalStateException();
   }
@@ -569,6 +616,15 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_catName = true && this.isSetCatName();
+    boolean that_present_catName = true && that.isSetCatName();
+    if (this_present_catName || that_present_catName) {
+      if (!(this_present_catName && that_present_catName))
+        return false;
+      if (!this.catName.equals(that.catName))
+        return false;
+    }
+
     return true;
   }
 
@@ -611,6 +667,11 @@ import org.slf4j.LoggerFactory;
     if (present_messageFormat)
       list.add(messageFormat);
 
+    boolean present_catName = true && (isSetCatName());
+    list.add(present_catName);
+    if (present_catName)
+      list.add(catName);
+
     return list.hashCode();
   }
 
@@ -692,6 +753,16 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetCatName()).compareTo(other.isSetCatName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catName, other.catName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -765,6 +836,16 @@ import org.slf4j.LoggerFactory;
       }
       first = false;
     }
+    if (isSetCatName()) {
+      if (!first) sb.append(", ");
+      sb.append("catName:");
+      if (this.catName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.catName);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -882,6 +963,14 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 8: // CAT_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.catName = iprot.readString();
+              struct.setCatNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -932,6 +1021,13 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldEnd();
         }
       }
+      if (struct.catName != null) {
+        if (struct.isSetCatName()) {
+          oprot.writeFieldBegin(CAT_NAME_FIELD_DESC);
+          oprot.writeString(struct.catName);
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -963,7 +1059,10 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetMessageFormat()) {
         optionals.set(2);
       }
-      oprot.writeBitSet(optionals, 3);
+      if (struct.isSetCatName()) {
+        optionals.set(3);
+      }
+      oprot.writeBitSet(optionals, 4);
       if (struct.isSetDbName()) {
         oprot.writeString(struct.dbName);
       }
@@ -973,6 +1072,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetMessageFormat()) {
         oprot.writeString(struct.messageFormat);
       }
+      if (struct.isSetCatName()) {
+        oprot.writeString(struct.catName);
+      }
     }
 
     @Override
@@ -986,7 +1088,7 @@ import org.slf4j.LoggerFactory;
       struct.setEventTypeIsSet(true);
       struct.message = iprot.readString();
       struct.setMessageIsSet(true);
-      BitSet incoming = iprot.readBitSet(3);
+      BitSet incoming = iprot.readBitSet(4);
       if (incoming.get(0)) {
         struct.dbName = iprot.readString();
         struct.setDbNameIsSet(true);
@@ -999,6 +1101,10 @@ import org.slf4j.LoggerFactory;
         struct.messageFormat = iprot.readString();
         struct.setMessageFormatIsSet(true);
       }
+      if (incoming.get(3)) {
+        struct.catName = iprot.readString();
+        struct.setCatNameIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
index ff40ab5..5045bda 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // EVENTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list684 = iprot.readListBegin();
-                struct.events = new ArrayList<NotificationEvent>(_list684.size);
-                NotificationEvent _elem685;
-                for (int _i686 = 0; _i686 < _list684.size; ++_i686)
+                org.apache.thrift.protocol.TList _list692 = iprot.readListBegin();
+                struct.events = new ArrayList<NotificationEvent>(_list692.size);
+                NotificationEvent _elem693;
+                for (int _i694 = 0; _i694 < _list692.size; ++_i694)
                 {
-                  _elem685 = new NotificationEvent();
-                  _elem685.read(iprot);
-                  struct.events.add(_elem685);
+                  _elem693 = new NotificationEvent();
+                  _elem693.read(iprot);
+                  struct.events.add(_elem693);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(EVENTS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.events.size()));
-          for (NotificationEvent _iter687 : struct.events)
+          for (NotificationEvent _iter695 : struct.events)
           {
-            _iter687.write(oprot);
+            _iter695.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.events.size());
-        for (NotificationEvent _iter688 : struct.events)
+        for (NotificationEvent _iter696 : struct.events)
         {
-          _iter688.write(oprot);
+          _iter696.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, NotificationEventResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list689 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.events = new ArrayList<NotificationEvent>(_list689.size);
-        NotificationEvent _elem690;
-        for (int _i691 = 0; _i691 < _list689.size; ++_i691)
+        org.apache.thrift.protocol.TList _list697 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.events = new ArrayList<NotificationEvent>(_list697.size);
+        NotificationEvent _elem698;
+        for (int _i699 = 0; _i699 < _list697.size; ++_i699)
         {
-          _elem690 = new NotificationEvent();
-          _elem690.read(iprot);
-          struct.events.add(_elem690);
+          _elem698 = new NotificationEvent();
+          _elem698.read(iprot);
+          struct.events.add(_elem698);
         }
       }
       struct.setEventsIsSet(true);


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
index 357ac2d..773e4f8 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.metastore;
 import org.apache.hadoop.hive.metastore.api.CreationMetadata;
 import org.apache.hadoop.hive.metastore.api.ISchemaName;
 import org.apache.hadoop.hive.metastore.api.SchemaVersionDescriptor;
+import org.apache.hadoop.hive.metastore.api.Catalog;
 import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
 
 import java.nio.ByteBuffer;
@@ -140,38 +141,64 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
+  public void createCatalog(Catalog cat) throws MetaException {
+
+  }
+
+  @Override
+  public Catalog getCatalog(String catalogName) throws NoSuchObjectException, MetaException {
+    return null;
+  }
+
+  @Override
+  public void alterCatalog(String catName, Catalog cat) throws MetaException,
+      InvalidOperationException {
+
+  }
+
+  @Override
+  public List<String> getCatalogs() throws MetaException {
+    return null;
+  }
+
+  @Override
+  public void dropCatalog(String catalogName) throws NoSuchObjectException, MetaException {
+
+  }
+
+  @Override
   public void createDatabase(Database db) throws InvalidObjectException, MetaException {
 
 
   }
 
   @Override
-  public Database getDatabase(String name) throws NoSuchObjectException {
+  public Database getDatabase(String catName, String name) throws NoSuchObjectException {
 
     return null;
   }
 
   @Override
-  public boolean dropDatabase(String dbname) throws NoSuchObjectException, MetaException {
+  public boolean dropDatabase(String catName, String dbname) throws NoSuchObjectException, MetaException {
 
     return false;
   }
 
   @Override
-  public boolean alterDatabase(String dbname, Database db) throws NoSuchObjectException,
+  public boolean alterDatabase(String catName, String dbname, Database db) throws NoSuchObjectException,
       MetaException {
 
     return false;
   }
 
   @Override
-  public List<String> getDatabases(String pattern) throws MetaException {
+  public List<String> getDatabases(String catName, String pattern) throws MetaException {
 
     return Collections.emptyList();
   }
 
   @Override
-  public List<String> getAllDatabases() throws MetaException {
+  public List<String> getAllDatabases(String catName) throws MetaException {
 
     return Collections.emptyList();
   }
@@ -201,13 +228,13 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
-  public boolean dropTable(String dbName, String tableName) throws MetaException {
+  public boolean dropTable(String catName, String dbName, String tableName) throws MetaException {
 
     return false;
   }
 
   @Override
-  public Table getTable(String dbName, String tableName) throws MetaException {
+  public Table getTable(String catName, String dbName, String tableName) throws MetaException {
 
     return null;
   }
@@ -219,144 +246,141 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
-  public Partition getPartition(String dbName, String tableName, List<String> part_vals)
+  public Partition getPartition(String catName, String dbName, String tableName, List<String> part_vals)
       throws MetaException, NoSuchObjectException {
 
     return null;
   }
 
   @Override
-  public boolean dropPartition(String dbName, String tableName, List<String> part_vals)
+  public boolean dropPartition(String catName, String dbName, String tableName, List<String> part_vals)
       throws MetaException {
 
     return false;
   }
 
   @Override
-  public List<Partition> getPartitions(String dbName, String tableName, int max)
+  public List<Partition> getPartitions(String catName, String dbName, String tableName, int max)
       throws MetaException {
 
     return Collections.emptyList();
   }
 
   @Override
-  public void alterTable(String dbname, String name, Table newTable) throws InvalidObjectException,
-      MetaException {
-
-
+  public void alterTable(String catName, String dbname, String name, Table newTable)
+      throws InvalidObjectException, MetaException {
   }
 
   @Override
-  public void updateCreationMetadata(String dbname, String tablename, CreationMetadata cm)
+  public void updateCreationMetadata(String catName, String dbname, String tablename, CreationMetadata cm)
       throws MetaException {
   }
 
-  @Override
-  public List<String> getTables(String dbName, String pattern) throws MetaException {
-
+  public List<String> getTables(String catName, String dbName, String pattern) throws MetaException {
     return Collections.emptyList();
   }
 
   @Override
-  public List<String> getTables(String dbName, String pattern, TableType tableType) throws MetaException {
+  public List<String> getTables(String catName, String dbName, String pattern, TableType tableType) throws MetaException {
     return Collections.emptyList();
   }
 
   @Override
-  public List<String> getMaterializedViewsForRewriting(String dbName)
+  public List<String> getMaterializedViewsForRewriting(String catName, String dbName)
       throws MetaException, NoSuchObjectException {
     return Collections.emptyList();
   }
 
   @Override
-  public List<TableMeta> getTableMeta(String dbNames, String tableNames, List<String> tableTypes)
+  public List<TableMeta> getTableMeta(String catName, String dbNames, String tableNames, List<String> tableTypes)
       throws MetaException {
     return Collections.emptyList();
   }
 
   @Override
-  public List<Table> getTableObjectsByName(String dbname, List<String> tableNames)
+  public List<Table> getTableObjectsByName(String catName, String dbname, List<String> tableNames)
       throws MetaException, UnknownDBException {
 
     return Collections.emptyList();
   }
 
   @Override
-  public List<String> getAllTables(String dbName) throws MetaException {
+  public List<String> getAllTables(String catName, String dbName) throws MetaException {
 
     return Collections.emptyList();
   }
 
   @Override
-  public List<String> listTableNamesByFilter(String dbName, String filter, short max_tables)
+  public List<String> listTableNamesByFilter(String catName, String dbName, String filter, short max_tables)
       throws MetaException, UnknownDBException {
 
     return Collections.emptyList();
   }
 
   @Override
-  public List<String> listPartitionNames(String db_name, String tbl_name, short max_parts)
+  public List<String> listPartitionNames(String catName, String db_name, String tbl_name, short max_parts)
       throws MetaException {
 
     return Collections.emptyList();
   }
 
   @Override
-  public PartitionValuesResponse listPartitionValues(String db_name, String tbl_name, List<FieldSchema> cols, boolean applyDistinct, String filter, boolean ascending, List<FieldSchema> order, long maxParts) throws MetaException {
+  public PartitionValuesResponse listPartitionValues(String catName, String db_name,
+                                                     String tbl_name, List<FieldSchema> cols,
+                                                     boolean applyDistinct, String filter,
+                                                     boolean ascending, List<FieldSchema> order,
+                                                     long maxParts) throws MetaException {
     return null;
   }
 
   @Override
-  public List<String> listPartitionNamesByFilter(String db_name, String tbl_name, String filter,
-      short max_parts) throws MetaException {
-
-    return Collections.emptyList();
-  }
-
-  @Override
-  public void alterPartition(String db_name, String tbl_name, List<String> part_vals,
+  public void alterPartition(String catName, String db_name, String tbl_name, List<String> part_vals,
       Partition new_part) throws InvalidObjectException, MetaException {
   }
 
   @Override
-  public void alterPartitions(String db_name, String tbl_name, List<List<String>> part_vals_list,
-      List<Partition> new_parts) throws InvalidObjectException, MetaException {
+  public void alterPartitions(String catName, String db_name, String tbl_name,
+                              List<List<String>> part_vals_list, List<Partition> new_parts)
+      throws InvalidObjectException, MetaException {
+
+
   }
 
   @Override
-  public List<Partition> getPartitionsByFilter(String dbName, String tblName, String filter,
-      short maxParts) throws MetaException, NoSuchObjectException {
+  public List<Partition> getPartitionsByFilter(String catName, String dbName, String tblName,
+                                               String filter, short maxParts)
+      throws MetaException, NoSuchObjectException {
 
     return Collections.emptyList();
   }
 
   @Override
-  public List<Partition> getPartitionsByNames(String dbName, String tblName,
+  public List<Partition> getPartitionsByNames(String catName, String dbName, String tblName,
       List<String> partNames) throws MetaException, NoSuchObjectException {
 
     return Collections.emptyList();
   }
 
   @Override
-  public boolean getPartitionsByExpr(String dbName, String tblName, byte[] expr,
+  public boolean getPartitionsByExpr(String catName, String dbName, String tblName, byte[] expr,
       String defaultPartitionName, short maxParts, List<Partition> result) throws TException {
     return false;
   }
 
   @Override
-  public int getNumPartitionsByFilter(String dbName, String tblName, String filter)
+  public int getNumPartitionsByFilter(String catName, String dbName, String tblName, String filter)
     throws MetaException, NoSuchObjectException {
     return -1;
   }
 
   @Override
-  public int getNumPartitionsByExpr(String dbName, String tblName, byte[] expr)
+  public int getNumPartitionsByExpr(String catName, String dbName, String tblName, byte[] expr)
       throws MetaException, NoSuchObjectException {
     return -1;
   }
 
   @Override
-  public Table markPartitionForEvent(String dbName, String tblName, Map<String, String> partVals,
+  public Table markPartitionForEvent(String catName, String dbName, String tblName, Map<String, String> partVals,
       PartitionEventType evtType) throws MetaException, UnknownTableException,
       InvalidPartitionException, UnknownPartitionException {
 
@@ -364,7 +388,7 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
-  public boolean isPartitionMarkedForEvent(String dbName, String tblName,
+  public boolean isPartitionMarkedForEvent(String catName, String dbName, String tblName,
       Map<String, String> partName, PartitionEventType evtType) throws MetaException,
       UnknownTableException, InvalidPartitionException, UnknownPartitionException {
 
@@ -407,21 +431,21 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
-  public PrincipalPrivilegeSet getDBPrivilegeSet(String dbName, String userName,
+  public PrincipalPrivilegeSet getDBPrivilegeSet(String catName, String dbName, String userName,
       List<String> groupNames) throws InvalidObjectException, MetaException {
 
     return null;
   }
 
   @Override
-  public PrincipalPrivilegeSet getTablePrivilegeSet(String dbName, String tableName,
+  public PrincipalPrivilegeSet getTablePrivilegeSet(String catName, String dbName, String tableName,
       String userName, List<String> groupNames) throws InvalidObjectException, MetaException {
 
     return null;
   }
 
   @Override
-  public PrincipalPrivilegeSet getPartitionPrivilegeSet(String dbName, String tableName,
+  public PrincipalPrivilegeSet getPartitionPrivilegeSet(String catName, String dbName, String tableName,
       String partition, String userName, List<String> groupNames) throws InvalidObjectException,
       MetaException {
 
@@ -429,7 +453,7 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
-  public PrincipalPrivilegeSet getColumnPrivilegeSet(String dbName, String tableName,
+  public PrincipalPrivilegeSet getColumnPrivilegeSet(String catName, String dbName, String tableName,
       String partitionName, String columnName, String userName, List<String> groupNames)
       throws InvalidObjectException, MetaException {
 
@@ -445,21 +469,21 @@ public class DummyRawStoreForJdoConnection implements RawStore {
 
   @Override
   public List<HiveObjectPrivilege> listPrincipalDBGrants(String principalName,
-      PrincipalType principalType, String dbName) {
+      PrincipalType principalType, String catName, String dbName) {
 
     return Collections.emptyList();
   }
 
   @Override
   public List<HiveObjectPrivilege> listAllTableGrants(String principalName,
-      PrincipalType principalType, String dbName, String tableName) {
+      PrincipalType principalType, String catName, String dbName, String tableName) {
 
     return Collections.emptyList();
   }
 
   @Override
   public List<HiveObjectPrivilege> listPrincipalPartitionGrants(String principalName,
-      PrincipalType principalType, String dbName, String tableName, List<String> partValues,
+      PrincipalType principalType, String catName, String dbName, String tableName, List<String> partValues,
       String partName) {
 
     return Collections.emptyList();
@@ -467,14 +491,14 @@ public class DummyRawStoreForJdoConnection implements RawStore {
 
   @Override
   public List<HiveObjectPrivilege> listPrincipalTableColumnGrants(String principalName,
-      PrincipalType principalType, String dbName, String tableName, String columnName) {
+      PrincipalType principalType, String catName, String dbName, String tableName, String columnName) {
 
     return Collections.emptyList();
   }
 
   @Override
   public List<HiveObjectPrivilege> listPrincipalPartitionColumnGrants(String principalName,
-      PrincipalType principalType, String dbName, String tableName, List<String> partVals,
+      PrincipalType principalType, String catName, String dbName, String tableName, List<String> partVals,
       String partName, String columnName) {
 
     return Collections.emptyList();
@@ -524,7 +548,7 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
-  public Partition getPartitionWithAuth(String dbName, String tblName, List<String> partVals,
+  public Partition getPartitionWithAuth(String catName, String dbName, String tblName, List<String> partVals,
       String user_name, List<String> group_names) throws MetaException, NoSuchObjectException,
       InvalidObjectException {
 
@@ -532,7 +556,7 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
-  public List<Partition> getPartitionsWithAuth(String dbName, String tblName, short maxParts,
+  public List<Partition> getPartitionsWithAuth(String catName, String dbName, String tblName, short maxParts,
       String userName, List<String> groupNames) throws MetaException, NoSuchObjectException,
       InvalidObjectException {
 
@@ -540,14 +564,14 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
-  public List<String> listPartitionNamesPs(String db_name, String tbl_name, List<String> part_vals,
+  public List<String> listPartitionNamesPs(String catName, String db_name, String tbl_name, List<String> part_vals,
       short max_parts) throws MetaException, NoSuchObjectException {
 
     return Collections.emptyList();
   }
 
   @Override
-  public List<Partition> listPartitionsPsWithAuth(String db_name, String tbl_name,
+  public List<Partition> listPartitionsPsWithAuth(String catName, String db_name, String tbl_name,
       List<String> part_vals, short max_parts, String userName, List<String> groupNames)
       throws MetaException, InvalidObjectException, NoSuchObjectException {
 
@@ -635,46 +659,46 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
-  public List<HiveObjectPrivilege> listDBGrantsAll(String dbName) {
+  public List<HiveObjectPrivilege> listDBGrantsAll(String catName, String dbName) {
     return Collections.emptyList();
   }
 
   @Override
-  public List<HiveObjectPrivilege> listPartitionColumnGrantsAll(String dbName, String tableName, String partitionName, String columnName) {
+  public List<HiveObjectPrivilege> listPartitionColumnGrantsAll(String catName, String dbName, String tableName, String partitionName, String columnName) {
     return Collections.emptyList();
   }
 
   @Override
-  public List<HiveObjectPrivilege> listTableGrantsAll(String dbName, String tableName) {
+  public List<HiveObjectPrivilege> listTableGrantsAll(String catName, String dbName, String tableName) {
     return Collections.emptyList();
   }
 
   @Override
-  public List<HiveObjectPrivilege> listPartitionGrantsAll(String dbName, String tableName, String partitionName) {
+  public List<HiveObjectPrivilege> listPartitionGrantsAll(String catName, String dbName, String tableName, String partitionName) {
     return Collections.emptyList();
   }
 
   @Override
-  public List<HiveObjectPrivilege> listTableColumnGrantsAll(String dbName, String tableName, String columnName) {
+  public List<HiveObjectPrivilege> listTableColumnGrantsAll(String catName, String dbName, String tableName, String columnName) {
     return Collections.emptyList();
   }
 
   @Override
-  public  ColumnStatistics getTableColumnStatistics(String dbName, String tableName,
+  public  ColumnStatistics getTableColumnStatistics(String catName, String dbName, String tableName,
       List<String> colName) throws MetaException, NoSuchObjectException {
     return null;
   }
 
   @Override
-  public boolean deleteTableColumnStatistics(String dbName, String tableName,
-                                              String colName)
+  public boolean deleteTableColumnStatistics(String catName, String dbName, String tableName,
+                                             String colName)
       throws NoSuchObjectException, MetaException, InvalidObjectException {
     return false;
   }
 
 
   @Override
-  public boolean deletePartitionColumnStatistics(String dbName, String tableName,
+  public boolean deletePartitionColumnStatistics(String catName, String dbName, String tableName,
     String partName, List<String> partVals, String colName)
     throws NoSuchObjectException, MetaException, InvalidObjectException,
     InvalidInputException {
@@ -708,31 +732,31 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
-  public List<ColumnStatistics> getPartitionColumnStatistics(String dbName,
+  public List<ColumnStatistics> getPartitionColumnStatistics(String catName, String dbName,
       String tblName, List<String> colNames, List<String> partNames)
       throws MetaException, NoSuchObjectException {
     return Collections.emptyList();
   }
 
   @Override
-  public boolean doesPartitionExist(String dbName, String tableName,
+  public boolean doesPartitionExist(String catName, String dbName, String tableName,
       List<String> partVals) throws MetaException, NoSuchObjectException {
     return false;
   }
 
   @Override
-  public boolean addPartitions(String dbName, String tblName, List<Partition> parts)
+  public boolean addPartitions(String catName, String dbName, String tblName, List<Partition> parts)
       throws InvalidObjectException, MetaException {
     return false;
   }
 
   @Override
-  public boolean addPartitions(String dbName, String tblName, PartitionSpecProxy partitionSpec, boolean ifNotExists) throws InvalidObjectException, MetaException {
+  public boolean addPartitions(String catName, String dbName, String tblName, PartitionSpecProxy partitionSpec, boolean ifNotExists) throws InvalidObjectException, MetaException {
     return false;
   }
 
   @Override
-  public void dropPartitions(String dbName, String tblName, List<String> partNames) {
+  public void dropPartitions(String catName, String dbName, String tblName, List<String> partNames) {
   }
 
   @Override
@@ -741,36 +765,36 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
-  public void alterFunction(String dbName, String funcName, Function newFunction)
+  public void alterFunction(String catName, String dbName, String funcName, Function newFunction)
       throws InvalidObjectException, MetaException {
   }
 
   @Override
-  public void dropFunction(String dbName, String funcName)
+  public void dropFunction(String catName, String dbName, String funcName)
       throws MetaException, NoSuchObjectException, InvalidObjectException,
       InvalidInputException {
   }
 
   @Override
-  public Function getFunction(String dbName, String funcName)
+  public Function getFunction(String catName, String dbName, String funcName)
       throws MetaException {
     return null;
   }
 
   @Override
-  public List<Function> getAllFunctions()
+  public List<Function> getAllFunctions(String catName)
           throws MetaException {
     return Collections.emptyList();
   }
 
   @Override
-  public List<String> getFunctions(String dbName, String pattern)
+  public List<String> getFunctions(String catName, String dbName, String pattern)
       throws MetaException {
     return Collections.emptyList();
   }
 
   @Override
-  public AggrStats get_aggr_stats_for(String dbName,
+  public AggrStats get_aggr_stats_for(String catName, String dbName,
       String tblName, List<String> partNames, List<String> colNames)
       throws MetaException {
     return null;
@@ -847,14 +871,14 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
-  public List<SQLPrimaryKey> getPrimaryKeys(String db_name, String tbl_name)
+  public List<SQLPrimaryKey> getPrimaryKeys(String catName, String db_name, String tbl_name)
     throws MetaException {
     // TODO Auto-generated method stub
     return null;
   }
 
   @Override
-  public List<SQLForeignKey> getForeignKeys(String parent_db_name,
+  public List<SQLForeignKey> getForeignKeys(String catName, String parent_db_name,
     String parent_tbl_name, String foreign_db_name, String foreign_tbl_name)
     throws MetaException {
     // TODO Auto-generated method stub
@@ -862,28 +886,28 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
-  public List<SQLUniqueConstraint> getUniqueConstraints(String db_name, String tbl_name)
+  public List<SQLUniqueConstraint> getUniqueConstraints(String catName, String db_name, String tbl_name)
     throws MetaException {
     // TODO Auto-generated method stub
     return null;
   }
 
   @Override
-  public List<SQLNotNullConstraint> getNotNullConstraints(String db_name, String tbl_name)
+  public List<SQLNotNullConstraint> getNotNullConstraints(String catName, String db_name, String tbl_name)
     throws MetaException {
     // TODO Auto-generated method stub
     return null;
   }
 
   @Override
-  public List<SQLDefaultConstraint> getDefaultConstraints(String db_name, String tbl_name)
+  public List<SQLDefaultConstraint> getDefaultConstraints(String catName, String db_name, String tbl_name)
       throws MetaException {
     // TODO Auto-generated method stub
     return null;
   }
 
   @Override
-  public List<SQLCheckConstraint> getCheckConstraints(String db_name, String tbl_name)
+  public List<SQLCheckConstraint> getCheckConstraints(String catName, String db_name, String tbl_name)
       throws MetaException {
     // TODO Auto-generated method stub
     return null;
@@ -902,8 +926,8 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
-  public void dropConstraint(String dbName, String tableName,
-  String constraintName) throws NoSuchObjectException {
+  public void dropConstraint(String catName, String dbName, String tableName,
+  String constraintName, boolean missingOk) throws NoSuchObjectException {
     // TODO Auto-generated method stub
   }
 
@@ -1049,6 +1073,13 @@ public class DummyRawStoreForJdoConnection implements RawStore {
       String poolPath) throws NoSuchObjectException, InvalidOperationException, MetaException {
   }
 
+  @Override
+  public List<ColStatsObjWithSourceInfo> getPartitionColStatsForDatabase(String catName, String dbName)
+      throws MetaException, NoSuchObjectException {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
   public void createISchema(ISchema schema) throws AlreadyExistsException, MetaException {
 
   }


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
index 1d072ad..c47856d 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
@@ -17,15 +17,13 @@
  */
 package org.apache.hadoop.hive.metastore.cache;
 
-import org.apache.hadoop.hive.metastore.api.CreationMetadata;
-import org.apache.hadoop.hive.metastore.api.ISchemaName;
-import org.apache.hadoop.hive.metastore.api.SchemaVersionDescriptor;
-import org.apache.hadoop.hive.metastore.api.SerDeInfo;
-import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
 
+import java.io.Closeable;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
@@ -35,6 +33,8 @@ import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
@@ -51,9 +51,11 @@ import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
+import org.apache.hadoop.hive.metastore.api.Catalog;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
+import org.apache.hadoop.hive.metastore.api.CreationMetadata;
 import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
@@ -61,6 +63,7 @@ import org.apache.hadoop.hive.metastore.api.FileMetadataExprType;
 import org.apache.hadoop.hive.metastore.api.Function;
 import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege;
 import org.apache.hadoop.hive.metastore.api.ISchema;
+import org.apache.hadoop.hive.metastore.api.ISchemaName;
 import org.apache.hadoop.hive.metastore.api.InvalidInputException;
 import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
 import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
@@ -95,12 +98,16 @@ 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.SchemaVersion;
+import org.apache.hadoop.hive.metastore.api.SchemaVersionDescriptor;
+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.Type;
 import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 import org.apache.hadoop.hive.metastore.api.UnknownPartitionException;
 import org.apache.hadoop.hive.metastore.api.UnknownTableException;
+import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMMapping;
 import org.apache.hadoop.hive.metastore.api.WMPool;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
@@ -117,6 +124,10 @@ import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
 
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+import static org.apache.hadoop.hive.metastore.utils.StringUtils.normalizeIdentifier;
+
 // TODO filter->expr
 // TODO functionCache
 // TODO constraintCache
@@ -187,9 +198,6 @@ public class CachedStore implements RawStore, Configurable {
   /**
    * This initializes the caches in SharedCache by getting the objects from Metastore DB via
    * ObjectStore and populating the respective caches
-   *
-   * @param rawStore
-   * @throws Exception
    */
   static void prewarm(RawStore rawStore) {
     if (isCachePrewarmed.get()) {
@@ -200,46 +208,68 @@ public class CachedStore implements RawStore, Configurable {
     while (!isCachePrewarmed.get()) {
       // Prevents throwing exceptions in our raw store calls since we're not using RawStoreProxy
       Deadline.registerIfNot(1000000);
-      List<String> dbNames;
+      Collection<String> catalogsToCache;
       try {
-        dbNames = rawStore.getAllDatabases();
-      } catch (MetaException e) {
-        // Try again
+        catalogsToCache = catalogsToCache(rawStore);
+        LOG.info("Going to cache catalogs: " +
+            org.apache.commons.lang.StringUtils.join(catalogsToCache, ", "));
+        List<Catalog> catalogs = new ArrayList<>(catalogsToCache.size());
+        for (String catName : catalogsToCache) catalogs.add(rawStore.getCatalog(catName));
+        sharedCache.populateCatalogsInCache(catalogs);
+      } catch (MetaException|NoSuchObjectException e) {
+        LOG.warn("Failed to populate catalogs in cache, going to try again", e);
+        // try again
         continue;
       }
-      LOG.info("Number of databases to prewarm: {}", dbNames.size());
-      List<Database> databases = new ArrayList<>(dbNames.size());
-      for (String dbName : dbNames) {
+      LOG.info("Finished prewarming catalogs, starting on databases");
+      List<Database> databases = new ArrayList<>();
+      for (String catName : catalogsToCache) {
         try {
-          databases.add(rawStore.getDatabase(dbName));
-        } catch (NoSuchObjectException e) {
-          // Continue with next database
-          continue;
+          List<String> dbNames = rawStore.getAllDatabases(catName);
+          LOG.info("Number of databases to prewarm in catalog {}: {}", catName, dbNames.size());
+          for (String dbName : dbNames) {
+            try {
+              databases.add(rawStore.getDatabase(catName, dbName));
+            } catch (NoSuchObjectException e) {
+              // Continue with next database
+              LOG.warn("Failed to cache database " +
+                  Warehouse.getCatalogQualifiedDbName(catName, dbName) + ", moving on", e);
+            }
+          }
+        } catch (MetaException e) {
+          LOG.warn("Failed to cache databases in catalog " + catName + ", moving on", e);
         }
       }
       sharedCache.populateDatabasesInCache(databases);
       LOG.debug(
           "Databases cache is now prewarmed. Now adding tables, partitions and statistics to the cache");
       int numberOfDatabasesCachedSoFar = 0;
-      for (String dbName : dbNames) {
-        dbName = StringUtils.normalizeIdentifier(dbName);
+      for (Database db : databases) {
+        String catName = StringUtils.normalizeIdentifier(db.getCatalogName());
+        String dbName = StringUtils.normalizeIdentifier(db.getName());
         List<String> tblNames;
         try {
-          tblNames = rawStore.getAllTables(dbName);
+          tblNames = rawStore.getAllTables(catName, dbName);
         } catch (MetaException e) {
+          LOG.warn("Failed to cache tables for database " +
+              Warehouse.getCatalogQualifiedDbName(catName, dbName) + ", moving on");
           // Continue with next database
           continue;
         }
         int numberOfTablesCachedSoFar = 0;
         for (String tblName : tblNames) {
           tblName = StringUtils.normalizeIdentifier(tblName);
-          if (!shouldCacheTable(dbName, tblName)) {
+          if (!shouldCacheTable(catName, dbName, tblName)) {
             continue;
+
           }
           Table table;
           try {
-            table = rawStore.getTable(dbName, tblName);
+            table = rawStore.getTable(catName, dbName, tblName);
           } catch (MetaException e) {
+            LOG.warn("Failed cache table " +
+                Warehouse.getCatalogQualifiedTableName(catName, dbName, tblName) +
+                ", moving on");
             // It is possible the table is deleted during fetching tables of the database,
             // in that case, continue with the next table
             continue;
@@ -253,7 +283,7 @@ public class CachedStore implements RawStore, Configurable {
             AggrStats aggrStatsAllButDefaultPartition = null;
             if (table.isSetPartitionKeys()) {
               Deadline.startTimer("getPartitions");
-              partitions = rawStore.getPartitions(dbName, tblName, Integer.MAX_VALUE);
+              partitions = rawStore.getPartitions(catName, dbName, tblName, Integer.MAX_VALUE);
               Deadline.stopTimer();
               List<String> partNames = new ArrayList<>(partitions.size());
               for (Partition p : partitions) {
@@ -263,13 +293,13 @@ public class CachedStore implements RawStore, Configurable {
                 // Get partition column stats for this table
                 Deadline.startTimer("getPartitionColumnStatistics");
                 partitionColStats =
-                    rawStore.getPartitionColumnStatistics(dbName, tblName, partNames, colNames);
+                    rawStore.getPartitionColumnStatistics(catName, dbName, tblName, partNames, colNames);
                 Deadline.stopTimer();
                 // Get aggregate stats for all partitions of a table and for all but default
                 // partition
                 Deadline.startTimer("getAggrPartitionColumnStatistics");
                 aggrStatsAllPartitions =
-                    rawStore.get_aggr_stats_for(dbName, tblName, partNames, colNames);
+                    rawStore.get_aggr_stats_for(catName, dbName, tblName, partNames, colNames);
                 Deadline.stopTimer();
                 // Remove default partition from partition names and get aggregate
                 // stats again
@@ -286,12 +316,12 @@ public class CachedStore implements RawStore, Configurable {
                 partNames.remove(defaultPartitionName);
                 Deadline.startTimer("getAggrPartitionColumnStatistics");
                 aggrStatsAllButDefaultPartition =
-                    rawStore.get_aggr_stats_for(dbName, tblName, partNames, colNames);
+                    rawStore.get_aggr_stats_for(catName, dbName, tblName, partNames, colNames);
                 Deadline.stopTimer();
               }
             } else {
               Deadline.startTimer("getTableColumnStatistics");
-              tableColStats = rawStore.getTableColumnStatistics(dbName, tblName, colNames);
+              tableColStats = rawStore.getTableColumnStatistics(catName, dbName, tblName, colNames);
               Deadline.stopTimer();
             }
             sharedCache.populateTableInCache(table, tableColStats, partitions, partitionColStats,
@@ -304,7 +334,7 @@ public class CachedStore implements RawStore, Configurable {
               tblName, ++numberOfTablesCachedSoFar, tblNames.size());
         }
         LOG.debug("Processed database: {}. Cached {} / {} databases so far.", dbName,
-            ++numberOfDatabasesCachedSoFar, dbNames.size());
+            ++numberOfDatabasesCachedSoFar, databases.size());
       }
       isCachePrewarmed.set(true);
     }
@@ -328,6 +358,17 @@ public class CachedStore implements RawStore, Configurable {
     }
   }
 
+  private static Collection<String> catalogsToCache(RawStore rs) throws MetaException {
+    Collection<String> confValue =
+        MetastoreConf.getStringCollection(rs.getConf(), ConfVars.CATALOGS_TO_CACHE);
+    if (confValue == null || confValue.isEmpty() ||
+        (confValue.size() == 1 && confValue.contains(""))) {
+      return rs.getCatalogs();
+    } else {
+      return confValue;
+    }
+  }
+
   @VisibleForTesting
   /**
    * This starts a background thread, which initially populates the SharedCache and later
@@ -426,85 +467,91 @@ public class CachedStore implements RawStore, Configurable {
     void update() {
       Deadline.registerIfNot(1000000);
       LOG.debug("CachedStore: updating cached objects");
-      List<String> dbNames;
       try {
-        dbNames = rawStore.getAllDatabases();
-      } catch (MetaException e) {
-        LOG.error("Updating CachedStore: error happen when refresh; skipping this iteration", e);
-        return;
-      }
-      // Update the database in cache
-      updateDatabases(rawStore, dbNames);
-      for (String dbName : dbNames) {
-        // Update the tables in cache
-        updateTables(rawStore, dbName);
-        List<String> tblNames;
-        try {
-          tblNames = rawStore.getAllTables(dbName);
-        } catch (MetaException e) {
-          // Continue with next database
-          continue;
-        }
-        for (String tblName : tblNames) {
-          if (!shouldCacheTable(dbName, tblName)) {
-            continue;
+        for (String catName : catalogsToCache(rawStore)) {
+          List<String> dbNames = rawStore.getAllDatabases(catName);
+          // Update the database in cache
+          updateDatabases(rawStore, catName, dbNames);
+          for (String dbName : dbNames) {
+            // Update the tables in cache
+            updateTables(rawStore, catName, dbName);
+            List<String> tblNames;
+            try {
+              tblNames = rawStore.getAllTables(catName, dbName);
+            } catch (MetaException e) {
+              // Continue with next database
+              continue;
+            }
+            for (String tblName : tblNames) {
+              if (!shouldCacheTable(catName, dbName, tblName)) {
+                continue;
+              }
+              // Update the table column stats for a table in cache
+              updateTableColStats(rawStore, catName, dbName, tblName);
+              // Update the partitions for a table in cache
+              updateTablePartitions(rawStore, catName, dbName, tblName);
+              // Update the partition col stats for a table in cache
+              updateTablePartitionColStats(rawStore, catName, dbName, tblName);
+              // Update aggregate partition column stats for a table in cache
+              updateTableAggregatePartitionColStats(rawStore, catName, dbName, tblName);
+            }
           }
-          // Update the table column stats for a table in cache
-          updateTableColStats(rawStore, dbName, tblName);
-          // Update the partitions for a table in cache
-          updateTablePartitions(rawStore, dbName, tblName);
-          // Update the partition col stats for a table in cache
-          updateTablePartitionColStats(rawStore, dbName, tblName);
-          // Update aggregate partition column stats for a table in cache
-          updateTableAggregatePartitionColStats(rawStore, dbName, tblName);
-        }
       }
       sharedCache.incrementUpdateCount();
+      } catch (MetaException e) {
+        LOG.error("Updating CachedStore: error happen when refresh; skipping this iteration", e);
+      }
     }
 
-    private void updateDatabases(RawStore rawStore, List<String> dbNames) {
-      List<Database> databases = new ArrayList<>(dbNames.size());
+
+    private void updateDatabases(RawStore rawStore, String catName, List<String> dbNames) {
+      // Prepare the list of databases
+      List<Database> databases = new ArrayList<>();
       for (String dbName : dbNames) {
         Database db;
         try {
-          db = rawStore.getDatabase(dbName);
+          db = rawStore.getDatabase(catName, dbName);
           databases.add(db);
         } catch (NoSuchObjectException e) {
-          LOG.info("Updating CachedStore: database - " + dbName + " does not exist.", e);
+          LOG.info("Updating CachedStore: database - " + catName + "." + dbName
+              + " does not exist.", e);
         }
       }
       sharedCache.refreshDatabasesInCache(databases);
     }
 
-    private void updateTables(RawStore rawStore, String dbName) {
+    private void updateTables(RawStore rawStore, String catName, String dbName) {
       List<Table> tables = new ArrayList<>();
       try {
-        List<String> tblNames = rawStore.getAllTables(dbName);
+        List<String> tblNames = rawStore.getAllTables(catName, dbName);
         for (String tblName : tblNames) {
-          if (!shouldCacheTable(dbName, tblName)) {
+          if (!shouldCacheTable(catName, dbName, tblName)) {
             continue;
           }
-          Table table = rawStore.getTable(StringUtils.normalizeIdentifier(dbName),
+          Table table = rawStore.getTable(StringUtils.normalizeIdentifier(catName),
+              StringUtils.normalizeIdentifier(dbName),
               StringUtils.normalizeIdentifier(tblName));
           tables.add(table);
         }
-        sharedCache.refreshTablesInCache(dbName, tables);
+        sharedCache.refreshTablesInCache(catName, dbName, tables);
       } catch (MetaException e) {
         LOG.debug("Unable to refresh cached tables for database: " + dbName, e);
       }
     }
 
-    private void updateTableColStats(RawStore rawStore, String dbName, String tblName) {
+
+    private void updateTableColStats(RawStore rawStore, String catName, String dbName, String tblName) {
       try {
-        Table table = rawStore.getTable(dbName, tblName);
+        Table table = rawStore.getTable(catName, dbName, tblName);
         if (!table.isSetPartitionKeys()) {
           List<String> colNames = MetaStoreUtils.getColumnNamesForTable(table);
           Deadline.startTimer("getTableColumnStatistics");
           ColumnStatistics tableColStats =
-              rawStore.getTableColumnStatistics(dbName, tblName, colNames);
+              rawStore.getTableColumnStatistics(catName, dbName, tblName, colNames);
           Deadline.stopTimer();
           if (tableColStats != null) {
-            sharedCache.refreshTableColStatsInCache(StringUtils.normalizeIdentifier(dbName),
+            sharedCache.refreshTableColStatsInCache(StringUtils.normalizeIdentifier(catName),
+                StringUtils.normalizeIdentifier(dbName),
                 StringUtils.normalizeIdentifier(tblName), tableColStats.getStatsObj());
           }
         }
@@ -513,29 +560,30 @@ public class CachedStore implements RawStore, Configurable {
       }
     }
 
-    private void updateTablePartitions(RawStore rawStore, String dbName, String tblName) {
+    private void updateTablePartitions(RawStore rawStore, String catName, String dbName, String tblName) {
       try {
         Deadline.startTimer("getPartitions");
-        List<Partition> partitions = rawStore.getPartitions(dbName, tblName, Integer.MAX_VALUE);
+        List<Partition> partitions = rawStore.getPartitions(catName, dbName, tblName, Integer.MAX_VALUE);
         Deadline.stopTimer();
-        sharedCache.refreshPartitionsInCache(StringUtils.normalizeIdentifier(dbName),
+        sharedCache.refreshPartitionsInCache(StringUtils.normalizeIdentifier(catName),
+            StringUtils.normalizeIdentifier(dbName),
             StringUtils.normalizeIdentifier(tblName), partitions);
       } catch (MetaException | NoSuchObjectException e) {
         LOG.info("Updating CachedStore: unable to read partitions of table: " + tblName, e);
       }
     }
 
-    private void updateTablePartitionColStats(RawStore rawStore, String dbName, String tblName) {
+    private void updateTablePartitionColStats(RawStore rawStore, String catName, String dbName, String tblName) {
       try {
-        Table table = rawStore.getTable(dbName, tblName);
+        Table table = rawStore.getTable(catName, dbName, tblName);
         List<String> colNames = MetaStoreUtils.getColumnNamesForTable(table);
-        List<String> partNames = rawStore.listPartitionNames(dbName, tblName, (short) -1);
+        List<String> partNames = rawStore.listPartitionNames(catName, dbName, tblName, (short) -1);
         // Get partition column stats for this table
         Deadline.startTimer("getPartitionColumnStatistics");
         List<ColumnStatistics> partitionColStats =
-            rawStore.getPartitionColumnStatistics(dbName, tblName, partNames, colNames);
+            rawStore.getPartitionColumnStatistics(catName, dbName, tblName, partNames, colNames);
         Deadline.stopTimer();
-        sharedCache.refreshPartitionColStatsInCache(dbName, tblName, partitionColStats);
+        sharedCache.refreshPartitionColStatsInCache(catName, dbName, tblName, partitionColStats);
       } catch (MetaException | NoSuchObjectException e) {
         LOG.info("Updating CachedStore: unable to read partitions of table: " + tblName, e);
       }
@@ -543,16 +591,16 @@ public class CachedStore implements RawStore, Configurable {
 
     // Update cached aggregate stats for all partitions of a table and for all
     // but default partition
-    private void updateTableAggregatePartitionColStats(RawStore rawStore, String dbName,
-        String tblName) {
+    private void updateTableAggregatePartitionColStats(RawStore rawStore, String catName, String dbName,
+                                                       String tblName) {
       try {
-        Table table = rawStore.getTable(dbName, tblName);
-        List<String> partNames = rawStore.listPartitionNames(dbName, tblName, (short) -1);
+        Table table = rawStore.getTable(catName, dbName, tblName);
+        List<String> partNames = rawStore.listPartitionNames(catName, dbName, tblName, (short) -1);
         List<String> colNames = MetaStoreUtils.getColumnNamesForTable(table);
         if ((partNames != null) && (partNames.size() > 0)) {
           Deadline.startTimer("getAggregareStatsForAllPartitions");
           AggrStats aggrStatsAllPartitions =
-              rawStore.get_aggr_stats_for(dbName, tblName, partNames, colNames);
+              rawStore.get_aggr_stats_for(catName, dbName, tblName, partNames, colNames);
           Deadline.stopTimer();
           // Remove default partition from partition names and get aggregate stats again
           List<FieldSchema> partKeys = table.getPartitionKeys();
@@ -568,9 +616,10 @@ public class CachedStore implements RawStore, Configurable {
           partNames.remove(defaultPartitionName);
           Deadline.startTimer("getAggregareStatsForAllPartitionsExceptDefault");
           AggrStats aggrStatsAllButDefaultPartition =
-              rawStore.get_aggr_stats_for(dbName, tblName, partNames, colNames);
+              rawStore.get_aggr_stats_for(catName, dbName, tblName, partNames, colNames);
           Deadline.stopTimer();
-          sharedCache.refreshAggregateStatsInCache(StringUtils.normalizeIdentifier(dbName),
+          sharedCache.refreshAggregateStatsInCache(StringUtils.normalizeIdentifier(catName),
+              StringUtils.normalizeIdentifier(dbName),
               StringUtils.normalizeIdentifier(tblName), aggrStatsAllPartitions,
               aggrStatsAllButDefaultPartition);
         }
@@ -612,19 +661,59 @@ public class CachedStore implements RawStore, Configurable {
   }
 
   @Override
+  public void createCatalog(Catalog cat) throws MetaException {
+    rawStore.createCatalog(cat);
+    sharedCache.addCatalogToCache(cat);
+  }
+
+  @Override
+  public void alterCatalog(String catName, Catalog cat) throws MetaException,
+      InvalidOperationException {
+    rawStore.alterCatalog(catName, cat);
+    sharedCache.alterCatalogInCache(StringUtils.normalizeIdentifier(catName), cat);
+  }
+
+  @Override
+  public Catalog getCatalog(String catalogName) throws NoSuchObjectException, MetaException {
+    if (!sharedCache.isCatalogCachePrewarmed()) {
+      return rawStore.getCatalog(catalogName);
+    }
+    Catalog cat = sharedCache.getCatalogFromCache(normalizeIdentifier(catalogName));
+    if (cat == null) {
+      throw new NoSuchObjectException();
+    }
+    return cat;
+  }
+
+  @Override
+  public List<String> getCatalogs() throws MetaException {
+    if (!sharedCache.isCatalogCachePrewarmed()) {
+      return rawStore.getCatalogs();
+    }
+    return sharedCache.listCachedCatalogs();
+  }
+
+  @Override
+  public void dropCatalog(String catalogName) throws NoSuchObjectException, MetaException {
+    rawStore.dropCatalog(catalogName);
+    catalogName = catalogName.toLowerCase();
+    sharedCache.removeCatalogFromCache(catalogName);
+  }
+
+  @Override
   public void createDatabase(Database db) throws InvalidObjectException, MetaException {
     rawStore.createDatabase(db);
     sharedCache.addDatabaseToCache(db);
   }
 
   @Override
-  public Database getDatabase(String dbName) throws NoSuchObjectException {
+  public Database getDatabase(String catName, String dbName) throws NoSuchObjectException {
     if (!sharedCache.isDatabaseCachePrewarmed()) {
-      return rawStore.getDatabase(dbName);
+      return rawStore.getDatabase(catName, dbName);
     }
     dbName = dbName.toLowerCase();
-    Database db =
-        sharedCache.getDatabaseFromCache(StringUtils.normalizeIdentifier(dbName));
+    Database db = sharedCache.getDatabaseFromCache(StringUtils.normalizeIdentifier(catName),
+            StringUtils.normalizeIdentifier(dbName));
     if (db == null) {
       throw new NoSuchObjectException();
     }
@@ -632,40 +721,40 @@ public class CachedStore implements RawStore, Configurable {
   }
 
   @Override
-  public boolean dropDatabase(String dbName) throws NoSuchObjectException, MetaException {
-    boolean succ = rawStore.dropDatabase(dbName);
+  public boolean dropDatabase(String catName, String dbName) throws NoSuchObjectException, MetaException {
+    boolean succ = rawStore.dropDatabase(catName, dbName);
     if (succ) {
-      dbName = dbName.toLowerCase();
-      sharedCache.removeDatabaseFromCache(StringUtils.normalizeIdentifier(dbName));
+      sharedCache.removeDatabaseFromCache(StringUtils.normalizeIdentifier(catName),
+          StringUtils.normalizeIdentifier(dbName));
     }
     return succ;
   }
 
   @Override
-  public boolean alterDatabase(String dbName, Database db)
+  public boolean alterDatabase(String catName, String dbName, Database db)
       throws NoSuchObjectException, MetaException {
-    boolean succ = rawStore.alterDatabase(dbName, db);
+    boolean succ = rawStore.alterDatabase(catName, dbName, db);
     if (succ) {
-      dbName = dbName.toLowerCase();
-      sharedCache.alterDatabaseInCache(StringUtils.normalizeIdentifier(dbName), db);
+      sharedCache.alterDatabaseInCache(StringUtils.normalizeIdentifier(catName),
+          StringUtils.normalizeIdentifier(dbName), db);
     }
     return succ;
   }
 
   @Override
-  public List<String> getDatabases(String pattern) throws MetaException {
+  public List<String> getDatabases(String catName, String pattern) throws MetaException {
     if (!sharedCache.isDatabaseCachePrewarmed()) {
-      return rawStore.getDatabases(pattern);
+      return rawStore.getDatabases(catName, pattern);
     }
-    return sharedCache.listCachedDatabases(pattern);
+    return sharedCache.listCachedDatabases(catName, pattern);
   }
 
   @Override
-  public List<String> getAllDatabases() throws MetaException {
+  public List<String> getAllDatabases(String catName) throws MetaException {
     if (!sharedCache.isDatabaseCachePrewarmed()) {
-      return rawStore.getAllDatabases();
+      return rawStore.getAllDatabases(catName);
     }
-    return sharedCache.listCachedDatabases();
+    return sharedCache.listCachedDatabases(catName);
   }
 
   @Override
@@ -704,41 +793,44 @@ public class CachedStore implements RawStore, Configurable {
   @Override
   public void createTable(Table tbl) throws InvalidObjectException, MetaException {
     rawStore.createTable(tbl);
-    String dbName = StringUtils.normalizeIdentifier(tbl.getDbName());
-    String tblName = StringUtils.normalizeIdentifier(tbl.getTableName());
-    if (!shouldCacheTable(dbName, tblName)) {
+    String catName = normalizeIdentifier(tbl.getCatName());
+    String dbName = normalizeIdentifier(tbl.getDbName());
+    String tblName = normalizeIdentifier(tbl.getTableName());
+    if (!shouldCacheTable(catName, dbName, tblName)) {
       return;
     }
     validateTableType(tbl);
-    sharedCache.addTableToCache(dbName, tblName, tbl);
+    sharedCache.addTableToCache(catName, dbName, tblName, tbl);
   }
 
   @Override
-  public boolean dropTable(String dbName, String tblName)
+  public boolean dropTable(String catName, String dbName, String tblName)
       throws MetaException, NoSuchObjectException, InvalidObjectException, InvalidInputException {
-    boolean succ = rawStore.dropTable(dbName, tblName);
+    boolean succ = rawStore.dropTable(catName, dbName, tblName);
     if (succ) {
-      dbName = StringUtils.normalizeIdentifier(dbName);
-      tblName = StringUtils.normalizeIdentifier(tblName);
-      if (!shouldCacheTable(dbName, tblName)) {
+      catName = normalizeIdentifier(catName);
+      dbName = normalizeIdentifier(dbName);
+      tblName = normalizeIdentifier(tblName);
+      if (!shouldCacheTable(catName, dbName, tblName)) {
         return succ;
       }
-      sharedCache.removeTableFromCache(dbName, tblName);
+      sharedCache.removeTableFromCache(catName, dbName, tblName);
     }
     return succ;
   }
 
   @Override
-  public Table getTable(String dbName, String tblName) throws MetaException {
+  public Table getTable(String catName, String dbName, String tblName) throws MetaException {
+    catName = normalizeIdentifier(catName);
     dbName = StringUtils.normalizeIdentifier(dbName);
     tblName = StringUtils.normalizeIdentifier(tblName);
-    if (!shouldCacheTable(dbName, tblName)) {
-      return rawStore.getTable(dbName, tblName);
+    if (!shouldCacheTable(catName, dbName, tblName)) {
+      return rawStore.getTable(catName, dbName, tblName);
     }
-    Table tbl = sharedCache.getTableFromCache(dbName, tblName);
+    Table tbl = sharedCache.getTableFromCache(catName, dbName, tblName);
     if (tbl == null) {
       // This table is not yet loaded in cache
-      return rawStore.getTable(dbName, tblName);
+      return rawStore.getTable(catName, dbName, tblName);
     }
     if (tbl != null) {
       tbl.unsetPrivileges();
@@ -751,220 +843,232 @@ public class CachedStore implements RawStore, Configurable {
   public boolean addPartition(Partition part) throws InvalidObjectException, MetaException {
     boolean succ = rawStore.addPartition(part);
     if (succ) {
-      String dbName = StringUtils.normalizeIdentifier(part.getDbName());
-      String tblName = StringUtils.normalizeIdentifier(part.getTableName());
-      if (!shouldCacheTable(dbName, tblName)) {
+      String dbName = normalizeIdentifier(part.getDbName());
+      String tblName = normalizeIdentifier(part.getTableName());
+      String catName = part.isSetCatName() ? normalizeIdentifier(part.getCatName()) : DEFAULT_CATALOG_NAME;
+      if (!shouldCacheTable(catName, dbName, tblName)) {
         return succ;
       }
-      sharedCache.addPartitionToCache(dbName, tblName, part);
+      sharedCache.addPartitionToCache(catName, dbName, tblName, part);
     }
     return succ;
   }
 
   @Override
-  public boolean addPartitions(String dbName, String tblName, List<Partition> parts)
+  public boolean addPartitions(String catName, String dbName, String tblName, List<Partition> parts)
       throws InvalidObjectException, MetaException {
-    boolean succ = rawStore.addPartitions(dbName, tblName, parts);
+    boolean succ = rawStore.addPartitions(catName, dbName, tblName, parts);
     if (succ) {
-      dbName = StringUtils.normalizeIdentifier(dbName);
-      tblName = StringUtils.normalizeIdentifier(tblName);
-      if (!shouldCacheTable(dbName, tblName)) {
+      catName = normalizeIdentifier(catName);
+      dbName = normalizeIdentifier(dbName);
+      tblName = normalizeIdentifier(tblName);
+      if (!shouldCacheTable(catName, dbName, tblName)) {
         return succ;
       }
-      sharedCache.addPartitionsToCache(dbName, tblName, parts);
+      sharedCache.addPartitionsToCache(catName, dbName, tblName, parts);
     }
     return succ;
   }
 
   @Override
-  public boolean addPartitions(String dbName, String tblName, PartitionSpecProxy partitionSpec,
+  public boolean addPartitions(String catName, String dbName, String tblName, PartitionSpecProxy partitionSpec,
       boolean ifNotExists) throws InvalidObjectException, MetaException {
-    boolean succ = rawStore.addPartitions(dbName, tblName, partitionSpec, ifNotExists);
+    boolean succ = rawStore.addPartitions(catName, dbName, tblName, partitionSpec, ifNotExists);
     if (succ) {
-      dbName = StringUtils.normalizeIdentifier(dbName);
-      tblName = StringUtils.normalizeIdentifier(tblName);
-      if (!shouldCacheTable(dbName, tblName)) {
+      catName = normalizeIdentifier(catName);
+      dbName = normalizeIdentifier(dbName);
+      tblName = normalizeIdentifier(tblName);
+      if (!shouldCacheTable(catName, dbName, tblName)) {
         return succ;
       }
       PartitionSpecProxy.PartitionIterator iterator = partitionSpec.getPartitionIterator();
       while (iterator.hasNext()) {
         Partition part = iterator.next();
-        sharedCache.addPartitionToCache(dbName, tblName, part);
+        sharedCache.addPartitionToCache(catName, dbName, tblName, part);
       }
     }
     return succ;
   }
 
   @Override
-  public Partition getPartition(String dbName, String tblName, List<String> part_vals)
+  public Partition getPartition(String catName, String dbName, String tblName, List<String> part_vals)
       throws MetaException, NoSuchObjectException {
+    catName = normalizeIdentifier(catName);
     dbName = StringUtils.normalizeIdentifier(dbName);
     tblName = StringUtils.normalizeIdentifier(tblName);
-    if (!shouldCacheTable(dbName, tblName)) {
-      return rawStore.getPartition(dbName, tblName, part_vals);
+    if (!shouldCacheTable(catName, dbName, tblName)) {
+      return rawStore.getPartition(catName, dbName, tblName, part_vals);
     }
-    Partition part = sharedCache.getPartitionFromCache(dbName, tblName, part_vals);
+    Partition part = sharedCache.getPartitionFromCache(catName, dbName, tblName, part_vals);
     if (part == null) {
       // The table containing the partition is not yet loaded in cache
-      return rawStore.getPartition(dbName, tblName, part_vals);
+      return rawStore.getPartition(catName, dbName, tblName, part_vals);
     }
     return part;
   }
 
   @Override
-  public boolean doesPartitionExist(String dbName, String tblName,
+  public boolean doesPartitionExist(String catName, String dbName, String tblName,
       List<String> part_vals) throws MetaException, NoSuchObjectException {
+    catName = normalizeIdentifier(catName);
     dbName = StringUtils.normalizeIdentifier(dbName);
     tblName = StringUtils.normalizeIdentifier(tblName);
-    if (!shouldCacheTable(dbName, tblName)) {
-      return rawStore.doesPartitionExist(dbName, tblName, part_vals);
+    if (!shouldCacheTable(catName, dbName, tblName)) {
+      return rawStore.doesPartitionExist(catName, dbName, tblName, part_vals);
     }
-    Table tbl = sharedCache.getTableFromCache(dbName, tblName);
+    Table tbl = sharedCache.getTableFromCache(catName, dbName, tblName);
     if (tbl == null) {
       // The table containing the partition is not yet loaded in cache
-      return rawStore.doesPartitionExist(dbName, tblName, part_vals);
+      return rawStore.doesPartitionExist(catName, dbName, tblName, part_vals);
     }
-    return sharedCache.existPartitionFromCache(dbName, tblName, part_vals);
+    return sharedCache.existPartitionFromCache(catName, dbName, tblName, part_vals);
   }
 
   @Override
-  public boolean dropPartition(String dbName, String tblName, List<String> part_vals)
+  public boolean dropPartition(String catName, String dbName, String tblName, List<String> part_vals)
       throws MetaException, NoSuchObjectException, InvalidObjectException, InvalidInputException {
-    boolean succ = rawStore.dropPartition(dbName, tblName, part_vals);
+    boolean succ = rawStore.dropPartition(catName, dbName, tblName, part_vals);
     if (succ) {
-      dbName = StringUtils.normalizeIdentifier(dbName);
-      tblName = StringUtils.normalizeIdentifier(tblName);
-      if (!shouldCacheTable(dbName, tblName)) {
+      catName = normalizeIdentifier(catName);
+      dbName = normalizeIdentifier(dbName);
+      tblName = normalizeIdentifier(tblName);
+      if (!shouldCacheTable(catName, dbName, tblName)) {
         return succ;
       }
-      sharedCache.removePartitionFromCache(dbName, tblName, part_vals);
+      sharedCache.removePartitionFromCache(catName, dbName, tblName, part_vals);
     }
     return succ;
   }
 
   @Override
-  public void dropPartitions(String dbName, String tblName, List<String> partNames)
+  public void dropPartitions(String catName, String dbName, String tblName, List<String> partNames)
       throws MetaException, NoSuchObjectException {
-    rawStore.dropPartitions(dbName, tblName, partNames);
+    rawStore.dropPartitions(catName, dbName, tblName, partNames);
+    catName = normalizeIdentifier(catName);
     dbName = StringUtils.normalizeIdentifier(dbName);
     tblName = StringUtils.normalizeIdentifier(tblName);
-    if (!shouldCacheTable(dbName, tblName)) {
+    if (!shouldCacheTable(catName, dbName, tblName)) {
       return;
     }
-    List<List<String>> partVals = new ArrayList<List<String>>();
+    List<List<String>> partVals = new ArrayList<>();
     for (String partName : partNames) {
       partVals.add(partNameToVals(partName));
     }
-    sharedCache.removePartitionsFromCache(dbName, tblName, partVals);
+    sharedCache.removePartitionsFromCache(catName, dbName, tblName, partVals);
   }
 
   @Override
-  public List<Partition> getPartitions(String dbName, String tblName, int max)
+  public List<Partition> getPartitions(String catName, String dbName, String tblName, int max)
       throws MetaException, NoSuchObjectException {
+    catName = normalizeIdentifier(catName);
     dbName = StringUtils.normalizeIdentifier(dbName);
     tblName = StringUtils.normalizeIdentifier(tblName);
-    if (!shouldCacheTable(dbName, tblName)) {
-      return rawStore.getPartitions(dbName, tblName, max);
+    if (!shouldCacheTable(catName, dbName, tblName)) {
+      return rawStore.getPartitions(catName, dbName, tblName, max);
     }
-    Table tbl = sharedCache.getTableFromCache(dbName, tblName);
+    Table tbl = sharedCache.getTableFromCache(catName, dbName, tblName);
     if (tbl == null) {
       // The table containing the partitions is not yet loaded in cache
-      return rawStore.getPartitions(dbName, tblName, max);
+      return rawStore.getPartitions(catName, dbName, tblName, max);
     }
-    List<Partition> parts = sharedCache.listCachedPartitions(dbName, tblName, max);
+    List<Partition> parts = sharedCache.listCachedPartitions(catName, dbName, tblName, max);
     return parts;
   }
 
   @Override
-  public void alterTable(String dbName, String tblName, Table newTable)
+  public void alterTable(String catName, String dbName, String tblName, Table newTable)
       throws InvalidObjectException, MetaException {
-    rawStore.alterTable(dbName, tblName, newTable);
-    dbName = StringUtils.normalizeIdentifier(dbName);
-    tblName = StringUtils.normalizeIdentifier(tblName);
-    String newTblName = StringUtils.normalizeIdentifier(newTable.getTableName());
-    if (!shouldCacheTable(dbName, tblName) && !shouldCacheTable(dbName, newTblName)) {
+    rawStore.alterTable(catName, dbName, tblName, newTable);
+    catName = normalizeIdentifier(catName);
+    dbName = normalizeIdentifier(dbName);
+    tblName = normalizeIdentifier(tblName);
+    String newTblName = normalizeIdentifier(newTable.getTableName());
+    if (!shouldCacheTable(catName, dbName, tblName) &&
+        !shouldCacheTable(catName, dbName, newTblName)) {
       return;
     }
-    Table tbl = sharedCache.getTableFromCache(dbName, tblName);
+    Table tbl = sharedCache.getTableFromCache(catName, dbName, tblName);
     if (tbl == null) {
       // The table is not yet loaded in cache
       return;
     }
-    if (shouldCacheTable(dbName, tblName) && shouldCacheTable(dbName, newTblName)) {
+    if (shouldCacheTable(catName, dbName, tblName) && shouldCacheTable(catName, dbName, newTblName)) {
       // If old table is in the cache and the new table can also be cached
-      sharedCache.alterTableInCache(dbName, tblName, newTable);
-    } else if (!shouldCacheTable(dbName, tblName) && shouldCacheTable(dbName, newTblName)) {
+      sharedCache.alterTableInCache(catName, dbName, tblName, newTable);
+    } else if (!shouldCacheTable(catName, dbName, tblName) && shouldCacheTable(catName, dbName, newTblName)) {
       // If old table is *not* in the cache but the new table can be cached
-      sharedCache.addTableToCache(dbName, newTblName, newTable);
-    } else if (shouldCacheTable(dbName, tblName) && !shouldCacheTable(dbName, newTblName)) {
+      sharedCache.addTableToCache(catName, dbName, newTblName, newTable);
+    } else if (shouldCacheTable(catName, dbName, tblName) && !shouldCacheTable(catName, dbName, newTblName)) {
       // If old table is in the cache but the new table *cannot* be cached
-      sharedCache.removeTableFromCache(dbName, tblName);
+      sharedCache.removeTableFromCache(catName, dbName, tblName);
     }
   }
 
   @Override
-  public void updateCreationMetadata(String dbname, String tablename, CreationMetadata cm)
+  public void updateCreationMetadata(String catName, String dbname, String tablename, CreationMetadata cm)
       throws MetaException {
-    rawStore.updateCreationMetadata(dbname, tablename, cm);
+    rawStore.updateCreationMetadata(catName, dbname, tablename, cm);
   }
 
   @Override
-  public List<String> getTables(String dbName, String pattern) throws MetaException {
+  public List<String> getTables(String catName, String dbName, String pattern) throws MetaException {
     if (!isBlacklistWhitelistEmpty(conf) || !isCachePrewarmed.get()) {
-      return rawStore.getTables(dbName, pattern);
+      return rawStore.getTables(catName, dbName, pattern);
     }
-    return sharedCache.listCachedTableNames(StringUtils.normalizeIdentifier(dbName), pattern,
-        (short) -1);
+    return sharedCache.listCachedTableNames(StringUtils.normalizeIdentifier(catName),
+        StringUtils.normalizeIdentifier(dbName), pattern, (short) -1);
   }
 
   @Override
-  public List<String> getTables(String dbName, String pattern, TableType tableType)
+  public List<String> getTables(String catName, String dbName, String pattern, TableType tableType)
       throws MetaException {
     if (!isBlacklistWhitelistEmpty(conf) || !isCachePrewarmed.get()) {
-      return rawStore.getTables(dbName, pattern, tableType);
+      return rawStore.getTables(catName, dbName, pattern, tableType);
     }
-    return sharedCache.listCachedTableNames(StringUtils.normalizeIdentifier(dbName), pattern,
-        tableType);
+    return sharedCache.listCachedTableNames(StringUtils.normalizeIdentifier(catName),
+        StringUtils.normalizeIdentifier(dbName), pattern, tableType);
   }
 
   @Override
-  public List<String> getMaterializedViewsForRewriting(String dbName)
+  public List<String> getMaterializedViewsForRewriting(String catName, String dbName)
       throws MetaException, NoSuchObjectException {
-    return rawStore.getMaterializedViewsForRewriting(dbName);
+    return rawStore.getMaterializedViewsForRewriting(catName, dbName);
   }
 
   @Override
-  public List<TableMeta> getTableMeta(String dbNames, String tableNames, List<String> tableTypes)
-      throws MetaException {
+  public List<TableMeta> getTableMeta(String catName, String dbNames, String tableNames,
+                                      List<String> tableTypes) throws MetaException {
     // TODO Check if all required tables are allowed, if so, get it from cache
     if (!isBlacklistWhitelistEmpty(conf) || !isCachePrewarmed.get()) {
-      return rawStore.getTableMeta(dbNames, tableNames, tableTypes);
+      return rawStore.getTableMeta(catName, dbNames, tableNames, tableTypes);
     }
-    return sharedCache.getTableMeta(StringUtils.normalizeIdentifier(dbNames),
+    return sharedCache.getTableMeta(StringUtils.normalizeIdentifier(catName),
+        StringUtils.normalizeIdentifier(dbNames),
         StringUtils.normalizeIdentifier(tableNames), tableTypes);
   }
 
   @Override
-  public List<Table> getTableObjectsByName(String dbName, List<String> tblNames)
+  public List<Table> getTableObjectsByName(String catName, String dbName, List<String> tblNames)
       throws MetaException, UnknownDBException {
-    dbName = StringUtils.normalizeIdentifier(dbName);
+    dbName = normalizeIdentifier(dbName);
+    catName = normalizeIdentifier(catName);
     boolean missSomeInCache = false;
     for (String tblName : tblNames) {
-      tblName = StringUtils.normalizeIdentifier(tblName);
-      if (!shouldCacheTable(dbName, tblName)) {
+      tblName = normalizeIdentifier(tblName);
+      if (!shouldCacheTable(catName, dbName, tblName)) {
         missSomeInCache = true;
         break;
       }
     }
     if (!isCachePrewarmed.get() || missSomeInCache) {
-      return rawStore.getTableObjectsByName(dbName, tblNames);
+      return rawStore.getTableObjectsByName(catName, dbName, tblNames);
     }
     List<Table> tables = new ArrayList<>();
     for (String tblName : tblNames) {
-      tblName = StringUtils.normalizeIdentifier(tblName);
-      Table tbl = sharedCache.getTableFromCache(dbName, tblName);
+      tblName = normalizeIdentifier(tblName);
+      Table tbl = sharedCache.getTableFromCache(catName, dbName, tblName);
       if (tbl == null) {
-        tbl = rawStore.getTable(dbName, tblName);
+        tbl = rawStore.getTable(catName, dbName, tblName);
       }
       tables.add(tbl);
     }
@@ -972,39 +1076,42 @@ public class CachedStore implements RawStore, Configurable {
   }
 
   @Override
-  public List<String> getAllTables(String dbName) throws MetaException {
+  public List<String> getAllTables(String catName, String dbName) throws MetaException {
     if (!isBlacklistWhitelistEmpty(conf) || !isCachePrewarmed.get()) {
-      return rawStore.getAllTables(dbName);
+      return rawStore.getAllTables(catName, dbName);
     }
-    return sharedCache.listCachedTableNames(StringUtils.normalizeIdentifier(dbName));
+    return sharedCache.listCachedTableNames(StringUtils.normalizeIdentifier(catName),
+        StringUtils.normalizeIdentifier(dbName));
   }
 
   @Override
-  public List<String> listTableNamesByFilter(String dbName, String filter, short max_tables)
+  public List<String> listTableNamesByFilter(String catName, String dbName, String filter,
+                                             short max_tables)
       throws MetaException, UnknownDBException {
     if (!isBlacklistWhitelistEmpty(conf) || !isCachePrewarmed.get()) {
-      return rawStore.listTableNamesByFilter(dbName, filter, max_tables);
+      return rawStore.listTableNamesByFilter(catName, dbName, filter, max_tables);
     }
-    return sharedCache.listCachedTableNames(StringUtils.normalizeIdentifier(dbName), filter,
-        max_tables);
+    return sharedCache.listCachedTableNames(StringUtils.normalizeIdentifier(catName),
+        StringUtils.normalizeIdentifier(dbName), filter, max_tables);
   }
 
   @Override
-  public List<String> listPartitionNames(String dbName, String tblName,
+  public List<String> listPartitionNames(String catName, String dbName, String tblName,
       short max_parts) throws MetaException {
+    catName = StringUtils.normalizeIdentifier(catName);
     dbName = StringUtils.normalizeIdentifier(dbName);
     tblName = StringUtils.normalizeIdentifier(tblName);
-    if (!shouldCacheTable(dbName, tblName)) {
-      return rawStore.listPartitionNames(dbName, tblName, max_parts);
+    if (!shouldCacheTable(catName, dbName, tblName)) {
+      return rawStore.listPartitionNames(catName, dbName, tblName, max_parts);
     }
-    Table tbl = sharedCache.getTableFromCache(dbName, tblName);
+    Table tbl = sharedCache.getTableFromCache(catName, dbName, tblName);
     if (tbl == null) {
       // The table is not yet loaded in cache
-      return rawStore.listPartitionNames(dbName, tblName, max_parts);
+      return rawStore.listPartitionNames(catName, dbName, tblName, max_parts);
     }
     List<String> partitionNames = new ArrayList<>();
     int count = 0;
-    for (Partition part : sharedCache.listCachedPartitions(dbName, tblName, max_parts)) {
+    for (Partition part : sharedCache.listCachedPartitions(catName, dbName, tblName, max_parts)) {
       if (max_parts == -1 || count < max_parts) {
         partitionNames.add(Warehouse.makePartName(tbl.getPartitionKeys(), part.getValues()));
       }
@@ -1013,48 +1120,45 @@ public class CachedStore implements RawStore, Configurable {
   }
 
   @Override
-  public PartitionValuesResponse listPartitionValues(String db_name, String tbl_name,
+  public PartitionValuesResponse listPartitionValues(String catName, String db_name, String tbl_name,
       List<FieldSchema> cols, boolean applyDistinct, String filter, boolean ascending,
       List<FieldSchema> order, long maxParts) throws MetaException {
     throw new UnsupportedOperationException();
   }
 
   @Override
-  public List<String> listPartitionNamesByFilter(String dbName,
-      String tblName, String filter, short max_parts) throws MetaException {
-    // TODO Translate filter -> expr
-    return rawStore.listPartitionNamesByFilter(dbName, tblName, filter, max_parts);
-  }
-
-  @Override
-  public void alterPartition(String dbName, String tblName, List<String> partVals, Partition newPart)
-      throws InvalidObjectException, MetaException {
-    rawStore.alterPartition(dbName, tblName, partVals, newPart);
-    dbName = StringUtils.normalizeIdentifier(dbName);
-    tblName = StringUtils.normalizeIdentifier(tblName);
-    if (!shouldCacheTable(dbName, tblName)) {
+  public void alterPartition(String catName, String dbName, String tblName, List<String> partVals,
+                             Partition newPart) throws InvalidObjectException, MetaException {
+    rawStore.alterPartition(catName, dbName, tblName, partVals, newPart);
+    catName = normalizeIdentifier(catName);
+    dbName = normalizeIdentifier(dbName);
+    tblName = normalizeIdentifier(tblName);
+    if (!shouldCacheTable(catName, dbName, tblName)) {
       return;
     }
-    sharedCache.alterPartitionInCache(dbName, tblName, partVals, newPart);
+    sharedCache.alterPartitionInCache(catName, dbName, tblName, partVals, newPart);
   }
 
   @Override
-  public void alterPartitions(String dbName, String tblName, List<List<String>> partValsList,
-      List<Partition> newParts) throws InvalidObjectException, MetaException {
-    rawStore.alterPartitions(dbName, tblName, partValsList, newParts);
-    dbName = StringUtils.normalizeIdentifier(dbName);
-    tblName = StringUtils.normalizeIdentifier(tblName);
-    if (!shouldCacheTable(dbName, tblName)) {
+  public void alterPartitions(String catName, String dbName, String tblName,
+                              List<List<String>> partValsList, List<Partition> newParts)
+      throws InvalidObjectException, MetaException {
+    rawStore.alterPartitions(catName, dbName, tblName, partValsList, newParts);
+    catName = normalizeIdentifier(catName);
+    dbName = normalizeIdentifier(dbName);
+    tblName = normalizeIdentifier(tblName);
+    if (!shouldCacheTable(catName, dbName, tblName)) {
       return;
     }
-    sharedCache.alterPartitionsInCache(dbName, tblName, partValsList, newParts);
+    sharedCache.alterPartitionsInCache(catName, dbName, tblName, partValsList, newParts);
   }
 
   private boolean getPartitionNamesPrunedByExprNoTxn(Table table, byte[] expr,
       String defaultPartName, short maxParts, List<String> result, SharedCache sharedCache)
       throws MetaException, NoSuchObjectException {
     List<Partition> parts =
-        sharedCache.listCachedPartitions(StringUtils.normalizeIdentifier(table.getDbName()),
+        sharedCache.listCachedPartitions(StringUtils.normalizeIdentifier(table.getCatName()),
+            StringUtils.normalizeIdentifier(table.getDbName()),
             StringUtils.normalizeIdentifier(table.getTableName()), maxParts);
     for (Partition part : parts) {
       result.add(Warehouse.makePartName(table.getPartitionKeys(), part.getValues()));
@@ -1067,26 +1171,27 @@ public class CachedStore implements RawStore, Configurable {
   }
 
   @Override
-  public List<Partition> getPartitionsByFilter(String dbName, String tblName,
+  public List<Partition> getPartitionsByFilter(String catName, String dbName, String tblName,
       String filter, short maxParts)
       throws MetaException, NoSuchObjectException {
-    return rawStore.getPartitionsByFilter(dbName, tblName, filter, maxParts);
+    return rawStore.getPartitionsByFilter(catName, dbName, tblName, filter, maxParts);
   }
 
   @Override
-  public boolean getPartitionsByExpr(String dbName, String tblName, byte[] expr,
+  public boolean getPartitionsByExpr(String catName, String dbName, String tblName, byte[] expr,
       String defaultPartitionName, short maxParts, List<Partition> result) throws TException {
+    catName = StringUtils.normalizeIdentifier(catName);
     dbName = StringUtils.normalizeIdentifier(dbName);
     tblName = StringUtils.normalizeIdentifier(tblName);
-    if (!shouldCacheTable(dbName, tblName)) {
-      return rawStore.getPartitionsByExpr(dbName, tblName, expr, defaultPartitionName, maxParts,
+    if (!shouldCacheTable(catName, dbName, tblName)) {
+      return rawStore.getPartitionsByExpr(catName, dbName, tblName, expr, defaultPartitionName, maxParts,
           result);
     }
     List<String> partNames = new LinkedList<>();
-    Table table = sharedCache.getTableFromCache(dbName, tblName);
+    Table table = sharedCache.getTableFromCache(catName, dbName, tblName);
     if (table == null) {
       // The table is not yet loaded in cache
-      return rawStore.getPartitionsByExpr(dbName, tblName, expr, defaultPartitionName, maxParts,
+      return rawStore.getPartitionsByExpr(catName, dbName, tblName, expr, defaultPartitionName, maxParts,
           result);
     }
     boolean hasUnknownPartitions = getPartitionNamesPrunedByExprNoTxn(table, expr,
@@ -1095,25 +1200,26 @@ public class CachedStore implements RawStore, Configurable {
   }
 
   @Override
-  public int getNumPartitionsByFilter(String dbName, String tblName, String filter)
+  public int getNumPartitionsByFilter(String catName, String dbName, String tblName, String filter)
       throws MetaException, NoSuchObjectException {
-    return rawStore.getNumPartitionsByFilter(dbName, tblName, filter);
+    return rawStore.getNumPartitionsByFilter(catName, dbName, tblName, filter);
   }
 
   @Override
-  public int getNumPartitionsByExpr(String dbName, String tblName, byte[] expr)
+  public int getNumPartitionsByExpr(String catName, String dbName, String tblName, byte[] expr)
       throws MetaException, NoSuchObjectException {
+    catName = normalizeIdentifier(catName);
     dbName = StringUtils.normalizeIdentifier(dbName);
     tblName = StringUtils.normalizeIdentifier(tblName);
-    if (!shouldCacheTable(dbName, tblName)) {
-      return rawStore.getNumPartitionsByExpr(dbName, tblName, expr);
+    if (!shouldCacheTable(catName, dbName, tblName)) {
+      return rawStore.getNumPartitionsByExpr(catName, dbName, tblName, expr);
     }
     String defaultPartName = MetastoreConf.getVar(getConf(), ConfVars.DEFAULTPARTITIONNAME);
     List<String> partNames = new LinkedList<>();
-    Table table = sharedCache.getTableFromCache(dbName, tblName);
+    Table table = sharedCache.getTableFromCache(catName, dbName, tblName);
     if (table == null) {
       // The table is not yet loaded in cache
-      return rawStore.getNumPartitionsByExpr(dbName, tblName, expr);
+      return rawStore.getNumPartitionsByExpr(catName, dbName, tblName, expr);
     }
     getPartitionNamesPrunedByExprNoTxn(table, expr, defaultPartName, Short.MAX_VALUE, partNames,
         sharedCache);
@@ -1133,21 +1239,22 @@ public class CachedStore implements RawStore, Configurable {
   }
 
   @Override
-  public List<Partition> getPartitionsByNames(String dbName, String tblName,
+  public List<Partition> getPartitionsByNames(String catName, String dbName, String tblName,
       List<String> partNames) throws MetaException, NoSuchObjectException {
+    catName = StringUtils.normalizeIdentifier(catName);
     dbName = StringUtils.normalizeIdentifier(dbName);
     tblName = StringUtils.normalizeIdentifier(tblName);
-    if (!shouldCacheTable(dbName, tblName)) {
-      return rawStore.getPartitionsByNames(dbName, tblName, partNames);
+    if (!shouldCacheTable(catName, dbName, tblName)) {
+      return rawStore.getPartitionsByNames(catName, dbName, tblName, partNames);
     }
-    Table table = sharedCache.getTableFromCache(dbName, tblName);
+    Table table = sharedCache.getTableFromCache(catName, dbName, tblName);
     if (table == null) {
       // The table is not yet loaded in cache
-      return rawStore.getPartitionsByNames(dbName, tblName, partNames);
+      return rawStore.getPartitionsByNames(catName, dbName, tblName, partNames);
     }
     List<Partition> partitions = new ArrayList<>();
     for (String partName : partNames) {
-      Partition part = sharedCache.getPartitionFromCache(dbName, tblName, partNameToVals(partName));
+      Partition part = sharedCache.getPartitionFromCache(catName, dbName, tblName, partNameToVals(partName));
       if (part!=null) {
         partitions.add(part);
       }
@@ -1156,19 +1263,19 @@ public class CachedStore implements RawStore, Configurable {
   }
 
   @Override
-  public Table markPartitionForEvent(String dbName, String tblName,
+  public Table markPartitionForEvent(String catName, String dbName, String tblName,
       Map<String, String> partVals, PartitionEventType evtType)
       throws MetaException, UnknownTableException, InvalidPartitionException,
       UnknownPartitionException {
-    return rawStore.markPartitionForEvent(dbName, tblName, partVals, evtType);
+    return rawStore.markPartitionForEvent(catName, dbName, tblName, partVals, evtType);
   }
 
   @Override
-  public boolean isPartitionMarkedForEvent(String dbName, String tblName,
+  public boolean isPartitionMarkedForEvent(String catName, String dbName, String tblName,
       Map<String, String> partName, PartitionEventType evtType)
       throws MetaException, UnknownTableException, InvalidPartitionException,
       UnknownPartitionException {
-    return rawStore.isPartitionMarkedForEvent(dbName, tblName, partName, evtType);
+    return rawStore.isPartitionMarkedForEvent(catName, dbName, tblName, partName, evtType);
   }
 
   @Override
@@ -1205,31 +1312,31 @@ public class CachedStore implements RawStore, Configurable {
   }
 
   @Override
-  public PrincipalPrivilegeSet getDBPrivilegeSet(String dbName, String userName,
+  public PrincipalPrivilegeSet getDBPrivilegeSet(String catName, String dbName, String userName,
       List<String> groupNames) throws InvalidObjectException, MetaException {
-    return rawStore.getDBPrivilegeSet(dbName, userName, groupNames);
+    return rawStore.getDBPrivilegeSet(catName, dbName, userName, groupNames);
   }
 
   @Override
-  public PrincipalPrivilegeSet getTablePrivilegeSet(String dbName,
+  public PrincipalPrivilegeSet getTablePrivilegeSet(String catName, String dbName,
       String tableName, String userName, List<String> groupNames)
       throws InvalidObjectException, MetaException {
-    return rawStore.getTablePrivilegeSet(dbName, tableName, userName, groupNames);
+    return rawStore.getTablePrivilegeSet(catName, dbName, tableName, userName, groupNames);
   }
 
   @Override
-  public PrincipalPrivilegeSet getPartitionPrivilegeSet(String dbName,
+  public PrincipalPrivilegeSet getPartitionPrivilegeSet(String catName, String dbName,
       String tableName, String partition, String userName,
       List<String> groupNames) throws InvalidObjectException, MetaException {
-    return rawStore.getPartitionPrivilegeSet(dbName, tableName, partition, userName, groupNames);
+    return rawStore.getPartitionPrivilegeSet(catName, dbName, tableName, partition, userName, groupNames);
   }
 
   @Override
-  public PrincipalPrivilegeSet getColumnPrivilegeSet(String dbName,
+  public PrincipalPrivilegeSet getColumnPrivilegeSet(String catName, String dbName,
       String tableName, String partitionName, String columnName,
       String userName, List<String> groupNames)
       throws InvalidObjectException, MetaException {
-    return rawStore.getColumnPrivilegeSet(dbName, tableName, partitionName, columnName, userName, groupNames);
+    return rawStore.getColumnPrivilegeSet(catName, dbName, tableName, partitionName, columnName, userName, groupNames);
   }
 
   @Override
@@ -1240,36 +1347,36 @@ public class CachedStore implements RawStore, Configurable {
 
   @Override
   public List<HiveObjectPrivilege> listPrincipalDBGrants(String principalName,
-      PrincipalType principalType, String dbName) {
-    return rawStore.listPrincipalDBGrants(principalName, principalType, dbName);
+      PrincipalType principalType, String catName, String dbName) {
+    return rawStore.listPrincipalDBGrants(principalName, principalType, catName, dbName);
   }
 
   @Override
   public List<HiveObjectPrivilege> listAllTableGrants(String principalName,
-      PrincipalType principalType, String dbName, String tableName) {
-    return rawStore.listAllTableGrants(principalName, principalType, dbName, tableName);
+      PrincipalType principalType, String catName, String dbName, String tableName) {
+    return rawStore.listAllTableGrants(principalName, principalType, catName, dbName, tableName);
   }
 
   @Override
   public List<HiveObjectPrivilege> listPrincipalPartitionGrants(
-      String principalName, PrincipalType principalType, String dbName,
+      String principalName, PrincipalType principalType, String catName, String dbName,
       String tableName, List<String> partValues, String partName) {
-    return rawStore.listPrincipalPartitionGrants(principalName, principalType, dbName, tableName, partValues, partName);
+    return rawStore.listPrincipalPartitionGrants(principalName, principalType, catName, dbName, tableName, partValues, partName);
   }
 
   @Override
   public List<HiveObjectPrivilege> listPrincipalTableColumnGrants(
-      String principalName, PrincipalType principalType, String dbName,
+      String principalName, PrincipalType principalType, String catName, String dbName,
       String tableName, String columnName) {
-    return rawStore.listPrincipalTableColumnGrants(principalName, principalType, dbName, tableName, columnName);
+    return rawStore.listPrincipalTableColumnGrants(principalName, principalType, catName, dbName, tableName, columnName);
   }
 
   @Override
   public List<HiveObjectPrivilege> listPrincipalPartitionColumnGrants(
-      String principalName, PrincipalType principalType, String dbName,
+      String principalName, PrincipalType principalType, String catName, String dbName,
       String tableName, List<String> partValues, String partName,
       String columnName) {
-    return rawStore.listPrincipalPartitionColumnGrants(principalName, principalType, dbName, tableName, partValues, partName, columnName);
+    return rawStore.listPrincipalPartitionColumnGrants(principalName, principalType, catName, dbName, tableName, partValues, partName, columnName);
   }
 
   @Override
@@ -1312,23 +1419,24 @@ public class CachedStore implements RawStore, Configurable {
   }
 
   @Override
-  public Partition getPartitionWithAuth(String dbName, String tblName,
+  public Partition getPartitionWithAuth(String catName, String dbName, String tblName,
       List<String> partVals, String userName, List<String> groupNames)
       throws MetaException, NoSuchObjectException, InvalidObjectException {
+    catName = StringUtils.normalizeIdentifier(catName);
     dbName = StringUtils.normalizeIdentifier(dbName);
     tblName = StringUtils.normalizeIdentifier(tblName);
-    if (!shouldCacheTable(dbName, tblName)) {
-      return rawStore.getPartitionWithAuth(dbName, tblName, partVals, userName, groupNames);
+    if (!shouldCacheTable(catName, dbName, tblName)) {
+      return rawStore.getPartitionWithAuth(catName, dbName, tblName, partVals, userName, groupNames);
     }
-    Table table = sharedCache.getTableFromCache(dbName, tblName);
+    Table table = sharedCache.getTableFromCache(catName, dbName, tblName);
     if (table == null) {
       // The table is not yet loaded in cache
-      return rawStore.getPartitionWithAuth(dbName, tblName, partVals, userName, groupNames);
+      return rawStore.getPartitionWithAuth(catName, dbName, tblName, partVals, userName, groupNames);
     }
-    Partition p = sharedCache.getPartitionFromCache(dbName, tblName, partVals);
+    Partition p = sharedCache.getPartitionFromCache(catName, dbName, tblName, partVals);
     if (p != null) {
       String partName = Warehouse.makePartName(table.getPartitionKeys(), partVals);
-      PrincipalPrivilegeSet privs = getPartitionPrivilegeSet(dbName, tblName, partName,
+      PrincipalPrivilegeSet privs = getPartitionPrivilegeSet(catName, dbName, tblName, partName,
           userName, groupNames);
       p.setPrivileges(privs);
     }
@@ -1336,25 +1444,26 @@ public class CachedStore implements RawStore, Configurable {
   }
 
   @Override
-  public List<Partition> getPartitionsWithAuth(String dbName, String tblName,
+  public List<Partition> getPartitionsWithAuth(String catName, String dbName, String tblName,
       short maxParts, String userName, List<String> groupNames)
       throws MetaException, NoSuchObjectException, InvalidObjectException {
+    catName = StringUtils.normalizeIdentifier(catName);
     dbName = StringUtils.normalizeIdentifier(dbName);
     tblName = StringUtils.normalizeIdentifier(tblName);
-    if (!shouldCacheTable(dbName, tblName)) {
-      return rawStore.getPartitionsWithAuth(dbName, tblName, maxParts, userName, groupNames);
+    if (!shouldCacheTable(catName, dbName, tblName)) {
+      return rawStore.getPartitionsWithAuth(catName, dbName, tblName, maxParts, userName, groupNames);
     }
-    Table table = sharedCache.getTableFromCache(dbName, tblName);
+    Table table = sharedCache.getTableFromCache(catName, dbName, tblName);
     if (table == null) {
       // The table is not yet loaded in cache
-      return rawStore.getPartitionsWithAuth(dbName, tblName, maxParts, userName, groupNames);
+      return rawStore.getPartitionsWithAuth(catName, dbName, tblName, maxParts, userName, groupNames);
     }
     List<Partition> partitions = new ArrayList<>();
     int count = 0;
-    for (Partition part : sharedCache.listCachedPartitions(dbName, tblName, maxParts)) {
+    for (Partition part : sharedCache.listCachedPartitions(catName, dbName, tblName, maxParts)) {
       if (maxParts == -1 || count < maxParts) {
         String partName = Warehouse.makePartName(table.getPartitionKeys(), part.getValues());
-        PrincipalPrivilegeSet privs = getPartitionPrivilegeSet(dbName, tblName, partName,
+        PrincipalPrivilegeSet privs = getPartitionPrivilegeSet(catName, dbName, tblName, partName,
             userName, groupNames);
         part.setPrivileges(privs);
         partitions.add(part);
@@ -1365,22 +1474,23 @@ public class CachedStore implements RawStore, Configurable {
   }
 
   @Override
-  public List<String> listPartitionNamesPs(String dbName, String tblName,
+  public List<String> listPartitionNamesPs(String catName, String dbName, String tblName,
       List<String> partVals, short maxParts)
       throws MetaException, NoSuchObjectException {
+    catName = StringUtils.normalizeIdentifier(catName);
     dbName = StringUtils.normalizeIdentifier(dbName);
     tblName = StringUtils.normalizeIdentifier(tblName);
-    if (!shouldCacheTable(dbName, tblName)) {
-      return rawStore.listPartitionNamesPs(dbName, tblName, partVals, maxParts);
+    if (!shouldCacheTable(catName, dbName, tblName)) {
+      return rawStore.listPartitionNamesPs(catName, dbName, tblName, partVals, maxParts);
     }
-    Table table = sharedCache.getTableFromCache(dbName, tblName);
+    Table table = sharedCache.getTableFromCache(catName, dbName, tblName);
     if (table == null) {
       // The table is not yet loaded in cache
-      return rawStore.listPartitionNamesPs(dbName, tblName, partVals, maxParts);
+      return rawStore.listPartitionNamesPs(catName, dbName, tblName, partVals, maxParts);
     }
     List<String> partNames = new ArrayList<>();
     int count = 0;
-    for (Partition part : sharedCache.listCachedPartitions(dbName, tblName, maxParts)) {
+    for (Partition part : sharedCache.listCachedPartitions(catName, dbName, tblName, maxParts)) {
       boolean psMatch = true;
       for (int i=0;i<partVals.size();i++) {
         String psVal = partVals.get(i);
@@ -1402,24 +1512,25 @@ public class CachedStore implements RawStore, Configurable {
   }
 
   @Override
-  public List<Partition> listPartitionsPsWithAuth(String dbName, String tblName,
+  public List<Partition> listPartitionsPsWithAuth(String catName, String dbName, String tblName,
       List<String> partVals, short maxParts, String userName, List<String> groupNames)
       throws MetaException, InvalidObjectException, NoSuchObjectException {
+    catName = StringUtils.normalizeIdentifier(catName);
     dbName = StringUtils.normalizeIdentifier(dbName);
     tblName = StringUtils.normalizeIdentifier(tblName);
-    if (!shouldCacheTable(dbName, tblName)) {
-      return rawStore.listPartitionsPsWithAuth(dbName, tblName, partVals, maxParts, userName,
+    if (!shouldCacheTable(catName, dbName, tblName)) {
+      return rawStore.listPartitionsPsWithAuth(catName, dbName, tblName, partVals, maxParts, userName,
           groupNames);
     }
-    Table table = sharedCache.getTableFromCache(dbName, tblName);
+    Table table = sharedCache.getTableFromCache(catName, dbName, tblName);
     if (table == null) {
       // The table is not yet loaded in cache
-      return rawStore.listPartitionsPsWithAuth(dbName, tblName, partVals, maxParts, userName,
+      return rawStore.listPartitionsPsWithAuth(catName, dbName, tblName, partVals, maxParts, userName,
           groupNames);
     }
     List<Partition> partitions = new ArrayList<>();
     int count = 0;
-    for (Partition part : sharedCache.listCachedPartitions(dbName, tblName, maxParts)) {
+    for (Partition part : sharedCache.listCachedPartitions(catName, dbName, tblName, maxParts)) {
       boolean psMatch = true;
       for (int i = 0; i < partVals.size(); i++) {
         String psVal = partVals.get(i);
@@ -1435,7 +1546,7 @@ public class CachedStore implements RawStore, Configurable {
       if (maxParts == -1 || count < maxParts) {
         String partName = Warehouse.makePartName(table.getPartitionKeys(), part.getValues());
         PrincipalPrivilegeSet privs =
-            getPartitionPrivilegeSet(dbName, tblName, partName, userName, groupNames);
+            getPartitionPrivilegeSet(catName, dbName, tblName, partName, userName, groupNames);
         part.setPrivileges(privs);
         partitions.add(part);
       }
@@ -1448,12 +1559,15 @@ public class CachedStore implements RawStore, Configurable {
       throws NoSuchObjectException, MetaException, InvalidObjectException, InvalidInputException {
     boolean succ = rawStore.updateTableColumnStatistics(colStats);
     if (succ) {
-      String dbName = StringUtils.normalizeIdentifier(colStats.getStatsDesc().getDbName());
-      String tblName = StringUtils.normalizeIdentifier(colStats.getStatsDesc().getTableName());
-      if (!shouldCacheTable(dbName, tblName)) {
+      String catName = colStats.getStatsDesc().isSetCatName() ?
+          normalizeIdentifier(colStats.getStatsDesc().getCatName()) :
+          getDefaultCatalog(conf);
+      String dbName = normalizeIdentifier(colStats.getStatsDesc().getDbName());
+      String tblName = normalizeIdentifier(colStats.getStatsDesc().getTableName());
+      if (!shouldCacheTable(catName, dbName, tblName)) {
         return succ;
       }
-      Table table = sharedCache.getTableFromCache(dbName, tblName);
+      Table table = sharedCache.getTableFromCache(catName, dbName, tblName);
       if (table == null) {
         // The table is not yet loaded in cache
         return succ;
@@ -1464,42 +1578,45 @@ public class CachedStore implements RawStore, Configurable {
         colNames.add(statsObj.getColName());
       }
       StatsSetupConst.setColumnStatsState(table.getParameters(), colNames);
-      sharedCache.alterTableInCache(dbName, tblName, table);
-      sharedCache.updateTableColStatsInCache(dbName, tblName, statsObjs);
+      sharedCache.alterTableInCache(catName, dbName, tblName, table);
+      sharedCache.updateTableColStatsInCache(catName, dbName, tblName, statsObjs);
     }
     return succ;
   }
 
   @Override
-  public ColumnStatistics getTableColumnStatistics(String dbName, String tblName,
+  public ColumnStatistics getTableColumnStatistics(String catName, String dbName, String tblName,
       List<String> colNames) throws MetaException, NoSuchObjectException {
+    catName = StringUtils.normalizeIdentifier(catName);
     dbName = StringUtils.normalizeIdentifier(dbName);
     tblName = StringUtils.normalizeIdentifier(tblName);
-    if (!shouldCacheTable(dbName, tblName)) {
-      return rawStore.getTableColumnStatistics(dbName, tblName, colNames);
+    if (!shouldCacheTable(catName, dbName, tblName)) {
+      return rawStore.getTableColumnStatistics(catName, dbName, tblName, colNames);
     }
-    Table table = sharedCache.getTableFromCache(dbName, tblName);
+    Table table = sharedCache.getTableFromCache(catName, dbName, tblName);
     if (table == null) {
       // The table is not yet loaded in cache
-      return rawStore.getTableColumnStatistics(dbName, tblName, colNames);
+      return rawStore.getTableColumnStatistics(catName, dbName, tblName, colNames);
     }
     ColumnStatisticsDesc csd = new ColumnStatisticsDesc(true, dbName, tblName);
     List<ColumnStatisticsObj> colStatObjs =
-        sharedCache.getTableColStatsFromCache(dbName, tblName, colNames);
+        sharedCache.getTableColStatsFromCache(catName, dbName, tblName, colNames);
     return new ColumnStatistics(csd, colStatObjs);
   }
 
   @Override
-  public boolean deleteTableColumnStatistics(String dbName, String tblName, String colName)
+  public boolean deleteTableColumnStatistics(String catName, String dbName, String tblName,
+                                             String colName)
       throws NoSuchObjectException, MetaException, InvalidObjectException, InvalidInputException {
-    boolean succ = rawStore.deleteTableColumnStatistics(dbName, tblName, colName);
+    boolean succ = rawStore.deleteTableColumnStatistics(catName, dbName, tblName, colName);
     if (succ) {
-      dbName = StringUtils.normalizeIdentifier(dbName);
-      tblName = StringUtils.normalizeIdentifier(tblName);
-      if (!shouldCacheTable(dbName, tblName)) {
+      catName = normalizeIdentifier(catName);
+      dbName = normalizeIdentifier(dbName);
+      tblName = normalizeIdentifier(tblName);
+      if (!shouldCacheTable(catName, dbName, tblName)) {
         return succ;
       }
-      sharedCache.removeTableColStatsFromCache(dbName, tblName, colName);
+      sharedCache.removeTableColStatsFromCache(catName, dbName, tblName, colName);
     }
     return succ;
   }
@@ -1509,65 +1626,69 @@ public class CachedStore implements RawStore, Configurable {
       throws NoSuchObjectException, MetaException, InvalidObjectException, InvalidInputException {
     boolean succ = rawStore.updatePartitionColumnStatistics(colStats, partVals);
     if (succ) {
-      String dbName = StringUtils.normalizeIdentifier(colStats.getStatsDesc().getDbName());
-      String tblName = StringUtils.normalizeIdentifier(colStats.getStatsDesc().getTableName());
-      if (!shouldCacheTable(dbName, tblName)) {
+      String catName = colStats.getStatsDesc().isSetCatName() ?
+          normalizeIdentifier(colStats.getStatsDesc().getCatName()) : DEFAULT_CATALOG_NAME;
+      String dbName = normalizeIdentifier(colStats.getStatsDesc().getDbName());
+      String tblName = normalizeIdentifier(colStats.getStatsDesc().getTableName());
+      if (!shouldCacheTable(catName, dbName, tblName)) {
         return succ;
       }
       List<ColumnStatisticsObj> statsObjs = colStats.getStatsObj();
-      Partition part = getPartition(dbName, tblName, partVals);
+      Partition part = getPartition(catName, dbName, tblName, partVals);
       List<String> colNames = new ArrayList<>();
       for (ColumnStatisticsObj statsObj : statsObjs) {
         colNames.add(statsObj.getColName());
       }
       StatsSetupConst.setColumnStatsState(part.getParameters(), colNames);
-      sharedCache.alterPartitionInCache(dbName, tblName, partVals, part);
-      sharedCache.updatePartitionColStatsInCache(dbName, tblName, partVals, colStats.getStatsObj());
+      sharedCache.alterPartitionInCache(catName, dbName, tblName, partVals, part);
+      sharedCache.updatePartitionColStatsInCache(catName, dbName, tblName, partVals, colStats.getStatsObj());
     }
     return succ;
   }
 
   @Override
   // TODO: calculate from cached values.
-  public List<ColumnStatistics> getPartitionColumnStatistics(String dbName, String tblName,
+  public List<ColumnStatistics> getPartitionColumnStatistics(String catName, String dbName, String tblName,
       List<String> partNames, List<String> colNames) throws MetaException, NoSuchObjectException {
-    return rawStore.getPartitionColumnStatistics(dbName, tblName, partNames, colNames);
+    return rawStore.getPartitionColumnStatistics(catName, dbName, tblName, partNames, colNames);
   }
 
   @Override
-  public boolean deletePartitionColumnStatistics(String dbName, String tblName, String partName,
+  public boolean deletePartitionColumnStatistics(String catName, String dbName, String tblName, String partName,
       List<String> partVals, String colName)
       throws NoSuchObjectException, MetaException, InvalidObjectException, InvalidInputException {
     boolean succ =
-        rawStore.deletePartitionColumnStatistics(dbName, tblName, partName, partVals, colName);
+        rawStore.deletePartitionColumnStatistics(catName, dbName, tblName, partName, partVals, colName);
     if (succ) {
-      dbName = StringUtils.normalizeIdentifier(dbName);
-      tblName = StringUtils.normalizeIdentifier(tblName);
-      if (!shouldCacheTable(dbName, tblName)) {
+      catName = normalizeIdentifier(catName);
+      dbName = normalizeIdentifier(dbName);
+      tblName = normalizeIdentifier(tblName);
+      if (!shouldCacheTable(catName, dbName, tblName)) {
         return succ;
       }
-      sharedCache.removePartitionColStatsFromCache(dbName, tblName, partVals, colName);
+      sharedCache.removePartitionColStatsFromCache(catName, dbName, tblName, partVals, colName);
     }
     return succ;
   }
 
   @Override
-  public AggrStats get_aggr_stats_for(String dbName, String tblName, List<String> partNames,
+  public AggrStats get_aggr_stats_for(String catName, String dbName, String tblName, List<String> partNames,
       List<String> colNames) throws MetaException, NoSuchObjectException {
     List<ColumnStatisticsObj> colStats;
+    catName = normalizeIdentifier(catName);
     dbName = StringUtils.normalizeIdentifier(dbName);
     tblName = StringUtils.normalizeIdentifier(tblName);
-    if (!shouldCacheTable(dbName, tblName)) {
-      rawStore.get_aggr_stats_for(dbName, tblName, partNames, colNames);
+    if (!shouldCacheTable(catName, dbName, tblName)) {
+      rawStore.get_aggr_stats_for(catName, dbName, tblName, partNames, colNames);
     }
-    Table table = sharedCache.getTableFromCache(dbName, tblName);
+    Table table = sharedCache.getTableFromCache(catName, dbName, tblName);
     if (table == null) {
       // The table is not yet loaded in cache
-      return rawStore.get_aggr_stats_for(dbName, tblName, partNames, colNames);
+      return rawStore.get_aggr_stats_for(catName, dbName, tblName, partNames, colNames);
     }
-    List<String> allPartNames = rawStore.listPartitionNames(dbName, tblName, (short) -1);
+    List<String> allPartNames = rawStore.listPartitionNames(catName, dbName, tblName, (short) -1);
     if (partNames.size() == allPartNames.size()) {
-      colStats = sharedCache.getAggrStatsFromCache(dbName, tblName, colNames, StatsType.ALL);
+      colStats = sharedCache.getAggrStatsFromCache(catName, dbName, tblName, colNames, StatsType.ALL);
       if (colStats != null) {
         return new AggrStats(colStats, partNames.size());
       }
@@ -1575,7 +1696,7 @@ public class CachedStore implements RawStore, Configurable {
       String defaultPartitionName = MetastoreConf.getVar(getConf(), ConfVars.DEFAULTPARTITIONNAME);
       if (!partNames.contains(defaultPartitionName)) {
         colStats =
-            sharedCache.getAggrStatsFromCache(dbName, tblName, colNames, StatsType.ALLBUTDEFAULT);
+            sharedCache.getAggrStatsFromCache(catName, dbName, tblName, colNames, StatsType.ALLBUTDEFAULT);
         if (colStats != null) {
           return new AggrStats(colStats, partNames.size());
         }
@@ -1584,30 +1705,29 @@ public class CachedStore implements RawStore, Configurable {
     LOG.debug("Didn't find aggr stats in cache. Merging them. tblName= {}, parts= {}, cols= {}",
         tblName, partNames, colNames);
     MergedColumnStatsForPartitions mergedColStats =
-        mergeColStatsForPartitions(dbName, tblName, partNames, colNames, sharedCache);
+        mergeColStatsForPartitions(catName, dbName, tblName, partNames, colNames, sharedCache);
     return new AggrStats(mergedColStats.getColStats(), mergedColStats.getPartsFound());
   }
 
-  private MergedColumnStatsForPartitions mergeColStatsForPartitions(String dbName, String tblName,
-      List<String> partNames, List<String> colNames, SharedCache sharedCache) throws MetaException {
+  private MergedColumnStatsForPartitions mergeColStatsForPartitions(
+      String catName, String dbName, String tblName, List<String> partNames, List<String> colNames,
+      SharedCache sharedCache) throws MetaException {
     final boolean useDensityFunctionForNDVEstimation =
         MetastoreConf.getBoolVar(getConf(), ConfVars.STATS_NDV_DENSITY_FUNCTION);
     final double ndvTuner = MetastoreConf.getDoubleVar(getConf(), ConfVars.STATS_NDV_TUNER);
-    Map<ColumnStatsAggregator, List<ColStatsObjWithSourceInfo>> colStatsMap =
-        new HashMap<ColumnStatsAggregator, List<ColStatsObjWithSourceInfo>>();
+    Map<ColumnStatsAggregator, List<ColStatsObjWithSourceInfo>> colStatsMap = new HashMap<>();
     boolean areAllPartsFound = true;
     long partsFound = 0;
     for (String colName : colNames) {
       long partsFoundForColumn = 0;
       ColumnStatsAggregator colStatsAggregator = null;
-      List<ColStatsObjWithSourceInfo> colStatsWithPartInfoList =
-          new ArrayList<ColStatsObjWithSourceInfo>();
+      List<ColStatsObjWithSourceInfo> colStatsWithPartInfoList = new ArrayList<>();
       for (String partName : partNames) {
         ColumnStatisticsObj colStatsForPart =
-            sharedCache.getPartitionColStatsFromCache(dbName, tblName, partNameToVals(partName), colName);
+            sharedCache.getPartitionColStatsFromCache(catName, dbName, tblName, partNameToVals(partName), colName);
         if (colStatsForPart != null) {
           ColStatsObjWithSourceInfo colStatsWithPartInfo =
-              new ColStatsObjWithSourceInfo(colStatsForPart, dbName, tblName, partName);
+              new ColStatsObjWithSourceInfo(colStatsForPart, catName, dbName, tblName, partName);
           colStatsWithPartInfoList.add(colStatsWithPartInfo);
           if (colStatsAggregator == null) {
             colStatsAggregator = ColumnStatsAggregatorFactory.getColumnStatsAggregator(
@@ -1755,32 +1875,32 @@ public class CachedStore implements RawStore, Configurable {
   }
 
   @Override
-  public List<HiveObjectPrivilege> listDBGrantsAll(String dbName) {
-    return rawStore.listDBGrantsAll(dbName);
+  public List<HiveObjectPrivilege> listDBGrantsAll(String catName, String dbName) {
+    return rawStore.listDBGrantsAll(catName, dbName);
   }
 
   @Override
-  public List<HiveObjectPrivilege> listPartitionColumnGrantsAll(String dbName,
+  public List<HiveObjectPrivilege> listPartitionColumnGrantsAll(String catName, String dbName,
       String tableName, String partitionName, String columnName) {
-    return rawStore.listPartitionColumnGrantsAll(dbName, tableName, partitionName, columnName);
+    return rawStore.listPartitionColumnGrantsAll(catName, dbName, tableName, partitionName, columnName);
   }
 
   @Override
-  public List<HiveObjectPrivilege> listTableGrantsAll(String dbName,
+  public List<HiveObjectPrivilege> listTableGrantsAll(String catName, String dbName,
       String tableName) {
-    return rawStore.listTableGrantsAll(dbName, tableName);
+    return rawStore.listTableGrantsAll(catName, dbName, tableName);
   }
 
   @Override
-  public List<HiveObjectPrivilege> listPartitionGrantsAll(String dbName,
+  public List<HiveObjectPrivilege> listPartitionGrantsAll(String catName, String dbName,
       String tableName, String partitionName) {
-    return rawStore.listPartitionGrantsAll(dbName, tableName, partitionName);
+    return rawStore.listPartitionGrantsAll(catName, dbName, tableName, partitionName);
   }
 
   @Override
-  public List<HiveObjectPrivilege> listTableColumnGrantsAll(String dbName,
+  public List<HiveObjectPrivilege> listTableColumnGrantsAll(String catName, String dbName,
       String tableName, String columnName) {
-    return rawStore.listTableColumnGrantsAll(dbName, tableName, columnName);
+    return rawStore.listTableColumnGrantsAll(catName, dbName, tableName, columnName);
   }
 
   @Override
@@ -1791,37 +1911,37 @@ public class CachedStore implements RawStore, Configurable {
   }
 
   @Override
-  public void alterFunction(String dbName, String funcName,
+  public void alterFunction(String catName, String dbName, String funcName,
       Function newFunction) throws InvalidObjectException, MetaException {
     // TODO fucntionCache
-    rawStore.alterFunction(dbName, funcName, newFunction);
+    rawStore.alterFunction(catName, dbName, funcName, newFunction);
   }
 
   @Override
-  public void dropFunction(String dbName, String funcName) throws MetaException,
+  public void dropFunction(String catName, String dbName, String funcName) throws MetaException,
       NoSuchObjectException, InvalidObjectException, InvalidInputException {
     // TODO fucntionCache
-    rawStore.dropFunction(dbName, funcName);
+    rawStore.dropFunction(catName, dbName, funcName);
   }
 
   @Override
-  public Function getFunction(String dbName, String funcName)
+  public Function getFunction(String catName, String dbName, String funcName)
       throws MetaException {
     // TODO fucntionCache
-    return rawStore.getFunction(dbName, funcName);
+    return rawStore.getFunction(catName, dbName, funcName);
   }
 
   @Override
-  public List<Function> getAllFunctions() throws MetaException {
+  public List<Function> getAllFunctions(String catName) throws MetaException {
     // TODO fucntionCache
-    return rawStore.getAllFunctions();
+    return rawStore.getAllFunctions(catName);
   }
 
   @Override
-  public List<String> getFunctions(String dbName, String pattern)
+  public List<String> getFunctions(String catName, String dbName, String pattern)
       throws MetaException {
     // TODO fucntionCache
-    return rawStore.getFunctions(dbName, pattern);
+    return rawStore.getFunctions(catName, dbName, pattern);
   }
 
   @Override
@@ -1899,46 +2019,46 @@ public class CachedStore implements RawStore, Configurable {
   }
 
   @Override
-  public List<SQLPrimaryKey> getPrimaryKeys(String db_name, String tbl_name)
+  public List<SQLPrimaryKey> getPrimaryKeys(String catName, String db_name, String tbl_name)
       throws MetaException {
     // TODO constraintCache
-    return rawStore.getPrimaryKeys(db_name, tbl_name);
+    return rawStore.getPrimaryKeys(catName, db_name, tbl_name);
   }
 
   @Override
-  public List<SQLForeignKey> getForeignKeys(String parent_db_name,
+  public List<SQLForeignKey> getForeignKeys(String catName, String parent_db_name,
       String parent_tbl_name, String foreign_db_name, String foreign_tbl_name)
       throws MetaE

<TRUNCATED>

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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListenerOnlyOnCommit.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListenerOnlyOnCommit.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListenerOnlyOnCommit.java
index a91d1c8..546422d 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListenerOnlyOnCommit.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListenerOnlyOnCommit.java
@@ -39,7 +39,6 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-import junit.framework.TestCase;
 import org.junit.experimental.categories.Category;
 
 /**
@@ -78,8 +77,8 @@ public class TestMetaStoreEventListenerOnlyOnCommit {
     String dbName = "tmpDb";
     Database db = new DatabaseBuilder()
         .setName(dbName)
-        .build();
-    msc.createDatabase(db);
+        .setCatalogName(Warehouse.DEFAULT_CATALOG_NAME)
+        .create(msc, conf);
 
     listSize += 1;
     notifyList = DummyListener.notifyList;
@@ -88,22 +87,20 @@ public class TestMetaStoreEventListenerOnlyOnCommit {
 
     String tableName = "unittest_TestMetaStoreEventListenerOnlyOnCommit";
     Table table = new TableBuilder()
-        .setDbName(db)
+        .inDb(db)
         .setTableName(tableName)
         .addCol("id", "int")
         .addPartCol("ds", "string")
-        .build();
-    msc.createTable(table);
+        .create(msc, conf);
     listSize += 1;
     notifyList = DummyListener.notifyList;
     assertEquals(notifyList.size(), listSize);
     assertTrue(DummyListener.getLastEvent().getStatus());
 
-    Partition part = new PartitionBuilder()
-        .fromTable(table)
+    new PartitionBuilder()
+        .inTable(table)
         .addValue("foo1")
-        .build();
-    msc.add_partition(part);
+        .addToTable(msc, conf);
     listSize += 1;
     notifyList = DummyListener.notifyList;
     assertEquals(notifyList.size(), listSize);
@@ -111,11 +108,10 @@ public class TestMetaStoreEventListenerOnlyOnCommit {
 
     DummyRawStoreControlledCommit.setCommitSucceed(false);
 
-    part = new PartitionBuilder()
-        .fromTable(table)
+    new PartitionBuilder()
+        .inTable(table)
         .addValue("foo2")
-        .build();
-    msc.add_partition(part);
+        .addToTable(msc, conf);
     listSize += 1;
     notifyList = DummyListener.notifyList;
     assertEquals(notifyList.size(), listSize);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreMaterializationsCacheCleaner.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreMaterializationsCacheCleaner.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreMaterializationsCacheCleaner.java
index 6d3f68c..7a871e1 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreMaterializationsCacheCleaner.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreMaterializationsCacheCleaner.java
@@ -31,6 +31,7 @@ import org.junit.runners.MethodSorters;
 
 import java.util.Map;
 
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -82,7 +83,7 @@ public class TestMetaStoreMaterializationsCacheCleaner {
     when(mv1.getDbName()).thenReturn(DB_NAME);
     when(mv1.getTableName()).thenReturn(MV_NAME_1);
     CreationMetadata mockCM1 = new CreationMetadata(
-        DB_NAME, MV_NAME_1,
+        DEFAULT_CATALOG_NAME, DB_NAME, MV_NAME_1,
         ImmutableSet.of(
             DB_NAME + "." + TBL_NAME_1,
             DB_NAME + "." + TBL_NAME_2));
@@ -115,7 +116,7 @@ public class TestMetaStoreMaterializationsCacheCleaner {
     when(mv2.getDbName()).thenReturn(DB_NAME);
     when(mv2.getTableName()).thenReturn(MV_NAME_2);
     CreationMetadata mockCM2 = new CreationMetadata(
-        DB_NAME, MV_NAME_2,
+        DEFAULT_CATALOG_NAME, DB_NAME, MV_NAME_2,
         ImmutableSet.of(
             DB_NAME + "." + TBL_NAME_1,
             DB_NAME + "." + TBL_NAME_2));
@@ -222,7 +223,7 @@ public class TestMetaStoreMaterializationsCacheCleaner {
     when(mv1.getDbName()).thenReturn(DB_NAME);
     when(mv1.getTableName()).thenReturn(MV_NAME_1);
     CreationMetadata mockCM1 = new CreationMetadata(
-        DB_NAME, MV_NAME_1,
+        DEFAULT_CATALOG_NAME, DB_NAME, MV_NAME_1,
         ImmutableSet.of(
             DB_NAME + "." + TBL_NAME_1,
             DB_NAME + "." + TBL_NAME_2));
@@ -255,7 +256,7 @@ public class TestMetaStoreMaterializationsCacheCleaner {
     when(mv2.getDbName()).thenReturn(DB_NAME);
     when(mv2.getTableName()).thenReturn(MV_NAME_2);
     CreationMetadata mockCM2 = new CreationMetadata(
-        DB_NAME, MV_NAME_2,
+        DEFAULT_CATALOG_NAME, DB_NAME, MV_NAME_2,
         ImmutableSet.of(
             DB_NAME + "." + TBL_NAME_1,
             DB_NAME + "." + TBL_NAME_2));

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/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
new file mode 100644
index 0000000..55ef885
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestNonCatCallsWithCatalog.java
@@ -0,0 +1,1126 @@
+/*
+ * 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);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java
index ca33b7d..9490586 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java
@@ -17,13 +17,13 @@
  */
 package org.apache.hadoop.hive.metastore;
 
-
 import com.codahale.metrics.Counter;
 import com.google.common.base.Supplier;
 import com.google.common.collect.ImmutableList;
 import org.apache.hadoop.hive.metastore.ObjectStore.RetryingExecutor;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.annotation.MetastoreUnitTest;
+import org.apache.hadoop.hive.metastore.api.Catalog;
 import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
@@ -43,6 +43,8 @@ import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 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.client.builder.CatalogBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.messaging.EventMessage;
 import org.apache.hadoop.hive.metastore.metrics.Metrics;
@@ -72,9 +74,12 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
+
 @Category(MetastoreUnitTest.class)
 public class TestObjectStore {
   private ObjectStore objectStore = null;
+  private Configuration conf;
 
   private static final String DB1 = "testobjectstoredb1";
   private static final String DB2 = "testobjectstoredb2";
@@ -98,37 +103,88 @@ public class TestObjectStore {
 
   @Before
   public void setUp() throws Exception {
-    Configuration conf = MetastoreConf.newMetastoreConf();
+    conf = MetastoreConf.newMetastoreConf();
     MetaStoreTestUtils.setConfForStandloneMode(conf);
 
     objectStore = new ObjectStore();
     objectStore.setConf(conf);
     dropAllStoreObjects(objectStore);
+    HiveMetaStore.HMSHandler.createDefaultCatalog(objectStore, new Warehouse(conf));
+  }
+
+  @Test
+  public void catalogs() throws MetaException, NoSuchObjectException {
+    final String names[] = {"cat1", "cat2"};
+    final String locations[] = {"loc1", "loc2"};
+    final String descriptions[] = {"description 1", "description 2"};
+
+    for (int i = 0; i < names.length; i++) {
+      Catalog cat = new CatalogBuilder()
+          .setName(names[i])
+          .setLocation(locations[i])
+          .setDescription(descriptions[i])
+          .build();
+      objectStore.createCatalog(cat);
+    }
+
+    List<String> fetchedNames = objectStore.getCatalogs();
+    Assert.assertEquals(3, fetchedNames.size());
+    for (int i = 0; i < names.length - 1; i++) {
+      Assert.assertEquals(names[i], fetchedNames.get(i));
+      Catalog cat = objectStore.getCatalog(fetchedNames.get(i));
+      Assert.assertEquals(names[i], cat.getName());
+      Assert.assertEquals(descriptions[i], cat.getDescription());
+      Assert.assertEquals(locations[i], cat.getLocationUri());
+    }
+    Catalog cat = objectStore.getCatalog(fetchedNames.get(2));
+    Assert.assertEquals(DEFAULT_CATALOG_NAME, cat.getName());
+    Assert.assertEquals(Warehouse.DEFAULT_CATALOG_COMMENT, cat.getDescription());
+    // Location will vary by system.
+
+    for (int i = 0; i < names.length; i++) objectStore.dropCatalog(names[i]);
+    fetchedNames = objectStore.getCatalogs();
+    Assert.assertEquals(1, fetchedNames.size());
   }
 
+  @Test(expected = NoSuchObjectException.class)
+  public void getNoSuchCatalog() throws MetaException, NoSuchObjectException {
+    objectStore.getCatalog("no_such_catalog");
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void dropNoSuchCatalog() throws MetaException, NoSuchObjectException {
+    objectStore.dropCatalog("no_such_catalog");
+  }
+
+  // TODO test dropping non-empty catalog
+
   /**
    * Test database operations
    */
   @Test
   public void testDatabaseOps() throws MetaException, InvalidObjectException,
       NoSuchObjectException {
+    String catName = "tdo1_cat";
+    createTestCatalog(catName);
     Database db1 = new Database(DB1, "description", "locationurl", null);
     Database db2 = new Database(DB2, "description", "locationurl", null);
+    db1.setCatalogName(catName);
+    db2.setCatalogName(catName);
     objectStore.createDatabase(db1);
     objectStore.createDatabase(db2);
 
-    List<String> databases = objectStore.getAllDatabases();
+    List<String> databases = objectStore.getAllDatabases(catName);
     LOG.info("databases: " + databases);
     Assert.assertEquals(2, databases.size());
     Assert.assertEquals(DB1, databases.get(0));
     Assert.assertEquals(DB2, databases.get(1));
 
-    objectStore.dropDatabase(DB1);
-    databases = objectStore.getAllDatabases();
+    objectStore.dropDatabase(catName, DB1);
+    databases = objectStore.getAllDatabases(catName);
     Assert.assertEquals(1, databases.size());
     Assert.assertEquals(DB2, databases.get(0));
 
-    objectStore.dropDatabase(DB2);
+    objectStore.dropDatabase(catName, DB2);
   }
 
   /**
@@ -137,7 +193,11 @@ public class TestObjectStore {
   @Test
   public void testTableOps() throws MetaException, InvalidObjectException, NoSuchObjectException,
       InvalidInputException {
-    Database db1 = new Database(DB1, "description", "locationurl", null);
+    Database db1 = new DatabaseBuilder()
+        .setName(DB1)
+        .setDescription("description")
+        .setLocation("locationurl")
+        .build(conf);
     objectStore.createDatabase(db1);
     StorageDescriptor sd1 =
         new StorageDescriptor(ImmutableList.of(new FieldSchema("pk_col", "double", null)),
@@ -149,7 +209,7 @@ public class TestObjectStore {
         new Table(TABLE1, DB1, "owner", 1, 2, 3, sd1, null, params, null, null, "MANAGED_TABLE");
     objectStore.createTable(tbl1);
 
-    List<String> tables = objectStore.getAllTables(DB1);
+    List<String> tables = objectStore.getAllTables(DEFAULT_CATALOG_NAME, DB1);
     Assert.assertEquals(1, tables.size());
     Assert.assertEquals(TABLE1, tables.get(0));
 
@@ -159,20 +219,21 @@ public class TestObjectStore {
             null, null, null);
     Table newTbl1 = new Table("new" + TABLE1, DB1, "owner", 1, 2, 3, sd2, null, params, null, null,
         "MANAGED_TABLE");
-    objectStore.alterTable(DB1, TABLE1, newTbl1);
-    tables = objectStore.getTables(DB1, "new*");
+    objectStore.alterTable(DEFAULT_CATALOG_NAME, DB1, TABLE1, newTbl1);
+    tables = objectStore.getTables(DEFAULT_CATALOG_NAME, DB1, "new*");
     Assert.assertEquals(1, tables.size());
     Assert.assertEquals("new" + TABLE1, tables.get(0));
 
     objectStore.createTable(tbl1);
-    tables = objectStore.getAllTables(DB1);
+    tables = objectStore.getAllTables(DEFAULT_CATALOG_NAME, DB1);
     Assert.assertEquals(2, tables.size());
 
-    List<SQLForeignKey> foreignKeys = objectStore.getForeignKeys(DB1, TABLE1, null, null);
+    List<SQLForeignKey> foreignKeys = objectStore.getForeignKeys(DEFAULT_CATALOG_NAME, DB1, TABLE1, null, null);
     Assert.assertEquals(0, foreignKeys.size());
 
     SQLPrimaryKey pk = new SQLPrimaryKey(DB1, TABLE1, "pk_col", 1,
         "pk_const_1", false, false, false);
+    pk.setCatName(DEFAULT_CATALOG_NAME);
     objectStore.addPrimaryKeys(ImmutableList.of(pk));
     SQLForeignKey fk = new SQLForeignKey(DB1, TABLE1, "pk_col",
         DB1, "new" + TABLE1, "fk_col", 1,
@@ -180,32 +241,32 @@ public class TestObjectStore {
     objectStore.addForeignKeys(ImmutableList.of(fk));
 
     // Retrieve from PK side
-    foreignKeys = objectStore.getForeignKeys(null, null, DB1, "new" + TABLE1);
+    foreignKeys = objectStore.getForeignKeys(DEFAULT_CATALOG_NAME, null, null, DB1, "new" + TABLE1);
     Assert.assertEquals(1, foreignKeys.size());
 
-    List<SQLForeignKey> fks = objectStore.getForeignKeys(null, null, DB1, "new" + TABLE1);
+    List<SQLForeignKey> fks = objectStore.getForeignKeys(DEFAULT_CATALOG_NAME, null, null, DB1, "new" + TABLE1);
     if (fks != null) {
       for (SQLForeignKey fkcol : fks) {
-        objectStore.dropConstraint(fkcol.getFktable_db(), fkcol.getFktable_name(),
+        objectStore.dropConstraint(fkcol.getCatName(), fkcol.getFktable_db(), fkcol.getFktable_name(),
             fkcol.getFk_name());
       }
     }
     // Retrieve from FK side
-    foreignKeys = objectStore.getForeignKeys(DB1, TABLE1, null, null);
+    foreignKeys = objectStore.getForeignKeys(DEFAULT_CATALOG_NAME, DB1, TABLE1, null, null);
     Assert.assertEquals(0, foreignKeys.size());
     // Retrieve from PK side
-    foreignKeys = objectStore.getForeignKeys(null, null, DB1, "new" + TABLE1);
+    foreignKeys = objectStore.getForeignKeys(DEFAULT_CATALOG_NAME, null, null, DB1, "new" + TABLE1);
     Assert.assertEquals(0, foreignKeys.size());
 
-    objectStore.dropTable(DB1, TABLE1);
-    tables = objectStore.getAllTables(DB1);
+    objectStore.dropTable(DEFAULT_CATALOG_NAME, DB1, TABLE1);
+    tables = objectStore.getAllTables(DEFAULT_CATALOG_NAME, DB1);
     Assert.assertEquals(1, tables.size());
 
-    objectStore.dropTable(DB1, "new" + TABLE1);
-    tables = objectStore.getAllTables(DB1);
+    objectStore.dropTable(DEFAULT_CATALOG_NAME, DB1, "new" + TABLE1);
+    tables = objectStore.getAllTables(DEFAULT_CATALOG_NAME, DB1);
     Assert.assertEquals(0, tables.size());
 
-    objectStore.dropDatabase(DB1);
+    objectStore.dropDatabase(db1.getCatalogName(), DB1);
   }
 
   private StorageDescriptor createFakeSd(String location) {
@@ -220,7 +281,11 @@ public class TestObjectStore {
   @Test
   public void testPartitionOps() throws MetaException, InvalidObjectException,
       NoSuchObjectException, InvalidInputException {
-    Database db1 = new Database(DB1, "description", "locationurl", null);
+    Database db1 = new DatabaseBuilder()
+        .setName(DB1)
+        .setDescription("description")
+        .setLocation("locationurl")
+        .build(conf);
     objectStore.createDatabase(db1);
     StorageDescriptor sd = createFakeSd("location");
     HashMap<String, String> tableParams = new HashMap<>();
@@ -235,31 +300,33 @@ public class TestObjectStore {
     partitionParams.put("PARTITION_LEVEL_PRIVILEGE", "true");
     List<String> value1 = Arrays.asList("US", "CA");
     Partition part1 = new Partition(value1, DB1, TABLE1, 111, 111, sd, partitionParams);
+    part1.setCatName(DEFAULT_CATALOG_NAME);
     objectStore.addPartition(part1);
     List<String> value2 = Arrays.asList("US", "MA");
     Partition part2 = new Partition(value2, DB1, TABLE1, 222, 222, sd, partitionParams);
+    part2.setCatName(DEFAULT_CATALOG_NAME);
     objectStore.addPartition(part2);
 
     Deadline.startTimer("getPartition");
-    List<Partition> partitions = objectStore.getPartitions(DB1, TABLE1, 10);
+    List<Partition> partitions = objectStore.getPartitions(DEFAULT_CATALOG_NAME, DB1, TABLE1, 10);
     Assert.assertEquals(2, partitions.size());
     Assert.assertEquals(111, partitions.get(0).getCreateTime());
     Assert.assertEquals(222, partitions.get(1).getCreateTime());
 
-    int numPartitions = objectStore.getNumPartitionsByFilter(DB1, TABLE1, "");
+    int numPartitions = objectStore.getNumPartitionsByFilter(DEFAULT_CATALOG_NAME, DB1, TABLE1, "");
     Assert.assertEquals(partitions.size(), numPartitions);
 
-    numPartitions = objectStore.getNumPartitionsByFilter(DB1, TABLE1, "country = \"US\"");
+    numPartitions = objectStore.getNumPartitionsByFilter(DEFAULT_CATALOG_NAME, DB1, TABLE1, "country = \"US\"");
     Assert.assertEquals(2, numPartitions);
 
-    objectStore.dropPartition(DB1, TABLE1, value1);
-    partitions = objectStore.getPartitions(DB1, TABLE1, 10);
+    objectStore.dropPartition(DEFAULT_CATALOG_NAME, DB1, TABLE1, value1);
+    partitions = objectStore.getPartitions(DEFAULT_CATALOG_NAME, DB1, TABLE1, 10);
     Assert.assertEquals(1, partitions.size());
     Assert.assertEquals(222, partitions.get(0).getCreateTime());
 
-    objectStore.dropPartition(DB1, TABLE1, value2);
-    objectStore.dropTable(DB1, TABLE1);
-    objectStore.dropDatabase(DB1);
+    objectStore.dropPartition(DEFAULT_CATALOG_NAME, DB1, TABLE1, value2);
+    objectStore.dropTable(DEFAULT_CATALOG_NAME, DB1, TABLE1);
+    objectStore.dropDatabase(db1.getCatalogName(), DB1);
   }
 
   /**
@@ -322,7 +389,7 @@ public class TestObjectStore {
     Counter directSqlErrors =
         Metrics.getRegistry().getCounters().get(MetricsConstants.DIRECTSQL_ERRORS);
 
-    objectStore.new GetDbHelper("foo", true, true) {
+    objectStore.new GetDbHelper(DEFAULT_CATALOG_NAME, "foo", true, true) {
       @Override
       protected Database getSqlResult(ObjectStore.GetHelper<Database> ctx) throws MetaException {
         return null;
@@ -337,7 +404,7 @@ public class TestObjectStore {
 
     Assert.assertEquals(0, directSqlErrors.getCount());
 
-    objectStore.new GetDbHelper("foo", true, true) {
+    objectStore.new GetDbHelper(DEFAULT_CATALOG_NAME, "foo", true, true) {
       @Override
       protected Database getSqlResult(ObjectStore.GetHelper<Database> ctx) throws MetaException {
         throw new RuntimeException();
@@ -357,39 +424,42 @@ public class TestObjectStore {
       throws MetaException, InvalidObjectException, InvalidInputException {
     try {
       Deadline.registerIfNot(100000);
-      List<Function> functions = store.getAllFunctions();
+      List<Function> functions = store.getAllFunctions(DEFAULT_CATALOG_NAME);
       for (Function func : functions) {
-        store.dropFunction(func.getDbName(), func.getFunctionName());
+        store.dropFunction(DEFAULT_CATALOG_NAME, func.getDbName(), func.getFunctionName());
       }
-      List<String> dbs = store.getAllDatabases();
-      for (String db : dbs) {
-        List<String> tbls = store.getAllTables(db);
-        for (String tbl : tbls) {
-          Deadline.startTimer("getPartition");
-          List<Partition> parts = store.getPartitions(db, tbl, 100);
-          for (Partition part : parts) {
-            store.dropPartition(db, tbl, part.getValues());
-          }
-          // Find any constraints and drop them
-          Set<String> constraints = new HashSet<>();
-          List<SQLPrimaryKey> pk = store.getPrimaryKeys(db, tbl);
-          if (pk != null) {
-            for (SQLPrimaryKey pkcol : pk) {
-              constraints.add(pkcol.getPk_name());
+      for (String catName : store.getCatalogs()) {
+        List<String> dbs = store.getAllDatabases(catName);
+        for (String db : dbs) {
+          List<String> tbls = store.getAllTables(DEFAULT_CATALOG_NAME, db);
+          for (String tbl : tbls) {
+            Deadline.startTimer("getPartition");
+            List<Partition> parts = store.getPartitions(DEFAULT_CATALOG_NAME, db, tbl, 100);
+            for (Partition part : parts) {
+              store.dropPartition(DEFAULT_CATALOG_NAME, db, tbl, part.getValues());
             }
-          }
-          List<SQLForeignKey> fks = store.getForeignKeys(null, null, db, tbl);
-          if (fks != null) {
-            for (SQLForeignKey fkcol : fks) {
-              constraints.add(fkcol.getFk_name());
+            // Find any constraints and drop them
+            Set<String> constraints = new HashSet<>();
+            List<SQLPrimaryKey> pk = store.getPrimaryKeys(DEFAULT_CATALOG_NAME, db, tbl);
+            if (pk != null) {
+              for (SQLPrimaryKey pkcol : pk) {
+                constraints.add(pkcol.getPk_name());
+              }
             }
+            List<SQLForeignKey> fks = store.getForeignKeys(DEFAULT_CATALOG_NAME, null, null, db, tbl);
+            if (fks != null) {
+              for (SQLForeignKey fkcol : fks) {
+                constraints.add(fkcol.getFk_name());
+              }
+            }
+            for (String constraint : constraints) {
+              store.dropConstraint(DEFAULT_CATALOG_NAME, db, tbl, constraint);
+            }
+            store.dropTable(DEFAULT_CATALOG_NAME, db, tbl);
           }
-          for (String constraint : constraints) {
-            store.dropConstraint(db, tbl, constraint);
-          }
-          store.dropTable(db, tbl);
+          store.dropDatabase(catName, db);
         }
-        store.dropDatabase(db);
+        store.dropCatalog(catName);
       }
       List<String> roles = store.listRoleNames();
       for (String role : roles) {
@@ -402,9 +472,9 @@ public class TestObjectStore {
   @Test
   public void testQueryCloseOnError() throws Exception {
     ObjectStore spy = Mockito.spy(objectStore);
-    spy.getAllDatabases();
-    spy.getAllFunctions();
-    spy.getAllTables(DB1);
+    spy.getAllDatabases(DEFAULT_CATALOG_NAME);
+    spy.getAllFunctions(DEFAULT_CATALOG_NAME);
+    spy.getAllTables(DEFAULT_CATALOG_NAME, DB1);
     spy.getPartitionCount();
     Mockito.verify(spy, Mockito.times(3))
         .rollbackAndCleanup(Mockito.anyBoolean(), Mockito.<Query>anyObject());
@@ -566,5 +636,13 @@ public class TestObjectStore {
       previousId = event.getEventId();
     }
   }
+
+  private void createTestCatalog(String catName) throws MetaException {
+    Catalog cat = new CatalogBuilder()
+        .setName(catName)
+        .setLocation("/tmp")
+        .build();
+    objectStore.createCatalog(cat);
+  }
 }
 


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java
index 1e6f973..dd3a127 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java
@@ -43,6 +43,7 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField PARTS_FIELD_DESC = new org.apache.thrift.protocol.TField("parts", org.apache.thrift.protocol.TType.LIST, (short)3);
   private static final org.apache.thrift.protocol.TField IF_NOT_EXISTS_FIELD_DESC = new org.apache.thrift.protocol.TField("ifNotExists", org.apache.thrift.protocol.TType.BOOL, (short)4);
   private static final org.apache.thrift.protocol.TField NEED_RESULT_FIELD_DESC = new org.apache.thrift.protocol.TField("needResult", org.apache.thrift.protocol.TType.BOOL, (short)5);
+  private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)6);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -55,6 +56,7 @@ import org.slf4j.LoggerFactory;
   private List<Partition> parts; // required
   private boolean ifNotExists; // required
   private boolean needResult; // optional
+  private String catName; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -62,7 +64,8 @@ import org.slf4j.LoggerFactory;
     TBL_NAME((short)2, "tblName"),
     PARTS((short)3, "parts"),
     IF_NOT_EXISTS((short)4, "ifNotExists"),
-    NEED_RESULT((short)5, "needResult");
+    NEED_RESULT((short)5, "needResult"),
+    CAT_NAME((short)6, "catName");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -87,6 +90,8 @@ import org.slf4j.LoggerFactory;
           return IF_NOT_EXISTS;
         case 5: // NEED_RESULT
           return NEED_RESULT;
+        case 6: // CAT_NAME
+          return CAT_NAME;
         default:
           return null;
       }
@@ -130,7 +135,7 @@ import org.slf4j.LoggerFactory;
   private static final int __IFNOTEXISTS_ISSET_ID = 0;
   private static final int __NEEDRESULT_ISSET_ID = 1;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.NEED_RESULT};
+  private static final _Fields optionals[] = {_Fields.NEED_RESULT,_Fields.CAT_NAME};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -145,6 +150,8 @@ import org.slf4j.LoggerFactory;
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
     tmpMap.put(_Fields.NEED_RESULT, new org.apache.thrift.meta_data.FieldMetaData("needResult", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(AddPartitionsRequest.class, metaDataMap);
   }
@@ -188,6 +195,9 @@ import org.slf4j.LoggerFactory;
     }
     this.ifNotExists = other.ifNotExists;
     this.needResult = other.needResult;
+    if (other.isSetCatName()) {
+      this.catName = other.catName;
+    }
   }
 
   public AddPartitionsRequest deepCopy() {
@@ -203,6 +213,7 @@ import org.slf4j.LoggerFactory;
     this.ifNotExists = false;
     this.needResult = true;
 
+    this.catName = null;
   }
 
   public String getDbName() {
@@ -333,6 +344,29 @@ import org.slf4j.LoggerFactory;
     __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NEEDRESULT_ISSET_ID, value);
   }
 
+  public String getCatName() {
+    return this.catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
+  public void unsetCatName() {
+    this.catName = null;
+  }
+
+  /** Returns true if field catName is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatName() {
+    return this.catName != null;
+  }
+
+  public void setCatNameIsSet(boolean value) {
+    if (!value) {
+      this.catName = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case DB_NAME:
@@ -375,6 +409,14 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case CAT_NAME:
+      if (value == null) {
+        unsetCatName();
+      } else {
+        setCatName((String)value);
+      }
+      break;
+
     }
   }
 
@@ -395,6 +437,9 @@ import org.slf4j.LoggerFactory;
     case NEED_RESULT:
       return isNeedResult();
 
+    case CAT_NAME:
+      return getCatName();
+
     }
     throw new IllegalStateException();
   }
@@ -416,6 +461,8 @@ import org.slf4j.LoggerFactory;
       return isSetIfNotExists();
     case NEED_RESULT:
       return isSetNeedResult();
+    case CAT_NAME:
+      return isSetCatName();
     }
     throw new IllegalStateException();
   }
@@ -478,6 +525,15 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_catName = true && this.isSetCatName();
+    boolean that_present_catName = true && that.isSetCatName();
+    if (this_present_catName || that_present_catName) {
+      if (!(this_present_catName && that_present_catName))
+        return false;
+      if (!this.catName.equals(that.catName))
+        return false;
+    }
+
     return true;
   }
 
@@ -510,6 +566,11 @@ import org.slf4j.LoggerFactory;
     if (present_needResult)
       list.add(needResult);
 
+    boolean present_catName = true && (isSetCatName());
+    list.add(present_catName);
+    if (present_catName)
+      list.add(catName);
+
     return list.hashCode();
   }
 
@@ -571,6 +632,16 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetCatName()).compareTo(other.isSetCatName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catName, other.catName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -624,6 +695,16 @@ import org.slf4j.LoggerFactory;
       sb.append(this.needResult);
       first = false;
     }
+    if (isSetCatName()) {
+      if (!first) sb.append(", ");
+      sb.append("catName:");
+      if (this.catName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.catName);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -704,14 +785,14 @@ import org.slf4j.LoggerFactory;
           case 3: // PARTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list474 = iprot.readListBegin();
-                struct.parts = new ArrayList<Partition>(_list474.size);
-                Partition _elem475;
-                for (int _i476 = 0; _i476 < _list474.size; ++_i476)
+                org.apache.thrift.protocol.TList _list482 = iprot.readListBegin();
+                struct.parts = new ArrayList<Partition>(_list482.size);
+                Partition _elem483;
+                for (int _i484 = 0; _i484 < _list482.size; ++_i484)
                 {
-                  _elem475 = new Partition();
-                  _elem475.read(iprot);
-                  struct.parts.add(_elem475);
+                  _elem483 = new Partition();
+                  _elem483.read(iprot);
+                  struct.parts.add(_elem483);
                 }
                 iprot.readListEnd();
               }
@@ -736,6 +817,14 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 6: // CAT_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.catName = iprot.readString();
+              struct.setCatNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -763,9 +852,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PARTS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.parts.size()));
-          for (Partition _iter477 : struct.parts)
+          for (Partition _iter485 : struct.parts)
           {
-            _iter477.write(oprot);
+            _iter485.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -779,6 +868,13 @@ import org.slf4j.LoggerFactory;
         oprot.writeBool(struct.needResult);
         oprot.writeFieldEnd();
       }
+      if (struct.catName != null) {
+        if (struct.isSetCatName()) {
+          oprot.writeFieldBegin(CAT_NAME_FIELD_DESC);
+          oprot.writeString(struct.catName);
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -800,9 +896,9 @@ import org.slf4j.LoggerFactory;
       oprot.writeString(struct.tblName);
       {
         oprot.writeI32(struct.parts.size());
-        for (Partition _iter478 : struct.parts)
+        for (Partition _iter486 : struct.parts)
         {
-          _iter478.write(oprot);
+          _iter486.write(oprot);
         }
       }
       oprot.writeBool(struct.ifNotExists);
@@ -810,10 +906,16 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetNeedResult()) {
         optionals.set(0);
       }
-      oprot.writeBitSet(optionals, 1);
+      if (struct.isSetCatName()) {
+        optionals.set(1);
+      }
+      oprot.writeBitSet(optionals, 2);
       if (struct.isSetNeedResult()) {
         oprot.writeBool(struct.needResult);
       }
+      if (struct.isSetCatName()) {
+        oprot.writeString(struct.catName);
+      }
     }
 
     @Override
@@ -824,24 +926,28 @@ import org.slf4j.LoggerFactory;
       struct.tblName = iprot.readString();
       struct.setTblNameIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list479 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.parts = new ArrayList<Partition>(_list479.size);
-        Partition _elem480;
-        for (int _i481 = 0; _i481 < _list479.size; ++_i481)
+        org.apache.thrift.protocol.TList _list487 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.parts = new ArrayList<Partition>(_list487.size);
+        Partition _elem488;
+        for (int _i489 = 0; _i489 < _list487.size; ++_i489)
         {
-          _elem480 = new Partition();
-          _elem480.read(iprot);
-          struct.parts.add(_elem480);
+          _elem488 = new Partition();
+          _elem488.read(iprot);
+          struct.parts.add(_elem488);
         }
       }
       struct.setPartsIsSet(true);
       struct.ifNotExists = iprot.readBool();
       struct.setIfNotExistsIsSet(true);
-      BitSet incoming = iprot.readBitSet(1);
+      BitSet incoming = iprot.readBitSet(2);
       if (incoming.get(0)) {
         struct.needResult = iprot.readBool();
         struct.setNeedResultIsSet(true);
       }
+      if (incoming.get(1)) {
+        struct.catName = iprot.readString();
+        struct.setCatNameIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsResult.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsResult.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsResult.java
index fb21b45..fe41b8c 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsResult.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsResult.java
@@ -346,14 +346,14 @@ import org.slf4j.LoggerFactory;
           case 1: // PARTITIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list466 = iprot.readListBegin();
-                struct.partitions = new ArrayList<Partition>(_list466.size);
-                Partition _elem467;
-                for (int _i468 = 0; _i468 < _list466.size; ++_i468)
+                org.apache.thrift.protocol.TList _list474 = iprot.readListBegin();
+                struct.partitions = new ArrayList<Partition>(_list474.size);
+                Partition _elem475;
+                for (int _i476 = 0; _i476 < _list474.size; ++_i476)
                 {
-                  _elem467 = new Partition();
-                  _elem467.read(iprot);
-                  struct.partitions.add(_elem467);
+                  _elem475 = new Partition();
+                  _elem475.read(iprot);
+                  struct.partitions.add(_elem475);
                 }
                 iprot.readListEnd();
               }
@@ -380,9 +380,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PARTITIONS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitions.size()));
-            for (Partition _iter469 : struct.partitions)
+            for (Partition _iter477 : struct.partitions)
             {
-              _iter469.write(oprot);
+              _iter477.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetPartitions()) {
         {
           oprot.writeI32(struct.partitions.size());
-          for (Partition _iter470 : struct.partitions)
+          for (Partition _iter478 : struct.partitions)
           {
-            _iter470.write(oprot);
+            _iter478.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list471 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.partitions = new ArrayList<Partition>(_list471.size);
-          Partition _elem472;
-          for (int _i473 = 0; _i473 < _list471.size; ++_i473)
+          org.apache.thrift.protocol.TList _list479 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.partitions = new ArrayList<Partition>(_list479.size);
+          Partition _elem480;
+          for (int _i481 = 0; _i481 < _list479.size; ++_i481)
           {
-            _elem472 = new Partition();
-            _elem472.read(iprot);
-            struct.partitions.add(_elem472);
+            _elem480 = new Partition();
+            _elem480.read(iprot);
+            struct.partitions.add(_elem480);
           }
         }
         struct.setPartitionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPrimaryKeyRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPrimaryKeyRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPrimaryKeyRequest.java
index 79c7930..39bb6be 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPrimaryKeyRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPrimaryKeyRequest.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // PRIMARY_KEY_COLS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list360 = iprot.readListBegin();
-                struct.primaryKeyCols = new ArrayList<SQLPrimaryKey>(_list360.size);
-                SQLPrimaryKey _elem361;
-                for (int _i362 = 0; _i362 < _list360.size; ++_i362)
+                org.apache.thrift.protocol.TList _list368 = iprot.readListBegin();
+                struct.primaryKeyCols = new ArrayList<SQLPrimaryKey>(_list368.size);
+                SQLPrimaryKey _elem369;
+                for (int _i370 = 0; _i370 < _list368.size; ++_i370)
                 {
-                  _elem361 = new SQLPrimaryKey();
-                  _elem361.read(iprot);
-                  struct.primaryKeyCols.add(_elem361);
+                  _elem369 = new SQLPrimaryKey();
+                  _elem369.read(iprot);
+                  struct.primaryKeyCols.add(_elem369);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PRIMARY_KEY_COLS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.primaryKeyCols.size()));
-          for (SQLPrimaryKey _iter363 : struct.primaryKeyCols)
+          for (SQLPrimaryKey _iter371 : struct.primaryKeyCols)
           {
-            _iter363.write(oprot);
+            _iter371.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.primaryKeyCols.size());
-        for (SQLPrimaryKey _iter364 : struct.primaryKeyCols)
+        for (SQLPrimaryKey _iter372 : struct.primaryKeyCols)
         {
-          _iter364.write(oprot);
+          _iter372.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, AddPrimaryKeyRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list365 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.primaryKeyCols = new ArrayList<SQLPrimaryKey>(_list365.size);
-        SQLPrimaryKey _elem366;
-        for (int _i367 = 0; _i367 < _list365.size; ++_i367)
+        org.apache.thrift.protocol.TList _list373 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.primaryKeyCols = new ArrayList<SQLPrimaryKey>(_list373.size);
+        SQLPrimaryKey _elem374;
+        for (int _i375 = 0; _i375 < _list373.size; ++_i375)
         {
-          _elem366 = new SQLPrimaryKey();
-          _elem366.read(iprot);
-          struct.primaryKeyCols.add(_elem366);
+          _elem374 = new SQLPrimaryKey();
+          _elem374.read(iprot);
+          struct.primaryKeyCols.add(_elem374);
         }
       }
       struct.setPrimaryKeyColsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddUniqueConstraintRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddUniqueConstraintRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddUniqueConstraintRequest.java
index 0cfee8a..bcb1e6b 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddUniqueConstraintRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddUniqueConstraintRequest.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // UNIQUE_CONSTRAINT_COLS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list376 = iprot.readListBegin();
-                struct.uniqueConstraintCols = new ArrayList<SQLUniqueConstraint>(_list376.size);
-                SQLUniqueConstraint _elem377;
-                for (int _i378 = 0; _i378 < _list376.size; ++_i378)
+                org.apache.thrift.protocol.TList _list384 = iprot.readListBegin();
+                struct.uniqueConstraintCols = new ArrayList<SQLUniqueConstraint>(_list384.size);
+                SQLUniqueConstraint _elem385;
+                for (int _i386 = 0; _i386 < _list384.size; ++_i386)
                 {
-                  _elem377 = new SQLUniqueConstraint();
-                  _elem377.read(iprot);
-                  struct.uniqueConstraintCols.add(_elem377);
+                  _elem385 = new SQLUniqueConstraint();
+                  _elem385.read(iprot);
+                  struct.uniqueConstraintCols.add(_elem385);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(UNIQUE_CONSTRAINT_COLS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.uniqueConstraintCols.size()));
-          for (SQLUniqueConstraint _iter379 : struct.uniqueConstraintCols)
+          for (SQLUniqueConstraint _iter387 : struct.uniqueConstraintCols)
           {
-            _iter379.write(oprot);
+            _iter387.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.uniqueConstraintCols.size());
-        for (SQLUniqueConstraint _iter380 : struct.uniqueConstraintCols)
+        for (SQLUniqueConstraint _iter388 : struct.uniqueConstraintCols)
         {
-          _iter380.write(oprot);
+          _iter388.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, AddUniqueConstraintRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list381 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.uniqueConstraintCols = new ArrayList<SQLUniqueConstraint>(_list381.size);
-        SQLUniqueConstraint _elem382;
-        for (int _i383 = 0; _i383 < _list381.size; ++_i383)
+        org.apache.thrift.protocol.TList _list389 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.uniqueConstraintCols = new ArrayList<SQLUniqueConstraint>(_list389.size);
+        SQLUniqueConstraint _elem390;
+        for (int _i391 = 0; _i391 < _list389.size; ++_i391)
         {
-          _elem382 = new SQLUniqueConstraint();
-          _elem382.read(iprot);
-          struct.uniqueConstraintCols.add(_elem382);
+          _elem390 = new SQLUniqueConstraint();
+          _elem390.read(iprot);
+          struct.uniqueConstraintCols.add(_elem390);
         }
       }
       struct.setUniqueConstraintColsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AggrStats.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AggrStats.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AggrStats.java
index 718a637..fff212d 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AggrStats.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AggrStats.java
@@ -439,14 +439,14 @@ import org.slf4j.LoggerFactory;
           case 1: // COL_STATS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list268 = iprot.readListBegin();
-                struct.colStats = new ArrayList<ColumnStatisticsObj>(_list268.size);
-                ColumnStatisticsObj _elem269;
-                for (int _i270 = 0; _i270 < _list268.size; ++_i270)
+                org.apache.thrift.protocol.TList _list276 = iprot.readListBegin();
+                struct.colStats = new ArrayList<ColumnStatisticsObj>(_list276.size);
+                ColumnStatisticsObj _elem277;
+                for (int _i278 = 0; _i278 < _list276.size; ++_i278)
                 {
-                  _elem269 = new ColumnStatisticsObj();
-                  _elem269.read(iprot);
-                  struct.colStats.add(_elem269);
+                  _elem277 = new ColumnStatisticsObj();
+                  _elem277.read(iprot);
+                  struct.colStats.add(_elem277);
                 }
                 iprot.readListEnd();
               }
@@ -480,9 +480,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(COL_STATS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.colStats.size()));
-          for (ColumnStatisticsObj _iter271 : struct.colStats)
+          for (ColumnStatisticsObj _iter279 : struct.colStats)
           {
-            _iter271.write(oprot);
+            _iter279.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -510,9 +510,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.colStats.size());
-        for (ColumnStatisticsObj _iter272 : struct.colStats)
+        for (ColumnStatisticsObj _iter280 : struct.colStats)
         {
-          _iter272.write(oprot);
+          _iter280.write(oprot);
         }
       }
       oprot.writeI64(struct.partsFound);
@@ -522,14 +522,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, AggrStats struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list273 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.colStats = new ArrayList<ColumnStatisticsObj>(_list273.size);
-        ColumnStatisticsObj _elem274;
-        for (int _i275 = 0; _i275 < _list273.size; ++_i275)
+        org.apache.thrift.protocol.TList _list281 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.colStats = new ArrayList<ColumnStatisticsObj>(_list281.size);
+        ColumnStatisticsObj _elem282;
+        for (int _i283 = 0; _i283 < _list281.size; ++_i283)
         {
-          _elem274 = new ColumnStatisticsObj();
-          _elem274.read(iprot);
-          struct.colStats.add(_elem274);
+          _elem282 = new ColumnStatisticsObj();
+          _elem282.read(iprot);
+          struct.colStats.add(_elem282);
         }
       }
       struct.setColStatsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsRequest.java
index bf95854..fd0d3c9 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsRequest.java
@@ -521,13 +521,13 @@ import org.slf4j.LoggerFactory;
           case 1: // TXN_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list602 = iprot.readListBegin();
-                struct.txnIds = new ArrayList<Long>(_list602.size);
-                long _elem603;
-                for (int _i604 = 0; _i604 < _list602.size; ++_i604)
+                org.apache.thrift.protocol.TList _list610 = iprot.readListBegin();
+                struct.txnIds = new ArrayList<Long>(_list610.size);
+                long _elem611;
+                for (int _i612 = 0; _i612 < _list610.size; ++_i612)
                 {
-                  _elem603 = iprot.readI64();
-                  struct.txnIds.add(_elem603);
+                  _elem611 = iprot.readI64();
+                  struct.txnIds.add(_elem611);
                 }
                 iprot.readListEnd();
               }
@@ -569,9 +569,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TXN_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.txnIds.size()));
-          for (long _iter605 : struct.txnIds)
+          for (long _iter613 : struct.txnIds)
           {
-            oprot.writeI64(_iter605);
+            oprot.writeI64(_iter613);
           }
           oprot.writeListEnd();
         }
@@ -606,9 +606,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.txnIds.size());
-        for (long _iter606 : struct.txnIds)
+        for (long _iter614 : struct.txnIds)
         {
-          oprot.writeI64(_iter606);
+          oprot.writeI64(_iter614);
         }
       }
       oprot.writeString(struct.dbName);
@@ -619,13 +619,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, AllocateTableWriteIdsRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list607 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.txnIds = new ArrayList<Long>(_list607.size);
-        long _elem608;
-        for (int _i609 = 0; _i609 < _list607.size; ++_i609)
+        org.apache.thrift.protocol.TList _list615 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.txnIds = new ArrayList<Long>(_list615.size);
+        long _elem616;
+        for (int _i617 = 0; _i617 < _list615.size; ++_i617)
         {
-          _elem608 = iprot.readI64();
-          struct.txnIds.add(_elem608);
+          _elem616 = iprot.readI64();
+          struct.txnIds.add(_elem616);
         }
       }
       struct.setTxnIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsResponse.java
index 5ce8d51..fb47073 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsResponse.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // TXN_TO_WRITE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list610 = iprot.readListBegin();
-                struct.txnToWriteIds = new ArrayList<TxnToWriteId>(_list610.size);
-                TxnToWriteId _elem611;
-                for (int _i612 = 0; _i612 < _list610.size; ++_i612)
+                org.apache.thrift.protocol.TList _list618 = iprot.readListBegin();
+                struct.txnToWriteIds = new ArrayList<TxnToWriteId>(_list618.size);
+                TxnToWriteId _elem619;
+                for (int _i620 = 0; _i620 < _list618.size; ++_i620)
                 {
-                  _elem611 = new TxnToWriteId();
-                  _elem611.read(iprot);
-                  struct.txnToWriteIds.add(_elem611);
+                  _elem619 = new TxnToWriteId();
+                  _elem619.read(iprot);
+                  struct.txnToWriteIds.add(_elem619);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TXN_TO_WRITE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.txnToWriteIds.size()));
-          for (TxnToWriteId _iter613 : struct.txnToWriteIds)
+          for (TxnToWriteId _iter621 : struct.txnToWriteIds)
           {
-            _iter613.write(oprot);
+            _iter621.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.txnToWriteIds.size());
-        for (TxnToWriteId _iter614 : struct.txnToWriteIds)
+        for (TxnToWriteId _iter622 : struct.txnToWriteIds)
         {
-          _iter614.write(oprot);
+          _iter622.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, AllocateTableWriteIdsResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list615 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.txnToWriteIds = new ArrayList<TxnToWriteId>(_list615.size);
-        TxnToWriteId _elem616;
-        for (int _i617 = 0; _i617 < _list615.size; ++_i617)
+        org.apache.thrift.protocol.TList _list623 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.txnToWriteIds = new ArrayList<TxnToWriteId>(_list623.size);
+        TxnToWriteId _elem624;
+        for (int _i625 = 0; _i625 < _list623.size; ++_i625)
         {
-          _elem616 = new TxnToWriteId();
-          _elem616.read(iprot);
-          struct.txnToWriteIds.add(_elem616);
+          _elem624 = new TxnToWriteId();
+          _elem624.read(iprot);
+          struct.txnToWriteIds.add(_elem624);
         }
       }
       struct.setTxnToWriteIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Catalog.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Catalog.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Catalog.java
new file mode 100644
index 0000000..3eb4dbd
--- /dev/null
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Catalog.java
@@ -0,0 +1,606 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class Catalog implements org.apache.thrift.TBase<Catalog, Catalog._Fields>, java.io.Serializable, Cloneable, Comparable<Catalog> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Catalog");
+
+  private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField DESCRIPTION_FIELD_DESC = new org.apache.thrift.protocol.TField("description", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField LOCATION_URI_FIELD_DESC = new org.apache.thrift.protocol.TField("locationUri", org.apache.thrift.protocol.TType.STRING, (short)3);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new CatalogStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new CatalogTupleSchemeFactory());
+  }
+
+  private String name; // required
+  private String description; // optional
+  private String locationUri; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    NAME((short)1, "name"),
+    DESCRIPTION((short)2, "description"),
+    LOCATION_URI((short)3, "locationUri");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // NAME
+          return NAME;
+        case 2: // DESCRIPTION
+          return DESCRIPTION;
+        case 3: // LOCATION_URI
+          return LOCATION_URI;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final _Fields optionals[] = {_Fields.DESCRIPTION};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.DESCRIPTION, new org.apache.thrift.meta_data.FieldMetaData("description", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.LOCATION_URI, new org.apache.thrift.meta_data.FieldMetaData("locationUri", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(Catalog.class, metaDataMap);
+  }
+
+  public Catalog() {
+  }
+
+  public Catalog(
+    String name,
+    String locationUri)
+  {
+    this();
+    this.name = name;
+    this.locationUri = locationUri;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public Catalog(Catalog other) {
+    if (other.isSetName()) {
+      this.name = other.name;
+    }
+    if (other.isSetDescription()) {
+      this.description = other.description;
+    }
+    if (other.isSetLocationUri()) {
+      this.locationUri = other.locationUri;
+    }
+  }
+
+  public Catalog deepCopy() {
+    return new Catalog(this);
+  }
+
+  @Override
+  public void clear() {
+    this.name = null;
+    this.description = null;
+    this.locationUri = null;
+  }
+
+  public String getName() {
+    return this.name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  public void unsetName() {
+    this.name = null;
+  }
+
+  /** Returns true if field name is set (has been assigned a value) and false otherwise */
+  public boolean isSetName() {
+    return this.name != null;
+  }
+
+  public void setNameIsSet(boolean value) {
+    if (!value) {
+      this.name = null;
+    }
+  }
+
+  public String getDescription() {
+    return this.description;
+  }
+
+  public void setDescription(String description) {
+    this.description = description;
+  }
+
+  public void unsetDescription() {
+    this.description = null;
+  }
+
+  /** Returns true if field description is set (has been assigned a value) and false otherwise */
+  public boolean isSetDescription() {
+    return this.description != null;
+  }
+
+  public void setDescriptionIsSet(boolean value) {
+    if (!value) {
+      this.description = null;
+    }
+  }
+
+  public String getLocationUri() {
+    return this.locationUri;
+  }
+
+  public void setLocationUri(String locationUri) {
+    this.locationUri = locationUri;
+  }
+
+  public void unsetLocationUri() {
+    this.locationUri = null;
+  }
+
+  /** Returns true if field locationUri is set (has been assigned a value) and false otherwise */
+  public boolean isSetLocationUri() {
+    return this.locationUri != null;
+  }
+
+  public void setLocationUriIsSet(boolean value) {
+    if (!value) {
+      this.locationUri = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case NAME:
+      if (value == null) {
+        unsetName();
+      } else {
+        setName((String)value);
+      }
+      break;
+
+    case DESCRIPTION:
+      if (value == null) {
+        unsetDescription();
+      } else {
+        setDescription((String)value);
+      }
+      break;
+
+    case LOCATION_URI:
+      if (value == null) {
+        unsetLocationUri();
+      } else {
+        setLocationUri((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case NAME:
+      return getName();
+
+    case DESCRIPTION:
+      return getDescription();
+
+    case LOCATION_URI:
+      return getLocationUri();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case NAME:
+      return isSetName();
+    case DESCRIPTION:
+      return isSetDescription();
+    case LOCATION_URI:
+      return isSetLocationUri();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof Catalog)
+      return this.equals((Catalog)that);
+    return false;
+  }
+
+  public boolean equals(Catalog that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_name = true && this.isSetName();
+    boolean that_present_name = true && that.isSetName();
+    if (this_present_name || that_present_name) {
+      if (!(this_present_name && that_present_name))
+        return false;
+      if (!this.name.equals(that.name))
+        return false;
+    }
+
+    boolean this_present_description = true && this.isSetDescription();
+    boolean that_present_description = true && that.isSetDescription();
+    if (this_present_description || that_present_description) {
+      if (!(this_present_description && that_present_description))
+        return false;
+      if (!this.description.equals(that.description))
+        return false;
+    }
+
+    boolean this_present_locationUri = true && this.isSetLocationUri();
+    boolean that_present_locationUri = true && that.isSetLocationUri();
+    if (this_present_locationUri || that_present_locationUri) {
+      if (!(this_present_locationUri && that_present_locationUri))
+        return false;
+      if (!this.locationUri.equals(that.locationUri))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_name = true && (isSetName());
+    list.add(present_name);
+    if (present_name)
+      list.add(name);
+
+    boolean present_description = true && (isSetDescription());
+    list.add(present_description);
+    if (present_description)
+      list.add(description);
+
+    boolean present_locationUri = true && (isSetLocationUri());
+    list.add(present_locationUri);
+    if (present_locationUri)
+      list.add(locationUri);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(Catalog other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetName()).compareTo(other.isSetName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetDescription()).compareTo(other.isSetDescription());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetDescription()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.description, other.description);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetLocationUri()).compareTo(other.isSetLocationUri());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetLocationUri()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.locationUri, other.locationUri);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("Catalog(");
+    boolean first = true;
+
+    sb.append("name:");
+    if (this.name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.name);
+    }
+    first = false;
+    if (isSetDescription()) {
+      if (!first) sb.append(", ");
+      sb.append("description:");
+      if (this.description == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.description);
+      }
+      first = false;
+    }
+    if (!first) sb.append(", ");
+    sb.append("locationUri:");
+    if (this.locationUri == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.locationUri);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class CatalogStandardSchemeFactory implements SchemeFactory {
+    public CatalogStandardScheme getScheme() {
+      return new CatalogStandardScheme();
+    }
+  }
+
+  private static class CatalogStandardScheme extends StandardScheme<Catalog> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, Catalog struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.name = iprot.readString();
+              struct.setNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // DESCRIPTION
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.description = iprot.readString();
+              struct.setDescriptionIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // LOCATION_URI
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.locationUri = iprot.readString();
+              struct.setLocationUriIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, Catalog struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.name != null) {
+        oprot.writeFieldBegin(NAME_FIELD_DESC);
+        oprot.writeString(struct.name);
+        oprot.writeFieldEnd();
+      }
+      if (struct.description != null) {
+        if (struct.isSetDescription()) {
+          oprot.writeFieldBegin(DESCRIPTION_FIELD_DESC);
+          oprot.writeString(struct.description);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.locationUri != null) {
+        oprot.writeFieldBegin(LOCATION_URI_FIELD_DESC);
+        oprot.writeString(struct.locationUri);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class CatalogTupleSchemeFactory implements SchemeFactory {
+    public CatalogTupleScheme getScheme() {
+      return new CatalogTupleScheme();
+    }
+  }
+
+  private static class CatalogTupleScheme extends TupleScheme<Catalog> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, Catalog struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.isSetName()) {
+        optionals.set(0);
+      }
+      if (struct.isSetDescription()) {
+        optionals.set(1);
+      }
+      if (struct.isSetLocationUri()) {
+        optionals.set(2);
+      }
+      oprot.writeBitSet(optionals, 3);
+      if (struct.isSetName()) {
+        oprot.writeString(struct.name);
+      }
+      if (struct.isSetDescription()) {
+        oprot.writeString(struct.description);
+      }
+      if (struct.isSetLocationUri()) {
+        oprot.writeString(struct.locationUri);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, Catalog struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(3);
+      if (incoming.get(0)) {
+        struct.name = iprot.readString();
+        struct.setNameIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.description = iprot.readString();
+        struct.setDescriptionIsSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.locationUri = iprot.readString();
+        struct.setLocationUriIsSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CheckConstraintsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CheckConstraintsRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CheckConstraintsRequest.java
index 2a8d81a..2ba0407 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CheckConstraintsRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CheckConstraintsRequest.java
@@ -38,8 +38,9 @@ import org.slf4j.LoggerFactory;
 @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class CheckConstraintsRequest implements org.apache.thrift.TBase<CheckConstraintsRequest, CheckConstraintsRequest._Fields>, java.io.Serializable, Cloneable, Comparable<CheckConstraintsRequest> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CheckConstraintsRequest");
 
-  private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("db_name", org.apache.thrift.protocol.TType.STRING, (short)1);
-  private static final org.apache.thrift.protocol.TField TBL_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tbl_name", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("db_name", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField TBL_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tbl_name", org.apache.thrift.protocol.TType.STRING, (short)3);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -47,13 +48,15 @@ import org.slf4j.LoggerFactory;
     schemes.put(TupleScheme.class, new CheckConstraintsRequestTupleSchemeFactory());
   }
 
+  private String catName; // required
   private String db_name; // required
   private String tbl_name; // required
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-    DB_NAME((short)1, "db_name"),
-    TBL_NAME((short)2, "tbl_name");
+    CAT_NAME((short)1, "catName"),
+    DB_NAME((short)2, "db_name"),
+    TBL_NAME((short)3, "tbl_name");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -68,9 +71,11 @@ import org.slf4j.LoggerFactory;
      */
     public static _Fields findByThriftId(int fieldId) {
       switch(fieldId) {
-        case 1: // DB_NAME
+        case 1: // CAT_NAME
+          return CAT_NAME;
+        case 2: // DB_NAME
           return DB_NAME;
-        case 2: // TBL_NAME
+        case 3: // TBL_NAME
           return TBL_NAME;
         default:
           return null;
@@ -115,6 +120,8 @@ import org.slf4j.LoggerFactory;
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.DB_NAME, new org.apache.thrift.meta_data.FieldMetaData("db_name", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.TBL_NAME, new org.apache.thrift.meta_data.FieldMetaData("tbl_name", org.apache.thrift.TFieldRequirementType.REQUIRED, 
@@ -127,10 +134,12 @@ import org.slf4j.LoggerFactory;
   }
 
   public CheckConstraintsRequest(
+    String catName,
     String db_name,
     String tbl_name)
   {
     this();
+    this.catName = catName;
     this.db_name = db_name;
     this.tbl_name = tbl_name;
   }
@@ -139,6 +148,9 @@ import org.slf4j.LoggerFactory;
    * Performs a deep copy on <i>other</i>.
    */
   public CheckConstraintsRequest(CheckConstraintsRequest other) {
+    if (other.isSetCatName()) {
+      this.catName = other.catName;
+    }
     if (other.isSetDb_name()) {
       this.db_name = other.db_name;
     }
@@ -153,10 +165,34 @@ import org.slf4j.LoggerFactory;
 
   @Override
   public void clear() {
+    this.catName = null;
     this.db_name = null;
     this.tbl_name = null;
   }
 
+  public String getCatName() {
+    return this.catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
+  public void unsetCatName() {
+    this.catName = null;
+  }
+
+  /** Returns true if field catName is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatName() {
+    return this.catName != null;
+  }
+
+  public void setCatNameIsSet(boolean value) {
+    if (!value) {
+      this.catName = null;
+    }
+  }
+
   public String getDb_name() {
     return this.db_name;
   }
@@ -205,6 +241,14 @@ import org.slf4j.LoggerFactory;
 
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
+    case CAT_NAME:
+      if (value == null) {
+        unsetCatName();
+      } else {
+        setCatName((String)value);
+      }
+      break;
+
     case DB_NAME:
       if (value == null) {
         unsetDb_name();
@@ -226,6 +270,9 @@ import org.slf4j.LoggerFactory;
 
   public Object getFieldValue(_Fields field) {
     switch (field) {
+    case CAT_NAME:
+      return getCatName();
+
     case DB_NAME:
       return getDb_name();
 
@@ -243,6 +290,8 @@ import org.slf4j.LoggerFactory;
     }
 
     switch (field) {
+    case CAT_NAME:
+      return isSetCatName();
     case DB_NAME:
       return isSetDb_name();
     case TBL_NAME:
@@ -264,6 +313,15 @@ import org.slf4j.LoggerFactory;
     if (that == null)
       return false;
 
+    boolean this_present_catName = true && this.isSetCatName();
+    boolean that_present_catName = true && that.isSetCatName();
+    if (this_present_catName || that_present_catName) {
+      if (!(this_present_catName && that_present_catName))
+        return false;
+      if (!this.catName.equals(that.catName))
+        return false;
+    }
+
     boolean this_present_db_name = true && this.isSetDb_name();
     boolean that_present_db_name = true && that.isSetDb_name();
     if (this_present_db_name || that_present_db_name) {
@@ -289,6 +347,11 @@ import org.slf4j.LoggerFactory;
   public int hashCode() {
     List<Object> list = new ArrayList<Object>();
 
+    boolean present_catName = true && (isSetCatName());
+    list.add(present_catName);
+    if (present_catName)
+      list.add(catName);
+
     boolean present_db_name = true && (isSetDb_name());
     list.add(present_db_name);
     if (present_db_name)
@@ -310,6 +373,16 @@ import org.slf4j.LoggerFactory;
 
     int lastComparison = 0;
 
+    lastComparison = Boolean.valueOf(isSetCatName()).compareTo(other.isSetCatName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catName, other.catName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     lastComparison = Boolean.valueOf(isSetDb_name()).compareTo(other.isSetDb_name());
     if (lastComparison != 0) {
       return lastComparison;
@@ -350,6 +423,14 @@ import org.slf4j.LoggerFactory;
     StringBuilder sb = new StringBuilder("CheckConstraintsRequest(");
     boolean first = true;
 
+    sb.append("catName:");
+    if (this.catName == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.catName);
+    }
+    first = false;
+    if (!first) sb.append(", ");
     sb.append("db_name:");
     if (this.db_name == null) {
       sb.append("null");
@@ -371,6 +452,10 @@ import org.slf4j.LoggerFactory;
 
   public void validate() throws org.apache.thrift.TException {
     // check for required fields
+    if (!isSetCatName()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'catName' is unset! Struct:" + toString());
+    }
+
     if (!isSetDb_name()) {
       throw new org.apache.thrift.protocol.TProtocolException("Required field 'db_name' is unset! Struct:" + toString());
     }
@@ -416,7 +501,15 @@ import org.slf4j.LoggerFactory;
           break;
         }
         switch (schemeField.id) {
-          case 1: // DB_NAME
+          case 1: // CAT_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.catName = iprot.readString();
+              struct.setCatNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // DB_NAME
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.db_name = iprot.readString();
               struct.setDb_nameIsSet(true);
@@ -424,7 +517,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 2: // TBL_NAME
+          case 3: // TBL_NAME
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.tbl_name = iprot.readString();
               struct.setTbl_nameIsSet(true);
@@ -445,6 +538,11 @@ import org.slf4j.LoggerFactory;
       struct.validate();
 
       oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.catName != null) {
+        oprot.writeFieldBegin(CAT_NAME_FIELD_DESC);
+        oprot.writeString(struct.catName);
+        oprot.writeFieldEnd();
+      }
       if (struct.db_name != null) {
         oprot.writeFieldBegin(DB_NAME_FIELD_DESC);
         oprot.writeString(struct.db_name);
@@ -472,6 +570,7 @@ import org.slf4j.LoggerFactory;
     @Override
     public void write(org.apache.thrift.protocol.TProtocol prot, CheckConstraintsRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.catName);
       oprot.writeString(struct.db_name);
       oprot.writeString(struct.tbl_name);
     }
@@ -479,6 +578,8 @@ import org.slf4j.LoggerFactory;
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, CheckConstraintsRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.catName = iprot.readString();
+      struct.setCatNameIsSet(true);
       struct.db_name = iprot.readString();
       struct.setDb_nameIsSet(true);
       struct.tbl_name = iprot.readString();

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CheckConstraintsResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CheckConstraintsResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CheckConstraintsResponse.java
index a0a4422..8d4f7be 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CheckConstraintsResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CheckConstraintsResponse.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // CHECK_CONSTRAINTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list352 = iprot.readListBegin();
-                struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list352.size);
-                SQLCheckConstraint _elem353;
-                for (int _i354 = 0; _i354 < _list352.size; ++_i354)
+                org.apache.thrift.protocol.TList _list360 = iprot.readListBegin();
+                struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list360.size);
+                SQLCheckConstraint _elem361;
+                for (int _i362 = 0; _i362 < _list360.size; ++_i362)
                 {
-                  _elem353 = new SQLCheckConstraint();
-                  _elem353.read(iprot);
-                  struct.checkConstraints.add(_elem353);
+                  _elem361 = new SQLCheckConstraint();
+                  _elem361.read(iprot);
+                  struct.checkConstraints.add(_elem361);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(CHECK_CONSTRAINTS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.checkConstraints.size()));
-          for (SQLCheckConstraint _iter355 : struct.checkConstraints)
+          for (SQLCheckConstraint _iter363 : struct.checkConstraints)
           {
-            _iter355.write(oprot);
+            _iter363.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.checkConstraints.size());
-        for (SQLCheckConstraint _iter356 : struct.checkConstraints)
+        for (SQLCheckConstraint _iter364 : struct.checkConstraints)
         {
-          _iter356.write(oprot);
+          _iter364.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, CheckConstraintsResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list357 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list357.size);
-        SQLCheckConstraint _elem358;
-        for (int _i359 = 0; _i359 < _list357.size; ++_i359)
+        org.apache.thrift.protocol.TList _list365 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list365.size);
+        SQLCheckConstraint _elem366;
+        for (int _i367 = 0; _i367 < _list365.size; ++_i367)
         {
-          _elem358 = new SQLCheckConstraint();
-          _elem358.read(iprot);
-          struct.checkConstraints.add(_elem358);
+          _elem366 = new SQLCheckConstraint();
+          _elem366.read(iprot);
+          struct.checkConstraints.add(_elem366);
         }
       }
       struct.setCheckConstraintsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java
index c4c1835..b4bf2ce 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java
@@ -351,13 +351,13 @@ import org.slf4j.LoggerFactory;
           case 1: // FILE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list768 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list768.size);
-                long _elem769;
-                for (int _i770 = 0; _i770 < _list768.size; ++_i770)
+                org.apache.thrift.protocol.TList _list776 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list776.size);
+                long _elem777;
+                for (int _i778 = 0; _i778 < _list776.size; ++_i778)
                 {
-                  _elem769 = iprot.readI64();
-                  struct.fileIds.add(_elem769);
+                  _elem777 = iprot.readI64();
+                  struct.fileIds.add(_elem777);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FILE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.fileIds.size()));
-          for (long _iter771 : struct.fileIds)
+          for (long _iter779 : struct.fileIds)
           {
-            oprot.writeI64(_iter771);
+            oprot.writeI64(_iter779);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter772 : struct.fileIds)
+        for (long _iter780 : struct.fileIds)
         {
-          oprot.writeI64(_iter772);
+          oprot.writeI64(_iter780);
         }
       }
     }
@@ -421,13 +421,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, ClearFileMetadataRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list773 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list773.size);
-        long _elem774;
-        for (int _i775 = 0; _i775 < _list773.size; ++_i775)
+        org.apache.thrift.protocol.TList _list781 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list781.size);
+        long _elem782;
+        for (int _i783 = 0; _i783 < _list781.size; ++_i783)
         {
-          _elem774 = iprot.readI64();
-          struct.fileIds.add(_elem774);
+          _elem782 = iprot.readI64();
+          struct.fileIds.add(_elem782);
         }
       }
       struct.setFileIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java
index 3085522..a214a87 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java
@@ -354,13 +354,13 @@ import org.slf4j.LoggerFactory;
           case 1: // VALUES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list784 = iprot.readListBegin();
-                struct.values = new ArrayList<ClientCapability>(_list784.size);
-                ClientCapability _elem785;
-                for (int _i786 = 0; _i786 < _list784.size; ++_i786)
+                org.apache.thrift.protocol.TList _list792 = iprot.readListBegin();
+                struct.values = new ArrayList<ClientCapability>(_list792.size);
+                ClientCapability _elem793;
+                for (int _i794 = 0; _i794 < _list792.size; ++_i794)
                 {
-                  _elem785 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32());
-                  struct.values.add(_elem785);
+                  _elem793 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32());
+                  struct.values.add(_elem793);
                 }
                 iprot.readListEnd();
               }
@@ -386,9 +386,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(VALUES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, struct.values.size()));
-          for (ClientCapability _iter787 : struct.values)
+          for (ClientCapability _iter795 : struct.values)
           {
-            oprot.writeI32(_iter787.getValue());
+            oprot.writeI32(_iter795.getValue());
           }
           oprot.writeListEnd();
         }
@@ -413,9 +413,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.values.size());
-        for (ClientCapability _iter788 : struct.values)
+        for (ClientCapability _iter796 : struct.values)
         {
-          oprot.writeI32(_iter788.getValue());
+          oprot.writeI32(_iter796.getValue());
         }
       }
     }
@@ -424,13 +424,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, ClientCapabilities struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list789 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, iprot.readI32());
-        struct.values = new ArrayList<ClientCapability>(_list789.size);
-        ClientCapability _elem790;
-        for (int _i791 = 0; _i791 < _list789.size; ++_i791)
+        org.apache.thrift.protocol.TList _list797 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, iprot.readI32());
+        struct.values = new ArrayList<ClientCapability>(_list797.size);
+        ClientCapability _elem798;
+        for (int _i799 = 0; _i799 < _list797.size; ++_i799)
         {
-          _elem790 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32());
-          struct.values.add(_elem790);
+          _elem798 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32());
+          struct.values.add(_elem798);
         }
       }
       struct.setValuesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatistics.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatistics.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatistics.java
index 9eb4652..6ce7214 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatistics.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ColumnStatistics.java
@@ -451,14 +451,14 @@ import org.slf4j.LoggerFactory;
           case 2: // STATS_OBJ
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list260 = iprot.readListBegin();
-                struct.statsObj = new ArrayList<ColumnStatisticsObj>(_list260.size);
-                ColumnStatisticsObj _elem261;
-                for (int _i262 = 0; _i262 < _list260.size; ++_i262)
+                org.apache.thrift.protocol.TList _list268 = iprot.readListBegin();
+                struct.statsObj = new ArrayList<ColumnStatisticsObj>(_list268.size);
+                ColumnStatisticsObj _elem269;
+                for (int _i270 = 0; _i270 < _list268.size; ++_i270)
                 {
-                  _elem261 = new ColumnStatisticsObj();
-                  _elem261.read(iprot);
-                  struct.statsObj.add(_elem261);
+                  _elem269 = new ColumnStatisticsObj();
+                  _elem269.read(iprot);
+                  struct.statsObj.add(_elem269);
                 }
                 iprot.readListEnd();
               }
@@ -489,9 +489,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(STATS_OBJ_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.statsObj.size()));
-          for (ColumnStatisticsObj _iter263 : struct.statsObj)
+          for (ColumnStatisticsObj _iter271 : struct.statsObj)
           {
-            _iter263.write(oprot);
+            _iter271.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -517,9 +517,9 @@ import org.slf4j.LoggerFactory;
       struct.statsDesc.write(oprot);
       {
         oprot.writeI32(struct.statsObj.size());
-        for (ColumnStatisticsObj _iter264 : struct.statsObj)
+        for (ColumnStatisticsObj _iter272 : struct.statsObj)
         {
-          _iter264.write(oprot);
+          _iter272.write(oprot);
         }
       }
     }
@@ -531,14 +531,14 @@ import org.slf4j.LoggerFactory;
       struct.statsDesc.read(iprot);
       struct.setStatsDescIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list265 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.statsObj = new ArrayList<ColumnStatisticsObj>(_list265.size);
-        ColumnStatisticsObj _elem266;
-        for (int _i267 = 0; _i267 < _list265.size; ++_i267)
+        org.apache.thrift.protocol.TList _list273 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.statsObj = new ArrayList<ColumnStatisticsObj>(_list273.size);
+        ColumnStatisticsObj _elem274;
+        for (int _i275 = 0; _i275 < _list273.size; ++_i275)
         {
-          _elem266 = new ColumnStatisticsObj();
-          _elem266.read(iprot);
-          struct.statsObj.add(_elem266);
+          _elem274 = new ColumnStatisticsObj();
+          _elem274.read(iprot);
+          struct.statsObj.add(_elem274);
         }
       }
       struct.setStatsObjIsSet(true);


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql b/standalone-metastore/src/main/sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql
index 5a483ab..20f7c8d 100644
--- a/standalone-metastore/src/main/sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql
+++ b/standalone-metastore/src/main/sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql
@@ -129,6 +129,7 @@ CREATE TABLE `SCHEMA_VERSION` (
 -- 048-HIVE-14498
 CREATE TABLE IF NOT EXISTS `MV_CREATION_METADATA` (
   `MV_CREATION_METADATA_ID` bigint(20) NOT NULL,
+  `CAT_NAME` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
   `DB_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
   `TBL_NAME` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
   `TXN_LIST` TEXT DEFAULT NULL,
@@ -202,6 +203,62 @@ ALTER TABLE `KEY_CONSTRAINTS` ADD COLUMN `DEFAULT_VALUE` VARCHAR(400);
 
 ALTER TABLE `HIVE_LOCKS` CHANGE COLUMN `HL_TXNID` `HL_TXNID` bigint NOT NULL;
 
+-- HIVE-18755, add catalogs
+-- new catalogs table
+CREATE TABLE `CTLGS` (
+    `CTLG_ID` BIGINT PRIMARY KEY,
+    `NAME` VARCHAR(256),
+    `DESC` VARCHAR(4000),
+    `LOCATION_URI` VARCHAR(4000) NOT NULL,
+    UNIQUE KEY `UNIQUE_CATALOG` (`NAME`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
+-- Insert a default value.  The location is TBD.  Hive will fix this when it starts
+INSERT INTO `CTLGS` VALUES (1, 'hive', 'Default catalog for Hive', 'TBD');
+
+-- Drop the unique index on DBS
+ALTER TABLE `DBS` DROP KEY `UNIQUE_DATABASE`;
+
+-- Add the new column to the DBS table, can't put in the not null constraint yet
+ALTER TABLE `DBS` ADD COLUMN `CTLG_NAME` VARCHAR(256);
+
+-- Update all records in the DBS table to point to the Hive catalog
+UPDATE `DBS` 
+  SET `CTLG_NAME` = 'hive';
+
+-- Add the not null constraint
+ALTER TABLE `DBS` CHANGE COLUMN `CTLG_NAME` `CTLG_NAME` varchar(256) NOT NULL;
+
+-- Put back the unique index 
+ALTER TABLE `DBS` ADD UNIQUE KEY `UNIQUE_DATABASE` (`NAME`, `CTLG_NAME`);
+
+-- Add the foreign key
+ALTER TABLE `DBS` ADD CONSTRAINT `CTLG_FK1` FOREIGN KEY (`CTLG_NAME`) REFERENCES `CTLGS` (`NAME`);
+
+-- Add columns to table stats and part stats
+ALTER TABLE `TAB_COL_STATS` ADD COLUMN `CAT_NAME` varchar(256);
+ALTER TABLE `PART_COL_STATS` ADD COLUMN `CAT_NAME` varchar(256);
+
+-- Set the existing column names to Hive
+UPDATE `TAB_COL_STATS`
+  SET `CAT_NAME` = 'hive';
+UPDATE `PART_COL_STATS`
+  SET `CAT_NAME` = 'hive';
+
+-- Add the not null constraint
+ALTER TABLE `TAB_COL_STATS` CHANGE COLUMN `CAT_NAME` `CAT_NAME` varchar(256) NOT NULL;
+ALTER TABLE `PART_COL_STATS` CHANGE COLUMN `CAT_NAME` `CAT_NAME` varchar(256) NOT NULL;
+
+-- Rebuild the index for Part col stats.  No such index for table stats, which seems weird
+DROP INDEX `PCS_STATS_IDX` ON `PART_COL_STATS`;
+CREATE INDEX `PCS_STATS_IDX` ON `PART_COL_STATS` (`CAT_NAME`, `DB_NAME`, `TABLE_NAME`, `COLUMN_NAME`, `PARTITION_NAME`);
+
+-- Add column to partition events
+ALTER TABLE `PARTITION_EVENTS` ADD COLUMN `CAT_NAME` varchar(256);
+
+-- Add column to notification log
+ALTER TABLE `NOTIFICATION_LOG` ADD COLUMN `CAT_NAME` varchar(256);
+
 -- These lines need to be last.  Insert any changes above.
 UPDATE VERSION SET SCHEMA_VERSION='3.0.0', VERSION_COMMENT='Hive release version 3.0.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.3.0 to 3.0.0' AS ' ';

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/sql/oracle/hive-schema-3.0.0.oracle.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/oracle/hive-schema-3.0.0.oracle.sql b/standalone-metastore/src/main/sql/oracle/hive-schema-3.0.0.oracle.sql
index 09c40ad..755a8a8 100644
--- a/standalone-metastore/src/main/sql/oracle/hive-schema-3.0.0.oracle.sql
+++ b/standalone-metastore/src/main/sql/oracle/hive-schema-3.0.0.oracle.sql
@@ -72,6 +72,14 @@ CREATE TABLE PARTITION_KEY_VALS
 
 ALTER TABLE PARTITION_KEY_VALS ADD CONSTRAINT PARTITION_KEY_VALS_PK PRIMARY KEY (PART_ID,INTEGER_IDX);
 
+CREATE TABLE CTLGS (
+    CTLG_ID NUMBER PRIMARY KEY,
+    "NAME" VARCHAR2(256),
+    "DESC" VARCHAR2(4000),
+    LOCATION_URI VARCHAR2(4000) NOT NULL,
+    UNIQUE ("NAME")
+);
+
 -- Table DBS for classes [org.apache.hadoop.hive.metastore.model.MDatabase]
 CREATE TABLE DBS
 (
@@ -80,7 +88,8 @@ CREATE TABLE DBS
     DB_LOCATION_URI VARCHAR2(4000) NOT NULL,
     "NAME" VARCHAR2(128) NULL,
     OWNER_NAME VARCHAR2(128) NULL,
-    OWNER_TYPE VARCHAR2(10) NULL
+    OWNER_TYPE VARCHAR2(10) NULL,
+    CTLG_NAME VARCHAR2(256)
 );
 
 ALTER TABLE DBS ADD CONSTRAINT DBS_PK PRIMARY KEY (DB_ID);
@@ -389,6 +398,7 @@ ALTER TABLE TBLS ADD CONSTRAINT TBLS_PK PRIMARY KEY (TBL_ID);
 CREATE TABLE MV_CREATION_METADATA
 (
     MV_CREATION_METADATA_ID NUMBER NOT NULL,
+    CAT_NAME VARCHAR2(256) NOT NULL,
     DB_NAME VARCHAR2(128) NOT NULL,
     TBL_NAME VARCHAR2(256) NOT NULL,
     TXN_LIST CLOB NULL
@@ -409,6 +419,7 @@ CREATE TABLE MV_TABLES_USED
 CREATE TABLE PARTITION_EVENTS
 (
     PART_NAME_ID NUMBER NOT NULL,
+    CAT_NAME VARCHAR2(256) NULL,
     DB_NAME VARCHAR2(128) NULL,
     EVENT_TIME NUMBER NOT NULL,
     EVENT_TYPE NUMBER (10) NOT NULL,
@@ -486,10 +497,13 @@ ALTER TABLE SKEWED_VALUES ADD CONSTRAINT SKEWED_VALUES_FK1 FOREIGN KEY (STRING_L
 
 ALTER TABLE SKEWED_VALUES ADD CONSTRAINT SKEWED_VALUES_FK2 FOREIGN KEY (SD_ID_OID) REFERENCES SDS (SD_ID) INITIALLY DEFERRED ;
 
+ALTER TABLE DBS ADD CONSTRAINT CTLGS_FK FOREIGN KEY (CTLG_NAME) REFERENCES CTLGS ("NAME") INITIALLY DEFERRED;
+
 -- column statistics
 
 CREATE TABLE TAB_COL_STATS (
  CS_ID NUMBER NOT NULL,
+ CAT_NAME VARCHAR2(256) NOT NULL,
  DB_NAME VARCHAR2(128) NOT NULL,
  TABLE_NAME VARCHAR2(256) NOT NULL,
  COLUMN_NAME VARCHAR2(767) NOT NULL,
@@ -526,6 +540,7 @@ CREATE INDEX TAB_COL_STATS_N49 ON TAB_COL_STATS(TBL_ID);
 
 CREATE TABLE PART_COL_STATS (
  CS_ID NUMBER NOT NULL,
+ CAT_NAME VARCHAR2(256) NOT NULL,
  DB_NAME VARCHAR2(128) NOT NULL,
  TABLE_NAME VARCHAR2(256) NOT NULL,
  PARTITION_NAME VARCHAR2(767) NOT NULL,
@@ -554,7 +569,7 @@ ALTER TABLE PART_COL_STATS ADD CONSTRAINT PART_COL_STATS_FK FOREIGN KEY (PART_ID
 
 CREATE INDEX PART_COL_STATS_N49 ON PART_COL_STATS (PART_ID);
 
-CREATE INDEX PCS_STATS_IDX ON PART_COL_STATS (DB_NAME,TABLE_NAME,COLUMN_NAME,PARTITION_NAME);
+CREATE INDEX PCS_STATS_IDX ON PART_COL_STATS (CAT_NAME, DB_NAME,TABLE_NAME,COLUMN_NAME,PARTITION_NAME);
 
 CREATE TABLE FUNCS (
   FUNC_ID NUMBER NOT NULL,
@@ -584,6 +599,7 @@ CREATE TABLE NOTIFICATION_LOG
     EVENT_ID NUMBER NOT NULL,
     EVENT_TIME NUMBER(10) NOT NULL,
     EVENT_TYPE VARCHAR2(32) NOT NULL,
+    CAT_NAME VARCHAR2(256),
     DB_NAME VARCHAR2(128),
     TBL_NAME VARCHAR2(256),
     MESSAGE CLOB NULL,
@@ -678,7 +694,7 @@ CREATE INDEX PARTITION_KEY_VALS_N49 ON PARTITION_KEY_VALS (PART_ID);
 
 
 -- Constraints for table DBS for class(es) [org.apache.hadoop.hive.metastore.model.MDatabase]
-CREATE UNIQUE INDEX UNIQUE_DATABASE ON DBS ("NAME");
+CREATE UNIQUE INDEX UNIQUE_DATABASE ON DBS ("NAME", CTLG_NAME);
 
 
 -- Constraints for table PARTITION_PARAMS

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql b/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql
index f95819e..cd94c01 100644
--- a/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql
+++ b/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql
@@ -142,6 +142,7 @@ CREATE TABLE "SCHEMA_VERSION" (
 CREATE TABLE MV_CREATION_METADATA
 (
     MV_CREATION_METADATA_ID NUMBER NOT NULL,
+    CAT_NAME VARCHAR2(256) NOT NULL,
     DB_NAME VARCHAR2(128) NOT NULL,
     TBL_NAME VARCHAR2(256) NOT NULL,
     TXN_LIST CLOB NULL
@@ -188,7 +189,6 @@ UPDATE DBS
   SET DB_LOCATION_URI = 's3a' || SUBSTR(DB_LOCATION_URI, 4)
   WHERE DB_LOCATION_URI LIKE 's3n://%' ;
 
-
 -- HIVE-18192
 CREATE TABLE TXN_TO_WRITE_ID (
   T2W_TXNID number(19) NOT NULL,
@@ -222,6 +222,62 @@ ALTER TABLE KEY_CONSTRAINTS ADD DEFAULT_VALUE VARCHAR(400);
 
 ALTER TABLE HIVE_LOCKS MODIFY(HL_TXNID NOT NULL);
 
+-- HIVE-18755, add catalogs
+-- new catalogs table
+CREATE TABLE CTLGS (
+    CTLG_ID NUMBER PRIMARY KEY,
+    "NAME" VARCHAR2(256),
+    "DESC" VARCHAR2(4000),
+    LOCATION_URI VARCHAR2(4000) NOT NULL,
+    UNIQUE ("NAME")
+);
+
+-- Insert a default value.  The location is TBD.  Hive will fix this when it starts
+INSERT INTO CTLGS VALUES (1, 'hive', 'Default catalog for Hive', 'TBD');
+
+-- Drop the unique index on DBS
+DROP INDEX UNIQUE_DATABASE;
+
+-- Add the new column to the DBS table, can't put in the not null constraint yet
+ALTER TABLE DBS ADD CTLG_NAME VARCHAR2(256);
+
+-- Update all records in the DBS table to point to the Hive catalog
+UPDATE DBS 
+  SET "CTLG_NAME" = 'hive';
+
+-- Add the not null constraint
+ALTER TABLE DBS MODIFY CTLG_NAME NOT NULL;
+
+-- Put back the unique index 
+CREATE UNIQUE INDEX UNIQUE_DATABASE ON DBS ("NAME", CTLG_NAME);
+
+-- Add the foreign key
+ALTER TABLE DBS ADD CONSTRAINT CTLGS_FK FOREIGN KEY (CTLG_NAME) REFERENCES CTLGS ("NAME") INITIALLY DEFERRED;
+
+-- Add columns to table stats and part stats
+ALTER TABLE TAB_COL_STATS ADD CAT_NAME VARCHAR2(256);
+ALTER TABLE PART_COL_STATS ADD CAT_NAME VARCHAR2(256);
+
+-- Set the existing column names to Hive
+UPDATE TAB_COL_STATS
+  SET CAT_NAME = 'hive';
+UPDATE PART_COL_STATS
+  SET CAT_NAME = 'hive';
+
+-- Add the not null constraint
+ALTER TABLE TAB_COL_STATS MODIFY CAT_NAME NOT NULL;
+ALTER TABLE PART_COL_STATS MODIFY CAT_NAME NOT NULL;
+
+-- Rebuild the index for Part col stats.  No such index for table stats, which seems weird
+DROP INDEX PCS_STATS_IDX;
+CREATE INDEX PCS_STATS_IDX ON PART_COL_STATS (CAT_NAME, DB_NAME,TABLE_NAME,COLUMN_NAME,PARTITION_NAME);
+
+-- Add column to partition events
+ALTER TABLE PARTITION_EVENTS ADD CAT_NAME VARCHAR2(256);
+
+-- Add column to notification log
+ALTER TABLE NOTIFICATION_LOG ADD CAT_NAME VARCHAR2(256);
+
 -- These lines need to be last.  Insert any changes above.
 UPDATE VERSION SET SCHEMA_VERSION='3.0.0', VERSION_COMMENT='Hive release version 3.0.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.3.0 to 3.0.0' AS Status from dual;

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/sql/postgres/hive-schema-3.0.0.postgres.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/postgres/hive-schema-3.0.0.postgres.sql b/standalone-metastore/src/main/sql/postgres/hive-schema-3.0.0.postgres.sql
index 69317b0..72e5966 100644
--- a/standalone-metastore/src/main/sql/postgres/hive-schema-3.0.0.postgres.sql
+++ b/standalone-metastore/src/main/sql/postgres/hive-schema-3.0.0.postgres.sql
@@ -59,6 +59,13 @@ CREATE TABLE "DATABASE_PARAMS" (
 );
 
 
+CREATE TABLE "CTLGS" (
+    "CTLG_ID" BIGINT PRIMARY KEY,
+    "NAME" VARCHAR(256) UNIQUE,
+    "DESC" VARCHAR(4000),
+    "LOCATION_URI" VARCHAR(4000) NOT NULL
+);
+
 --
 -- Name: DBS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
 --
@@ -69,7 +76,8 @@ CREATE TABLE "DBS" (
     "DB_LOCATION_URI" character varying(4000) NOT NULL,
     "NAME" character varying(128) DEFAULT NULL::character varying,
     "OWNER_NAME" character varying(128) DEFAULT NULL::character varying,
-    "OWNER_TYPE" character varying(10) DEFAULT NULL::character varying
+    "OWNER_TYPE" character varying(10) DEFAULT NULL::character varying,
+    "CTLG_NAME" varchar(256)
 );
 
 
@@ -168,6 +176,7 @@ CREATE TABLE "PARTITIONS" (
 
 CREATE TABLE "PARTITION_EVENTS" (
     "PART_NAME_ID" bigint NOT NULL,
+    "CAT_NAME" character varying(256),
     "DB_NAME" character varying(128),
     "EVENT_TIME" bigint NOT NULL,
     "EVENT_TYPE" integer NOT NULL,
@@ -386,6 +395,7 @@ CREATE TABLE "TBLS" (
 
 CREATE TABLE "MV_CREATION_METADATA" (
     "MV_CREATION_METADATA_ID" bigint NOT NULL,
+    "CAT_NAME" character varying(256) NOT NULL,
     "DB_NAME" character varying(128) NOT NULL,
     "TBL_NAME" character varying(256) NOT NULL,
     "TXN_LIST" text
@@ -508,6 +518,7 @@ CREATE TABLE  "DELEGATION_TOKENS"
 
 CREATE TABLE "TAB_COL_STATS" (
  "CS_ID" bigint NOT NULL,
+ "CAT_NAME" character varying(256) DEFAULT NULL::character varying,
  "DB_NAME" character varying(128) DEFAULT NULL::character varying,
  "TABLE_NAME" character varying(256) DEFAULT NULL::character varying,
  "COLUMN_NAME" character varying(767) DEFAULT NULL::character varying,
@@ -544,6 +555,7 @@ CREATE TABLE "VERSION" (
 
 CREATE TABLE "PART_COL_STATS" (
  "CS_ID" bigint NOT NULL,
+ "CAT_NAME" character varying(256) DEFAULT NULL::character varying,
  "DB_NAME" character varying(128) DEFAULT NULL::character varying,
  "TABLE_NAME" character varying(256) DEFAULT NULL::character varying,
  "PARTITION_NAME" character varying(767) DEFAULT NULL::character varying,
@@ -598,6 +610,7 @@ CREATE TABLE "NOTIFICATION_LOG"
     "EVENT_ID" BIGINT NOT NULL,
     "EVENT_TIME" INTEGER NOT NULL,
     "EVENT_TYPE" VARCHAR(32) NOT NULL,
+    "CAT_NAME" VARCHAR(256),
     "DB_NAME" VARCHAR(128),
     "TBL_NAME" VARCHAR(256),
     "MESSAGE" text,
@@ -1182,7 +1195,7 @@ CREATE INDEX "PART_PRIVS_N49" ON "PART_PRIVS" USING btree ("PART_ID");
 -- Name: PCS_STATS_IDX; Type: INDEX; Schema: public; Owner: hiveuser; Tablespace:
 --
 
-CREATE INDEX "PCS_STATS_IDX" ON "PART_COL_STATS" USING btree ("DB_NAME","TABLE_NAME","COLUMN_NAME","PARTITION_NAME");
+CREATE INDEX "PCS_STATS_IDX" ON "PART_COL_STATS" USING btree ("CAT_NAME", "DB_NAME","TABLE_NAME","COLUMN_NAME","PARTITION_NAME");
 
 
 --
@@ -1556,6 +1569,7 @@ ALTER TABLE ONLY "TAB_COL_STATS" ADD CONSTRAINT "TAB_COL_STATS_fkey" FOREIGN KEY
 --
 ALTER TABLE ONLY "PART_COL_STATS" ADD CONSTRAINT "PART_COL_STATS_fkey" FOREIGN KEY("PART_ID") REFERENCES "PARTITIONS"("PART_ID") DEFERRABLE;
 
+ALTER TABLE "DBS" ADD CONSTRAINT "DBS_FK1" FOREIGN KEY ("CTLG_NAME") REFERENCES "CTLGS" ("NAME");
 
 ALTER TABLE ONLY "VERSION" ADD CONSTRAINT "VERSION_pkey" PRIMARY KEY ("VER_ID");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/sql/postgres/upgrade-2.3.0-to-3.0.0.postgres.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/postgres/upgrade-2.3.0-to-3.0.0.postgres.sql b/standalone-metastore/src/main/sql/postgres/upgrade-2.3.0-to-3.0.0.postgres.sql
index f7d8c73..7b4bd68 100644
--- a/standalone-metastore/src/main/sql/postgres/upgrade-2.3.0-to-3.0.0.postgres.sql
+++ b/standalone-metastore/src/main/sql/postgres/upgrade-2.3.0-to-3.0.0.postgres.sql
@@ -155,6 +155,7 @@ CREATE TABLE "SCHEMA_VERSION" (
 -- 047-HIVE-14498
 CREATE TABLE "MV_CREATION_METADATA" (
     "MV_CREATION_METADATA_ID" bigint NOT NULL,
+    "CAT_NAME" character varying(256) NOT NULL,
     "DB_NAME" character varying(128) NOT NULL,
     "TBL_NAME" character varying(256) NOT NULL,
     "TXN_LIST" text
@@ -237,6 +238,61 @@ ALTER TABLE "KEY_CONSTRAINTS" ADD COLUMN "DEFAULT_VALUE" VARCHAR(400);
 
 ALTER TABLE HIVE_LOCKS ALTER COLUMN HL_TXNID SET NOT NULL;
 
+-- HIVE-18755, add catalogs
+-- new catalogs table
+CREATE TABLE "CTLGS" (
+    "CTLG_ID" BIGINT PRIMARY KEY,
+    "NAME" VARCHAR(256) UNIQUE,
+    "DESC" VARCHAR(4000),
+    "LOCATION_URI" VARCHAR(4000) NOT NULL
+);
+
+-- Insert a default value.  The location is TBD.  Hive will fix this when it starts
+INSERT INTO "CTLGS" VALUES (1, 'hive', 'Default catalog for Hive', 'TBD');
+
+-- Drop the unique index on DBS
+ALTER TABLE "DBS" DROP CONSTRAINT "UNIQUE_DATABASE";
+
+-- Add the new column to the DBS table, can't put in the not null constraint yet
+ALTER TABLE "DBS" ADD "CTLG_NAME" VARCHAR(256);
+
+-- Update all records in the DBS table to point to the Hive catalog
+UPDATE "DBS" 
+  SET "CTLG_NAME" = 'hive';
+
+-- Add the not null constraint
+ALTER TABLE "DBS" ALTER COLUMN "CTLG_NAME" SET NOT NULL;
+
+-- Put back the unique index 
+ALTER TABLE "DBS" ADD CONSTRAINT "UNIQUE_DATABASE" UNIQUE ("NAME", "CTLG_NAME");
+
+-- Add the foreign key
+ALTER TABLE "DBS" ADD CONSTRAINT "DBS_FK1" FOREIGN KEY ("CTLG_NAME") REFERENCES "CTLGS" ("NAME");
+
+-- Add columns to table stats and part stats
+ALTER TABLE "TAB_COL_STATS" ADD "CAT_NAME" varchar(256);
+ALTER TABLE "PART_COL_STATS" ADD "CAT_NAME" varchar(256);
+
+-- Set the existing column names to Hive
+UPDATE "TAB_COL_STATS"
+  SET "CAT_NAME" = 'hive';
+UPDATE "PART_COL_STATS"
+  SET "CAT_NAME" = 'hive';
+
+-- Add the not null constraint
+ALTER TABLE "TAB_COL_STATS" ALTER COLUMN "CAT_NAME" SET NOT NULL;
+ALTER TABLE "PART_COL_STATS" ALTER COLUMN "CAT_NAME" SET NOT NULL;
+
+-- Rebuild the index for Part col stats.  No such index for table stats, which seems weird
+DROP INDEX "PCS_STATS_IDX";
+CREATE INDEX "PCS_STATS_IDX" ON "PART_COL_STATS" ("CAT_NAME", "DB_NAME", "TABLE_NAME", "COLUMN_NAME", "PARTITION_NAME");
+
+-- Add column to partition event
+ALTER TABLE "PARTITION_EVENTS" ADD "CAT_NAME" varchar(256);
+
+-- Add column to notification log
+ALTER TABLE "NOTIFICATION_LOG" ADD "CAT_NAME" varchar(256);
+
 -- These lines need to be last.  Insert any changes above.
 UPDATE "VERSION" SET "SCHEMA_VERSION"='3.0.0', "VERSION_COMMENT"='Hive release version 3.0.0' where "VER_ID"=1;
 SELECT 'Finished upgrading MetaStore schema from 2.3.0 to 3.0.0';

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/thrift/hive_metastore.thrift
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/thrift/hive_metastore.thrift b/standalone-metastore/src/main/thrift/hive_metastore.thrift
index 68d7b45..27917f0 100644
--- a/standalone-metastore/src/main/thrift/hive_metastore.thrift
+++ b/standalone-metastore/src/main/thrift/hive_metastore.thrift
@@ -48,8 +48,9 @@ struct SQLPrimaryKey {
   4: i32 key_seq,        // sequence number within primary key
   5: string pk_name,     // primary key name
   6: bool enable_cstr,   // Enable/Disable
-  7: bool validate_cstr,  // Validate/No validate
-  8: bool rely_cstr      // Rely/No Rely
+  7: bool validate_cstr, // Validate/No validate
+  8: bool rely_cstr,     // Rely/No Rely
+  9: optional string catName
 }
 
 struct SQLForeignKey {
@@ -66,50 +67,55 @@ struct SQLForeignKey {
   11: string pk_name,      // primary key name
   12: bool enable_cstr,    // Enable/Disable
   13: bool validate_cstr,  // Validate/No validate
-  14: bool rely_cstr       // Rely/No Rely
+  14: bool rely_cstr,      // Rely/No Rely
+  15: optional string catName
 }
 
 struct SQLUniqueConstraint {
-  1: string table_db,    // table schema
-  2: string table_name,  // table name
-  3: string column_name, // column name
-  4: i32 key_seq,        // sequence number within unique constraint
-  5: string uk_name,     // unique key name
-  6: bool enable_cstr,   // Enable/Disable
-  7: bool validate_cstr, // Validate/No validate
-  8: bool rely_cstr      // Rely/No Rely
+  1: string catName,     // table catalog
+  2: string table_db,    // table schema
+  3: string table_name,  // table name
+  4: string column_name, // column name
+  5: i32 key_seq,        // sequence number within unique constraint
+  6: string uk_name,     // unique key name
+  7: bool enable_cstr,   // Enable/Disable
+  8: bool validate_cstr, // Validate/No validate
+  9: bool rely_cstr,     // Rely/No Rely
 }
 
 struct SQLNotNullConstraint {
-  1: string table_db,    // table schema
-  2: string table_name,  // table name
-  3: string column_name, // column name
-  4: string nn_name,     // not null name
-  5: bool enable_cstr,   // Enable/Disable
-  6: bool validate_cstr, // Validate/No validate
-  7: bool rely_cstr      // Rely/No Rely
+  1: string catName,     // table catalog
+  2: string table_db,    // table schema
+  3: string table_name,  // table name
+  4: string column_name, // column name
+  5: string nn_name,     // not null name
+  6: bool enable_cstr,   // Enable/Disable
+  7: bool validate_cstr, // Validate/No validate
+  8: bool rely_cstr,     // Rely/No Rely
 }
 
 struct SQLDefaultConstraint {
-  1: string table_db,    // table schema
-  2: string table_name,  // table name
-  3: string column_name, // column name
-  4: string default_value,// default value
-  5: string dc_name,     // default name
-  6: bool enable_cstr,   // Enable/Disable
-  7: bool validate_cstr, // Validate/No validate
-  8: bool rely_cstr      // Rely/No Rely
+  1: string catName,     // catalog name
+  2: string table_db,    // table schema
+  3: string table_name,  // table name
+  4: string column_name, // column name
+  5: string default_value,// default value
+  6: string dc_name,     // default name
+  7: bool enable_cstr,   // Enable/Disable
+  8: bool validate_cstr, // Validate/No validate
+  9: bool rely_cstr      // Rely/No Rely
 }
 
 struct SQLCheckConstraint {
-  1: string table_db,    // table schema
-  2: string table_name,  // table name
-  3: string column_name, // column name
-  4: string check_expression,// check expression
-  5: string dc_name,     // default name
-  6: bool enable_cstr,   // Enable/Disable
-  7: bool validate_cstr, // Validate/No validate
-  8: bool rely_cstr      // Rely/No Rely
+  1: string catName,     // catalog name
+  2: string table_db,    // table schema
+  3: string table_name,  // table name
+  4: string column_name, // column name
+  5: string check_expression,// check expression
+  6: string dc_name,     // default name
+  7: bool enable_cstr,   // Enable/Disable
+  8: bool validate_cstr, // Validate/No validate
+  9: bool rely_cstr      // Rely/No Rely
 }
 
 struct Type {
@@ -233,6 +239,7 @@ struct HiveObjectRef{
   3: string objectName,
   4: list<string> partValues,
   5: string columnName,
+  6: optional string catName
 }
 
 struct PrivilegeGrantInfo {
@@ -318,6 +325,34 @@ struct GrantRevokeRoleResponse {
   1: optional bool success;
 }
 
+struct Catalog {
+  1: string name,                    // Name of the catalog
+  2: optional string description,    // description of the catalog
+  3: string locationUri              // default storage location.  When databases are created in
+                                     // this catalog, if they do not specify a location, they will
+                                     // be placed in this location.
+}
+
+struct CreateCatalogRequest {
+  1: Catalog catalog
+}
+
+struct GetCatalogRequest {
+  1: string name
+}
+
+struct GetCatalogResponse {
+  1: Catalog catalog
+}
+
+struct GetCatalogsResponse {
+  1: list<string> names
+}
+
+struct DropCatalogRequest {
+  1: string name
+}
+
 // namespace for tables
 struct Database {
   1: string name,
@@ -326,7 +361,8 @@ struct Database {
   4: map<string, string> parameters, // properties associated with the database
   5: optional PrincipalPrivilegeSet privileges,
   6: optional string ownerName,
-  7: optional PrincipalType ownerType
+  7: optional PrincipalType ownerType,
+  8: optional string catalogName
 }
 
 // This object holds the information needed by SerDes
@@ -386,7 +422,8 @@ struct Table {
   13: optional PrincipalPrivilegeSet privileges,
   14: optional bool temporary=false,
   15: optional bool rewriteEnabled,     // rewrite enabled or not
-  16: optional CreationMetadata creationMetadata   // only for MVs, it stores table names used and txn list at MV creation
+  16: optional CreationMetadata creationMetadata,   // only for MVs, it stores table names used and txn list at MV creation
+  17: optional string catName          // Name of the catalog the table is in
 }
 
 struct Partition {
@@ -397,7 +434,8 @@ struct Partition {
   5: i32          lastAccessTime,
   6: StorageDescriptor   sd,
   7: map<string, string> parameters,
-  8: optional PrincipalPrivilegeSet privileges
+  8: optional PrincipalPrivilegeSet privileges,
+  9: optional string catName
 }
 
 struct PartitionWithoutSD {
@@ -423,7 +461,8 @@ struct PartitionSpec {
   2: string tableName,
   3: string rootPath,
   4: optional PartitionSpecWithSharedSD sharedSDPartitionSpec,
-  5: optional PartitionListComposingSpec partitionList
+  5: optional PartitionListComposingSpec partitionList,
+  6: optional string catName
 }
 
 // column statistics
@@ -512,7 +551,8 @@ struct ColumnStatisticsDesc {
 2: required string dbName,
 3: required string tableName,
 4: optional string partName,
-5: optional i64 lastAnalyzed
+5: optional i64 lastAnalyzed,
+6: optional string catName
 }
 
 struct ColumnStatistics {
@@ -547,7 +587,8 @@ struct EnvironmentContext {
 
 struct PrimaryKeysRequest {
   1: required string db_name,
-  2: required string tbl_name
+  2: required string tbl_name,
+  3: optional string catName
 }
 
 struct PrimaryKeysResponse {
@@ -559,6 +600,7 @@ struct ForeignKeysRequest {
   2: string parent_tbl_name,
   3: string foreign_db_name,
   4: string foreign_tbl_name
+  5: optional string catName          // No cross catalog constraints
 }
 
 struct ForeignKeysResponse {
@@ -566,8 +608,9 @@ struct ForeignKeysResponse {
 }
 
 struct UniqueConstraintsRequest {
-  1: required string db_name,
-  2: required string tbl_name
+  1: required string catName,
+  2: required string db_name,
+  3: required string tbl_name,
 }
 
 struct UniqueConstraintsResponse {
@@ -575,8 +618,9 @@ struct UniqueConstraintsResponse {
 }
 
 struct NotNullConstraintsRequest {
-  1: required string db_name,
-  2: required string tbl_name
+  1: required string catName,
+  2: required string db_name,
+  3: required string tbl_name,
 }
 
 struct NotNullConstraintsResponse {
@@ -584,8 +628,9 @@ struct NotNullConstraintsResponse {
 }
 
 struct DefaultConstraintsRequest {
-  1: required string db_name,
-  2: required string tbl_name
+  1: required string catName,
+  2: required string db_name,
+  3: required string tbl_name
 }
 
 struct DefaultConstraintsResponse {
@@ -593,8 +638,9 @@ struct DefaultConstraintsResponse {
 }
 
 struct CheckConstraintsRequest {
-  1: required string db_name,
-  2: required string tbl_name
+  1: required string catName,
+  2: required string db_name,
+  3: required string tbl_name
 }
 
 struct CheckConstraintsResponse {
@@ -605,7 +651,8 @@ struct CheckConstraintsResponse {
 struct DropConstraintRequest {
   1: required string dbname, 
   2: required string tablename,
-  3: required string constraintname
+  3: required string constraintname,
+  4: optional string catName
 }
 
 struct AddPrimaryKeyRequest {
@@ -645,6 +692,7 @@ struct PartitionsByExprRequest {
   3: required binary expr,
   4: optional string defaultPartitionName,
   5: optional i16 maxParts=-1
+  6: optional string catName
 }
 
 struct TableStatsResult {
@@ -659,13 +707,15 @@ struct TableStatsRequest {
  1: required string dbName,
  2: required string tblName,
  3: required list<string> colNames
+ 4: optional string catName
 }
 
 struct PartitionsStatsRequest {
  1: required string dbName,
  2: required string tblName,
  3: required list<string> colNames,
- 4: required list<string> partNames
+ 4: required list<string> partNames,
+ 5: optional string catName
 }
 
 // Return type for add_partitions_req
@@ -679,7 +729,8 @@ struct AddPartitionsRequest {
   2: required string tblName,
   3: required list<Partition> parts,
   4: required bool ifNotExists,
-  5: optional bool needResult=true
+  5: optional bool needResult=true,
+  6: optional string catName
 }
 
 // Return type for drop_partitions_req
@@ -707,7 +758,8 @@ struct DropPartitionsRequest {
   5: optional bool ifExists=true, // currently verified on client
   6: optional bool ignoreProtection,
   7: optional EnvironmentContext environmentContext,
-  8: optional bool needResult=true
+  8: optional bool needResult=true,
+  9: optional string catName
 }
 
 struct PartitionValuesRequest {
@@ -719,6 +771,7 @@ struct PartitionValuesRequest {
   6: optional list<FieldSchema> partitionOrder;
   7: optional bool ascending = true;
   8: optional i64 maxParts = -1;
+  9: optional string catName
 }
 
 struct PartitionValuesRow {
@@ -754,6 +807,7 @@ struct Function {
   6: i32              createTime,
   7: FunctionType     functionType,
   8: list<ResourceUri> resourceUris,
+  9: optional string  catName
 }
 
 // Structs for transaction and locks
@@ -977,10 +1031,11 @@ struct BasicTxnInfo {
 }
 
 struct CreationMetadata {
-    1: required string dbName,
-    2: required string tblName,
-    3: required set<string> tablesUsed,
-    4: optional string validTxnList
+    1: required string catName
+    2: required string dbName,
+    3: required string tblName,
+    4: required set<string> tablesUsed,
+    5: optional string validTxnList,
 }
 
 struct NotificationEventRequest {
@@ -996,6 +1051,7 @@ struct NotificationEvent {
     5: optional string tableName,
     6: required string message,
     7: optional string messageFormat,
+    8: optional string catName
 }
 
 struct NotificationEventResponse {
@@ -1009,6 +1065,7 @@ struct CurrentNotificationEventId {
 struct NotificationEventsCountRequest {
     1: required i64 fromEventId,
     2: required string dbName,
+    3: optional string catName
 }
 
 struct NotificationEventsCountResponse {
@@ -1034,6 +1091,7 @@ struct FireEventRequest {
     3: optional string dbName,
     4: optional string tableName,
     5: optional list<string> partitionVals,
+    6: optional string catName,
 }
 
 struct FireEventResponse {
@@ -1125,7 +1183,8 @@ struct ClientCapabilities {
 struct GetTableRequest {
   1: required string dbName,
   2: required string tblName,
-  3: optional ClientCapabilities capabilities
+  3: optional ClientCapabilities capabilities,
+  4: optional string catName
 }
 
 struct GetTableResult {
@@ -1135,7 +1194,8 @@ struct GetTableResult {
 struct GetTablesRequest {
   1: required string dbName,
   2: optional list<string> tblNames,
-  3: optional ClientCapabilities capabilities
+  3: optional ClientCapabilities capabilities,
+  4: optional string catName
 }
 
 struct GetTablesResult {
@@ -1157,6 +1217,7 @@ struct TableMeta {
   2: required string tableName;
   3: required string tableType;
   4: optional string comments;
+  5: optional string catName;
 }
 
 struct Materialization {
@@ -1384,17 +1445,19 @@ struct WMCreateOrDropTriggerToPoolMappingResponse {
 struct ISchema {
   1: SchemaType schemaType,
   2: string name,
-  3: string dbName,
-  4: SchemaCompatibility compatibility,
-  5: SchemaValidation validationLevel,
-  6: bool canEvolve,
-  7: optional string schemaGroup,
-  8: optional string description
+  3: string catName,
+  4: string dbName,
+  5: SchemaCompatibility compatibility,
+  6: SchemaValidation validationLevel,
+  7: bool canEvolve,
+  8: optional string schemaGroup,
+  9: optional string description
 }
 
 struct ISchemaName {
-  1: string dbName,
-  2: string schemaName
+  1: string catName,
+  2: string dbName,
+  3: string schemaName
 }
 
 struct AlterISchemaRequest {
@@ -1515,6 +1578,11 @@ service ThriftHiveMetastore extends fb303.FacebookService
   string getMetaConf(1:string key) throws(1:MetaException o1)
   void setMetaConf(1:string key, 2:string value) throws(1:MetaException o1)
 
+  void create_catalog(1: CreateCatalogRequest catalog) throws (1:AlreadyExistsException o1, 2:InvalidObjectException o2, 3: MetaException o3)
+  GetCatalogResponse get_catalog(1: GetCatalogRequest catName) throws (1:NoSuchObjectException o1, 2:MetaException o2)
+  GetCatalogsResponse get_catalogs() throws (1:MetaException o1)
+  void drop_catalog(1: DropCatalogRequest catName) throws (1:NoSuchObjectException o1, 2:InvalidOperationException o2, 3:MetaException o3)
+
   void create_database(1:Database database) throws(1:AlreadyExistsException o1, 2:InvalidObjectException o2, 3:MetaException o3)
   Database get_database(1:string name) throws(1:NoSuchObjectException o1, 2:MetaException o2)
   void drop_database(1:string name, 2:bool deleteData, 3:bool cascade) throws(1:NoSuchObjectException o1, 2:InvalidOperationException o2, 3:MetaException o3)
@@ -1597,7 +1665,7 @@ service ThriftHiveMetastore extends fb303.FacebookService
 				   throws (1:MetaException o1, 2:InvalidOperationException o2, 3:UnknownDBException o3)
   map<string, Materialization> get_materialization_invalidation_info(1:string dbname, 2:list<string> tbl_names)
 				   throws (1:MetaException o1, 2:InvalidOperationException o2, 3:UnknownDBException o3)
-  void update_creation_metadata(1:string dbname, 2:string tbl_name, 3:CreationMetadata creation_metadata)
+  void update_creation_metadata(1: string catName, 2:string dbname, 3:string tbl_name, 4:CreationMetadata creation_metadata)
                    throws (1:MetaException o1, 2:InvalidOperationException o2, 3:UnknownDBException o3)
 
   // Get a list of table names that match a filter.

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
index acad676..304f567 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.metastore;
 import org.apache.hadoop.hive.metastore.api.CreationMetadata;
 import org.apache.hadoop.hive.metastore.api.ISchemaName;
 import org.apache.hadoop.hive.metastore.api.SchemaVersionDescriptor;
+import org.apache.hadoop.hive.metastore.api.Catalog;
 import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
 
 import java.nio.ByteBuffer;
@@ -152,36 +153,62 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
   }
 
   @Override
+  public void createCatalog(Catalog cat) throws MetaException {
+    objectStore.createCatalog(cat);
+  }
+
+  @Override
+  public void alterCatalog(String catName, Catalog cat) throws MetaException,
+      InvalidOperationException {
+    objectStore.alterCatalog(catName, cat);
+  }
+
+  @Override
+  public Catalog getCatalog(String catalogName) throws NoSuchObjectException, MetaException {
+    return objectStore.getCatalog(catalogName);
+  }
+
+  @Override
+  public List<String> getCatalogs() throws MetaException {
+    return objectStore.getCatalogs();
+  }
+
+  @Override
+  public void dropCatalog(String catalogName) throws NoSuchObjectException, MetaException {
+    objectStore.dropCatalog(catalogName);
+  }
+
+  @Override
   public void createDatabase(Database db) throws InvalidObjectException, MetaException {
     objectStore.createDatabase(db);
   }
 
   @Override
-  public Database getDatabase(String dbName) throws NoSuchObjectException {
-    return objectStore.getDatabase(dbName);
+  public Database getDatabase(String catName, String dbName) throws NoSuchObjectException {
+    return objectStore.getDatabase(catName, dbName);
   }
 
   @Override
-  public boolean dropDatabase(String dbName)
+  public boolean dropDatabase(String catName, String dbName)
       throws NoSuchObjectException, MetaException {
-    return objectStore.dropDatabase(dbName);
+    return objectStore.dropDatabase(catName, dbName);
   }
 
   @Override
-  public boolean alterDatabase(String dbName, Database db)
+  public boolean alterDatabase(String catName, String dbName, Database db)
       throws NoSuchObjectException, MetaException {
 
-    return objectStore.alterDatabase(dbName, db);
+    return objectStore.alterDatabase(catName, dbName, db);
   }
 
   @Override
-  public List<String> getDatabases(String pattern) throws MetaException {
-    return objectStore.getDatabases(pattern);
+  public List<String> getDatabases(String catName, String pattern) throws MetaException {
+    return objectStore.getDatabases(catName, pattern);
   }
 
   @Override
-  public List<String> getAllDatabases() throws MetaException {
-    return objectStore.getAllDatabases();
+  public List<String> getAllDatabases(String catName) throws MetaException {
+    return objectStore.getAllDatabases(catName);
   }
 
   @Override
@@ -205,15 +232,15 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
   }
 
   @Override
-  public boolean dropTable(String dbName, String tableName)
+  public boolean dropTable(String catName, String dbName, String tableName)
       throws MetaException, NoSuchObjectException,
       InvalidObjectException, InvalidInputException {
-    return objectStore.dropTable(dbName, tableName);
+    return objectStore.dropTable(catName, dbName, tableName);
   }
 
   @Override
-  public Table getTable(String dbName, String tableName) throws MetaException {
-    return objectStore.getTable(dbName, tableName);
+  public Table getTable(String catName, String dbName, String tableName) throws MetaException {
+    return objectStore.getTable(catName, dbName, tableName);
   }
 
   @Override
@@ -223,150 +250,145 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
   }
 
   @Override
-  public Partition getPartition(String dbName, String tableName, List<String> partVals)
+  public Partition getPartition(String catName, String dbName, String tableName, List<String> partVals)
       throws MetaException, NoSuchObjectException {
-    return objectStore.getPartition(dbName, tableName, partVals);
+    return objectStore.getPartition(catName, dbName, tableName, partVals);
   }
 
   @Override
-  public boolean dropPartition(String dbName, String tableName, List<String> partVals)
+  public boolean dropPartition(String catName, String dbName, String tableName, List<String> partVals)
       throws MetaException, NoSuchObjectException,
       InvalidObjectException, InvalidInputException {
-    return objectStore.dropPartition(dbName, tableName, partVals);
+    return objectStore.dropPartition(catName, dbName, tableName, partVals);
   }
 
   @Override
-  public List<Partition> getPartitions(String dbName, String tableName, int max)
+  public List<Partition> getPartitions(String catName, String dbName, String tableName, int max)
       throws MetaException, NoSuchObjectException {
-    return objectStore.getPartitions(dbName, tableName, max);
+    return objectStore.getPartitions(catName, dbName, tableName, max);
   }
 
   @Override
-  public void alterTable(String dbName, String name, Table newTable)
+  public void alterTable(String catName, String dbName, String name, Table newTable)
       throws InvalidObjectException, MetaException {
-    objectStore.alterTable(dbName, name, newTable);
+    objectStore.alterTable(catName, dbName, name, newTable);
   }
 
   @Override
-  public void updateCreationMetadata(String dbname, String tablename, CreationMetadata cm)
+  public void updateCreationMetadata(String catName, String dbname, String tablename, CreationMetadata cm)
       throws MetaException {
-    objectStore.updateCreationMetadata(dbname, tablename, cm);
+    objectStore.updateCreationMetadata(catName, dbname, tablename, cm);
   }
 
-  @Override
-  public List<String> getTables(String dbName, String pattern) throws MetaException {
-    return objectStore.getTables(dbName, pattern);
+  public List<String> getTables(String catName, String dbName, String pattern) throws MetaException {
+    return objectStore.getTables(catName, dbName, pattern);
   }
 
   @Override
-  public List<String> getTables(String dbName, String pattern, TableType tableType) throws MetaException {
-    return objectStore.getTables(dbName, pattern, tableType);
+  public List<String> getTables(String catName, String dbName, String pattern, TableType tableType) throws MetaException {
+    return objectStore.getTables(catName, dbName, pattern, tableType);
   }
 
   @Override
-  public List<String> getMaterializedViewsForRewriting(String dbName)
+  public List<String> getMaterializedViewsForRewriting(String catName, String dbName)
       throws MetaException, NoSuchObjectException {
-    return objectStore.getMaterializedViewsForRewriting(dbName);
+    return objectStore.getMaterializedViewsForRewriting(catName, dbName);
   }
 
   @Override
-  public List<TableMeta> getTableMeta(String dbNames, String tableNames, List<String> tableTypes)
+  public List<TableMeta> getTableMeta(String catName, String dbNames, String tableNames, List<String> tableTypes)
       throws MetaException {
-    return objectStore.getTableMeta(dbNames, tableNames, tableTypes);
+    return objectStore.getTableMeta(catName, dbNames, tableNames, tableTypes);
   }
 
   @Override
-  public List<Table> getTableObjectsByName(String dbName, List<String> tableNames)
+  public List<Table> getTableObjectsByName(String catName, String dbName, List<String> tableNames)
       throws MetaException, UnknownDBException {
-    return objectStore.getTableObjectsByName(dbName, tableNames);
+    return objectStore.getTableObjectsByName(catName, dbName, tableNames);
   }
 
   @Override
-  public List<String> getAllTables(String dbName) throws MetaException {
-    return objectStore.getAllTables(dbName);
+  public List<String> getAllTables(String catName, String dbName) throws MetaException {
+    return objectStore.getAllTables(catName, dbName);
   }
 
   @Override
-  public List<String> listTableNamesByFilter(String dbName, String filter,
+  public List<String> listTableNamesByFilter(String catName, String dbName, String filter,
       short maxTables) throws MetaException, UnknownDBException {
-    return objectStore.listTableNamesByFilter(dbName, filter, maxTables);
+    return objectStore.listTableNamesByFilter(catName, dbName, filter, maxTables);
   }
 
   @Override
-  public List<String> listPartitionNames(String dbName, String tblName, short maxParts)
+  public List<String> listPartitionNames(String catName, String dbName, String tblName, short maxParts)
       throws MetaException {
-    return objectStore.listPartitionNames(dbName, tblName, maxParts);
+    return objectStore.listPartitionNames(catName, dbName, tblName, maxParts);
   }
 
   @Override
-  public PartitionValuesResponse listPartitionValues(String db_name, String tbl_name, List<FieldSchema> cols, boolean applyDistinct, String filter, boolean ascending, List<FieldSchema> order, long maxParts) throws MetaException {
+  public PartitionValuesResponse listPartitionValues(String catName, String db_name,
+      String tbl_name, List<FieldSchema> cols, boolean applyDistinct, String filter,
+      boolean ascending, List<FieldSchema> order, long maxParts) throws MetaException {
     return null;
   }
 
   @Override
-  public List<String> listPartitionNamesByFilter(String dbName, String tblName,
-      String filter, short maxParts) throws MetaException {
-    return objectStore.listPartitionNamesByFilter(dbName, tblName, filter, maxParts);
-  }
-
-  @Override
-  public void alterPartition(String dbName, String tblName, List<String> partVals,
+  public void alterPartition(String catName, String dbName, String tblName, List<String> partVals,
       Partition newPart) throws InvalidObjectException, MetaException {
-    objectStore.alterPartition(dbName, tblName, partVals, newPart);
+    objectStore.alterPartition(catName, dbName, tblName, partVals, newPart);
   }
 
   @Override
-  public void alterPartitions(String dbName, String tblName,
+  public void alterPartitions(String catName, String dbName, String tblName,
       List<List<String>> partValsList, List<Partition> newParts)
       throws InvalidObjectException, MetaException {
-    objectStore.alterPartitions(dbName, tblName, partValsList, newParts);
+    objectStore.alterPartitions(catName, dbName, tblName, partValsList, newParts);
   }
 
   @Override
-  public List<Partition> getPartitionsByFilter(String dbName, String tblName,
+  public List<Partition> getPartitionsByFilter(String catName, String dbName, String tblName,
       String filter, short maxParts) throws MetaException, NoSuchObjectException {
-    return objectStore.getPartitionsByFilter(dbName, tblName, filter, maxParts);
+    return objectStore.getPartitionsByFilter(catName, dbName, tblName, filter, maxParts);
   }
 
   @Override
-  public int getNumPartitionsByFilter(String dbName, String tblName,
+  public int getNumPartitionsByFilter(String catName, String dbName, String tblName,
                                       String filter) throws MetaException, NoSuchObjectException {
-    return objectStore.getNumPartitionsByFilter(dbName, tblName, filter);
+    return objectStore.getNumPartitionsByFilter(catName, dbName, tblName, filter);
   }
 
   @Override
-  public int getNumPartitionsByExpr(String dbName, String tblName,
+  public int getNumPartitionsByExpr(String catName, String dbName, String tblName,
                                       byte[] expr) throws MetaException, NoSuchObjectException {
-    return objectStore.getNumPartitionsByExpr(dbName, tblName, expr);
+    return objectStore.getNumPartitionsByExpr(catName, dbName, tblName, expr);
   }
 
   @Override
-  public List<Partition> getPartitionsByNames(String dbName, String tblName,
+  public List<Partition> getPartitionsByNames(String catName, String dbName, String tblName,
       List<String> partNames) throws MetaException, NoSuchObjectException {
-    return objectStore.getPartitionsByNames(dbName, tblName, partNames);
+    return objectStore.getPartitionsByNames(catName, dbName, tblName, partNames);
   }
 
   @Override
-  public boolean getPartitionsByExpr(String dbName, String tblName, byte[] expr,
+  public boolean getPartitionsByExpr(String catName, String dbName, String tblName, byte[] expr,
       String defaultPartitionName, short maxParts, List<Partition> result) throws TException {
-    return objectStore.getPartitionsByExpr(
+    return objectStore.getPartitionsByExpr(catName,
         dbName, tblName, expr, defaultPartitionName, maxParts, result);
   }
 
   @Override
-  public Table markPartitionForEvent(String dbName, String tblName,
+  public Table markPartitionForEvent(String catName, String dbName, String tblName,
       Map<String, String> partVals, PartitionEventType evtType)
       throws MetaException, UnknownTableException, InvalidPartitionException,
       UnknownPartitionException {
-    return objectStore.markPartitionForEvent(dbName, tblName, partVals, evtType);
+    return objectStore.markPartitionForEvent(catName, dbName, tblName, partVals, evtType);
   }
 
   @Override
-  public boolean isPartitionMarkedForEvent(String dbName, String tblName,
+  public boolean isPartitionMarkedForEvent(String catName, String dbName, String tblName,
       Map<String, String> partName, PartitionEventType evtType)
       throws MetaException, UnknownTableException, InvalidPartitionException,
       UnknownPartitionException {
-    return objectStore.isPartitionMarkedForEvent(dbName, tblName, partName, evtType);
+    return objectStore.isPartitionMarkedForEvent(catName, dbName, tblName, partName, evtType);
   }
 
   @Override
@@ -402,31 +424,31 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
   }
 
   @Override
-  public PrincipalPrivilegeSet getDBPrivilegeSet(String dbName, String userName,
+  public PrincipalPrivilegeSet getDBPrivilegeSet(String catName, String dbName, String userName,
       List<String> groupNames) throws InvalidObjectException, MetaException {
-    return objectStore.getDBPrivilegeSet(dbName, userName, groupNames);
+    return objectStore.getDBPrivilegeSet(catName, dbName, userName, groupNames);
   }
 
   @Override
-  public PrincipalPrivilegeSet getTablePrivilegeSet(String dbName, String tableName,
+  public PrincipalPrivilegeSet getTablePrivilegeSet(String catName, String dbName, String tableName,
       String userName, List<String> groupNames)
       throws InvalidObjectException, MetaException {
-    return objectStore.getTablePrivilegeSet(dbName, tableName, userName, groupNames);
+    return objectStore.getTablePrivilegeSet(catName, dbName, tableName, userName, groupNames);
   }
 
   @Override
-  public PrincipalPrivilegeSet getPartitionPrivilegeSet(String dbName, String tableName,
+  public PrincipalPrivilegeSet getPartitionPrivilegeSet(String catName, String dbName, String tableName,
       String partition, String userName, List<String> groupNames)
       throws InvalidObjectException, MetaException {
-    return objectStore.getPartitionPrivilegeSet(dbName, tableName, partition,
+    return objectStore.getPartitionPrivilegeSet(catName, dbName, tableName, partition,
         userName, groupNames);
   }
 
   @Override
-  public PrincipalPrivilegeSet getColumnPrivilegeSet(String dbName, String tableName,
+  public PrincipalPrivilegeSet getColumnPrivilegeSet(String catName, String dbName, String tableName,
       String partitionName, String columnName, String userName, List<String> groupNames)
       throws InvalidObjectException, MetaException {
-    return objectStore.getColumnPrivilegeSet(dbName, tableName, partitionName,
+    return objectStore.getColumnPrivilegeSet(catName, dbName, tableName, partitionName,
         columnName, userName, groupNames);
   }
 
@@ -438,38 +460,38 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
 
   @Override
   public List<HiveObjectPrivilege> listPrincipalDBGrants(String principalName,
-      PrincipalType principalType, String dbName) {
-    return objectStore.listPrincipalDBGrants(principalName, principalType, dbName);
+      PrincipalType principalType, String catName, String dbName) {
+    return objectStore.listPrincipalDBGrants(principalName, principalType, catName, dbName);
   }
 
   @Override
   public List<HiveObjectPrivilege> listAllTableGrants(String principalName,
-      PrincipalType principalType, String dbName, String tableName) {
+      PrincipalType principalType, String catName, String dbName, String tableName) {
     return objectStore.listAllTableGrants(principalName, principalType,
-        dbName, tableName);
+        catName, dbName, tableName);
   }
 
   @Override
   public List<HiveObjectPrivilege> listPrincipalPartitionGrants(String principalName,
-      PrincipalType principalType, String dbName, String tableName, List<String> partValues,
+      PrincipalType principalType, String catName, String dbName, String tableName, List<String> partValues,
       String partName) {
     return objectStore.listPrincipalPartitionGrants(principalName, principalType,
-        dbName, tableName, partValues, partName);
+        catName, dbName, tableName, partValues, partName);
   }
 
   @Override
   public List<HiveObjectPrivilege> listPrincipalTableColumnGrants(String principalName,
-      PrincipalType principalType, String dbName, String tableName, String columnName) {
+      PrincipalType principalType, String catName, String dbName, String tableName, String columnName) {
     return objectStore.listPrincipalTableColumnGrants(principalName, principalType,
-        dbName, tableName, columnName);
+        catName, dbName, tableName, columnName);
   }
 
   @Override
   public List<HiveObjectPrivilege> listPrincipalPartitionColumnGrants(
-      String principalName, PrincipalType principalType, String dbName, String tableName,
+      String principalName, PrincipalType principalType, String catName, String dbName, String tableName,
       List<String> partVals, String partName, String columnName) {
     return objectStore.listPrincipalPartitionColumnGrants(principalName, principalType,
-        dbName, tableName, partVals, partName, columnName);
+        catName, dbName, tableName, partVals, partName, columnName);
   }
 
   @Override
@@ -511,33 +533,33 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
   }
 
   @Override
-  public Partition getPartitionWithAuth(String dbName, String tblName,
+  public Partition getPartitionWithAuth(String catName, String dbName, String tblName,
       List<String> partVals, String userName, List<String> groupNames)
       throws MetaException, NoSuchObjectException, InvalidObjectException {
-    return objectStore.getPartitionWithAuth(dbName, tblName, partVals, userName,
+    return objectStore.getPartitionWithAuth(catName, dbName, tblName, partVals, userName,
         groupNames);
   }
 
   @Override
-  public List<Partition> getPartitionsWithAuth(String dbName, String tblName,
+  public List<Partition> getPartitionsWithAuth(String catName, String dbName, String tblName,
       short maxParts, String userName, List<String> groupNames)
       throws MetaException, NoSuchObjectException, InvalidObjectException {
-    return objectStore.getPartitionsWithAuth(dbName, tblName, maxParts, userName,
+    return objectStore.getPartitionsWithAuth(catName, dbName, tblName, maxParts, userName,
         groupNames);
   }
 
   @Override
-  public List<String> listPartitionNamesPs(String dbName, String tblName,
+  public List<String> listPartitionNamesPs(String catName, String dbName, String tblName,
       List<String> partVals, short maxParts)
       throws MetaException, NoSuchObjectException {
-    return objectStore.listPartitionNamesPs(dbName, tblName, partVals, maxParts);
+    return objectStore.listPartitionNamesPs(catName, dbName, tblName, partVals, maxParts);
   }
 
   @Override
-  public List<Partition> listPartitionsPsWithAuth(String dbName, String tblName,
+  public List<Partition> listPartitionsPsWithAuth(String catName, String dbName, String tblName,
       List<String> partVals, short maxParts, String userName, List<String> groupNames)
       throws MetaException, InvalidObjectException, NoSuchObjectException {
-    return objectStore.listPartitionsPsWithAuth(dbName, tblName, partVals, maxParts,
+    return objectStore.listPartitionsPsWithAuth(catName, dbName, tblName, partVals, maxParts,
         userName, groupNames);
   }
 
@@ -582,53 +604,52 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
   }
 
   @Override
-  public List<HiveObjectPrivilege> listDBGrantsAll(String dbName) {
-    return objectStore.listDBGrantsAll(dbName);
+  public List<HiveObjectPrivilege> listDBGrantsAll(String catName, String dbName) {
+    return objectStore.listDBGrantsAll(catName, dbName);
   }
 
   @Override
-  public List<HiveObjectPrivilege> listPartitionColumnGrantsAll(String dbName, String tableName,
+  public List<HiveObjectPrivilege> listPartitionColumnGrantsAll(String catName, String dbName, String tableName,
       String partitionName, String columnName) {
-    return objectStore.listPartitionColumnGrantsAll(dbName, tableName, partitionName, columnName);
+    return objectStore.listPartitionColumnGrantsAll(catName, dbName, tableName, partitionName, columnName);
   }
 
   @Override
-  public List<HiveObjectPrivilege> listTableGrantsAll(String dbName, String tableName) {
-    return objectStore.listTableGrantsAll(dbName, tableName);
+  public List<HiveObjectPrivilege> listTableGrantsAll(String catName, String dbName, String tableName) {
+    return objectStore.listTableGrantsAll(catName, dbName, tableName);
   }
 
   @Override
-  public List<HiveObjectPrivilege> listPartitionGrantsAll(String dbName, String tableName,
+  public List<HiveObjectPrivilege> listPartitionGrantsAll(String catName, String dbName, String tableName,
       String partitionName) {
-    return objectStore.listPartitionGrantsAll(dbName, tableName, partitionName);
+    return objectStore.listPartitionGrantsAll(catName, dbName, tableName, partitionName);
   }
 
   @Override
-  public List<HiveObjectPrivilege> listTableColumnGrantsAll(String dbName, String tableName,
+  public List<HiveObjectPrivilege> listTableColumnGrantsAll(String catName, String dbName, String tableName,
       String columnName) {
-    return objectStore.listTableColumnGrantsAll(dbName, tableName, columnName);
+    return objectStore.listTableColumnGrantsAll(catName, dbName, tableName, columnName);
   }
 
   @Override
-  public ColumnStatistics getTableColumnStatistics(String dbName, String tableName,
+  public ColumnStatistics getTableColumnStatistics(String catName, String dbName, String tableName,
       List<String> colNames) throws MetaException, NoSuchObjectException {
-    return objectStore.getTableColumnStatistics(dbName, tableName, colNames);
+    return objectStore.getTableColumnStatistics(catName, dbName, tableName, colNames);
   }
 
   @Override
-  public boolean deleteTableColumnStatistics(String dbName, String tableName,
-      String colName)
-      throws NoSuchObjectException, MetaException, InvalidObjectException,
-      InvalidInputException {
-    return objectStore.deleteTableColumnStatistics(dbName, tableName, colName);
+  public boolean deleteTableColumnStatistics(String catName, String dbName, String tableName,
+                                             String colName)
+      throws NoSuchObjectException, MetaException, InvalidObjectException, InvalidInputException {
+    return objectStore.deleteTableColumnStatistics(catName, dbName, tableName, colName);
   }
 
   @Override
-  public boolean deletePartitionColumnStatistics(String dbName, String tableName,
+  public boolean deletePartitionColumnStatistics(String catName, String dbName, String tableName,
       String partName, List<String> partVals, String colName)
       throws NoSuchObjectException, MetaException, InvalidObjectException,
       InvalidInputException {
-    return objectStore.deletePartitionColumnStatistics(dbName, tableName, partName,
+    return objectStore.deletePartitionColumnStatistics(catName, dbName, tableName, partName,
         partVals, colName);
   }
 
@@ -702,33 +723,33 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
   }
 
   @Override
-  public List<ColumnStatistics> getPartitionColumnStatistics(String dbName,
+  public List<ColumnStatistics> getPartitionColumnStatistics(String catName, String dbName,
       String tblName, List<String> colNames, List<String> partNames)
       throws MetaException, NoSuchObjectException {
-    return objectStore.getPartitionColumnStatistics(dbName, tblName  , colNames, partNames);
+    return objectStore.getPartitionColumnStatistics(catName, dbName, tblName  , colNames, partNames);
   }
 
   @Override
-  public boolean doesPartitionExist(String dbName, String tableName,
+  public boolean doesPartitionExist(String catName, String dbName, String tableName,
       List<String> partVals) throws MetaException, NoSuchObjectException {
-    return objectStore.doesPartitionExist(dbName, tableName, partVals);
+    return objectStore.doesPartitionExist(catName, dbName, tableName, partVals);
   }
 
   @Override
-  public boolean addPartitions(String dbName, String tblName, List<Partition> parts)
+  public boolean addPartitions(String catName, String dbName, String tblName, List<Partition> parts)
       throws InvalidObjectException, MetaException {
-    return objectStore.addPartitions(dbName, tblName, parts);
+    return objectStore.addPartitions(catName, dbName, tblName, parts);
   }
 
   @Override
-  public boolean addPartitions(String dbName, String tblName, PartitionSpecProxy partitionSpec, boolean ifNotExists) throws InvalidObjectException, MetaException {
+  public boolean addPartitions(String catName, String dbName, String tblName, PartitionSpecProxy partitionSpec, boolean ifNotExists) throws InvalidObjectException, MetaException {
     return false;
   }
 
   @Override
-  public void dropPartitions(String dbName, String tblName, List<String> partNames)
+  public void dropPartitions(String catName, String dbName, String tblName, List<String> partNames)
       throws MetaException, NoSuchObjectException {
-    objectStore.dropPartitions(dbName, tblName, partNames);
+    objectStore.dropPartitions(catName, dbName, tblName, partNames);
   }
 
   @Override
@@ -738,38 +759,38 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
   }
 
   @Override
-  public void alterFunction(String dbName, String funcName, Function newFunction)
+  public void alterFunction(String catName, String dbName, String funcName, Function newFunction)
       throws InvalidObjectException, MetaException {
-    objectStore.alterFunction(dbName, funcName, newFunction);
+    objectStore.alterFunction(catName, dbName, funcName, newFunction);
   }
 
   @Override
-  public void dropFunction(String dbName, String funcName)
+  public void dropFunction(String catName, String dbName, String funcName)
       throws MetaException, NoSuchObjectException, InvalidObjectException,
       InvalidInputException {
-    objectStore.dropFunction(dbName, funcName);
+    objectStore.dropFunction(catName, dbName, funcName);
   }
 
   @Override
-  public Function getFunction(String dbName, String funcName)
+  public Function getFunction(String catName, String dbName, String funcName)
       throws MetaException {
-    return objectStore.getFunction(dbName, funcName);
+    return objectStore.getFunction(catName, dbName, funcName);
   }
 
   @Override
-  public List<Function> getAllFunctions()
+  public List<Function> getAllFunctions(String catName)
           throws MetaException {
     return Collections.emptyList();
   }
 
   @Override
-  public List<String> getFunctions(String dbName, String pattern)
+  public List<String> getFunctions(String catName, String dbName, String pattern)
       throws MetaException {
-    return objectStore.getFunctions(dbName, pattern);
+    return objectStore.getFunctions(catName, dbName, pattern);
   }
 
   @Override
-  public AggrStats get_aggr_stats_for(String dbName,
+  public AggrStats get_aggr_stats_for(String catName, String dbName,
       String tblName, List<String> partNames, List<String> colNames)
       throws MetaException {
     return null;
@@ -847,14 +868,14 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
   }
 
   @Override
-  public List<SQLPrimaryKey> getPrimaryKeys(String db_name, String tbl_name)
+  public List<SQLPrimaryKey> getPrimaryKeys(String catName, String db_name, String tbl_name)
     throws MetaException {
     // TODO Auto-generated method stub
     return null;
   }
 
   @Override
-  public List<SQLForeignKey> getForeignKeys(String parent_db_name,
+  public List<SQLForeignKey> getForeignKeys(String catName, String parent_db_name,
     String parent_tbl_name, String foreign_db_name, String foreign_tbl_name)
     throws MetaException {
     // TODO Auto-generated method stub
@@ -862,28 +883,28 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
   }
 
   @Override
-  public List<SQLUniqueConstraint> getUniqueConstraints(String db_name, String tbl_name)
+  public List<SQLUniqueConstraint> getUniqueConstraints(String catName, String db_name, String tbl_name)
     throws MetaException {
     // TODO Auto-generated method stub
     return null;
   }
 
   @Override
-  public List<SQLNotNullConstraint> getNotNullConstraints(String db_name, String tbl_name)
+  public List<SQLNotNullConstraint> getNotNullConstraints(String catName, String db_name, String tbl_name)
     throws MetaException {
     // TODO Auto-generated method stub
     return null;
   }
 
   @Override
-  public List<SQLDefaultConstraint> getDefaultConstraints(String db_name, String tbl_name)
+  public List<SQLDefaultConstraint> getDefaultConstraints(String catName, String db_name, String tbl_name)
       throws MetaException {
     // TODO Auto-generated method stub
     return null;
   }
 
   @Override
-  public List<SQLCheckConstraint> getCheckConstraints(String db_name, String tbl_name)
+  public List<SQLCheckConstraint> getCheckConstraints(String catName, String db_name, String tbl_name)
       throws MetaException {
     // TODO Auto-generated method stub
     return null;
@@ -902,8 +923,8 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
   }
 
   @Override
-  public void dropConstraint(String dbName, String tableName,
-   String constraintName) throws NoSuchObjectException {
+  public void dropConstraint(String catName, String dbName, String tableName,
+   String constraintName, boolean missingOk) throws NoSuchObjectException {
    // TODO Auto-generated method stub
   }
 
@@ -959,7 +980,8 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
   }
 
   @Override
-  public WMFullResourcePlan getResourcePlan(String name) throws NoSuchObjectException {
+  public WMFullResourcePlan getResourcePlan(String name) throws NoSuchObjectException,
+      MetaException {
     return objectStore.getResourcePlan(name);
   }
 
@@ -1062,6 +1084,14 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
     objectStore.dropWMTriggerToPoolMapping(resourcePlanName, triggerName, poolPath);
   }
 
+
+  @Override
+  public List<ColStatsObjWithSourceInfo> getPartitionColStatsForDatabase(String catName, String dbName)
+      throws MetaException, NoSuchObjectException {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
   public void createISchema(ISchema schema) throws AlreadyExistsException, MetaException,
       NoSuchObjectException {
     objectStore.createISchema(schema);


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
index 724f022..d4ee95b 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
@@ -287,6 +287,18 @@ class GrantRevokeRoleRequest;
 
 class GrantRevokeRoleResponse;
 
+class Catalog;
+
+class CreateCatalogRequest;
+
+class GetCatalogRequest;
+
+class GetCatalogResponse;
+
+class GetCatalogsResponse;
+
+class DropCatalogRequest;
+
 class Database;
 
 class SerDeInfo;
@@ -788,7 +800,7 @@ inline std::ostream& operator<<(std::ostream& out, const FieldSchema& obj)
 }
 
 typedef struct _SQLPrimaryKey__isset {
-  _SQLPrimaryKey__isset() : table_db(false), table_name(false), column_name(false), key_seq(false), pk_name(false), enable_cstr(false), validate_cstr(false), rely_cstr(false) {}
+  _SQLPrimaryKey__isset() : table_db(false), table_name(false), column_name(false), key_seq(false), pk_name(false), enable_cstr(false), validate_cstr(false), rely_cstr(false), catName(false) {}
   bool table_db :1;
   bool table_name :1;
   bool column_name :1;
@@ -797,6 +809,7 @@ typedef struct _SQLPrimaryKey__isset {
   bool enable_cstr :1;
   bool validate_cstr :1;
   bool rely_cstr :1;
+  bool catName :1;
 } _SQLPrimaryKey__isset;
 
 class SQLPrimaryKey {
@@ -804,7 +817,7 @@ class SQLPrimaryKey {
 
   SQLPrimaryKey(const SQLPrimaryKey&);
   SQLPrimaryKey& operator=(const SQLPrimaryKey&);
-  SQLPrimaryKey() : table_db(), table_name(), column_name(), key_seq(0), pk_name(), enable_cstr(0), validate_cstr(0), rely_cstr(0) {
+  SQLPrimaryKey() : table_db(), table_name(), column_name(), key_seq(0), pk_name(), enable_cstr(0), validate_cstr(0), rely_cstr(0), catName() {
   }
 
   virtual ~SQLPrimaryKey() throw();
@@ -816,6 +829,7 @@ class SQLPrimaryKey {
   bool enable_cstr;
   bool validate_cstr;
   bool rely_cstr;
+  std::string catName;
 
   _SQLPrimaryKey__isset __isset;
 
@@ -835,6 +849,8 @@ class SQLPrimaryKey {
 
   void __set_rely_cstr(const bool val);
 
+  void __set_catName(const std::string& val);
+
   bool operator == (const SQLPrimaryKey & rhs) const
   {
     if (!(table_db == rhs.table_db))
@@ -853,6 +869,10 @@ class SQLPrimaryKey {
       return false;
     if (!(rely_cstr == rhs.rely_cstr))
       return false;
+    if (__isset.catName != rhs.__isset.catName)
+      return false;
+    else if (__isset.catName && !(catName == rhs.catName))
+      return false;
     return true;
   }
   bool operator != (const SQLPrimaryKey &rhs) const {
@@ -876,7 +896,7 @@ inline std::ostream& operator<<(std::ostream& out, const SQLPrimaryKey& obj)
 }
 
 typedef struct _SQLForeignKey__isset {
-  _SQLForeignKey__isset() : pktable_db(false), pktable_name(false), pkcolumn_name(false), fktable_db(false), fktable_name(false), fkcolumn_name(false), key_seq(false), update_rule(false), delete_rule(false), fk_name(false), pk_name(false), enable_cstr(false), validate_cstr(false), rely_cstr(false) {}
+  _SQLForeignKey__isset() : pktable_db(false), pktable_name(false), pkcolumn_name(false), fktable_db(false), fktable_name(false), fkcolumn_name(false), key_seq(false), update_rule(false), delete_rule(false), fk_name(false), pk_name(false), enable_cstr(false), validate_cstr(false), rely_cstr(false), catName(false) {}
   bool pktable_db :1;
   bool pktable_name :1;
   bool pkcolumn_name :1;
@@ -891,6 +911,7 @@ typedef struct _SQLForeignKey__isset {
   bool enable_cstr :1;
   bool validate_cstr :1;
   bool rely_cstr :1;
+  bool catName :1;
 } _SQLForeignKey__isset;
 
 class SQLForeignKey {
@@ -898,7 +919,7 @@ class SQLForeignKey {
 
   SQLForeignKey(const SQLForeignKey&);
   SQLForeignKey& operator=(const SQLForeignKey&);
-  SQLForeignKey() : pktable_db(), pktable_name(), pkcolumn_name(), fktable_db(), fktable_name(), fkcolumn_name(), key_seq(0), update_rule(0), delete_rule(0), fk_name(), pk_name(), enable_cstr(0), validate_cstr(0), rely_cstr(0) {
+  SQLForeignKey() : pktable_db(), pktable_name(), pkcolumn_name(), fktable_db(), fktable_name(), fkcolumn_name(), key_seq(0), update_rule(0), delete_rule(0), fk_name(), pk_name(), enable_cstr(0), validate_cstr(0), rely_cstr(0), catName() {
   }
 
   virtual ~SQLForeignKey() throw();
@@ -916,6 +937,7 @@ class SQLForeignKey {
   bool enable_cstr;
   bool validate_cstr;
   bool rely_cstr;
+  std::string catName;
 
   _SQLForeignKey__isset __isset;
 
@@ -947,6 +969,8 @@ class SQLForeignKey {
 
   void __set_rely_cstr(const bool val);
 
+  void __set_catName(const std::string& val);
+
   bool operator == (const SQLForeignKey & rhs) const
   {
     if (!(pktable_db == rhs.pktable_db))
@@ -977,6 +1001,10 @@ class SQLForeignKey {
       return false;
     if (!(rely_cstr == rhs.rely_cstr))
       return false;
+    if (__isset.catName != rhs.__isset.catName)
+      return false;
+    else if (__isset.catName && !(catName == rhs.catName))
+      return false;
     return true;
   }
   bool operator != (const SQLForeignKey &rhs) const {
@@ -1000,7 +1028,8 @@ inline std::ostream& operator<<(std::ostream& out, const SQLForeignKey& obj)
 }
 
 typedef struct _SQLUniqueConstraint__isset {
-  _SQLUniqueConstraint__isset() : table_db(false), table_name(false), column_name(false), key_seq(false), uk_name(false), enable_cstr(false), validate_cstr(false), rely_cstr(false) {}
+  _SQLUniqueConstraint__isset() : catName(false), table_db(false), table_name(false), column_name(false), key_seq(false), uk_name(false), enable_cstr(false), validate_cstr(false), rely_cstr(false) {}
+  bool catName :1;
   bool table_db :1;
   bool table_name :1;
   bool column_name :1;
@@ -1016,10 +1045,11 @@ class SQLUniqueConstraint {
 
   SQLUniqueConstraint(const SQLUniqueConstraint&);
   SQLUniqueConstraint& operator=(const SQLUniqueConstraint&);
-  SQLUniqueConstraint() : table_db(), table_name(), column_name(), key_seq(0), uk_name(), enable_cstr(0), validate_cstr(0), rely_cstr(0) {
+  SQLUniqueConstraint() : catName(), table_db(), table_name(), column_name(), key_seq(0), uk_name(), enable_cstr(0), validate_cstr(0), rely_cstr(0) {
   }
 
   virtual ~SQLUniqueConstraint() throw();
+  std::string catName;
   std::string table_db;
   std::string table_name;
   std::string column_name;
@@ -1031,6 +1061,8 @@ class SQLUniqueConstraint {
 
   _SQLUniqueConstraint__isset __isset;
 
+  void __set_catName(const std::string& val);
+
   void __set_table_db(const std::string& val);
 
   void __set_table_name(const std::string& val);
@@ -1049,6 +1081,8 @@ class SQLUniqueConstraint {
 
   bool operator == (const SQLUniqueConstraint & rhs) const
   {
+    if (!(catName == rhs.catName))
+      return false;
     if (!(table_db == rhs.table_db))
       return false;
     if (!(table_name == rhs.table_name))
@@ -1088,7 +1122,8 @@ inline std::ostream& operator<<(std::ostream& out, const SQLUniqueConstraint& ob
 }
 
 typedef struct _SQLNotNullConstraint__isset {
-  _SQLNotNullConstraint__isset() : table_db(false), table_name(false), column_name(false), nn_name(false), enable_cstr(false), validate_cstr(false), rely_cstr(false) {}
+  _SQLNotNullConstraint__isset() : catName(false), table_db(false), table_name(false), column_name(false), nn_name(false), enable_cstr(false), validate_cstr(false), rely_cstr(false) {}
+  bool catName :1;
   bool table_db :1;
   bool table_name :1;
   bool column_name :1;
@@ -1103,10 +1138,11 @@ class SQLNotNullConstraint {
 
   SQLNotNullConstraint(const SQLNotNullConstraint&);
   SQLNotNullConstraint& operator=(const SQLNotNullConstraint&);
-  SQLNotNullConstraint() : table_db(), table_name(), column_name(), nn_name(), enable_cstr(0), validate_cstr(0), rely_cstr(0) {
+  SQLNotNullConstraint() : catName(), table_db(), table_name(), column_name(), nn_name(), enable_cstr(0), validate_cstr(0), rely_cstr(0) {
   }
 
   virtual ~SQLNotNullConstraint() throw();
+  std::string catName;
   std::string table_db;
   std::string table_name;
   std::string column_name;
@@ -1117,6 +1153,8 @@ class SQLNotNullConstraint {
 
   _SQLNotNullConstraint__isset __isset;
 
+  void __set_catName(const std::string& val);
+
   void __set_table_db(const std::string& val);
 
   void __set_table_name(const std::string& val);
@@ -1133,6 +1171,8 @@ class SQLNotNullConstraint {
 
   bool operator == (const SQLNotNullConstraint & rhs) const
   {
+    if (!(catName == rhs.catName))
+      return false;
     if (!(table_db == rhs.table_db))
       return false;
     if (!(table_name == rhs.table_name))
@@ -1170,7 +1210,8 @@ inline std::ostream& operator<<(std::ostream& out, const SQLNotNullConstraint& o
 }
 
 typedef struct _SQLDefaultConstraint__isset {
-  _SQLDefaultConstraint__isset() : table_db(false), table_name(false), column_name(false), default_value(false), dc_name(false), enable_cstr(false), validate_cstr(false), rely_cstr(false) {}
+  _SQLDefaultConstraint__isset() : catName(false), table_db(false), table_name(false), column_name(false), default_value(false), dc_name(false), enable_cstr(false), validate_cstr(false), rely_cstr(false) {}
+  bool catName :1;
   bool table_db :1;
   bool table_name :1;
   bool column_name :1;
@@ -1186,10 +1227,11 @@ class SQLDefaultConstraint {
 
   SQLDefaultConstraint(const SQLDefaultConstraint&);
   SQLDefaultConstraint& operator=(const SQLDefaultConstraint&);
-  SQLDefaultConstraint() : table_db(), table_name(), column_name(), default_value(), dc_name(), enable_cstr(0), validate_cstr(0), rely_cstr(0) {
+  SQLDefaultConstraint() : catName(), table_db(), table_name(), column_name(), default_value(), dc_name(), enable_cstr(0), validate_cstr(0), rely_cstr(0) {
   }
 
   virtual ~SQLDefaultConstraint() throw();
+  std::string catName;
   std::string table_db;
   std::string table_name;
   std::string column_name;
@@ -1201,6 +1243,8 @@ class SQLDefaultConstraint {
 
   _SQLDefaultConstraint__isset __isset;
 
+  void __set_catName(const std::string& val);
+
   void __set_table_db(const std::string& val);
 
   void __set_table_name(const std::string& val);
@@ -1219,6 +1263,8 @@ class SQLDefaultConstraint {
 
   bool operator == (const SQLDefaultConstraint & rhs) const
   {
+    if (!(catName == rhs.catName))
+      return false;
     if (!(table_db == rhs.table_db))
       return false;
     if (!(table_name == rhs.table_name))
@@ -1258,7 +1304,8 @@ inline std::ostream& operator<<(std::ostream& out, const SQLDefaultConstraint& o
 }
 
 typedef struct _SQLCheckConstraint__isset {
-  _SQLCheckConstraint__isset() : table_db(false), table_name(false), column_name(false), check_expression(false), dc_name(false), enable_cstr(false), validate_cstr(false), rely_cstr(false) {}
+  _SQLCheckConstraint__isset() : catName(false), table_db(false), table_name(false), column_name(false), check_expression(false), dc_name(false), enable_cstr(false), validate_cstr(false), rely_cstr(false) {}
+  bool catName :1;
   bool table_db :1;
   bool table_name :1;
   bool column_name :1;
@@ -1274,10 +1321,11 @@ class SQLCheckConstraint {
 
   SQLCheckConstraint(const SQLCheckConstraint&);
   SQLCheckConstraint& operator=(const SQLCheckConstraint&);
-  SQLCheckConstraint() : table_db(), table_name(), column_name(), check_expression(), dc_name(), enable_cstr(0), validate_cstr(0), rely_cstr(0) {
+  SQLCheckConstraint() : catName(), table_db(), table_name(), column_name(), check_expression(), dc_name(), enable_cstr(0), validate_cstr(0), rely_cstr(0) {
   }
 
   virtual ~SQLCheckConstraint() throw();
+  std::string catName;
   std::string table_db;
   std::string table_name;
   std::string column_name;
@@ -1289,6 +1337,8 @@ class SQLCheckConstraint {
 
   _SQLCheckConstraint__isset __isset;
 
+  void __set_catName(const std::string& val);
+
   void __set_table_db(const std::string& val);
 
   void __set_table_name(const std::string& val);
@@ -1307,6 +1357,8 @@ class SQLCheckConstraint {
 
   bool operator == (const SQLCheckConstraint & rhs) const
   {
+    if (!(catName == rhs.catName))
+      return false;
     if (!(table_db == rhs.table_db))
       return false;
     if (!(table_name == rhs.table_name))
@@ -1416,12 +1468,13 @@ inline std::ostream& operator<<(std::ostream& out, const Type& obj)
 }
 
 typedef struct _HiveObjectRef__isset {
-  _HiveObjectRef__isset() : objectType(false), dbName(false), objectName(false), partValues(false), columnName(false) {}
+  _HiveObjectRef__isset() : objectType(false), dbName(false), objectName(false), partValues(false), columnName(false), catName(false) {}
   bool objectType :1;
   bool dbName :1;
   bool objectName :1;
   bool partValues :1;
   bool columnName :1;
+  bool catName :1;
 } _HiveObjectRef__isset;
 
 class HiveObjectRef {
@@ -1429,7 +1482,7 @@ class HiveObjectRef {
 
   HiveObjectRef(const HiveObjectRef&);
   HiveObjectRef& operator=(const HiveObjectRef&);
-  HiveObjectRef() : objectType((HiveObjectType::type)0), dbName(), objectName(), columnName() {
+  HiveObjectRef() : objectType((HiveObjectType::type)0), dbName(), objectName(), columnName(), catName() {
   }
 
   virtual ~HiveObjectRef() throw();
@@ -1438,6 +1491,7 @@ class HiveObjectRef {
   std::string objectName;
   std::vector<std::string>  partValues;
   std::string columnName;
+  std::string catName;
 
   _HiveObjectRef__isset __isset;
 
@@ -1451,6 +1505,8 @@ class HiveObjectRef {
 
   void __set_columnName(const std::string& val);
 
+  void __set_catName(const std::string& val);
+
   bool operator == (const HiveObjectRef & rhs) const
   {
     if (!(objectType == rhs.objectType))
@@ -1463,6 +1519,10 @@ class HiveObjectRef {
       return false;
     if (!(columnName == rhs.columnName))
       return false;
+    if (__isset.catName != rhs.__isset.catName)
+      return false;
+    else if (__isset.catName && !(catName == rhs.catName))
+      return false;
     return true;
   }
   bool operator != (const HiveObjectRef &rhs) const {
@@ -2272,8 +2332,298 @@ inline std::ostream& operator<<(std::ostream& out, const GrantRevokeRoleResponse
   return out;
 }
 
+typedef struct _Catalog__isset {
+  _Catalog__isset() : name(false), description(false), locationUri(false) {}
+  bool name :1;
+  bool description :1;
+  bool locationUri :1;
+} _Catalog__isset;
+
+class Catalog {
+ public:
+
+  Catalog(const Catalog&);
+  Catalog& operator=(const Catalog&);
+  Catalog() : name(), description(), locationUri() {
+  }
+
+  virtual ~Catalog() throw();
+  std::string name;
+  std::string description;
+  std::string locationUri;
+
+  _Catalog__isset __isset;
+
+  void __set_name(const std::string& val);
+
+  void __set_description(const std::string& val);
+
+  void __set_locationUri(const std::string& val);
+
+  bool operator == (const Catalog & rhs) const
+  {
+    if (!(name == rhs.name))
+      return false;
+    if (__isset.description != rhs.__isset.description)
+      return false;
+    else if (__isset.description && !(description == rhs.description))
+      return false;
+    if (!(locationUri == rhs.locationUri))
+      return false;
+    return true;
+  }
+  bool operator != (const Catalog &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const Catalog & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(Catalog &a, Catalog &b);
+
+inline std::ostream& operator<<(std::ostream& out, const Catalog& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+typedef struct _CreateCatalogRequest__isset {
+  _CreateCatalogRequest__isset() : catalog(false) {}
+  bool catalog :1;
+} _CreateCatalogRequest__isset;
+
+class CreateCatalogRequest {
+ public:
+
+  CreateCatalogRequest(const CreateCatalogRequest&);
+  CreateCatalogRequest& operator=(const CreateCatalogRequest&);
+  CreateCatalogRequest() {
+  }
+
+  virtual ~CreateCatalogRequest() throw();
+  Catalog catalog;
+
+  _CreateCatalogRequest__isset __isset;
+
+  void __set_catalog(const Catalog& val);
+
+  bool operator == (const CreateCatalogRequest & rhs) const
+  {
+    if (!(catalog == rhs.catalog))
+      return false;
+    return true;
+  }
+  bool operator != (const CreateCatalogRequest &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const CreateCatalogRequest & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(CreateCatalogRequest &a, CreateCatalogRequest &b);
+
+inline std::ostream& operator<<(std::ostream& out, const CreateCatalogRequest& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+typedef struct _GetCatalogRequest__isset {
+  _GetCatalogRequest__isset() : name(false) {}
+  bool name :1;
+} _GetCatalogRequest__isset;
+
+class GetCatalogRequest {
+ public:
+
+  GetCatalogRequest(const GetCatalogRequest&);
+  GetCatalogRequest& operator=(const GetCatalogRequest&);
+  GetCatalogRequest() : name() {
+  }
+
+  virtual ~GetCatalogRequest() throw();
+  std::string name;
+
+  _GetCatalogRequest__isset __isset;
+
+  void __set_name(const std::string& val);
+
+  bool operator == (const GetCatalogRequest & rhs) const
+  {
+    if (!(name == rhs.name))
+      return false;
+    return true;
+  }
+  bool operator != (const GetCatalogRequest &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const GetCatalogRequest & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(GetCatalogRequest &a, GetCatalogRequest &b);
+
+inline std::ostream& operator<<(std::ostream& out, const GetCatalogRequest& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+typedef struct _GetCatalogResponse__isset {
+  _GetCatalogResponse__isset() : catalog(false) {}
+  bool catalog :1;
+} _GetCatalogResponse__isset;
+
+class GetCatalogResponse {
+ public:
+
+  GetCatalogResponse(const GetCatalogResponse&);
+  GetCatalogResponse& operator=(const GetCatalogResponse&);
+  GetCatalogResponse() {
+  }
+
+  virtual ~GetCatalogResponse() throw();
+  Catalog catalog;
+
+  _GetCatalogResponse__isset __isset;
+
+  void __set_catalog(const Catalog& val);
+
+  bool operator == (const GetCatalogResponse & rhs) const
+  {
+    if (!(catalog == rhs.catalog))
+      return false;
+    return true;
+  }
+  bool operator != (const GetCatalogResponse &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const GetCatalogResponse & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(GetCatalogResponse &a, GetCatalogResponse &b);
+
+inline std::ostream& operator<<(std::ostream& out, const GetCatalogResponse& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+typedef struct _GetCatalogsResponse__isset {
+  _GetCatalogsResponse__isset() : names(false) {}
+  bool names :1;
+} _GetCatalogsResponse__isset;
+
+class GetCatalogsResponse {
+ public:
+
+  GetCatalogsResponse(const GetCatalogsResponse&);
+  GetCatalogsResponse& operator=(const GetCatalogsResponse&);
+  GetCatalogsResponse() {
+  }
+
+  virtual ~GetCatalogsResponse() throw();
+  std::vector<std::string>  names;
+
+  _GetCatalogsResponse__isset __isset;
+
+  void __set_names(const std::vector<std::string> & val);
+
+  bool operator == (const GetCatalogsResponse & rhs) const
+  {
+    if (!(names == rhs.names))
+      return false;
+    return true;
+  }
+  bool operator != (const GetCatalogsResponse &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const GetCatalogsResponse & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(GetCatalogsResponse &a, GetCatalogsResponse &b);
+
+inline std::ostream& operator<<(std::ostream& out, const GetCatalogsResponse& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+typedef struct _DropCatalogRequest__isset {
+  _DropCatalogRequest__isset() : name(false) {}
+  bool name :1;
+} _DropCatalogRequest__isset;
+
+class DropCatalogRequest {
+ public:
+
+  DropCatalogRequest(const DropCatalogRequest&);
+  DropCatalogRequest& operator=(const DropCatalogRequest&);
+  DropCatalogRequest() : name() {
+  }
+
+  virtual ~DropCatalogRequest() throw();
+  std::string name;
+
+  _DropCatalogRequest__isset __isset;
+
+  void __set_name(const std::string& val);
+
+  bool operator == (const DropCatalogRequest & rhs) const
+  {
+    if (!(name == rhs.name))
+      return false;
+    return true;
+  }
+  bool operator != (const DropCatalogRequest &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const DropCatalogRequest & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(DropCatalogRequest &a, DropCatalogRequest &b);
+
+inline std::ostream& operator<<(std::ostream& out, const DropCatalogRequest& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
 typedef struct _Database__isset {
-  _Database__isset() : name(false), description(false), locationUri(false), parameters(false), privileges(false), ownerName(false), ownerType(false) {}
+  _Database__isset() : name(false), description(false), locationUri(false), parameters(false), privileges(false), ownerName(false), ownerType(false), catalogName(false) {}
   bool name :1;
   bool description :1;
   bool locationUri :1;
@@ -2281,6 +2631,7 @@ typedef struct _Database__isset {
   bool privileges :1;
   bool ownerName :1;
   bool ownerType :1;
+  bool catalogName :1;
 } _Database__isset;
 
 class Database {
@@ -2288,7 +2639,7 @@ class Database {
 
   Database(const Database&);
   Database& operator=(const Database&);
-  Database() : name(), description(), locationUri(), ownerName(), ownerType((PrincipalType::type)0) {
+  Database() : name(), description(), locationUri(), ownerName(), ownerType((PrincipalType::type)0), catalogName() {
   }
 
   virtual ~Database() throw();
@@ -2299,6 +2650,7 @@ class Database {
   PrincipalPrivilegeSet privileges;
   std::string ownerName;
   PrincipalType::type ownerType;
+  std::string catalogName;
 
   _Database__isset __isset;
 
@@ -2316,6 +2668,8 @@ class Database {
 
   void __set_ownerType(const PrincipalType::type val);
 
+  void __set_catalogName(const std::string& val);
+
   bool operator == (const Database & rhs) const
   {
     if (!(name == rhs.name))
@@ -2338,6 +2692,10 @@ class Database {
       return false;
     else if (__isset.ownerType && !(ownerType == rhs.ownerType))
       return false;
+    if (__isset.catalogName != rhs.__isset.catalogName)
+      return false;
+    else if (__isset.catalogName && !(catalogName == rhs.catalogName))
+      return false;
     return true;
   }
   bool operator != (const Database &rhs) const {
@@ -2677,7 +3035,7 @@ inline std::ostream& operator<<(std::ostream& out, const StorageDescriptor& obj)
 }
 
 typedef struct _Table__isset {
-  _Table__isset() : tableName(false), dbName(false), owner(false), createTime(false), lastAccessTime(false), retention(false), sd(false), partitionKeys(false), parameters(false), viewOriginalText(false), viewExpandedText(false), tableType(false), privileges(false), temporary(true), rewriteEnabled(false), creationMetadata(false) {}
+  _Table__isset() : tableName(false), dbName(false), owner(false), createTime(false), lastAccessTime(false), retention(false), sd(false), partitionKeys(false), parameters(false), viewOriginalText(false), viewExpandedText(false), tableType(false), privileges(false), temporary(true), rewriteEnabled(false), creationMetadata(false), catName(false) {}
   bool tableName :1;
   bool dbName :1;
   bool owner :1;
@@ -2694,6 +3052,7 @@ typedef struct _Table__isset {
   bool temporary :1;
   bool rewriteEnabled :1;
   bool creationMetadata :1;
+  bool catName :1;
 } _Table__isset;
 
 class Table {
@@ -2701,7 +3060,7 @@ class Table {
 
   Table(const Table&);
   Table& operator=(const Table&);
-  Table() : tableName(), dbName(), owner(), createTime(0), lastAccessTime(0), retention(0), viewOriginalText(), viewExpandedText(), tableType(), temporary(false), rewriteEnabled(0) {
+  Table() : tableName(), dbName(), owner(), createTime(0), lastAccessTime(0), retention(0), viewOriginalText(), viewExpandedText(), tableType(), temporary(false), rewriteEnabled(0), catName() {
   }
 
   virtual ~Table() throw();
@@ -2721,6 +3080,7 @@ class Table {
   bool temporary;
   bool rewriteEnabled;
   CreationMetadata creationMetadata;
+  std::string catName;
 
   _Table__isset __isset;
 
@@ -2756,6 +3116,8 @@ class Table {
 
   void __set_creationMetadata(const CreationMetadata& val);
 
+  void __set_catName(const std::string& val);
+
   bool operator == (const Table & rhs) const
   {
     if (!(tableName == rhs.tableName))
@@ -2798,6 +3160,10 @@ class Table {
       return false;
     else if (__isset.creationMetadata && !(creationMetadata == rhs.creationMetadata))
       return false;
+    if (__isset.catName != rhs.__isset.catName)
+      return false;
+    else if (__isset.catName && !(catName == rhs.catName))
+      return false;
     return true;
   }
   bool operator != (const Table &rhs) const {
@@ -2821,7 +3187,7 @@ inline std::ostream& operator<<(std::ostream& out, const Table& obj)
 }
 
 typedef struct _Partition__isset {
-  _Partition__isset() : values(false), dbName(false), tableName(false), createTime(false), lastAccessTime(false), sd(false), parameters(false), privileges(false) {}
+  _Partition__isset() : values(false), dbName(false), tableName(false), createTime(false), lastAccessTime(false), sd(false), parameters(false), privileges(false), catName(false) {}
   bool values :1;
   bool dbName :1;
   bool tableName :1;
@@ -2830,6 +3196,7 @@ typedef struct _Partition__isset {
   bool sd :1;
   bool parameters :1;
   bool privileges :1;
+  bool catName :1;
 } _Partition__isset;
 
 class Partition {
@@ -2837,7 +3204,7 @@ class Partition {
 
   Partition(const Partition&);
   Partition& operator=(const Partition&);
-  Partition() : dbName(), tableName(), createTime(0), lastAccessTime(0) {
+  Partition() : dbName(), tableName(), createTime(0), lastAccessTime(0), catName() {
   }
 
   virtual ~Partition() throw();
@@ -2849,6 +3216,7 @@ class Partition {
   StorageDescriptor sd;
   std::map<std::string, std::string>  parameters;
   PrincipalPrivilegeSet privileges;
+  std::string catName;
 
   _Partition__isset __isset;
 
@@ -2868,6 +3236,8 @@ class Partition {
 
   void __set_privileges(const PrincipalPrivilegeSet& val);
 
+  void __set_catName(const std::string& val);
+
   bool operator == (const Partition & rhs) const
   {
     if (!(values == rhs.values))
@@ -2888,6 +3258,10 @@ class Partition {
       return false;
     else if (__isset.privileges && !(privileges == rhs.privileges))
       return false;
+    if (__isset.catName != rhs.__isset.catName)
+      return false;
+    else if (__isset.catName && !(catName == rhs.catName))
+      return false;
     return true;
   }
   bool operator != (const Partition &rhs) const {
@@ -3087,12 +3461,13 @@ inline std::ostream& operator<<(std::ostream& out, const PartitionListComposingS
 }
 
 typedef struct _PartitionSpec__isset {
-  _PartitionSpec__isset() : dbName(false), tableName(false), rootPath(false), sharedSDPartitionSpec(false), partitionList(false) {}
+  _PartitionSpec__isset() : dbName(false), tableName(false), rootPath(false), sharedSDPartitionSpec(false), partitionList(false), catName(false) {}
   bool dbName :1;
   bool tableName :1;
   bool rootPath :1;
   bool sharedSDPartitionSpec :1;
   bool partitionList :1;
+  bool catName :1;
 } _PartitionSpec__isset;
 
 class PartitionSpec {
@@ -3100,7 +3475,7 @@ class PartitionSpec {
 
   PartitionSpec(const PartitionSpec&);
   PartitionSpec& operator=(const PartitionSpec&);
-  PartitionSpec() : dbName(), tableName(), rootPath() {
+  PartitionSpec() : dbName(), tableName(), rootPath(), catName() {
   }
 
   virtual ~PartitionSpec() throw();
@@ -3109,6 +3484,7 @@ class PartitionSpec {
   std::string rootPath;
   PartitionSpecWithSharedSD sharedSDPartitionSpec;
   PartitionListComposingSpec partitionList;
+  std::string catName;
 
   _PartitionSpec__isset __isset;
 
@@ -3122,6 +3498,8 @@ class PartitionSpec {
 
   void __set_partitionList(const PartitionListComposingSpec& val);
 
+  void __set_catName(const std::string& val);
+
   bool operator == (const PartitionSpec & rhs) const
   {
     if (!(dbName == rhs.dbName))
@@ -3138,6 +3516,10 @@ class PartitionSpec {
       return false;
     else if (__isset.partitionList && !(partitionList == rhs.partitionList))
       return false;
+    if (__isset.catName != rhs.__isset.catName)
+      return false;
+    else if (__isset.catName && !(catName == rhs.catName))
+      return false;
     return true;
   }
   bool operator != (const PartitionSpec &rhs) const {
@@ -3868,9 +4250,10 @@ inline std::ostream& operator<<(std::ostream& out, const ColumnStatisticsObj& ob
 }
 
 typedef struct _ColumnStatisticsDesc__isset {
-  _ColumnStatisticsDesc__isset() : partName(false), lastAnalyzed(false) {}
+  _ColumnStatisticsDesc__isset() : partName(false), lastAnalyzed(false), catName(false) {}
   bool partName :1;
   bool lastAnalyzed :1;
+  bool catName :1;
 } _ColumnStatisticsDesc__isset;
 
 class ColumnStatisticsDesc {
@@ -3878,7 +4261,7 @@ class ColumnStatisticsDesc {
 
   ColumnStatisticsDesc(const ColumnStatisticsDesc&);
   ColumnStatisticsDesc& operator=(const ColumnStatisticsDesc&);
-  ColumnStatisticsDesc() : isTblLevel(0), dbName(), tableName(), partName(), lastAnalyzed(0) {
+  ColumnStatisticsDesc() : isTblLevel(0), dbName(), tableName(), partName(), lastAnalyzed(0), catName() {
   }
 
   virtual ~ColumnStatisticsDesc() throw();
@@ -3887,6 +4270,7 @@ class ColumnStatisticsDesc {
   std::string tableName;
   std::string partName;
   int64_t lastAnalyzed;
+  std::string catName;
 
   _ColumnStatisticsDesc__isset __isset;
 
@@ -3900,6 +4284,8 @@ class ColumnStatisticsDesc {
 
   void __set_lastAnalyzed(const int64_t val);
 
+  void __set_catName(const std::string& val);
+
   bool operator == (const ColumnStatisticsDesc & rhs) const
   {
     if (!(isTblLevel == rhs.isTblLevel))
@@ -3916,6 +4302,10 @@ class ColumnStatisticsDesc {
       return false;
     else if (__isset.lastAnalyzed && !(lastAnalyzed == rhs.lastAnalyzed))
       return false;
+    if (__isset.catName != rhs.__isset.catName)
+      return false;
+    else if (__isset.catName && !(catName == rhs.catName))
+      return false;
     return true;
   }
   bool operator != (const ColumnStatisticsDesc &rhs) const {
@@ -4179,29 +4569,42 @@ inline std::ostream& operator<<(std::ostream& out, const EnvironmentContext& obj
   return out;
 }
 
+typedef struct _PrimaryKeysRequest__isset {
+  _PrimaryKeysRequest__isset() : catName(false) {}
+  bool catName :1;
+} _PrimaryKeysRequest__isset;
 
 class PrimaryKeysRequest {
  public:
 
   PrimaryKeysRequest(const PrimaryKeysRequest&);
   PrimaryKeysRequest& operator=(const PrimaryKeysRequest&);
-  PrimaryKeysRequest() : db_name(), tbl_name() {
+  PrimaryKeysRequest() : db_name(), tbl_name(), catName() {
   }
 
   virtual ~PrimaryKeysRequest() throw();
   std::string db_name;
   std::string tbl_name;
+  std::string catName;
+
+  _PrimaryKeysRequest__isset __isset;
 
   void __set_db_name(const std::string& val);
 
   void __set_tbl_name(const std::string& val);
 
+  void __set_catName(const std::string& val);
+
   bool operator == (const PrimaryKeysRequest & rhs) const
   {
     if (!(db_name == rhs.db_name))
       return false;
     if (!(tbl_name == rhs.tbl_name))
       return false;
+    if (__isset.catName != rhs.__isset.catName)
+      return false;
+    else if (__isset.catName && !(catName == rhs.catName))
+      return false;
     return true;
   }
   bool operator != (const PrimaryKeysRequest &rhs) const {
@@ -4265,11 +4668,12 @@ inline std::ostream& operator<<(std::ostream& out, const PrimaryKeysResponse& ob
 }
 
 typedef struct _ForeignKeysRequest__isset {
-  _ForeignKeysRequest__isset() : parent_db_name(false), parent_tbl_name(false), foreign_db_name(false), foreign_tbl_name(false) {}
+  _ForeignKeysRequest__isset() : parent_db_name(false), parent_tbl_name(false), foreign_db_name(false), foreign_tbl_name(false), catName(false) {}
   bool parent_db_name :1;
   bool parent_tbl_name :1;
   bool foreign_db_name :1;
   bool foreign_tbl_name :1;
+  bool catName :1;
 } _ForeignKeysRequest__isset;
 
 class ForeignKeysRequest {
@@ -4277,7 +4681,7 @@ class ForeignKeysRequest {
 
   ForeignKeysRequest(const ForeignKeysRequest&);
   ForeignKeysRequest& operator=(const ForeignKeysRequest&);
-  ForeignKeysRequest() : parent_db_name(), parent_tbl_name(), foreign_db_name(), foreign_tbl_name() {
+  ForeignKeysRequest() : parent_db_name(), parent_tbl_name(), foreign_db_name(), foreign_tbl_name(), catName() {
   }
 
   virtual ~ForeignKeysRequest() throw();
@@ -4285,6 +4689,7 @@ class ForeignKeysRequest {
   std::string parent_tbl_name;
   std::string foreign_db_name;
   std::string foreign_tbl_name;
+  std::string catName;
 
   _ForeignKeysRequest__isset __isset;
 
@@ -4296,6 +4701,8 @@ class ForeignKeysRequest {
 
   void __set_foreign_tbl_name(const std::string& val);
 
+  void __set_catName(const std::string& val);
+
   bool operator == (const ForeignKeysRequest & rhs) const
   {
     if (!(parent_db_name == rhs.parent_db_name))
@@ -4306,6 +4713,10 @@ class ForeignKeysRequest {
       return false;
     if (!(foreign_tbl_name == rhs.foreign_tbl_name))
       return false;
+    if (__isset.catName != rhs.__isset.catName)
+      return false;
+    else if (__isset.catName && !(catName == rhs.catName))
+      return false;
     return true;
   }
   bool operator != (const ForeignKeysRequest &rhs) const {
@@ -4374,19 +4785,24 @@ class UniqueConstraintsRequest {
 
   UniqueConstraintsRequest(const UniqueConstraintsRequest&);
   UniqueConstraintsRequest& operator=(const UniqueConstraintsRequest&);
-  UniqueConstraintsRequest() : db_name(), tbl_name() {
+  UniqueConstraintsRequest() : catName(), db_name(), tbl_name() {
   }
 
   virtual ~UniqueConstraintsRequest() throw();
+  std::string catName;
   std::string db_name;
   std::string tbl_name;
 
+  void __set_catName(const std::string& val);
+
   void __set_db_name(const std::string& val);
 
   void __set_tbl_name(const std::string& val);
 
   bool operator == (const UniqueConstraintsRequest & rhs) const
   {
+    if (!(catName == rhs.catName))
+      return false;
     if (!(db_name == rhs.db_name))
       return false;
     if (!(tbl_name == rhs.tbl_name))
@@ -4459,19 +4875,24 @@ class NotNullConstraintsRequest {
 
   NotNullConstraintsRequest(const NotNullConstraintsRequest&);
   NotNullConstraintsRequest& operator=(const NotNullConstraintsRequest&);
-  NotNullConstraintsRequest() : db_name(), tbl_name() {
+  NotNullConstraintsRequest() : catName(), db_name(), tbl_name() {
   }
 
   virtual ~NotNullConstraintsRequest() throw();
+  std::string catName;
   std::string db_name;
   std::string tbl_name;
 
+  void __set_catName(const std::string& val);
+
   void __set_db_name(const std::string& val);
 
   void __set_tbl_name(const std::string& val);
 
   bool operator == (const NotNullConstraintsRequest & rhs) const
   {
+    if (!(catName == rhs.catName))
+      return false;
     if (!(db_name == rhs.db_name))
       return false;
     if (!(tbl_name == rhs.tbl_name))
@@ -4544,19 +4965,24 @@ class DefaultConstraintsRequest {
 
   DefaultConstraintsRequest(const DefaultConstraintsRequest&);
   DefaultConstraintsRequest& operator=(const DefaultConstraintsRequest&);
-  DefaultConstraintsRequest() : db_name(), tbl_name() {
+  DefaultConstraintsRequest() : catName(), db_name(), tbl_name() {
   }
 
   virtual ~DefaultConstraintsRequest() throw();
+  std::string catName;
   std::string db_name;
   std::string tbl_name;
 
+  void __set_catName(const std::string& val);
+
   void __set_db_name(const std::string& val);
 
   void __set_tbl_name(const std::string& val);
 
   bool operator == (const DefaultConstraintsRequest & rhs) const
   {
+    if (!(catName == rhs.catName))
+      return false;
     if (!(db_name == rhs.db_name))
       return false;
     if (!(tbl_name == rhs.tbl_name))
@@ -4629,19 +5055,24 @@ class CheckConstraintsRequest {
 
   CheckConstraintsRequest(const CheckConstraintsRequest&);
   CheckConstraintsRequest& operator=(const CheckConstraintsRequest&);
-  CheckConstraintsRequest() : db_name(), tbl_name() {
+  CheckConstraintsRequest() : catName(), db_name(), tbl_name() {
   }
 
   virtual ~CheckConstraintsRequest() throw();
+  std::string catName;
   std::string db_name;
   std::string tbl_name;
 
+  void __set_catName(const std::string& val);
+
   void __set_db_name(const std::string& val);
 
   void __set_tbl_name(const std::string& val);
 
   bool operator == (const CheckConstraintsRequest & rhs) const
   {
+    if (!(catName == rhs.catName))
+      return false;
     if (!(db_name == rhs.db_name))
       return false;
     if (!(tbl_name == rhs.tbl_name))
@@ -4708,19 +5139,26 @@ inline std::ostream& operator<<(std::ostream& out, const CheckConstraintsRespons
   return out;
 }
 
+typedef struct _DropConstraintRequest__isset {
+  _DropConstraintRequest__isset() : catName(false) {}
+  bool catName :1;
+} _DropConstraintRequest__isset;
 
 class DropConstraintRequest {
  public:
 
   DropConstraintRequest(const DropConstraintRequest&);
   DropConstraintRequest& operator=(const DropConstraintRequest&);
-  DropConstraintRequest() : dbname(), tablename(), constraintname() {
+  DropConstraintRequest() : dbname(), tablename(), constraintname(), catName() {
   }
 
   virtual ~DropConstraintRequest() throw();
   std::string dbname;
   std::string tablename;
   std::string constraintname;
+  std::string catName;
+
+  _DropConstraintRequest__isset __isset;
 
   void __set_dbname(const std::string& val);
 
@@ -4728,6 +5166,8 @@ class DropConstraintRequest {
 
   void __set_constraintname(const std::string& val);
 
+  void __set_catName(const std::string& val);
+
   bool operator == (const DropConstraintRequest & rhs) const
   {
     if (!(dbname == rhs.dbname))
@@ -4736,6 +5176,10 @@ class DropConstraintRequest {
       return false;
     if (!(constraintname == rhs.constraintname))
       return false;
+    if (__isset.catName != rhs.__isset.catName)
+      return false;
+    else if (__isset.catName && !(catName == rhs.catName))
+      return false;
     return true;
   }
   bool operator != (const DropConstraintRequest &rhs) const {
@@ -5044,9 +5488,10 @@ inline std::ostream& operator<<(std::ostream& out, const PartitionsByExprResult&
 }
 
 typedef struct _PartitionsByExprRequest__isset {
-  _PartitionsByExprRequest__isset() : defaultPartitionName(false), maxParts(true) {}
+  _PartitionsByExprRequest__isset() : defaultPartitionName(false), maxParts(true), catName(false) {}
   bool defaultPartitionName :1;
   bool maxParts :1;
+  bool catName :1;
 } _PartitionsByExprRequest__isset;
 
 class PartitionsByExprRequest {
@@ -5054,7 +5499,7 @@ class PartitionsByExprRequest {
 
   PartitionsByExprRequest(const PartitionsByExprRequest&);
   PartitionsByExprRequest& operator=(const PartitionsByExprRequest&);
-  PartitionsByExprRequest() : dbName(), tblName(), expr(), defaultPartitionName(), maxParts(-1) {
+  PartitionsByExprRequest() : dbName(), tblName(), expr(), defaultPartitionName(), maxParts(-1), catName() {
   }
 
   virtual ~PartitionsByExprRequest() throw();
@@ -5063,6 +5508,7 @@ class PartitionsByExprRequest {
   std::string expr;
   std::string defaultPartitionName;
   int16_t maxParts;
+  std::string catName;
 
   _PartitionsByExprRequest__isset __isset;
 
@@ -5076,6 +5522,8 @@ class PartitionsByExprRequest {
 
   void __set_maxParts(const int16_t val);
 
+  void __set_catName(const std::string& val);
+
   bool operator == (const PartitionsByExprRequest & rhs) const
   {
     if (!(dbName == rhs.dbName))
@@ -5092,6 +5540,10 @@ class PartitionsByExprRequest {
       return false;
     else if (__isset.maxParts && !(maxParts == rhs.maxParts))
       return false;
+    if (__isset.catName != rhs.__isset.catName)
+      return false;
+    else if (__isset.catName && !(catName == rhs.catName))
+      return false;
     return true;
   }
   bool operator != (const PartitionsByExprRequest &rhs) const {
@@ -5194,19 +5646,26 @@ inline std::ostream& operator<<(std::ostream& out, const PartitionsStatsResult&
   return out;
 }
 
+typedef struct _TableStatsRequest__isset {
+  _TableStatsRequest__isset() : catName(false) {}
+  bool catName :1;
+} _TableStatsRequest__isset;
 
 class TableStatsRequest {
  public:
 
   TableStatsRequest(const TableStatsRequest&);
   TableStatsRequest& operator=(const TableStatsRequest&);
-  TableStatsRequest() : dbName(), tblName() {
+  TableStatsRequest() : dbName(), tblName(), catName() {
   }
 
   virtual ~TableStatsRequest() throw();
   std::string dbName;
   std::string tblName;
   std::vector<std::string>  colNames;
+  std::string catName;
+
+  _TableStatsRequest__isset __isset;
 
   void __set_dbName(const std::string& val);
 
@@ -5214,6 +5673,8 @@ class TableStatsRequest {
 
   void __set_colNames(const std::vector<std::string> & val);
 
+  void __set_catName(const std::string& val);
+
   bool operator == (const TableStatsRequest & rhs) const
   {
     if (!(dbName == rhs.dbName))
@@ -5222,6 +5683,10 @@ class TableStatsRequest {
       return false;
     if (!(colNames == rhs.colNames))
       return false;
+    if (__isset.catName != rhs.__isset.catName)
+      return false;
+    else if (__isset.catName && !(catName == rhs.catName))
+      return false;
     return true;
   }
   bool operator != (const TableStatsRequest &rhs) const {
@@ -5244,13 +5709,17 @@ inline std::ostream& operator<<(std::ostream& out, const TableStatsRequest& obj)
   return out;
 }
 
+typedef struct _PartitionsStatsRequest__isset {
+  _PartitionsStatsRequest__isset() : catName(false) {}
+  bool catName :1;
+} _PartitionsStatsRequest__isset;
 
 class PartitionsStatsRequest {
  public:
 
   PartitionsStatsRequest(const PartitionsStatsRequest&);
   PartitionsStatsRequest& operator=(const PartitionsStatsRequest&);
-  PartitionsStatsRequest() : dbName(), tblName() {
+  PartitionsStatsRequest() : dbName(), tblName(), catName() {
   }
 
   virtual ~PartitionsStatsRequest() throw();
@@ -5258,6 +5727,9 @@ class PartitionsStatsRequest {
   std::string tblName;
   std::vector<std::string>  colNames;
   std::vector<std::string>  partNames;
+  std::string catName;
+
+  _PartitionsStatsRequest__isset __isset;
 
   void __set_dbName(const std::string& val);
 
@@ -5267,6 +5739,8 @@ class PartitionsStatsRequest {
 
   void __set_partNames(const std::vector<std::string> & val);
 
+  void __set_catName(const std::string& val);
+
   bool operator == (const PartitionsStatsRequest & rhs) const
   {
     if (!(dbName == rhs.dbName))
@@ -5277,6 +5751,10 @@ class PartitionsStatsRequest {
       return false;
     if (!(partNames == rhs.partNames))
       return false;
+    if (__isset.catName != rhs.__isset.catName)
+      return false;
+    else if (__isset.catName && !(catName == rhs.catName))
+      return false;
     return true;
   }
   bool operator != (const PartitionsStatsRequest &rhs) const {
@@ -5348,8 +5826,9 @@ inline std::ostream& operator<<(std::ostream& out, const AddPartitionsResult& ob
 }
 
 typedef struct _AddPartitionsRequest__isset {
-  _AddPartitionsRequest__isset() : needResult(true) {}
+  _AddPartitionsRequest__isset() : needResult(true), catName(false) {}
   bool needResult :1;
+  bool catName :1;
 } _AddPartitionsRequest__isset;
 
 class AddPartitionsRequest {
@@ -5357,7 +5836,7 @@ class AddPartitionsRequest {
 
   AddPartitionsRequest(const AddPartitionsRequest&);
   AddPartitionsRequest& operator=(const AddPartitionsRequest&);
-  AddPartitionsRequest() : dbName(), tblName(), ifNotExists(0), needResult(true) {
+  AddPartitionsRequest() : dbName(), tblName(), ifNotExists(0), needResult(true), catName() {
   }
 
   virtual ~AddPartitionsRequest() throw();
@@ -5366,6 +5845,7 @@ class AddPartitionsRequest {
   std::vector<Partition>  parts;
   bool ifNotExists;
   bool needResult;
+  std::string catName;
 
   _AddPartitionsRequest__isset __isset;
 
@@ -5379,6 +5859,8 @@ class AddPartitionsRequest {
 
   void __set_needResult(const bool val);
 
+  void __set_catName(const std::string& val);
+
   bool operator == (const AddPartitionsRequest & rhs) const
   {
     if (!(dbName == rhs.dbName))
@@ -5393,6 +5875,10 @@ class AddPartitionsRequest {
       return false;
     else if (__isset.needResult && !(needResult == rhs.needResult))
       return false;
+    if (__isset.catName != rhs.__isset.catName)
+      return false;
+    else if (__isset.catName && !(catName == rhs.catName))
+      return false;
     return true;
   }
   bool operator != (const AddPartitionsRequest &rhs) const {
@@ -5569,12 +6055,13 @@ inline std::ostream& operator<<(std::ostream& out, const RequestPartsSpec& obj)
 }
 
 typedef struct _DropPartitionsRequest__isset {
-  _DropPartitionsRequest__isset() : deleteData(false), ifExists(true), ignoreProtection(false), environmentContext(false), needResult(true) {}
+  _DropPartitionsRequest__isset() : deleteData(false), ifExists(true), ignoreProtection(false), environmentContext(false), needResult(true), catName(false) {}
   bool deleteData :1;
   bool ifExists :1;
   bool ignoreProtection :1;
   bool environmentContext :1;
   bool needResult :1;
+  bool catName :1;
 } _DropPartitionsRequest__isset;
 
 class DropPartitionsRequest {
@@ -5582,7 +6069,7 @@ class DropPartitionsRequest {
 
   DropPartitionsRequest(const DropPartitionsRequest&);
   DropPartitionsRequest& operator=(const DropPartitionsRequest&);
-  DropPartitionsRequest() : dbName(), tblName(), deleteData(0), ifExists(true), ignoreProtection(0), needResult(true) {
+  DropPartitionsRequest() : dbName(), tblName(), deleteData(0), ifExists(true), ignoreProtection(0), needResult(true), catName() {
   }
 
   virtual ~DropPartitionsRequest() throw();
@@ -5594,6 +6081,7 @@ class DropPartitionsRequest {
   bool ignoreProtection;
   EnvironmentContext environmentContext;
   bool needResult;
+  std::string catName;
 
   _DropPartitionsRequest__isset __isset;
 
@@ -5613,6 +6101,8 @@ class DropPartitionsRequest {
 
   void __set_needResult(const bool val);
 
+  void __set_catName(const std::string& val);
+
   bool operator == (const DropPartitionsRequest & rhs) const
   {
     if (!(dbName == rhs.dbName))
@@ -5641,6 +6131,10 @@ class DropPartitionsRequest {
       return false;
     else if (__isset.needResult && !(needResult == rhs.needResult))
       return false;
+    if (__isset.catName != rhs.__isset.catName)
+      return false;
+    else if (__isset.catName && !(catName == rhs.catName))
+      return false;
     return true;
   }
   bool operator != (const DropPartitionsRequest &rhs) const {
@@ -5664,12 +6158,13 @@ inline std::ostream& operator<<(std::ostream& out, const DropPartitionsRequest&
 }
 
 typedef struct _PartitionValuesRequest__isset {
-  _PartitionValuesRequest__isset() : applyDistinct(true), filter(false), partitionOrder(false), ascending(true), maxParts(true) {}
+  _PartitionValuesRequest__isset() : applyDistinct(true), filter(false), partitionOrder(false), ascending(true), maxParts(true), catName(false) {}
   bool applyDistinct :1;
   bool filter :1;
   bool partitionOrder :1;
   bool ascending :1;
   bool maxParts :1;
+  bool catName :1;
 } _PartitionValuesRequest__isset;
 
 class PartitionValuesRequest {
@@ -5677,7 +6172,7 @@ class PartitionValuesRequest {
 
   PartitionValuesRequest(const PartitionValuesRequest&);
   PartitionValuesRequest& operator=(const PartitionValuesRequest&);
-  PartitionValuesRequest() : dbName(), tblName(), applyDistinct(true), filter(), ascending(true), maxParts(-1LL) {
+  PartitionValuesRequest() : dbName(), tblName(), applyDistinct(true), filter(), ascending(true), maxParts(-1LL), catName() {
   }
 
   virtual ~PartitionValuesRequest() throw();
@@ -5689,6 +6184,7 @@ class PartitionValuesRequest {
   std::vector<FieldSchema>  partitionOrder;
   bool ascending;
   int64_t maxParts;
+  std::string catName;
 
   _PartitionValuesRequest__isset __isset;
 
@@ -5708,6 +6204,8 @@ class PartitionValuesRequest {
 
   void __set_maxParts(const int64_t val);
 
+  void __set_catName(const std::string& val);
+
   bool operator == (const PartitionValuesRequest & rhs) const
   {
     if (!(dbName == rhs.dbName))
@@ -5736,6 +6234,10 @@ class PartitionValuesRequest {
       return false;
     else if (__isset.maxParts && !(maxParts == rhs.maxParts))
       return false;
+    if (__isset.catName != rhs.__isset.catName)
+      return false;
+    else if (__isset.catName && !(catName == rhs.catName))
+      return false;
     return true;
   }
   bool operator != (const PartitionValuesRequest &rhs) const {
@@ -5891,7 +6393,7 @@ inline std::ostream& operator<<(std::ostream& out, const ResourceUri& obj)
 }
 
 typedef struct _Function__isset {
-  _Function__isset() : functionName(false), dbName(false), className(false), ownerName(false), ownerType(false), createTime(false), functionType(false), resourceUris(false) {}
+  _Function__isset() : functionName(false), dbName(false), className(false), ownerName(false), ownerType(false), createTime(false), functionType(false), resourceUris(false), catName(false) {}
   bool functionName :1;
   bool dbName :1;
   bool className :1;
@@ -5900,6 +6402,7 @@ typedef struct _Function__isset {
   bool createTime :1;
   bool functionType :1;
   bool resourceUris :1;
+  bool catName :1;
 } _Function__isset;
 
 class Function {
@@ -5907,7 +6410,7 @@ class Function {
 
   Function(const Function&);
   Function& operator=(const Function&);
-  Function() : functionName(), dbName(), className(), ownerName(), ownerType((PrincipalType::type)0), createTime(0), functionType((FunctionType::type)0) {
+  Function() : functionName(), dbName(), className(), ownerName(), ownerType((PrincipalType::type)0), createTime(0), functionType((FunctionType::type)0), catName() {
   }
 
   virtual ~Function() throw();
@@ -5919,6 +6422,7 @@ class Function {
   int32_t createTime;
   FunctionType::type functionType;
   std::vector<ResourceUri>  resourceUris;
+  std::string catName;
 
   _Function__isset __isset;
 
@@ -5938,6 +6442,8 @@ class Function {
 
   void __set_resourceUris(const std::vector<ResourceUri> & val);
 
+  void __set_catName(const std::string& val);
+
   bool operator == (const Function & rhs) const
   {
     if (!(functionName == rhs.functionName))
@@ -5956,6 +6462,10 @@ class Function {
       return false;
     if (!(resourceUris == rhs.resourceUris))
       return false;
+    if (__isset.catName != rhs.__isset.catName)
+      return false;
+    else if (__isset.catName && !(catName == rhs.catName))
+      return false;
     return true;
   }
   bool operator != (const Function &rhs) const {
@@ -7928,10 +8438,11 @@ class CreationMetadata {
 
   CreationMetadata(const CreationMetadata&);
   CreationMetadata& operator=(const CreationMetadata&);
-  CreationMetadata() : dbName(), tblName(), validTxnList() {
+  CreationMetadata() : catName(), dbName(), tblName(), validTxnList() {
   }
 
   virtual ~CreationMetadata() throw();
+  std::string catName;
   std::string dbName;
   std::string tblName;
   std::set<std::string>  tablesUsed;
@@ -7939,6 +8450,8 @@ class CreationMetadata {
 
   _CreationMetadata__isset __isset;
 
+  void __set_catName(const std::string& val);
+
   void __set_dbName(const std::string& val);
 
   void __set_tblName(const std::string& val);
@@ -7949,6 +8462,8 @@ class CreationMetadata {
 
   bool operator == (const CreationMetadata & rhs) const
   {
+    if (!(catName == rhs.catName))
+      return false;
     if (!(dbName == rhs.dbName))
       return false;
     if (!(tblName == rhs.tblName))
@@ -8035,10 +8550,11 @@ inline std::ostream& operator<<(std::ostream& out, const NotificationEventReques
 }
 
 typedef struct _NotificationEvent__isset {
-  _NotificationEvent__isset() : dbName(false), tableName(false), messageFormat(false) {}
+  _NotificationEvent__isset() : dbName(false), tableName(false), messageFormat(false), catName(false) {}
   bool dbName :1;
   bool tableName :1;
   bool messageFormat :1;
+  bool catName :1;
 } _NotificationEvent__isset;
 
 class NotificationEvent {
@@ -8046,7 +8562,7 @@ class NotificationEvent {
 
   NotificationEvent(const NotificationEvent&);
   NotificationEvent& operator=(const NotificationEvent&);
-  NotificationEvent() : eventId(0), eventTime(0), eventType(), dbName(), tableName(), message(), messageFormat() {
+  NotificationEvent() : eventId(0), eventTime(0), eventType(), dbName(), tableName(), message(), messageFormat(), catName() {
   }
 
   virtual ~NotificationEvent() throw();
@@ -8057,6 +8573,7 @@ class NotificationEvent {
   std::string tableName;
   std::string message;
   std::string messageFormat;
+  std::string catName;
 
   _NotificationEvent__isset __isset;
 
@@ -8074,6 +8591,8 @@ class NotificationEvent {
 
   void __set_messageFormat(const std::string& val);
 
+  void __set_catName(const std::string& val);
+
   bool operator == (const NotificationEvent & rhs) const
   {
     if (!(eventId == rhs.eventId))
@@ -8096,6 +8615,10 @@ class NotificationEvent {
       return false;
     else if (__isset.messageFormat && !(messageFormat == rhs.messageFormat))
       return false;
+    if (__isset.catName != rhs.__isset.catName)
+      return false;
+    else if (__isset.catName && !(catName == rhs.catName))
+      return false;
     return true;
   }
   bool operator != (const NotificationEvent &rhs) const {
@@ -8198,29 +8721,42 @@ inline std::ostream& operator<<(std::ostream& out, const CurrentNotificationEven
   return out;
 }
 
+typedef struct _NotificationEventsCountRequest__isset {
+  _NotificationEventsCountRequest__isset() : catName(false) {}
+  bool catName :1;
+} _NotificationEventsCountRequest__isset;
 
 class NotificationEventsCountRequest {
  public:
 
   NotificationEventsCountRequest(const NotificationEventsCountRequest&);
   NotificationEventsCountRequest& operator=(const NotificationEventsCountRequest&);
-  NotificationEventsCountRequest() : fromEventId(0), dbName() {
+  NotificationEventsCountRequest() : fromEventId(0), dbName(), catName() {
   }
 
   virtual ~NotificationEventsCountRequest() throw();
   int64_t fromEventId;
   std::string dbName;
+  std::string catName;
+
+  _NotificationEventsCountRequest__isset __isset;
 
   void __set_fromEventId(const int64_t val);
 
   void __set_dbName(const std::string& val);
 
+  void __set_catName(const std::string& val);
+
   bool operator == (const NotificationEventsCountRequest & rhs) const
   {
     if (!(fromEventId == rhs.fromEventId))
       return false;
     if (!(dbName == rhs.dbName))
       return false;
+    if (__isset.catName != rhs.__isset.catName)
+      return false;
+    else if (__isset.catName && !(catName == rhs.catName))
+      return false;
     return true;
   }
   bool operator != (const NotificationEventsCountRequest &rhs) const {
@@ -8391,10 +8927,11 @@ inline std::ostream& operator<<(std::ostream& out, const FireEventRequestData& o
 }
 
 typedef struct _FireEventRequest__isset {
-  _FireEventRequest__isset() : dbName(false), tableName(false), partitionVals(false) {}
+  _FireEventRequest__isset() : dbName(false), tableName(false), partitionVals(false), catName(false) {}
   bool dbName :1;
   bool tableName :1;
   bool partitionVals :1;
+  bool catName :1;
 } _FireEventRequest__isset;
 
 class FireEventRequest {
@@ -8402,7 +8939,7 @@ class FireEventRequest {
 
   FireEventRequest(const FireEventRequest&);
   FireEventRequest& operator=(const FireEventRequest&);
-  FireEventRequest() : successful(0), dbName(), tableName() {
+  FireEventRequest() : successful(0), dbName(), tableName(), catName() {
   }
 
   virtual ~FireEventRequest() throw();
@@ -8411,6 +8948,7 @@ class FireEventRequest {
   std::string dbName;
   std::string tableName;
   std::vector<std::string>  partitionVals;
+  std::string catName;
 
   _FireEventRequest__isset __isset;
 
@@ -8424,6 +8962,8 @@ class FireEventRequest {
 
   void __set_partitionVals(const std::vector<std::string> & val);
 
+  void __set_catName(const std::string& val);
+
   bool operator == (const FireEventRequest & rhs) const
   {
     if (!(successful == rhs.successful))
@@ -8442,6 +8982,10 @@ class FireEventRequest {
       return false;
     else if (__isset.partitionVals && !(partitionVals == rhs.partitionVals))
       return false;
+    if (__isset.catName != rhs.__isset.catName)
+      return false;
+    else if (__isset.catName && !(catName == rhs.catName))
+      return false;
     return true;
   }
   bool operator != (const FireEventRequest &rhs) const {
@@ -9114,8 +9658,9 @@ inline std::ostream& operator<<(std::ostream& out, const ClientCapabilities& obj
 }
 
 typedef struct _GetTableRequest__isset {
-  _GetTableRequest__isset() : capabilities(false) {}
+  _GetTableRequest__isset() : capabilities(false), catName(false) {}
   bool capabilities :1;
+  bool catName :1;
 } _GetTableRequest__isset;
 
 class GetTableRequest {
@@ -9123,13 +9668,14 @@ class GetTableRequest {
 
   GetTableRequest(const GetTableRequest&);
   GetTableRequest& operator=(const GetTableRequest&);
-  GetTableRequest() : dbName(), tblName() {
+  GetTableRequest() : dbName(), tblName(), catName() {
   }
 
   virtual ~GetTableRequest() throw();
   std::string dbName;
   std::string tblName;
   ClientCapabilities capabilities;
+  std::string catName;
 
   _GetTableRequest__isset __isset;
 
@@ -9139,6 +9685,8 @@ class GetTableRequest {
 
   void __set_capabilities(const ClientCapabilities& val);
 
+  void __set_catName(const std::string& val);
+
   bool operator == (const GetTableRequest & rhs) const
   {
     if (!(dbName == rhs.dbName))
@@ -9149,6 +9697,10 @@ class GetTableRequest {
       return false;
     else if (__isset.capabilities && !(capabilities == rhs.capabilities))
       return false;
+    if (__isset.catName != rhs.__isset.catName)
+      return false;
+    else if (__isset.catName && !(catName == rhs.catName))
+      return false;
     return true;
   }
   bool operator != (const GetTableRequest &rhs) const {
@@ -9212,9 +9764,10 @@ inline std::ostream& operator<<(std::ostream& out, const GetTableResult& obj)
 }
 
 typedef struct _GetTablesRequest__isset {
-  _GetTablesRequest__isset() : tblNames(false), capabilities(false) {}
+  _GetTablesRequest__isset() : tblNames(false), capabilities(false), catName(false) {}
   bool tblNames :1;
   bool capabilities :1;
+  bool catName :1;
 } _GetTablesRequest__isset;
 
 class GetTablesRequest {
@@ -9222,13 +9775,14 @@ class GetTablesRequest {
 
   GetTablesRequest(const GetTablesRequest&);
   GetTablesRequest& operator=(const GetTablesRequest&);
-  GetTablesRequest() : dbName() {
+  GetTablesRequest() : dbName(), catName() {
   }
 
   virtual ~GetTablesRequest() throw();
   std::string dbName;
   std::vector<std::string>  tblNames;
   ClientCapabilities capabilities;
+  std::string catName;
 
   _GetTablesRequest__isset __isset;
 
@@ -9238,6 +9792,8 @@ class GetTablesRequest {
 
   void __set_capabilities(const ClientCapabilities& val);
 
+  void __set_catName(const std::string& val);
+
   bool operator == (const GetTablesRequest & rhs) const
   {
     if (!(dbName == rhs.dbName))
@@ -9250,6 +9806,10 @@ class GetTablesRequest {
       return false;
     else if (__isset.capabilities && !(capabilities == rhs.capabilities))
       return false;
+    if (__isset.catName != rhs.__isset.catName)
+      return false;
+    else if (__isset.catName && !(catName == rhs.catName))
+      return false;
     return true;
   }
   bool operator != (const GetTablesRequest &rhs) const {
@@ -9393,8 +9953,9 @@ inline std::ostream& operator<<(std::ostream& out, const CmRecycleResponse& obj)
 }
 
 typedef struct _TableMeta__isset {
-  _TableMeta__isset() : comments(false) {}
+  _TableMeta__isset() : comments(false), catName(false) {}
   bool comments :1;
+  bool catName :1;
 } _TableMeta__isset;
 
 class TableMeta {
@@ -9402,7 +9963,7 @@ class TableMeta {
 
   TableMeta(const TableMeta&);
   TableMeta& operator=(const TableMeta&);
-  TableMeta() : dbName(), tableName(), tableType(), comments() {
+  TableMeta() : dbName(), tableName(), tableType(), comments(), catName() {
   }
 
   virtual ~TableMeta() throw();
@@ -9410,6 +9971,7 @@ class TableMeta {
   std::string tableName;
   std::string tableType;
   std::string comments;
+  std::string catName;
 
   _TableMeta__isset __isset;
 
@@ -9421,6 +9983,8 @@ class TableMeta {
 
   void __set_comments(const std::string& val);
 
+  void __set_catName(const std::string& val);
+
   bool operator == (const TableMeta & rhs) const
   {
     if (!(dbName == rhs.dbName))
@@ -9433,6 +9997,10 @@ class TableMeta {
       return false;
     else if (__isset.comments && !(comments == rhs.comments))
       return false;
+    if (__isset.catName != rhs.__isset.catName)
+      return false;
+    else if (__isset.catName && !(catName == rhs.catName))
+      return false;
     return true;
   }
   bool operator != (const TableMeta &rhs) const {
@@ -11658,9 +12226,10 @@ inline std::ostream& operator<<(std::ostream& out, const WMCreateOrDropTriggerTo
 }
 
 typedef struct _ISchema__isset {
-  _ISchema__isset() : schemaType(false), name(false), dbName(false), compatibility(false), validationLevel(false), canEvolve(false), schemaGroup(false), description(false) {}
+  _ISchema__isset() : schemaType(false), name(false), catName(false), dbName(false), compatibility(false), validationLevel(false), canEvolve(false), schemaGroup(false), description(false) {}
   bool schemaType :1;
   bool name :1;
+  bool catName :1;
   bool dbName :1;
   bool compatibility :1;
   bool validationLevel :1;
@@ -11674,12 +12243,13 @@ class ISchema {
 
   ISchema(const ISchema&);
   ISchema& operator=(const ISchema&);
-  ISchema() : schemaType((SchemaType::type)0), name(), dbName(), compatibility((SchemaCompatibility::type)0), validationLevel((SchemaValidation::type)0), canEvolve(0), schemaGroup(), description() {
+  ISchema() : schemaType((SchemaType::type)0), name(), catName(), dbName(), compatibility((SchemaCompatibility::type)0), validationLevel((SchemaValidation::type)0), canEvolve(0), schemaGroup(), description() {
   }
 
   virtual ~ISchema() throw();
   SchemaType::type schemaType;
   std::string name;
+  std::string catName;
   std::string dbName;
   SchemaCompatibility::type compatibility;
   SchemaValidation::type validationLevel;
@@ -11693,6 +12263,8 @@ class ISchema {
 
   void __set_name(const std::string& val);
 
+  void __set_catName(const std::string& val);
+
   void __set_dbName(const std::string& val);
 
   void __set_compatibility(const SchemaCompatibility::type val);
@@ -11711,6 +12283,8 @@ class ISchema {
       return false;
     if (!(name == rhs.name))
       return false;
+    if (!(catName == rhs.catName))
+      return false;
     if (!(dbName == rhs.dbName))
       return false;
     if (!(compatibility == rhs.compatibility))
@@ -11750,7 +12324,8 @@ inline std::ostream& operator<<(std::ostream& out, const ISchema& obj)
 }
 
 typedef struct _ISchemaName__isset {
-  _ISchemaName__isset() : dbName(false), schemaName(false) {}
+  _ISchemaName__isset() : catName(false), dbName(false), schemaName(false) {}
+  bool catName :1;
   bool dbName :1;
   bool schemaName :1;
 } _ISchemaName__isset;
@@ -11760,21 +12335,26 @@ class ISchemaName {
 
   ISchemaName(const ISchemaName&);
   ISchemaName& operator=(const ISchemaName&);
-  ISchemaName() : dbName(), schemaName() {
+  ISchemaName() : catName(), dbName(), schemaName() {
   }
 
   virtual ~ISchemaName() throw();
+  std::string catName;
   std::string dbName;
   std::string schemaName;
 
   _ISchemaName__isset __isset;
 
+  void __set_catName(const std::string& val);
+
   void __set_dbName(const std::string& val);
 
   void __set_schemaName(const std::string& val);
 
   bool operator == (const ISchemaName & rhs) const
   {
+    if (!(catName == rhs.catName))
+      return false;
     if (!(dbName == rhs.dbName))
       return false;
     if (!(schemaName == rhs.schemaName))

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnsRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnsRequest.java
index 911e981..24c68d8 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnsRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnsRequest.java
@@ -351,13 +351,13 @@ import org.slf4j.LoggerFactory;
           case 1: // TXN_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list570 = iprot.readListBegin();
-                struct.txn_ids = new ArrayList<Long>(_list570.size);
-                long _elem571;
-                for (int _i572 = 0; _i572 < _list570.size; ++_i572)
+                org.apache.thrift.protocol.TList _list578 = iprot.readListBegin();
+                struct.txn_ids = new ArrayList<Long>(_list578.size);
+                long _elem579;
+                for (int _i580 = 0; _i580 < _list578.size; ++_i580)
                 {
-                  _elem571 = iprot.readI64();
-                  struct.txn_ids.add(_elem571);
+                  _elem579 = iprot.readI64();
+                  struct.txn_ids.add(_elem579);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TXN_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.txn_ids.size()));
-          for (long _iter573 : struct.txn_ids)
+          for (long _iter581 : struct.txn_ids)
           {
-            oprot.writeI64(_iter573);
+            oprot.writeI64(_iter581);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.txn_ids.size());
-        for (long _iter574 : struct.txn_ids)
+        for (long _iter582 : struct.txn_ids)
         {
-          oprot.writeI64(_iter574);
+          oprot.writeI64(_iter582);
         }
       }
     }
@@ -421,13 +421,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, AbortTxnsRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list575 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.txn_ids = new ArrayList<Long>(_list575.size);
-        long _elem576;
-        for (int _i577 = 0; _i577 < _list575.size; ++_i577)
+        org.apache.thrift.protocol.TList _list583 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.txn_ids = new ArrayList<Long>(_list583.size);
+        long _elem584;
+        for (int _i585 = 0; _i585 < _list583.size; ++_i585)
         {
-          _elem576 = iprot.readI64();
-          struct.txn_ids.add(_elem576);
+          _elem584 = iprot.readI64();
+          struct.txn_ids.add(_elem584);
         }
       }
       struct.setTxn_idsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddCheckConstraintRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddCheckConstraintRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddCheckConstraintRequest.java
index 8a37ee4..02d552d 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddCheckConstraintRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddCheckConstraintRequest.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // CHECK_CONSTRAINT_COLS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list400 = iprot.readListBegin();
-                struct.checkConstraintCols = new ArrayList<SQLCheckConstraint>(_list400.size);
-                SQLCheckConstraint _elem401;
-                for (int _i402 = 0; _i402 < _list400.size; ++_i402)
+                org.apache.thrift.protocol.TList _list408 = iprot.readListBegin();
+                struct.checkConstraintCols = new ArrayList<SQLCheckConstraint>(_list408.size);
+                SQLCheckConstraint _elem409;
+                for (int _i410 = 0; _i410 < _list408.size; ++_i410)
                 {
-                  _elem401 = new SQLCheckConstraint();
-                  _elem401.read(iprot);
-                  struct.checkConstraintCols.add(_elem401);
+                  _elem409 = new SQLCheckConstraint();
+                  _elem409.read(iprot);
+                  struct.checkConstraintCols.add(_elem409);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(CHECK_CONSTRAINT_COLS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.checkConstraintCols.size()));
-          for (SQLCheckConstraint _iter403 : struct.checkConstraintCols)
+          for (SQLCheckConstraint _iter411 : struct.checkConstraintCols)
           {
-            _iter403.write(oprot);
+            _iter411.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.checkConstraintCols.size());
-        for (SQLCheckConstraint _iter404 : struct.checkConstraintCols)
+        for (SQLCheckConstraint _iter412 : struct.checkConstraintCols)
         {
-          _iter404.write(oprot);
+          _iter412.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, AddCheckConstraintRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list405 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.checkConstraintCols = new ArrayList<SQLCheckConstraint>(_list405.size);
-        SQLCheckConstraint _elem406;
-        for (int _i407 = 0; _i407 < _list405.size; ++_i407)
+        org.apache.thrift.protocol.TList _list413 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.checkConstraintCols = new ArrayList<SQLCheckConstraint>(_list413.size);
+        SQLCheckConstraint _elem414;
+        for (int _i415 = 0; _i415 < _list413.size; ++_i415)
         {
-          _elem406 = new SQLCheckConstraint();
-          _elem406.read(iprot);
-          struct.checkConstraintCols.add(_elem406);
+          _elem414 = new SQLCheckConstraint();
+          _elem414.read(iprot);
+          struct.checkConstraintCols.add(_elem414);
         }
       }
       struct.setCheckConstraintColsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDefaultConstraintRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDefaultConstraintRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDefaultConstraintRequest.java
index b4b9cf2..6acc6f8 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDefaultConstraintRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDefaultConstraintRequest.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // DEFAULT_CONSTRAINT_COLS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list392 = iprot.readListBegin();
-                struct.defaultConstraintCols = new ArrayList<SQLDefaultConstraint>(_list392.size);
-                SQLDefaultConstraint _elem393;
-                for (int _i394 = 0; _i394 < _list392.size; ++_i394)
+                org.apache.thrift.protocol.TList _list400 = iprot.readListBegin();
+                struct.defaultConstraintCols = new ArrayList<SQLDefaultConstraint>(_list400.size);
+                SQLDefaultConstraint _elem401;
+                for (int _i402 = 0; _i402 < _list400.size; ++_i402)
                 {
-                  _elem393 = new SQLDefaultConstraint();
-                  _elem393.read(iprot);
-                  struct.defaultConstraintCols.add(_elem393);
+                  _elem401 = new SQLDefaultConstraint();
+                  _elem401.read(iprot);
+                  struct.defaultConstraintCols.add(_elem401);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(DEFAULT_CONSTRAINT_COLS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.defaultConstraintCols.size()));
-          for (SQLDefaultConstraint _iter395 : struct.defaultConstraintCols)
+          for (SQLDefaultConstraint _iter403 : struct.defaultConstraintCols)
           {
-            _iter395.write(oprot);
+            _iter403.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.defaultConstraintCols.size());
-        for (SQLDefaultConstraint _iter396 : struct.defaultConstraintCols)
+        for (SQLDefaultConstraint _iter404 : struct.defaultConstraintCols)
         {
-          _iter396.write(oprot);
+          _iter404.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, AddDefaultConstraintRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list397 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.defaultConstraintCols = new ArrayList<SQLDefaultConstraint>(_list397.size);
-        SQLDefaultConstraint _elem398;
-        for (int _i399 = 0; _i399 < _list397.size; ++_i399)
+        org.apache.thrift.protocol.TList _list405 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.defaultConstraintCols = new ArrayList<SQLDefaultConstraint>(_list405.size);
+        SQLDefaultConstraint _elem406;
+        for (int _i407 = 0; _i407 < _list405.size; ++_i407)
         {
-          _elem398 = new SQLDefaultConstraint();
-          _elem398.read(iprot);
-          struct.defaultConstraintCols.add(_elem398);
+          _elem406 = new SQLDefaultConstraint();
+          _elem406.read(iprot);
+          struct.defaultConstraintCols.add(_elem406);
         }
       }
       struct.setDefaultConstraintColsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
index 374cdc3..d6a071a 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
@@ -816,13 +816,13 @@ import org.slf4j.LoggerFactory;
           case 5: // PARTITIONNAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list668 = iprot.readListBegin();
-                struct.partitionnames = new ArrayList<String>(_list668.size);
-                String _elem669;
-                for (int _i670 = 0; _i670 < _list668.size; ++_i670)
+                org.apache.thrift.protocol.TList _list676 = iprot.readListBegin();
+                struct.partitionnames = new ArrayList<String>(_list676.size);
+                String _elem677;
+                for (int _i678 = 0; _i678 < _list676.size; ++_i678)
                 {
-                  _elem669 = iprot.readString();
-                  struct.partitionnames.add(_elem669);
+                  _elem677 = iprot.readString();
+                  struct.partitionnames.add(_elem677);
                 }
                 iprot.readListEnd();
               }
@@ -872,9 +872,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PARTITIONNAMES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partitionnames.size()));
-          for (String _iter671 : struct.partitionnames)
+          for (String _iter679 : struct.partitionnames)
           {
-            oprot.writeString(_iter671);
+            oprot.writeString(_iter679);
           }
           oprot.writeListEnd();
         }
@@ -910,9 +910,9 @@ import org.slf4j.LoggerFactory;
       oprot.writeString(struct.tablename);
       {
         oprot.writeI32(struct.partitionnames.size());
-        for (String _iter672 : struct.partitionnames)
+        for (String _iter680 : struct.partitionnames)
         {
-          oprot.writeString(_iter672);
+          oprot.writeString(_iter680);
         }
       }
       BitSet optionals = new BitSet();
@@ -937,13 +937,13 @@ import org.slf4j.LoggerFactory;
       struct.tablename = iprot.readString();
       struct.setTablenameIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list673 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.partitionnames = new ArrayList<String>(_list673.size);
-        String _elem674;
-        for (int _i675 = 0; _i675 < _list673.size; ++_i675)
+        org.apache.thrift.protocol.TList _list681 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.partitionnames = new ArrayList<String>(_list681.size);
+        String _elem682;
+        for (int _i683 = 0; _i683 < _list681.size; ++_i683)
         {
-          _elem674 = iprot.readString();
-          struct.partitionnames.add(_elem674);
+          _elem682 = iprot.readString();
+          struct.partitionnames.add(_elem682);
         }
       }
       struct.setPartitionnamesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddForeignKeyRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddForeignKeyRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddForeignKeyRequest.java
index 9a2087c..f57eb3b 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddForeignKeyRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddForeignKeyRequest.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // FOREIGN_KEY_COLS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list368 = iprot.readListBegin();
-                struct.foreignKeyCols = new ArrayList<SQLForeignKey>(_list368.size);
-                SQLForeignKey _elem369;
-                for (int _i370 = 0; _i370 < _list368.size; ++_i370)
+                org.apache.thrift.protocol.TList _list376 = iprot.readListBegin();
+                struct.foreignKeyCols = new ArrayList<SQLForeignKey>(_list376.size);
+                SQLForeignKey _elem377;
+                for (int _i378 = 0; _i378 < _list376.size; ++_i378)
                 {
-                  _elem369 = new SQLForeignKey();
-                  _elem369.read(iprot);
-                  struct.foreignKeyCols.add(_elem369);
+                  _elem377 = new SQLForeignKey();
+                  _elem377.read(iprot);
+                  struct.foreignKeyCols.add(_elem377);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FOREIGN_KEY_COLS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.foreignKeyCols.size()));
-          for (SQLForeignKey _iter371 : struct.foreignKeyCols)
+          for (SQLForeignKey _iter379 : struct.foreignKeyCols)
           {
-            _iter371.write(oprot);
+            _iter379.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.foreignKeyCols.size());
-        for (SQLForeignKey _iter372 : struct.foreignKeyCols)
+        for (SQLForeignKey _iter380 : struct.foreignKeyCols)
         {
-          _iter372.write(oprot);
+          _iter380.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, AddForeignKeyRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list373 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.foreignKeyCols = new ArrayList<SQLForeignKey>(_list373.size);
-        SQLForeignKey _elem374;
-        for (int _i375 = 0; _i375 < _list373.size; ++_i375)
+        org.apache.thrift.protocol.TList _list381 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.foreignKeyCols = new ArrayList<SQLForeignKey>(_list381.size);
+        SQLForeignKey _elem382;
+        for (int _i383 = 0; _i383 < _list381.size; ++_i383)
         {
-          _elem374 = new SQLForeignKey();
-          _elem374.read(iprot);
-          struct.foreignKeyCols.add(_elem374);
+          _elem382 = new SQLForeignKey();
+          _elem382.read(iprot);
+          struct.foreignKeyCols.add(_elem382);
         }
       }
       struct.setForeignKeyColsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddNotNullConstraintRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddNotNullConstraintRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddNotNullConstraintRequest.java
index d3d771c..e6bac16 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddNotNullConstraintRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddNotNullConstraintRequest.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // NOT_NULL_CONSTRAINT_COLS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list384 = iprot.readListBegin();
-                struct.notNullConstraintCols = new ArrayList<SQLNotNullConstraint>(_list384.size);
-                SQLNotNullConstraint _elem385;
-                for (int _i386 = 0; _i386 < _list384.size; ++_i386)
+                org.apache.thrift.protocol.TList _list392 = iprot.readListBegin();
+                struct.notNullConstraintCols = new ArrayList<SQLNotNullConstraint>(_list392.size);
+                SQLNotNullConstraint _elem393;
+                for (int _i394 = 0; _i394 < _list392.size; ++_i394)
                 {
-                  _elem385 = new SQLNotNullConstraint();
-                  _elem385.read(iprot);
-                  struct.notNullConstraintCols.add(_elem385);
+                  _elem393 = new SQLNotNullConstraint();
+                  _elem393.read(iprot);
+                  struct.notNullConstraintCols.add(_elem393);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(NOT_NULL_CONSTRAINT_COLS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.notNullConstraintCols.size()));
-          for (SQLNotNullConstraint _iter387 : struct.notNullConstraintCols)
+          for (SQLNotNullConstraint _iter395 : struct.notNullConstraintCols)
           {
-            _iter387.write(oprot);
+            _iter395.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.notNullConstraintCols.size());
-        for (SQLNotNullConstraint _iter388 : struct.notNullConstraintCols)
+        for (SQLNotNullConstraint _iter396 : struct.notNullConstraintCols)
         {
-          _iter388.write(oprot);
+          _iter396.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, AddNotNullConstraintRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list389 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.notNullConstraintCols = new ArrayList<SQLNotNullConstraint>(_list389.size);
-        SQLNotNullConstraint _elem390;
-        for (int _i391 = 0; _i391 < _list389.size; ++_i391)
+        org.apache.thrift.protocol.TList _list397 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.notNullConstraintCols = new ArrayList<SQLNotNullConstraint>(_list397.size);
+        SQLNotNullConstraint _elem398;
+        for (int _i399 = 0; _i399 < _list397.size; ++_i399)
         {
-          _elem390 = new SQLNotNullConstraint();
-          _elem390.read(iprot);
-          struct.notNullConstraintCols.add(_elem390);
+          _elem398 = new SQLNotNullConstraint();
+          _elem398.read(iprot);
+          struct.notNullConstraintCols.add(_elem398);
         }
       }
       struct.setNotNullConstraintColsIsSet(true);


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
index c69192b..2e146f3 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
@@ -27,6 +27,7 @@ import java.util.Map.Entry;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.common.ValidTxnWriteIdList;
@@ -37,6 +38,7 @@ import org.apache.hadoop.hive.metastore.api.AggrStats;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
 import org.apache.hadoop.hive.metastore.api.BasicTxnInfo;
 import org.apache.hadoop.hive.metastore.api.CheckConstraintsRequest;
+import org.apache.hadoop.hive.metastore.api.Catalog;
 import org.apache.hadoop.hive.metastore.api.CmRecycleRequest;
 import org.apache.hadoop.hive.metastore.api.CmRecycleResponse;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
@@ -125,6 +127,7 @@ import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMTrigger;
 import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
 import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.utils.ObjectPair;
 import org.apache.thrift.TException;
 
@@ -176,77 +179,219 @@ public interface IMetaStoreClient {
   String getMetaConf(String key) throws MetaException, TException;
 
   /**
-   * Get the names of all databases in the MetaStore that match the given pattern.
-   * @param databasePattern
+   * Create a new catalog.
+   * @param catalog catalog object to create.
+   * @throws AlreadyExistsException A catalog of this name already exists.
+   * @throws InvalidObjectException There is something wrong with the passed in catalog object.
+   * @throws MetaException something went wrong, usually either in the database or trying to
+   * create the directory for the catalog.
+   * @throws TException general thrift exception.
+   */
+  void createCatalog(Catalog catalog)
+      throws AlreadyExistsException, InvalidObjectException, MetaException, TException;
+
+  /**
+   * Get a catalog object.
+   * @param catName Name of the catalog to fetch.
+   * @return The catalog.
+   * @throws NoSuchObjectException no catalog of this name exists.
+   * @throws MetaException something went wrong, usually in the database.
+   * @throws TException general thrift exception.
+   */
+  Catalog getCatalog(String catName) throws NoSuchObjectException, MetaException, TException;
+
+  /**
+   * Get a list of all catalogs known to the system.
+   * @return list of catalog names
+   * @throws MetaException something went wrong, usually in the database.
+   * @throws TException general thrift exception.
+   */
+  List<String> getCatalogs() throws MetaException, TException;
+
+  /**
+   * Drop a catalog.  Catalogs must be empty to be dropped, there is no cascade for dropping a
+   * catalog.
+   * @param catName name of the catalog to drop
+   * @throws NoSuchObjectException no catalog of this name exists.
+   * @throws InvalidOperationException The catalog is not empty and cannot be dropped.
+   * @throws MetaException something went wrong, usually in the database.
+   * @throws TException general thrift exception.
+   */
+  void dropCatalog(String catName)
+      throws NoSuchObjectException, InvalidOperationException, MetaException, TException;
+
+  /**
+   * Get the names of all databases in the default catalog that match the given pattern.
+   * @param databasePattern pattern for the database name to patch
    * @return List of database names.
-   * @throws MetaException
-   * @throws TException
+   * @throws MetaException error accessing RDBMS.
+   * @throws TException thrift transport error
    */
   List<String> getDatabases(String databasePattern) throws MetaException, TException;
 
   /**
+   * Get all databases in a catalog whose names match a pattern.
+   * @param catName  catalog name.  Can be null, in which case the default catalog is assumed.
+   * @param databasePattern pattern for the database name to match
+   * @return list of database names
+   * @throws MetaException error accessing RDBMS.
+   * @throws TException thrift transport error
+   */
+  List<String> getDatabases(String catName, String databasePattern)
+      throws MetaException, TException;
+
+  /**
    * Get the names of all databases in the MetaStore.
-   * @return List of database names.
-   * @throws MetaException
-   * @throws TException
+   * @return List of database names in the default catalog.
+   * @throws MetaException error accessing RDBMS.
+   * @throws TException thrift transport error
    */
   List<String> getAllDatabases() throws MetaException, TException;
 
   /**
+   * Get all databases in a catalog.
+   * @param catName catalog name.  Can be null, in which case the default catalog is assumed.
+   * @return list of all database names
+   * @throws MetaException error accessing RDBMS.
+   * @throws TException thrift transport error
+   */
+  List<String> getAllDatabases(String catName) throws MetaException, TException;
+
+  /**
    * Get the names of all tables in the specified database that satisfy the supplied
    * table name pattern.
-   * @param dbName
-   * @param tablePattern
+   * @param dbName database name.
+   * @param tablePattern pattern for table name to conform to
    * @return List of table names.
-   * @throws MetaException
-   * @throws TException
-   * @throws UnknownDBException
+   * @throws MetaException error fetching information from the RDBMS
+   * @throws TException thrift transport error
+   * @throws UnknownDBException indicated database to search in does not exist.
    */
   List<String> getTables(String dbName, String tablePattern)
       throws MetaException, TException, UnknownDBException;
 
   /**
    * Get the names of all tables in the specified database that satisfy the supplied
+   * table name pattern.
+   * @param catName catalog name.
+   * @param dbName database name.
+   * @param tablePattern pattern for table name to conform to
+   * @return List of table names.
+   * @throws MetaException error fetching information from the RDBMS
+   * @throws TException general thrift error
+   * @throws UnknownDBException indicated database to search in does not exist.
+   */
+  List<String> getTables(String catName, String dbName, String tablePattern)
+      throws MetaException, TException, UnknownDBException;
+
+
+  /**
+   * Get the names of all tables in the specified database that satisfy the supplied
    * table name pattern and table type (MANAGED_TABLE || EXTERNAL_TABLE || VIRTUAL_VIEW)
    * @param dbName Name of the database to fetch tables in.
    * @param tablePattern pattern to match for table names.
    * @param tableType Type of the table in the HMS store. VIRTUAL_VIEW is for views.
    * @return List of table names.
-   * @throws MetaException
-   * @throws TException
-   * @throws UnknownDBException
+   * @throws MetaException error fetching information from the RDBMS
+   * @throws TException thrift transport error
+   * @throws UnknownDBException indicated database does not exist.
    */
   List<String> getTables(String dbName, String tablePattern, TableType tableType)
       throws MetaException, TException, UnknownDBException;
 
   /**
-   * Get materialized views that have rewriting enabled.
+   * Get the names of all tables in the specified database that satisfy the supplied
+   * table name pattern and table type (MANAGED_TABLE || EXTERNAL_TABLE || VIRTUAL_VIEW)
+   * @param catName catalog name.
+   * @param dbName Name of the database to fetch tables in.
+   * @param tablePattern pattern to match for table names.
+   * @param tableType Type of the table in the HMS store. VIRTUAL_VIEW is for views.
+   * @return List of table names.
+   * @throws MetaException error fetching information from the RDBMS
+   * @throws TException thrift transport error
+   * @throws UnknownDBException indicated database does not exist.
+   */
+  List<String> getTables(String catName, String dbName, String tablePattern, TableType tableType)
+      throws MetaException, TException, UnknownDBException;
+
+  /**
+   * Get materialized views that have rewriting enabled.  This will use the default catalog.
    * @param dbName Name of the database to fetch materialized views from.
    * @return List of materialized view names.
-   * @throws MetaException
-   * @throws TException
-   * @throws UnknownDBException
+   * @throws MetaException error fetching from the RDBMS
+   * @throws TException thrift transport error
+   * @throws UnknownDBException no such database
    */
   List<String> getMaterializedViewsForRewriting(String dbName)
       throws MetaException, TException, UnknownDBException;
 
   /**
-   * For quick GetTablesOperation
+   * Get materialized views that have rewriting enabled.
+   * @param catName catalog name.
+   * @param dbName Name of the database to fetch materialized views from.
+   * @return List of materialized view names.
+   * @throws MetaException error fetching from the RDBMS
+   * @throws TException thrift transport error
+   * @throws UnknownDBException no such database
+   */
+  List<String> getMaterializedViewsForRewriting(String catName, String dbName)
+      throws MetaException, TException, UnknownDBException;
+
+  /**
+   * Fetches just table name and comments.  Useful when you need full table name
+   * (catalog.database.table) but don't need extra information like partition columns that
+   * require additional fetches from the database.
+   * @param dbPatterns database pattern to match, or null for all databases
+   * @param tablePatterns table pattern to match.
+   * @param tableTypes list of table types to fetch.
+   * @return list of TableMeta objects with information on matching tables
+   * @throws MetaException something went wrong with the fetch from the RDBMS
+   * @throws TException thrift transport error
+   * @throws UnknownDBException No databases match the provided pattern.
    */
   List<TableMeta> getTableMeta(String dbPatterns, String tablePatterns, List<String> tableTypes)
       throws MetaException, TException, UnknownDBException;
 
   /**
+   * Fetches just table name and comments.  Useful when you need full table name
+   * (catalog.database.table) but don't need extra information like partition columns that
+   * require additional fetches from the database.
+   * @param catName catalog to search in.  Search cannot cross catalogs.
+   * @param dbPatterns database pattern to match, or null for all databases
+   * @param tablePatterns table pattern to match.
+   * @param tableTypes list of table types to fetch.
+   * @return list of TableMeta objects with information on matching tables
+   * @throws MetaException something went wrong with the fetch from the RDBMS
+   * @throws TException thrift transport error
+   * @throws UnknownDBException No databases match the provided pattern.
+   */
+  List<TableMeta> getTableMeta(String catName, String dbPatterns, String tablePatterns,
+                               List<String> tableTypes)
+      throws MetaException, TException, UnknownDBException;
+
+  /**
    * Get the names of all tables in the specified database.
-   * @param dbName
+   * @param dbName database name
    * @return List of table names.
-   * @throws MetaException
-   * @throws TException
-   * @throws UnknownDBException
+   * @throws MetaException something went wrong with the fetch from the RDBMS
+   * @throws TException thrift transport error
+   * @throws UnknownDBException No databases match the provided pattern.
    */
   List<String> getAllTables(String dbName) throws MetaException, TException, UnknownDBException;
 
   /**
+   * Get the names of all tables in the specified database.
+   * @param catName catalog name
+   * @param dbName database name
+   * @return List of table names.
+   * @throws MetaException something went wrong with the fetch from the RDBMS
+   * @throws TException thrift transport error
+   * @throws UnknownDBException No databases match the provided pattern.
+   */
+  List<String> getAllTables(String catName, String dbName)
+      throws MetaException, TException, UnknownDBException;
+
+  /**
    * Get a list of table names that match a filter.
    * The filter operators are LIKE, &lt;, &lt;=, &gt;, &gt;=, =, &lt;&gt;
    *
@@ -281,10 +426,55 @@ public interface IMetaStoreClient {
    * @param maxTables
    *          The maximum number of tables returned
    * @return  A list of table names that match the desired filter
+   * @throws InvalidOperationException invalid filter
+   * @throws UnknownDBException no such database
+   * @throws TException thrift transport error
    */
   List<String> listTableNamesByFilter(String dbName, String filter, short maxTables)
-      throws MetaException, TException, InvalidOperationException, UnknownDBException;
+      throws TException, InvalidOperationException, UnknownDBException;
 
+  /**
+   * Get a list of table names that match a filter.
+   * The filter operators are LIKE, &lt;, &lt;=, &gt;, &gt;=, =, &lt;&gt;
+   *
+   * In the filter statement, values interpreted as strings must be enclosed in quotes,
+   * while values interpreted as integers should not be.  Strings and integers are the only
+   * supported value types.
+   *
+   * The currently supported key names in the filter are:
+   * Constants.HIVE_FILTER_FIELD_OWNER, which filters on the tables' owner's name
+   *   and supports all filter operators
+   * Constants.HIVE_FILTER_FIELD_LAST_ACCESS, which filters on the last access times
+   *   and supports all filter operators except LIKE
+   * Constants.HIVE_FILTER_FIELD_PARAMS, which filters on the tables' parameter keys and values
+   *   and only supports the filter operators = and &lt;&gt;.
+   *   Append the parameter key name to HIVE_FILTER_FIELD_PARAMS in the filter statement.
+   *   For example, to filter on parameter keys called "retention", the key name in the filter
+   *   statement should be Constants.HIVE_FILTER_FIELD_PARAMS + "retention"
+   *   Also, = and &lt;&gt; only work for keys that exist in the tables.
+   *   E.g., filtering on tables where key1 &lt;&gt; value will only
+   *   return tables that have a value for the parameter key1.
+   * Some example filter statements include:
+   * filter = Constants.HIVE_FILTER_FIELD_OWNER + " like \".*test.*\" and " +
+   *   Constants.HIVE_FILTER_FIELD_LAST_ACCESS + " = 0";
+   * filter = Constants.HIVE_FILTER_FIELD_OWNER + " = \"test_user\" and (" +
+   *   Constants.HIVE_FILTER_FIELD_PARAMS + "retention = \"30\" or " +
+   *   Constants.HIVE_FILTER_FIELD_PARAMS + "retention = \"90\")"
+   *
+   * @param catName catalog name
+   * @param dbName
+   *          The name of the database from which you will retrieve the table names
+   * @param filter
+   *          The filter string
+   * @param maxTables
+   *          The maximum number of tables returned
+   * @return  A list of table names that match the desired filter
+   * @throws InvalidOperationException invalid filter
+   * @throws UnknownDBException no such database
+   * @throws TException thrift transport error
+   */
+  List<String> listTableNamesByFilter(String catName, String dbName, String filter, int maxTables)
+      throws TException, InvalidOperationException, UnknownDBException;
 
   /**
    * Drop the table.
@@ -303,50 +493,109 @@ public interface IMetaStoreClient {
    *           The table wasn't found.
    * @throws TException
    *           A thrift communication error occurred
+   *
    */
   void dropTable(String dbname, String tableName, boolean deleteData,
       boolean ignoreUnknownTab) throws MetaException, TException,
       NoSuchObjectException;
 
   /**
+   * Drop the table.
+   *
+   * @param dbname
+   *          The database for this table
+   * @param tableName
+   *          The table to drop
+   * @param deleteData
+   *          Should we delete the underlying data
+   * @param ignoreUnknownTab
+   *          don't throw if the requested table doesn't exist
    * @param ifPurge
    *          completely purge the table (skipping trash) while removing data from warehouse
-   * @see #dropTable(String, String, boolean, boolean)
+   * @throws MetaException
+   *           Could not drop table properly.
+   * @throws NoSuchObjectException
+   *           The table wasn't found.
+   * @throws TException
+   *           A thrift communication error occurred
    */
-  public void dropTable(String dbname, String tableName, boolean deleteData,
+  void dropTable(String dbname, String tableName, boolean deleteData,
       boolean ignoreUnknownTab, boolean ifPurge) throws MetaException, TException,
       NoSuchObjectException;
 
   /**
-   * Drop the table in the DEFAULT database.
+   * Drop the table.
    *
+   * @param dbname
+   *          The database for this table
    * @param tableName
    *          The table to drop
-   * @param deleteData
-   *          Should we delete the underlying data
    * @throws MetaException
    *           Could not drop table properly.
-   * @throws UnknownTableException
+   * @throws NoSuchObjectException
    *           The table wasn't found.
    * @throws TException
    *           A thrift communication error occurred
-   * @throws NoSuchObjectException
-   *           The table wasn't found.
-   *
-   * @deprecated As of release 0.6.0 replaced by {@link #dropTable(String, String, boolean, boolean)}.
-   *             This method will be removed in release 0.7.0.
-   */
-  @Deprecated
-  void dropTable(String tableName, boolean deleteData)
-      throws MetaException, UnknownTableException, TException, NoSuchObjectException;
-
-  /**
-   * @see #dropTable(String, String, boolean, boolean)
    */
   void dropTable(String dbname, String tableName)
       throws MetaException, TException, NoSuchObjectException;
 
   /**
+   * Drop a table.
+   * @param catName catalog the table is in.
+   * @param dbName database the table is in.
+   * @param tableName table name.
+   * @param deleteData whether associated data should be deleted.
+   * @param ignoreUnknownTable whether a non-existent table name should be ignored
+   * @param ifPurge whether dropped data should be immediately removed rather than placed in HDFS
+   *               trash.
+   * @throws MetaException something went wrong, usually in the RDBMS or storage.
+   * @throws NoSuchObjectException No table of this name exists, only thrown if
+   * ignoreUnknownTable is false.
+   * @throws TException general thrift error.
+   */
+  void dropTable(String catName, String dbName, String tableName, boolean deleteData,
+                 boolean ignoreUnknownTable, boolean ifPurge)
+    throws MetaException, NoSuchObjectException, TException;
+
+  /**
+   * Drop a table.  Equivalent to
+   * {@link #dropTable(String, String, String, boolean, boolean, boolean)} with ifPurge set to
+   * false.
+   * @param catName catalog the table is in.
+   * @param dbName database the table is in.
+   * @param tableName table name.
+   * @param deleteData whether associated data should be deleted.
+   * @param ignoreUnknownTable whether a non-existent table name should be ignored
+   * @throws MetaException something went wrong, usually in the RDBMS or storage.
+   * @throws NoSuchObjectException No table of this name exists, only thrown if
+   * ignoreUnknownTable is false.
+   * @throws TException general thrift error.
+   */
+  default void dropTable(String catName, String dbName, String tableName, boolean deleteData,
+                         boolean ignoreUnknownTable)
+    throws MetaException, NoSuchObjectException, TException {
+    dropTable(catName, dbName, tableName, deleteData, ignoreUnknownTable, false);
+  }
+
+  /**
+   * Drop a table.  Equivalent to
+   * {@link #dropTable(String, String, String, boolean, boolean, boolean)} with deleteData
+   * set and ignoreUnknownTable set to true and ifPurge set to false.
+   * @param catName catalog the table is in.
+   * @param dbName database the table is in.
+   * @param tableName table name.
+   * @throws MetaException something went wrong, usually in the RDBMS or storage.
+   * @throws NoSuchObjectException No table of this name exists, only thrown if
+   * ignoreUnknownTable is false.
+   * @throws TException general thrift error.
+   */
+  default void dropTable(String catName, String dbName, String tableName)
+      throws MetaException, NoSuchObjectException, TException {
+    dropTable(catName, dbName, tableName, true, true, false);
+  }
+
+  /**
    * Truncate the table/partitions in the DEFAULT database.
    * @param dbName
    *          The db to which the table to be truncate belongs to
@@ -354,13 +603,27 @@ public interface IMetaStoreClient {
    *          The table to truncate
    * @param partNames
    *          List of partitions to truncate. NULL will truncate the whole table/all partitions
-   * @throws MetaException
-   * @throws TException
-   *           Could not truncate table properly.
+   * @throws MetaException Failure in the RDBMS or storage
+   * @throws TException Thrift transport exception
    */
   void truncateTable(String dbName, String tableName, List<String> partNames) throws MetaException, TException;
 
   /**
+   * Truncate the table/partitions in the DEFAULT database.
+   * @param catName catalog name
+   * @param dbName
+   *          The db to which the table to be truncate belongs to
+   * @param tableName
+   *          The table to truncate
+   * @param partNames
+   *          List of partitions to truncate. NULL will truncate the whole table/all partitions
+   * @throws MetaException Failure in the RDBMS or storage
+   * @throws TException Thrift transport exception
+   */
+  void truncateTable(String catName, String dbName, String tableName, List<String> partNames)
+      throws MetaException, TException;
+
+  /**
    * Recycles the files recursively from the input path to the cmroot directory either by copying or moving it.
    *
    * @param request Inputs for path of the data files to be recycled to cmroot and
@@ -369,43 +632,33 @@ public interface IMetaStoreClient {
    */
   CmRecycleResponse recycleDirToCmPath(CmRecycleRequest request) throws MetaException, TException;
 
-  boolean tableExists(String databaseName, String tableName) throws MetaException,
-      TException, UnknownDBException;
-
   /**
-   * Check to see if the specified table exists in the DEFAULT database.
-   * @param tableName
-   * @return TRUE if DEFAULT.tableName exists, FALSE otherwise.
-   * @throws MetaException
-   * @throws TException
-   * @throws UnknownDBException
-   * @deprecated As of release 0.6.0 replaced by {@link #tableExists(String, String)}.
-   *             This method will be removed in release 0.7.0.
+   * Check whether a table exists in the default catalog.
+   * @param databaseName database name
+   * @param tableName table name
+   * @return true if the indicated table exists, false if not
+   * @throws MetaException error fetching form the RDBMS
+   * @throws TException thrift transport error
+   * @throws UnknownDBException the indicated database does not exist.
    */
-  @Deprecated
-  boolean tableExists(String tableName) throws MetaException,
-      TException, UnknownDBException;
+  boolean tableExists(String databaseName, String tableName)
+      throws MetaException, TException, UnknownDBException;
 
   /**
-   * Get a table object from the DEFAULT database.
-   *
-   * @param tableName
-   *          Name of the table to fetch.
-   * @return An object representing the table.
-   * @throws MetaException
-   *           Could not fetch the table
-   * @throws TException
-   *           A thrift communication error occurred
-   * @throws NoSuchObjectException
-   *           In case the table wasn't found.
-   * @deprecated As of release 0.6.0 replaced by {@link #getTable(String, String)}.
-   *             This method will be removed in release 0.7.0.
+   * Check whether a table exists.
+   * @param catName catalog name
+   * @param dbName database name
+   * @param tableName table name
+   * @return true if the indicated table exists, false if not
+   * @throws MetaException error fetching form the RDBMS
+   * @throws TException thrift transport error
+   * @throws UnknownDBException the indicated database does not exist.
    */
-  @Deprecated
-  Table getTable(String tableName) throws MetaException, TException, NoSuchObjectException;
+  boolean tableExists(String catName, String dbName, String tableName)
+      throws MetaException, TException, UnknownDBException;
 
   /**
-   * Get a Database Object
+   * Get a Database Object in the default catalog
    * @param databaseName  name of the database to fetch
    * @return the database
    * @throws NoSuchObjectException The database does not exist
@@ -415,9 +668,21 @@ public interface IMetaStoreClient {
   Database getDatabase(String databaseName)
       throws NoSuchObjectException, MetaException, TException;
 
+  /**
+   * Get a database.
+   * @param catalogName catalog name.  Can be null, in which case
+   * {@link Warehouse#DEFAULT_CATALOG_NAME} will be assumed.
+   * @param databaseName database name
+   * @return the database object
+   * @throws NoSuchObjectException No database with this name exists in the specified catalog
+   * @throws MetaException something went wrong, usually in the RDBMS
+   * @throws TException general thrift error
+   */
+  Database getDatabase(String catalogName, String databaseName)
+      throws NoSuchObjectException, MetaException, TException;
 
   /**
-   * Get a table object.
+   * Get a table object in the default catalog.
    *
    * @param dbName
    *          The database the table is located in.
@@ -435,7 +700,19 @@ public interface IMetaStoreClient {
       TException, NoSuchObjectException;
 
   /**
-   *
+   * Get a table object.
+   * @param catName catalog the table is in.
+   * @param dbName database the table is in.
+   * @param tableName table name.
+   * @return table object.
+   * @throws MetaException Something went wrong, usually in the RDBMS.
+   * @throws TException general thrift error.
+   */
+  Table getTable(String catName, String dbName, String tableName) throws MetaException, TException;
+
+  /**
+   * Get tables as objects (rather than just fetching their names).  This is more expensive and
+   * should only be used if you actually need all the information about the tables.
    * @param dbName
    *          The database the tables are located in.
    * @param tableNames
@@ -457,6 +734,30 @@ public interface IMetaStoreClient {
       throws MetaException, InvalidOperationException, UnknownDBException, TException;
 
   /**
+   * Get tables as objects (rather than just fetching their names).  This is more expensive and
+   * should only be used if you actually need all the information about the tables.
+   * @param catName catalog name
+   * @param dbName
+   *          The database the tables are located in.
+   * @param tableNames
+   *          The names of the tables to fetch
+   * @return A list of objects representing the tables.
+   *          Only the tables that can be retrieved from the database are returned.  For example,
+   *          if none of the requested tables could be retrieved, an empty list is returned.
+   *          There is no guarantee of ordering of the returned tables.
+   * @throws InvalidOperationException
+   *          The input to this operation is invalid (e.g., the list of tables names is null)
+   * @throws UnknownDBException
+   *          The requested database could not be fetched.
+   * @throws TException
+   *          A thrift communication error occurred
+   * @throws MetaException
+   *          Any other errors
+   */
+  List<Table> getTableObjectsByName(String catName, String dbName, List<String> tableNames)
+      throws MetaException, InvalidOperationException, UnknownDBException, TException;
+
+  /**
    * Returns the invalidation information for the materialized views given as input.
    */
   Map<String, Materialization> getMaterializationsInvalidationInfo(String dbName, List<String> viewNames)
@@ -469,22 +770,72 @@ public interface IMetaStoreClient {
       throws MetaException, TException;
 
   /**
-   * @param tableName
-   * @param dbName
-   * @param partVals
+   * Updates the creation metadata for the materialized view.
+   */
+  void updateCreationMetadata(String catName, String dbName, String tableName, CreationMetadata cm)
+      throws MetaException, TException;
+
+  /**
+  /**
+   * Add a partition to a table and get back the resulting Partition object.  This creates an
+   * empty default partition with just the partition values set.
+   * @param dbName database name
+   * @param tableName table name
+   * @param partVals partition values
    * @return the partition object
-   * @throws InvalidObjectException
-   * @throws AlreadyExistsException
-   * @throws MetaException
-   * @throws TException
-   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#append_partition(java.lang.String,
-   *      java.lang.String, java.util.List)
+   * @throws InvalidObjectException no such table
+   * @throws AlreadyExistsException a partition with these values already exists
+   * @throws MetaException error accessing the RDBMS
+   * @throws TException thrift transport error
    */
-  Partition appendPartition(String tableName, String dbName,
-      List<String> partVals) throws InvalidObjectException,
-      AlreadyExistsException, MetaException, TException;
+  Partition appendPartition(String dbName, String tableName, List<String> partVals)
+      throws InvalidObjectException, AlreadyExistsException, MetaException, TException;
 
-  Partition appendPartition(String tableName, String dbName, String name)
+  /**
+   * Add a partition to a table and get back the resulting Partition object.  This creates an
+   * empty default partition with just the partition values set.
+   * @param catName catalog name
+   * @param dbName database name
+   * @param tableName table name
+   * @param partVals partition values
+   * @return the partition object
+   * @throws InvalidObjectException no such table
+   * @throws AlreadyExistsException a partition with these values already exists
+   * @throws MetaException error accessing the RDBMS
+   * @throws TException thrift transport error
+   */
+  Partition appendPartition(String catName, String dbName, String tableName, List<String> partVals)
+      throws InvalidObjectException, AlreadyExistsException, MetaException, TException;
+
+  /**
+   * Add a partition to a table and get back the resulting Partition object.  This creates an
+   * empty default partition with just the partition value set.
+   * @param dbName database name.
+   * @param tableName table name.
+   * @param name name of the partition, should be in the form partkey=partval.
+   * @return new partition object.
+   * @throws InvalidObjectException No such table.
+   * @throws AlreadyExistsException Partition of this name already exists.
+   * @throws MetaException error accessing the RDBMS
+   * @throws TException thrift transport error
+   */
+  Partition appendPartition(String dbName, String tableName, String name)
+      throws InvalidObjectException, AlreadyExistsException, MetaException, TException;
+
+  /**
+   * Add a partition to a table and get back the resulting Partition object.  This creates an
+   * empty default partition with just the partition value set.
+   * @param catName catalog name.
+   * @param dbName database name.
+   * @param tableName table name.
+   * @param name name of the partition, should be in the form partkey=partval.
+   * @return new partition object.
+   * @throws InvalidObjectException No such table.
+   * @throws AlreadyExistsException Partition of this name already exists.
+   * @throws MetaException error accessing the RDBMS
+   * @throws TException thrift transport error
+   */
+  Partition appendPartition(String catName, String dbName, String tableName, String name)
       throws InvalidObjectException, AlreadyExistsException, MetaException, TException;
 
   /**
@@ -522,6 +873,15 @@ public interface IMetaStoreClient {
   int add_partitions(List<Partition> partitions)
       throws InvalidObjectException, AlreadyExistsException, MetaException, TException;
 
+  /**
+   * Add a partitions using a spec proxy.
+   * @param partitionSpec partition spec proxy
+   * @return number of partitions that were added
+   * @throws InvalidObjectException the partitionSpec is malformed.
+   * @throws AlreadyExistsException one or more of the partitions already exist.
+   * @throws MetaException error accessing the RDBMS or storage.
+   * @throws TException thrift transport error
+   */
   int add_partitions_pspec(PartitionSpecProxy partitionSpec)
       throws InvalidObjectException, AlreadyExistsException, MetaException, TException;
 
@@ -538,25 +898,46 @@ public interface IMetaStoreClient {
       throws InvalidObjectException, AlreadyExistsException, MetaException, TException;
 
   /**
-   * @param dbName
-   * @param tblName
-   * @param partVals
+   * Get a partition.
+   * @param dbName database name
+   * @param tblName table name
+   * @param partVals partition values for this partition, must be in the same order as the
+   *                 partition keys of the table.
    * @return the partition object
-   * @throws MetaException
-   * @throws TException
-   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#get_partition(java.lang.String,
-   *      java.lang.String, java.util.List)
+   * @throws NoSuchObjectException no such partition
+   * @throws MetaException error access the RDBMS.
+   * @throws TException thrift transport error
+   */
+  Partition getPartition(String dbName, String tblName, List<String> partVals)
+      throws NoSuchObjectException, MetaException, TException;
+
+  /**
+   * Get a partition.
+   * @param catName catalog name
+   * @param dbName database name
+   * @param tblName table name
+   * @param partVals partition values for this partition, must be in the same order as the
+   *                 partition keys of the table.
+   * @return the partition object
+   * @throws NoSuchObjectException no such partition
+   * @throws MetaException error access the RDBMS.
+   * @throws TException thrift transport error
    */
-  Partition getPartition(String dbName, String tblName,
-      List<String> partVals) throws NoSuchObjectException, MetaException, TException;
+  Partition getPartition(String catName, String dbName, String tblName, List<String> partVals)
+      throws NoSuchObjectException, MetaException, TException;
 
   /**
-   * @param partitionSpecs
-   * @param sourceDb
-   * @param sourceTable
-   * @param destdb
-   * @param destTableName
+   * Move a partition from one table to another
+   * @param partitionSpecs key value pairs that describe the partition to be moved.
+   * @param sourceDb database of the source table
+   * @param sourceTable name of the source table
+   * @param destdb database of the destination table
+   * @param destTableName name of the destination table
    * @return partition object
+   * @throws MetaException error accessing the RDBMS or storage
+   * @throws NoSuchObjectException no such table, for either source or destination table
+   * @throws InvalidObjectException error in partition specifications
+   * @throws TException thrift transport error
    */
   Partition exchange_partition(Map<String, String> partitionSpecs,
       String sourceDb, String sourceTable, String destdb,
@@ -564,14 +945,38 @@ public interface IMetaStoreClient {
       InvalidObjectException, TException;
 
   /**
+   * Move a partition from one table to another
+   * @param partitionSpecs key value pairs that describe the partition to be moved.
+   * @param sourceCat catalog of the source table
+   * @param sourceDb database of the source table
+   * @param sourceTable name of the source table
+   * @param destCat catalog of the destination table, for now must the same as sourceCat
+   * @param destdb database of the destination table
+   * @param destTableName name of the destination table
+   * @return partition object
+   * @throws MetaException error accessing the RDBMS or storage
+   * @throws NoSuchObjectException no such table, for either source or destination table
+   * @throws InvalidObjectException error in partition specifications
+   * @throws TException thrift transport error
+   */
+  Partition exchange_partition(Map<String, String> partitionSpecs, String sourceCat,
+                               String sourceDb, String sourceTable, String destCat, String destdb,
+                               String destTableName) throws MetaException, NoSuchObjectException,
+      InvalidObjectException, TException;
+
+  /**
    * With the one partitionSpecs to exchange, multiple partitions could be exchanged.
    * e.g., year=2015/month/day, exchanging partition year=2015 results to all the partitions
    * belonging to it exchanged. This function returns the list of affected partitions.
-   * @param partitionSpecs
-   * @param sourceDb
-   * @param sourceTable
-   * @param destdb
-   * @param destTableName
+   * @param partitionSpecs key value pairs that describe the partition(s) to be moved.
+   * @param sourceDb database of the source table
+   * @param sourceTable name of the source table
+   * @param destdb database of the destination table
+   * @param destTableName name of the destination table
+   * @throws MetaException error accessing the RDBMS or storage
+   * @throws NoSuchObjectException no such table, for either source or destination table
+   * @throws InvalidObjectException error in partition specifications
+   * @throws TException thrift transport error
    * @return the list of the new partitions
    */
   List<Partition> exchange_partitions(Map<String, String> partitionSpecs,
@@ -580,60 +985,243 @@ public interface IMetaStoreClient {
       InvalidObjectException, TException;
 
   /**
-   * @param dbName
-   * @param tblName
+   * With the one partitionSpecs to exchange, multiple partitions could be exchanged.
+   * e.g., year=2015/month/day, exchanging partition year=2015 results to all the partitions
+   * belonging to it exchanged. This function returns the list of affected partitions.
+   * @param partitionSpecs key value pairs that describe the partition(s) to be moved.
+   * @param sourceCat catalog of the source table
+   * @param sourceDb database of the source table
+   * @param sourceTable name of the source table
+   * @param destCat catalog of the destination table, for now must the same as sourceCat
+   * @param destdb database of the destination table
+   * @param destTableName name of the destination table
+   * @throws MetaException error accessing the RDBMS or storage
+   * @throws NoSuchObjectException no such table, for either source or destination table
+   * @throws InvalidObjectException error in partition specifications
+   * @throws TException thrift transport error
+   * @return the list of the new partitions
+   */
+  List<Partition> exchange_partitions(Map<String, String> partitionSpecs, String sourceCat,
+                                      String sourceDb, String sourceTable, String destCat,
+                                      String destdb, String destTableName)
+      throws MetaException, NoSuchObjectException, InvalidObjectException, TException;
+
+  /**
+   * Get a Partition by name.
+   * @param dbName database name.
+   * @param tblName table name.
    * @param name - partition name i.e. 'ds=2010-02-03/ts=2010-02-03 18%3A16%3A01'
    * @return the partition object
-   * @throws MetaException
-   * @throws TException
-   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#get_partition(java.lang.String,
-   *      java.lang.String, java.util.List)
+   * @throws MetaException error access the RDBMS.
+   * @throws TException thrift transport error
+   */
+  Partition getPartition(String dbName, String tblName, String name)
+      throws MetaException, UnknownTableException, NoSuchObjectException, TException;
+
+  /**
+   * Get a Partition by name.
+   * @param catName catalog name.
+   * @param dbName database name.
+   * @param tblName table name.
+   * @param name - partition name i.e. 'ds=2010-02-03/ts=2010-02-03 18%3A16%3A01'
+   * @return the partition object
+   * @throws MetaException error access the RDBMS.
+   * @throws TException thrift transport error
    */
-  Partition getPartition(String dbName, String tblName,
-      String name) throws MetaException, UnknownTableException, NoSuchObjectException, TException;
+  Partition getPartition(String catName, String dbName, String tblName, String name)
+      throws MetaException, UnknownTableException, NoSuchObjectException, TException;
 
 
   /**
-   * @param dbName
-   * @param tableName
-   * @param pvals
-   * @param userName
-   * @param groupNames
+   * Get a Partition along with authorization information.
+   * @param dbName database name
+   * @param tableName table name
+   * @param pvals partition values, must be in the same order as the tables partition keys
+   * @param userName name of the calling user
+   * @param groupNames groups the call
    * @return the partition
-   * @throws MetaException
-   * @throws UnknownTableException
-   * @throws NoSuchObjectException
-   * @throws TException
+   * @throws MetaException error accessing the RDBMS
+   * @throws UnknownTableException no such table
+   * @throws NoSuchObjectException no such partition
+   * @throws TException thrift transport error
    */
   Partition getPartitionWithAuthInfo(String dbName, String tableName,
       List<String> pvals, String userName, List<String> groupNames)
       throws MetaException, UnknownTableException, NoSuchObjectException, TException;
 
   /**
-   * @param tbl_name
-   * @param db_name
-   * @param max_parts
+   * Get a Partition along with authorization information.
+   * @param catName catalog name
+   * @param dbName database name
+   * @param tableName table name
+   * @param pvals partition values, must be in the same order as the tables partition keys
+   * @param userName name of the calling user
+   * @param groupNames groups the call
+   * @return the partition
+   * @throws MetaException error accessing the RDBMS
+   * @throws UnknownTableException no such table
+   * @throws NoSuchObjectException no such partition
+   * @throws TException thrift transport error
+   */
+  Partition getPartitionWithAuthInfo(String catName, String dbName, String tableName,
+                                     List<String> pvals, String userName, List<String> groupNames)
+      throws MetaException, UnknownTableException, NoSuchObjectException, TException;
+
+  /**
+   * Get a list of partittions for a table.
+   * @param db_name database name
+   * @param tbl_name table name
+   * @param max_parts maximum number of parts to return, -1 for all
    * @return the list of partitions
-   * @throws NoSuchObjectException
-   * @throws MetaException
-   * @throws TException
+   * @throws NoSuchObjectException No such table.
+   * @throws MetaException error accessing RDBMS.
+   * @throws TException thrift transport error
    */
-  List<Partition> listPartitions(String db_name, String tbl_name,
-      short max_parts) throws NoSuchObjectException, MetaException, TException;
+  List<Partition> listPartitions(String db_name, String tbl_name, short max_parts)
+      throws NoSuchObjectException, MetaException, TException;
+
+  /**
+   * Get a list of partittions for a table.
+   * @param catName catalog name
+   * @param db_name database name
+   * @param tbl_name table name
+   * @param max_parts maximum number of parts to return, -1 for all
+   * @return the list of partitions
+   * @throws NoSuchObjectException No such table.
+   * @throws MetaException error accessing RDBMS.
+   * @throws TException thrift transport error
+   */
+  List<Partition> listPartitions(String catName, String db_name, String tbl_name, int max_parts)
+      throws NoSuchObjectException, MetaException, TException;
 
-  public PartitionSpecProxy listPartitionSpecs(String dbName, String tableName, int maxParts)
+  /**
+   * Get a list of partitions from a table, returned in the form of PartitionSpecProxy
+   * @param dbName database name.
+   * @param tableName table name.
+   * @param maxParts maximum number of partitions to return, or -1 for all
+   * @return a PartitionSpecProxy
+   * @throws TException thrift transport error
+   */
+  PartitionSpecProxy listPartitionSpecs(String dbName, String tableName, int maxParts)
     throws TException;
+
+  /**
+   * Get a list of partitions from a table, returned in the form of PartitionSpecProxy
+   * @param catName catalog name.
+   * @param dbName database name.
+   * @param tableName table name.
+   * @param maxParts maximum number of partitions to return, or -1 for all
+   * @return a PartitionSpecProxy
+   * @throws TException thrift transport error
+   */
+  PartitionSpecProxy listPartitionSpecs(String catName, String dbName, String tableName,
+                                        int maxParts) throws TException;
+
+  /**
+   * Get a list of partitions based on a (possibly partial) list of partition values.
+   * @param db_name database name.
+   * @param tbl_name table name.
+   * @param part_vals partition values, in order of the table partition keys.  These can be
+   *                  partial, or .* to match all values for a particular key.
+   * @param max_parts maximum number of partitions to return, or -1 for all.
+   * @return list of partitions
+   * @throws NoSuchObjectException no such table.
+   * @throws MetaException error accessing the database or processing the partition values.
+   * @throws TException thrift transport error.
+   */
   List<Partition> listPartitions(String db_name, String tbl_name,
       List<String> part_vals, short max_parts) throws NoSuchObjectException, MetaException, TException;
 
+  /**
+   * Get a list of partitions based on a (possibly partial) list of partition values.
+   * @param catName catalog name.
+   * @param db_name database name.
+   * @param tbl_name table name.
+   * @param part_vals partition values, in order of the table partition keys.  These can be
+   *                  partial, or .* to match all values for a particular key.
+   * @param max_parts maximum number of partitions to return, or -1 for all.
+   * @return list of partitions
+   * @throws NoSuchObjectException no such table.
+   * @throws MetaException error accessing the database or processing the partition values.
+   * @throws TException thrift transport error.
+   */
+  List<Partition> listPartitions(String catName, String db_name, String tbl_name,
+                                 List<String> part_vals, int max_parts)
+      throws NoSuchObjectException, MetaException, TException;
+
+  /**
+   * List Names of partitions in a table.
+   * @param db_name database name.
+   * @param tbl_name table name.
+   * @param max_parts maximum number of parts of fetch, or -1 to fetch them all.
+   * @return list of partition names.
+   * @throws NoSuchObjectException No such table.
+   * @throws MetaException Error accessing the RDBMS.
+   * @throws TException thrift transport error
+   */
   List<String> listPartitionNames(String db_name, String tbl_name,
       short max_parts) throws NoSuchObjectException, MetaException, TException;
 
+  /**
+   * List Names of partitions in a table.
+   * @param catName catalog name.
+   * @param db_name database name.
+   * @param tbl_name table name.
+   * @param max_parts maximum number of parts of fetch, or -1 to fetch them all.
+   * @return list of partition names.
+   * @throws NoSuchObjectException No such table.
+   * @throws MetaException Error accessing the RDBMS.
+   * @throws TException thrift transport error
+   */
+  List<String> listPartitionNames(String catName, String db_name, String tbl_name,
+                                  int max_parts) throws NoSuchObjectException, MetaException, TException;
+
+  /**
+   * Get a list of partition names matching a partial specification of the partition values.
+   * @param db_name database name.
+   * @param tbl_name table name.
+   * @param part_vals partial list of partition values.  These must be given in the order of the
+   *                  partition keys.  If you wish to accept any value for a particular key you
+   *                  can pass ".*" for that value in this list.
+   * @param max_parts maximum number of partition names to return, or -1 to return all that are
+   *                  found.
+   * @return list of matching partition names.
+   * @throws MetaException error accessing the RDBMS.
+   * @throws TException thrift transport error.
+   * @throws NoSuchObjectException no such table.
+   */
   List<String> listPartitionNames(String db_name, String tbl_name,
       List<String> part_vals, short max_parts)
       throws MetaException, TException, NoSuchObjectException;
 
-  public PartitionValuesResponse listPartitionValues(PartitionValuesRequest request)
+  /**
+   * Get a list of partition names matching a partial specification of the partition values.
+   * @param catName catalog name.
+   * @param db_name database name.
+   * @param tbl_name table name.
+   * @param part_vals partial list of partition values.  These must be given in the order of the
+   *                  partition keys.  If you wish to accept any value for a particular key you
+   *                  can pass ".*" for that value in this list.
+   * @param max_parts maximum number of partition names to return, or -1 to return all that are
+   *                  found.
+   * @return list of matching partition names.
+   * @throws MetaException error accessing the RDBMS.
+   * @throws TException thrift transport error.
+   * @throws NoSuchObjectException no such table.
+   */
+  List<String> listPartitionNames(String catName, String db_name, String tbl_name,
+                                  List<String> part_vals, int max_parts)
+      throws MetaException, TException, NoSuchObjectException;
+
+  /**
+   * Get a list of partition values
+   * @param request request
+   * @return reponse
+   * @throws MetaException error accessing RDBMS
+   * @throws TException thrift transport error
+   * @throws NoSuchObjectException no such table
+   */
+  PartitionValuesResponse listPartitionValues(PartitionValuesRequest request)
       throws MetaException, TException, NoSuchObjectException;
 
   /**
@@ -644,15 +1232,31 @@ public interface IMetaStoreClient {
    *    for example "part1 = \"p1_abc\" and part2 &lt;= "\p2_test\"". Filtering can
    *    be done only on string partition keys.
    * @return number of partitions
-   * @throws MetaException
-   * @throws NoSuchObjectException
-   * @throws TException
+   * @throws MetaException error accessing RDBMS or processing the filter
+   * @throws NoSuchObjectException no such table
+   * @throws TException thrift transport error
+   */
+  int getNumPartitionsByFilter(String dbName, String tableName,
+                               String filter) throws MetaException, NoSuchObjectException, TException;
+
+  /**
+   * Get number of partitions matching specified filter
+   * @param catName catalog name
+   * @param dbName the database name
+   * @param tableName the table name
+   * @param filter the filter string,
+   *    for example "part1 = \"p1_abc\" and part2 &lt;= "\p2_test\"". Filtering can
+   *    be done only on string partition keys.
+   * @return number of partitions
+   * @throws MetaException error accessing RDBMS or processing the filter
+   * @throws NoSuchObjectException no such table
+   * @throws TException thrift transport error
    */
-  public int getNumPartitionsByFilter(String dbName, String tableName,
-                                      String filter) throws MetaException, NoSuchObjectException, TException;
+  int getNumPartitionsByFilter(String catName, String dbName, String tableName,
+                               String filter) throws MetaException, NoSuchObjectException, TException;
 
 
-    /**
+  /**
    * Get list of partitions matching specified filter
    * @param db_name the database name
    * @param tbl_name the table name
@@ -662,17 +1266,64 @@ public interface IMetaStoreClient {
    * @param max_parts the maximum number of partitions to return,
    *    all partitions are returned if -1 is passed
    * @return list of partitions
-   * @throws MetaException
-   * @throws NoSuchObjectException
-   * @throws TException
+   * @throws MetaException Error accessing the RDBMS or processing the filter.
+   * @throws NoSuchObjectException No such table.
+   * @throws TException thrift transport error
    */
   List<Partition> listPartitionsByFilter(String db_name, String tbl_name,
-      String filter, short max_parts) throws MetaException,
-         NoSuchObjectException, TException;
+      String filter, short max_parts) throws MetaException, NoSuchObjectException, TException;
+
+  /**
+   * Get list of partitions matching specified filter
+   * @param catName catalog name.
+   * @param db_name the database name
+   * @param tbl_name the table name
+   * @param filter the filter string,
+   *    for example "part1 = \"p1_abc\" and part2 &lt;= "\p2_test\"". Filtering can
+   *    be done only on string partition keys.
+   * @param max_parts the maximum number of partitions to return,
+   *    all partitions are returned if -1 is passed
+   * @return list of partitions
+   * @throws MetaException Error accessing the RDBMS or processing the filter.
+   * @throws NoSuchObjectException No such table.
+   * @throws TException thrift transport error
+   */
+  List<Partition> listPartitionsByFilter(String catName, String db_name, String tbl_name,
+                                         String filter, int max_parts)
+      throws MetaException, NoSuchObjectException, TException;
 
+  /**
+   * Get a list of partitions in a PartitionSpec, using a filter to select which partitions to
+   * fetch.
+   * @param db_name database name
+   * @param tbl_name table name
+   * @param filter SQL where clause filter
+   * @param max_parts maximum number of partitions to fetch, or -1 for all
+   * @return PartitionSpec
+   * @throws MetaException error accessing RDBMS or processing the filter
+   * @throws NoSuchObjectException No table matches the request
+   * @throws TException thrift transport error
+   */
   PartitionSpecProxy listPartitionSpecsByFilter(String db_name, String tbl_name,
-                                                       String filter, int max_parts) throws MetaException,
-         NoSuchObjectException, TException;
+                                                String filter, int max_parts)
+      throws MetaException, NoSuchObjectException, TException;
+
+  /**
+   * Get a list of partitions in a PartitionSpec, using a filter to select which partitions to
+   * fetch.
+   * @param catName catalog name
+   * @param db_name database name
+   * @param tbl_name table name
+   * @param filter SQL where clause filter
+   * @param max_parts maximum number of partitions to fetch, or -1 for all
+   * @return PartitionSpec
+   * @throws MetaException error accessing RDBMS or processing the filter
+   * @throws NoSuchObjectException No table matches the request
+   * @throws TException thrift transport error
+   */
+  PartitionSpecProxy listPartitionSpecsByFilter(String catName, String db_name, String tbl_name,
+                                                String filter, int max_parts)
+      throws MetaException, NoSuchObjectException, TException;
 
   /**
    * Get list of partitions matching specified serialized expression
@@ -685,22 +1336,61 @@ public interface IMetaStoreClient {
    *    metastore server-side configuration is used.
    * @param result the resulting list of partitions
    * @return whether the resulting list contains partitions which may or may not match the expr
+   * @throws TException thrift transport error or error executing the filter.
    */
   boolean listPartitionsByExpr(String db_name, String tbl_name,
       byte[] expr, String default_partition_name, short max_parts, List<Partition> result)
           throws TException;
 
   /**
-   * @param dbName
-   * @param tableName
-   * @param s
-   * @param userName
-   * @param groupNames
+   * Get list of partitions matching specified serialized expression
+   * @param catName catalog name
+   * @param db_name the database name
+   * @param tbl_name the table name
+   * @param expr expression, serialized from ExprNodeDesc
+   * @param max_parts the maximum number of partitions to return,
+   *    all partitions are returned if -1 is passed
+   * @param default_partition_name Default partition name from configuration. If blank, the
+   *    metastore server-side configuration is used.
+   * @param result the resulting list of partitions
+   * @return whether the resulting list contains partitions which may or may not match the expr
+   * @throws TException thrift transport error or error executing the filter.
+   */
+  boolean listPartitionsByExpr(String catName, String db_name, String tbl_name, byte[] expr,
+                               String default_partition_name, int max_parts, List<Partition> result)
+      throws TException;
+
+  /**
+   * List partitions, fetching the authorization information along with the partitions.
+   * @param dbName database name
+   * @param tableName table name
+   * @param maxParts maximum number of partitions to fetch, or -1 for all
+   * @param userName user to fetch privileges for
+   * @param groupNames groups to fetch privileges for
    * @return the list of partitions
-   * @throws NoSuchObjectException
+   * @throws NoSuchObjectException no partitions matching the criteria were found
+   * @throws MetaException error accessing the RDBMS
+   * @throws TException thrift transport error
    */
   List<Partition> listPartitionsWithAuthInfo(String dbName,
-      String tableName, short s, String userName, List<String> groupNames)
+      String tableName, short maxParts, String userName, List<String> groupNames)
+      throws MetaException, TException, NoSuchObjectException;
+
+  /**
+   * List partitions, fetching the authorization information along with the partitions.
+   * @param catName catalog name
+   * @param dbName database name
+   * @param tableName table name
+   * @param maxParts maximum number of partitions to fetch, or -1 for all
+   * @param userName user to fetch privileges for
+   * @param groupNames groups to fetch privileges for
+   * @return the list of partitions
+   * @throws NoSuchObjectException no partitions matching the criteria were found
+   * @throws MetaException error accessing the RDBMS
+   * @throws TException thrift transport error
+   */
+  List<Partition> listPartitionsWithAuthInfo(String catName, String dbName, String tableName,
+                                             int maxParts, String userName, List<String> groupNames)
       throws MetaException, TException, NoSuchObjectException;
 
   /**
@@ -709,62 +1399,138 @@ public interface IMetaStoreClient {
    * @param tbl_name table name
    * @param part_names list of partition names
    * @return list of Partition objects
-   * @throws NoSuchObjectException
-   * @throws MetaException
-   * @throws TException
+   * @throws NoSuchObjectException No such partitions
+   * @throws MetaException error accessing the RDBMS.
+   * @throws TException thrift transport error
    */
   List<Partition> getPartitionsByNames(String db_name, String tbl_name,
       List<String> part_names) throws NoSuchObjectException, MetaException, TException;
 
   /**
-   * @param dbName
-   * @param tableName
-   * @param partialPvals
-   * @param s
-   * @param userName
-   * @param groupNames
-   * @return the list of paritions
-   * @throws NoSuchObjectException
+   * Get partitions by a list of partition names.
+   * @param catName catalog name
+   * @param db_name database name
+   * @param tbl_name table name
+   * @param part_names list of partition names
+   * @return list of Partition objects
+   * @throws NoSuchObjectException No such partitions
+   * @throws MetaException error accessing the RDBMS.
+   * @throws TException thrift transport error
+   */
+  List<Partition> getPartitionsByNames(String catName, String db_name, String tbl_name,
+                                       List<String> part_names)
+      throws NoSuchObjectException, MetaException, TException;
+
+  /**
+   * List partitions along with privilege information for a user or groups
+   * @param dbName database name
+   * @param tableName table name
+   * @param partialPvals partition values, can be partial
+   * @param maxParts maximum number of partitions to fetch, or -1 for all
+   * @param userName user to fetch privilege information for
+   * @param groupNames group to fetch privilege information for
+   * @return the list of partitions
+   * @throws NoSuchObjectException no partitions matching the criteria were found
+   * @throws MetaException error accessing the RDBMS
+   * @throws TException thrift transport error
    */
   List<Partition> listPartitionsWithAuthInfo(String dbName,
-      String tableName, List<String> partialPvals, short s, String userName,
+      String tableName, List<String> partialPvals, short maxParts, String userName,
       List<String> groupNames) throws MetaException, TException, NoSuchObjectException;
 
   /**
-   * @param db_name
-   * @param tbl_name
-   * @param partKVs
-   * @param eventType
-   * @throws MetaException
-   * @throws NoSuchObjectException
-   * @throws TException
-   * @throws UnknownTableException
-   * @throws UnknownDBException
-   * @throws UnknownPartitionException
-   * @throws InvalidPartitionException
+   * List partitions along with privilege information for a user or groups
+   * @param dbName database name
+   * @param tableName table name
+   * @param partialPvals partition values, can be partial
+   * @param maxParts maximum number of partitions to fetch, or -1 for all
+   * @param userName user to fetch privilege information for
+   * @param groupNames group to fetch privilege information for
+   * @return the list of partitions
+   * @throws NoSuchObjectException no partitions matching the criteria were found
+   * @throws MetaException error accessing the RDBMS
+   * @throws TException thrift transport error
+   */
+  List<Partition> listPartitionsWithAuthInfo(String catName, String dbName, String tableName,
+                                             List<String> partialPvals, int maxParts, String userName,
+                                             List<String> groupNames)
+      throws MetaException, TException, NoSuchObjectException;
+
+  /**
+   * Mark an event as having occurred on a partition.
+   * @param db_name database name
+   * @param tbl_name table name
+   * @param partKVs key value pairs that describe the partition
+   * @param eventType type of the event
+   * @throws MetaException error access the RDBMS
+   * @throws NoSuchObjectException never throws this AFAICT
+   * @throws TException thrift transport error
+   * @throws UnknownTableException no such table
+   * @throws UnknownDBException no such database
+   * @throws UnknownPartitionException no such partition
+   * @throws InvalidPartitionException partition partKVs is invalid
    */
   void markPartitionForEvent(String db_name, String tbl_name, Map<String,String> partKVs,
       PartitionEventType eventType) throws MetaException, NoSuchObjectException, TException,
       UnknownTableException, UnknownDBException, UnknownPartitionException, InvalidPartitionException;
 
   /**
-   * @param db_name
-   * @param tbl_name
-   * @param partKVs
-   * @param eventType
-   * @throws MetaException
-   * @throws NoSuchObjectException
-   * @throws TException
-   * @throws UnknownTableException
-   * @throws UnknownDBException
-   * @throws UnknownPartitionException
-   * @throws InvalidPartitionException
+   * Mark an event as having occurred on a partition.
+   * @param catName catalog name
+   * @param db_name database name
+   * @param tbl_name table name
+   * @param partKVs key value pairs that describe the partition
+   * @param eventType type of the event
+   * @throws MetaException error access the RDBMS
+   * @throws NoSuchObjectException never throws this AFAICT
+   * @throws TException thrift transport error
+   * @throws UnknownTableException no such table
+   * @throws UnknownDBException no such database
+   * @throws UnknownPartitionException no such partition
+   * @throws InvalidPartitionException partition partKVs is invalid
+   */
+  void markPartitionForEvent(String catName, String db_name, String tbl_name, Map<String,String> partKVs,
+                             PartitionEventType eventType) throws MetaException, NoSuchObjectException, TException,
+      UnknownTableException, UnknownDBException, UnknownPartitionException, InvalidPartitionException;
+
+  /**
+   * Determine whether a partition has been marked with a particular event type.
+   * @param db_name database name
+   * @param tbl_name table name.
+   * @param partKVs key value pairs that describe the partition.
+   * @param eventType event type
+   * @throws MetaException error access the RDBMS
+   * @throws NoSuchObjectException never throws this AFAICT
+   * @throws TException thrift transport error
+   * @throws UnknownTableException no such table
+   * @throws UnknownDBException no such database
+   * @throws UnknownPartitionException no such partition
+   * @throws InvalidPartitionException partition partKVs is invalid
    */
   boolean isPartitionMarkedForEvent(String db_name, String tbl_name, Map<String,String> partKVs,
       PartitionEventType eventType) throws MetaException, NoSuchObjectException, TException,
       UnknownTableException, UnknownDBException, UnknownPartitionException, InvalidPartitionException;
 
   /**
+   * Determine whether a partition has been marked with a particular event type.
+   * @param catName catalog name
+   * @param db_name database name
+   * @param tbl_name table name.
+   * @param partKVs key value pairs that describe the partition.
+   * @param eventType event type
+   * @throws MetaException error access the RDBMS
+   * @throws NoSuchObjectException never throws this AFAICT
+   * @throws TException thrift transport error
+   * @throws UnknownTableException no such table
+   * @throws UnknownDBException no such database
+   * @throws UnknownPartitionException no such partition
+   * @throws InvalidPartitionException partition partKVs is invalid
+   */
+  boolean isPartitionMarkedForEvent(String catName, String db_name, String tbl_name, Map<String,String> partKVs,
+                                    PartitionEventType eventType) throws MetaException, NoSuchObjectException, TException,
+      UnknownTableException, UnknownDBException, UnknownPartitionException, InvalidPartitionException;
+
+  /**
    * @param partVals
    * @throws TException
    * @throws MetaException
@@ -784,92 +1550,449 @@ public interface IMetaStoreClient {
   void createTable(Table tbl) throws AlreadyExistsException,
       InvalidObjectException, MetaException, NoSuchObjectException, TException;
 
-  void alter_table(String defaultDatabaseName, String tblName,
-      Table table) throws InvalidOperationException, MetaException, TException;
+  /**
+   * Alter a table
+   * @param databaseName database name
+   * @param tblName table name
+   * @param table new table object, should be complete representation of the table, not just the
+   *             things you want to change.
+   * @throws InvalidOperationException something is wrong with the new table object or an
+   * operation was attempted that is not allowed (such as changing partition columns).
+   * @throws MetaException something went wrong, usually in the RDBMS
+   * @throws TException general thrift exception
+   */
+  void alter_table(String databaseName, String tblName, Table table)
+      throws InvalidOperationException, MetaException, TException;
+
+  /**
+   * Alter a table. Equivalent to
+   * {@link #alter_table(String, String, String, Table, EnvironmentContext)} with
+   * EnvironmentContext set to null.
+   * @param catName catalog name.
+   * @param dbName database name.
+   * @param tblName table name.
+   * @param newTable new table object, should be complete representation of the table, not just the
+   *                 things you want to change.
+   * @throws InvalidOperationException something is wrong with the new table object or an
+   * operation was attempted that is not allowed (such as changing partition columns).
+   * @throws MetaException something went wrong, usually in the RDBMS
+   * @throws TException general thrift exception
+   */
+  default void alter_table(String catName, String dbName, String tblName, Table newTable)
+      throws InvalidOperationException, MetaException, TException {
+    alter_table(catName, dbName, tblName, newTable, null);
+  }
+
+  /**
+   * Alter a table.
+   * @param catName catalog name.
+   * @param dbName database name.
+   * @param tblName table name.
+   * @param newTable new table object, should be complete representation of the table, not just the
+   *                 things you want to change.
+   * @param envContext options for the alter.
+   * @throws InvalidOperationException something is wrong with the new table object or an
+   * operation was attempted that is not allowed (such as changing partition columns).
+   * @throws MetaException something went wrong, usually in the RDBMS
+   * @throws TException general thrift exception
+   */
+  void alter_table(String catName, String dbName, String tblName, Table newTable,
+                  EnvironmentContext envContext)
+      throws InvalidOperationException, MetaException, TException;
 
   /**
-   * Use alter_table_with_environmentContext instead of alter_table with cascade option
+   * @deprecated Use alter_table_with_environmentContext instead of alter_table with cascade option
    * passed in EnvironmentContext using {@code StatsSetupConst.CASCADE}
    */
   @Deprecated
   void alter_table(String defaultDatabaseName, String tblName, Table table,
       boolean cascade) throws InvalidOperationException, MetaException, TException;
 
-  //wrapper of alter_table_with_cascade
-  void alter_table_with_environmentContext(String defaultDatabaseName, String tblName, Table table,
+  /**
+   * Alter a table.
+   * @param databaseName database name
+   * @param tblName table name
+   * @param table new table object, should be complete representation of the table, not just the
+   *              things you want to change.
+   * @param environmentContext options for the alter.
+   * @throws InvalidOperationException something is wrong with the new table object or an
+   * operation was attempted that is not allowed (such as changing partition columns).
+   * @throws MetaException something went wrong, usually in the RDBMS
+   * @throws TException general thrift exception
+   */
+  void alter_table_with_environmentContext(String databaseName, String tblName, Table table,
       EnvironmentContext environmentContext) throws InvalidOperationException, MetaException,
       TException;
 
+  /**
+   * Create a new database.
+   * @param db database object.  If the catalog name is null it will be assumed to be
+   *           {@link Warehouse#DEFAULT_CATALOG_NAME}.
+   * @throws InvalidObjectException There is something wrong with the database object.
+   * @throws AlreadyExistsException There is already a database of this name in the specified
+   * catalog.
+   * @throws MetaException something went wrong, usually in the RDBMS
+   * @throws TException general thrift error
+   */
   void createDatabase(Database db)
       throws InvalidObjectException, AlreadyExistsException, MetaException, TException;
 
+  /**
+   * Drop a database.
+   * @param name name of the database to drop.
+   * @throws NoSuchObjectException No such database exists.
+   * @throws InvalidOperationException The database cannot be dropped because it is not empty.
+   * @throws MetaException something went wrong, usually either in the RDMBS or in storage.
+   * @throws TException general thrift error.
+   */
   void dropDatabase(String name)
       throws NoSuchObjectException, InvalidOperationException, MetaException, TException;
 
+  /**
+   *
+   * Drop a database.
+   * @param name name of the database to drop.
+   * @param deleteData whether to drop the underlying HDFS directory.
+   * @param ignoreUnknownDb whether to ignore an attempt to drop a non-existant database
+   * @throws NoSuchObjectException No database of this name exists in the specified catalog and
+   * ignoreUnknownDb is false.
+   * @throws InvalidOperationException The database cannot be dropped because it is not empty.
+   * @throws MetaException something went wrong, usually either in the RDMBS or in storage.
+   * @throws TException general thrift error.
+   */
   void dropDatabase(String name, boolean deleteData, boolean ignoreUnknownDb)
       throws NoSuchObjectException, InvalidOperationException, MetaException, TException;
 
+  /**
+   *
+   * Drop a database.
+   * @param name database name.
+   * @param deleteData whether to drop the underlying HDFS directory.
+   * @param ignoreUnknownDb whether to ignore an attempt to drop a non-existant database
+   * @param cascade whether to drop contained tables, etc.  If this is false and there are
+   *                objects still in the database the drop will fail.
+   * @throws NoSuchObjectException No database of this name exists in the specified catalog and
+   * ignoreUnknownDb is false.
+   * @throws InvalidOperationException The database contains objects and cascade is false.
+   * @throws MetaException something went wrong, usually either in the RDBMS or storage.
+   * @throws TException general thrift error.
+   */
   void dropDatabase(String name, boolean deleteData, boolean ignoreUnknownDb, boolean cascade)
       throws NoSuchObjectException, InvalidOperationException, MetaException, TException;
 
+  /**
+   * Drop a database.
+   * @param catName Catalog name.  This can be null, in which case
+   *                {@link Warehouse#DEFAULT_CATALOG_NAME} will be assumed.
+   * @param dbName database name.
+   * @param deleteData whether to drop the underlying HDFS directory.
+   * @param ignoreUnknownDb whether to ignore an attempt to drop a non-existant database
+   * @param cascade whether to drop contained tables, etc.  If this is false and there are
+   *                objects still in the database the drop will fail.
+   * @throws NoSuchObjectException No database of this name exists in the specified catalog and
+   * ignoreUnknownDb is false.
+   * @throws InvalidOperationException The database contains objects and cascade is false.
+   * @throws MetaException something went wrong, usually either in the RDBMS or storage.
+   * @throws TException general thrift error.
+   */
+  void dropDatabase(String catName, String dbName, boolean deleteData, boolean ignoreUnknownDb,
+                    boolean cascade)
+      throws NoSuchObjectException, InvalidOperationException, MetaException, TException;
+
+  /**
+   * Drop a database.  Equivalent to
+   * {@link #dropDatabase(String, String, boolean, boolean, boolean)} with cascade = false.
+   * @param catName Catalog name.  This can be null, in which case
+   *                {@link Warehouse#DEFAULT_CATALOG_NAME} will be assumed.
+   * @param dbName database name.
+   * @param deleteData whether to drop the underlying HDFS directory.
+   * @param ignoreUnknownDb whether to ignore an attempt to drop a non-existant database
+   * @throws NoSuchObjectException No database of this name exists in the specified catalog and
+   * ignoreUnknownDb is false.
+   * @throws InvalidOperationException The database contains objects and cascade is false.
+   * @throws MetaException something went wrong, usually either in the RDBMS or storage.
+   * @throws TException general thrift error.
+   */
+  default void dropDatabase(String catName, String dbName, boolean deleteData,
+                            boolean ignoreUnknownDb)
+      throws NoSuchObjectException, InvalidOperationException, MetaException, TException {
+    dropDatabase(catName, dbName, deleteData, ignoreUnknownDb, false);
+  }
+
+  /**
+   * Drop a database.  Equivalent to
+   * {@link #dropDatabase(String, String, boolean, boolean, boolean)} with deleteData =
+   * true, ignoreUnknownDb = false, cascade = false.
+   * @param catName Catalog name.  This can be null, in which case
+   *                {@link Warehouse#DEFAULT_CATALOG_NAME} will be assumed.
+   * @param dbName database name.
+   * @throws NoSuchObjectException No database of this name exists in the specified catalog and
+   * ignoreUnknownDb is false.
+   * @throws InvalidOperationException The database contains objects and cascade is false.
+   * @throws MetaException something went wrong, usually either in the RDBMS or storage.
+   * @throws TException general thrift error.
+   */
+  default void dropDatabase(String catName, String dbName)
+      throws NoSuchObjectException, InvalidOperationException, MetaException, TException {
+    dropDatabase(catName, dbName, true, false, false);
+  }
+
+
+  /**
+   * Alter a database.
+   * @param name database name.
+   * @param db new database object.
+   * @throws NoSuchObjectException No database of this name exists in the specified catalog.
+   * @throws MetaException something went wrong, usually in the RDBMS.
+   * @throws TException general thrift error.
+   */
   void alterDatabase(String name, Database db)
       throws NoSuchObjectException, MetaException, TException;
 
   /**
-   * @param db_name
-   * @param tbl_name
-   * @param part_vals
+   * Alter a database.
+   * @param catName Catalog name.  This can be null, in which case
+   *                {@link Warehouse#DEFAULT_CATALOG_NAME} will be assumed.
+   * @param dbName database name.
+   * @param newDb new database object.
+   * @throws NoSuchObjectException No database of this name exists in the specified catalog.
+   * @throws MetaException something went wrong, usually in the RDBMS.
+   * @throws TException general thrift error.
+   */
+  void alterDatabase(String catName, String dbName, Database newDb)
+      throws NoSuchObjectException, MetaException, TException;
+
+  /**
+   * Drop a partition.
+   * @param db_name database name
+   * @param tbl_name table name
+   * @param part_vals partition values, in the same order as the partition keys
    * @param deleteData
-   *          delete the underlying data or just delete the table in metadata
+   *          delete the underlying data or just delete the partition in metadata
    * @return true or false
-   * @throws NoSuchObjectException
-   * @throws MetaException
-   * @throws TException
-   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#drop_partition(java.lang.String,
-   *      java.lang.String, java.util.List, boolean)
+   * @throws NoSuchObjectException partition does not exist
+   * @throws MetaException error accessing the RDBMS or the storage.
+   * @throws TException thrift transport error
+   */
+  boolean dropPartition(String db_name, String tbl_name,
+      List<String> part_vals, boolean deleteData) throws NoSuchObjectException,
+      MetaException, TException;
+
+  /**
+   * Drop a partition.
+   * @param catName catalog name.
+   * @param db_name database name
+   * @param tbl_name table name
+   * @param part_vals partition values, in the same order as the partition keys
+   * @param deleteData
+   *          delete the underlying data or just delete the partition in metadata
+   * @return true or false
+   * @throws NoSuchObjectException partition does not exist
+   * @throws MetaException error accessing the RDBMS or the storage.
+   * @throws TException thrift transport error
+   */
+  boolean dropPartition(String catName, String db_name, String tbl_name,
+                        List<String> part_vals, boolean deleteData) throws NoSuchObjectException,
+      MetaException, TException;
+
+  /**
+   * Drop a partition with the option to purge the partition data directly,
+   * rather than to move data to trash.
+   * @param db_name Name of the database.
+   * @param tbl_name Name of the table.
+   * @param part_vals Specification of the partitions being dropped.
+   * @param options PartitionDropOptions for the operation.
+   * @return True (if partitions are dropped), else false.
+   * @throws NoSuchObjectException partition does not exist
+   * @throws MetaException error accessing the RDBMS or the storage.
+   * @throws TException thrift transport error.
    */
-  boolean dropPartition(String db_name, String tbl_name,
-      List<String> part_vals, boolean deleteData) throws NoSuchObjectException,
-      MetaException, TException;
+  boolean dropPartition(String db_name, String tbl_name, List<String> part_vals,
+                        PartitionDropOptions options)
+      throws NoSuchObjectException, MetaException, TException;
 
   /**
-   * Method to dropPartitions() with the option to purge the partition data directly,
+   * Drop a partition with the option to purge the partition data directly,
    * rather than to move data to trash.
+   * @param catName catalog name.
    * @param db_name Name of the database.
    * @param tbl_name Name of the table.
    * @param part_vals Specification of the partitions being dropped.
    * @param options PartitionDropOptions for the operation.
    * @return True (if partitions are dropped), else false.
-   * @throws TException
+   * @throws NoSuchObjectException partition does not exist
+   * @throws MetaException error accessing the RDBMS or the storage.
+   * @throws TException thrift transport error.
    */
-  boolean dropPartition(String db_name, String tbl_name, List<String> part_vals,
-                        PartitionDropOptions options) throws TException;
+  boolean dropPartition(String catName, String db_name, String tbl_name, List<String> part_vals,
+                        PartitionDropOptions options)
+      throws NoSuchObjectException, MetaException, TException;
 
+  /**
+   * Drop partitions based on an expression.
+   * @param dbName database name.
+   * @param tblName table name.
+   * @param partExprs I don't understand this fully, so can't completely explain it.  The second
+   *                  half of the object pair is an expression used to determine which partitions
+   *                  to drop.  The first half has something to do with archive level, but I
+   *                  don't understand what.  I'm also not sure what happens if you pass multiple
+   *                  expressions.
+   * @param deleteData whether to delete the data as well as the metadata.
+   * @param ifExists if true, it is not an error if no partitions match the expression(s).
+   * @return list of deleted partitions.
+   * @throws NoSuchObjectException No partition matches the expression(s), and ifExists was false.
+   * @throws MetaException error access the RDBMS or storage.
+   * @throws TException Thrift transport error.
+   */
   List<Partition> dropPartitions(String dbName, String tblName,
                                  List<ObjectPair<Integer, byte[]>> partExprs, boolean deleteData,
                                  boolean ifExists) throws NoSuchObjectException, MetaException, TException;
 
+  /**
+   * Drop partitions based on an expression.
+   * @param catName catalog name.
+   * @param dbName database name.
+   * @param tblName table name.
+   * @param partExprs I don't understand this fully, so can't completely explain it.  The second
+   *                  half of the object pair is an expression used to determine which partitions
+   *                  to drop.  The first half has something to do with archive level, but I
+   *                  don't understand what.  I'm also not sure what happens if you pass multiple
+   *                  expressions.
+   * @param deleteData whether to delete the data as well as the metadata.
+   * @param ifExists if true, it is not an error if no partitions match the expression(s).
+   * @return list of deleted partitions.
+   * @throws NoSuchObjectException No partition matches the expression(s), and ifExists was false.
+   * @throws MetaException error access the RDBMS or storage.
+   * @throws TException Thrift transport error.
+   */
+  default List<Partition> dropPartitions(String catName, String dbName, String tblName,
+                                         List<ObjectPair<Integer, byte[]>> partExprs,
+                                         boolean deleteData, boolean ifExists)
+      throws NoSuchObjectException, MetaException, TException {
+    return dropPartitions(catName, dbName, tblName, partExprs,
+        PartitionDropOptions.instance()
+            .deleteData(deleteData)
+            .ifExists(ifExists));
+  }
+
+  /**
+   * Drop partitions based on an expression.
+   * @param dbName database name.
+   * @param tblName table name.
+   * @param partExprs I don't understand this fully, so can't completely explain it.  The second
+   *                  half of the object pair is an expression used to determine which partitions
+   *                  to drop.  The first half has something to do with archive level, but I
+   *                  don't understand what.  I'm also not sure what happens if you pass multiple
+   *                  expressions.
+   * @param deleteData whether to delete the data as well as the metadata.
+   * @param ifExists if true, it is not an error if no partitions match the expression(s).
+   * @param needResults if true, the list of deleted partitions will be returned, if not, null
+   *                    will be returned.
+   * @return list of deleted partitions.
+   * @throws NoSuchObjectException No partition matches the expression(s), and ifExists was false.
+   * @throws MetaException error access the RDBMS or storage.
+   * @throws TException Thrift transport error.
+   * @deprecated Use {@link #dropPartitions(String, String, String, List, boolean, boolean, boolean)}
+   */
   List<Partition> dropPartitions(String dbName, String tblName,
       List<ObjectPair<Integer, byte[]>> partExprs, boolean deleteData,
       boolean ifExists, boolean needResults) throws NoSuchObjectException, MetaException, TException;
 
   /**
+   * Drop partitions based on an expression.
+   * @param catName catalog name.
+   * @param dbName database name.
+   * @param tblName table name.
+   * @param partExprs I don't understand this fully, so can't completely explain it.  The second
+   *                  half of the object pair is an expression used to determine which partitions
+   *                  to drop.  The first half has something to do with archive level, but I
+   *                  don't understand what.  I'm also not sure what happens if you pass multiple
+   *                  expressions.
+   * @param deleteData whether to delete the data as well as the metadata.
+   * @param ifExists if true, it is not an error if no partitions match the expression(s).
+   * @param needResults if true, the list of deleted partitions will be returned, if not, null
+   *                    will be returned.
+   * @return list of deleted partitions, if needResults is true
+   * @throws NoSuchObjectException No partition matches the expression(s), and ifExists was false.
+   * @throws MetaException error access the RDBMS or storage.
+   * @throws TException Thrift transport error.
+   */
+  default List<Partition> dropPartitions(String catName, String dbName, String tblName,
+                                         List<ObjectPair<Integer, byte[]>> partExprs, boolean deleteData,
+                                         boolean ifExists, boolean needResults)
+      throws NoSuchObjectException, MetaException, TException {
+    return dropPartitions(catName, dbName, tblName, partExprs,
+        PartitionDropOptions.instance()
+            .deleteData(deleteData)
+            .ifExists(ifExists)
+            .returnResults(needResults));
+  }
+
+  /**
    * Generalization of dropPartitions(),
    * @param dbName Name of the database
    * @param tblName Name of the table
    * @param partExprs Partition-specification
    * @param options Boolean options for dropping partitions
    * @return List of Partitions dropped
+   *

<TRUNCATED>

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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/SharedCache.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/SharedCache.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/SharedCache.java
index cf92eda..89b4006 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/SharedCache.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/SharedCache.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.metastore.cache;
 import java.security.MessageDigest;
 import java.security.NoSuchAlgorithmException;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -30,11 +31,15 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.TreeMap;
+
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.StatObjectConverter;
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
+import org.apache.hadoop.hive.metastore.api.Catalog;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
@@ -50,29 +55,38 @@ import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
 
+import static org.apache.hadoop.hive.metastore.utils.StringUtils.normalizeIdentifier;
+
 public class SharedCache {
   private static ReentrantReadWriteLock cacheLock = new ReentrantReadWriteLock(true);
+  private boolean isCatalogCachePrewarmed = false;
+  private Map<String, Catalog> catalogCache = new TreeMap<>();
+  private HashSet<String> catalogsDeletedDuringPrewarm = new HashSet<>();
+  private AtomicBoolean isCatalogCacheDirty = new AtomicBoolean(false);
+
   // For caching Database objects. Key is database name
-  private Map<String, Database> databaseCache = new ConcurrentHashMap<String, Database>();
+  private Map<String, Database> databaseCache = new TreeMap<>();
   private boolean isDatabaseCachePrewarmed = false;
-  private HashSet<String> databasesDeletedDuringPrewarm = new HashSet<String>();
+  private HashSet<String> databasesDeletedDuringPrewarm = new HashSet<>();
   private AtomicBoolean isDatabaseCacheDirty = new AtomicBoolean(false);
+
   // For caching TableWrapper objects. Key is aggregate of database name and table name
-  private Map<String, TableWrapper> tableCache = new ConcurrentHashMap<String, TableWrapper>();
+  private Map<String, TableWrapper> tableCache = new TreeMap<>();
   private boolean isTableCachePrewarmed = false;
-  private HashSet<String> tablesDeletedDuringPrewarm = new HashSet<String>();
+  private HashSet<String> tablesDeletedDuringPrewarm = new HashSet<>();
   private AtomicBoolean isTableCacheDirty = new AtomicBoolean(false);
-  private Map<ByteArrayWrapper, StorageDescriptorWrapper> sdCache = new ConcurrentHashMap<>();
+  private Map<ByteArrayWrapper, StorageDescriptorWrapper> sdCache = new HashMap<>();
+  private Configuration conf;
   private static MessageDigest md;
   static final private Logger LOG = LoggerFactory.getLogger(SharedCache.class.getName());
   private AtomicLong cacheUpdateCount = new AtomicLong(0);
 
-  static enum StatsType {
+  enum StatsType {
     ALL(0), ALLBUTDEFAULT(1);
 
     private final int position;
 
-    private StatsType(int position) {
+    StatsType(int position) {
       this.position = position;
     }
 
@@ -155,6 +169,10 @@ public class SharedCache {
       this.parameters = parameters;
     }
 
+    boolean sameDatabase(String catName, String dbName) {
+      return catName.equals(t.getCatName()) && dbName.equals(t.getDbName());
+    }
+
     void cachePartition(Partition part, SharedCache sharedCache) {
       try {
         tableLock.writeLock().lock();
@@ -669,12 +687,102 @@ public class SharedCache {
     }
   }
 
-  public Database getDatabaseFromCache(String name) {
+  public void populateCatalogsInCache(Collection<Catalog> catalogs) {
+    for (Catalog cat : catalogs) {
+      Catalog catCopy = cat.deepCopy();
+      // ObjectStore also stores db name in lowercase
+      catCopy.setName(catCopy.getName().toLowerCase());
+      try {
+        cacheLock.writeLock().lock();
+        // Since we allow write operations on cache while prewarm is happening:
+        // 1. Don't add databases that were deleted while we were preparing list for prewarm
+        // 2. Skip overwriting exisiting db object
+        // (which is present because it was added after prewarm started)
+        if (catalogsDeletedDuringPrewarm.contains(catCopy.getName())) {
+          continue;
+        }
+        catalogCache.putIfAbsent(catCopy.getName(), catCopy);
+        catalogsDeletedDuringPrewarm.clear();
+        isCatalogCachePrewarmed = true;
+      } finally {
+        cacheLock.writeLock().unlock();
+      }
+    }
+  }
+
+  public Catalog getCatalogFromCache(String name) {
+    Catalog cat = null;
+    try {
+      cacheLock.readLock().lock();
+      if (catalogCache.get(name) != null) {
+        cat = catalogCache.get(name).deepCopy();
+      }
+    } finally {
+      cacheLock.readLock().unlock();
+    }
+    return cat;
+  }
+
+  public void addCatalogToCache(Catalog cat) {
+    try {
+      cacheLock.writeLock().lock();
+      Catalog catCopy = cat.deepCopy();
+      // ObjectStore also stores db name in lowercase
+      catCopy.setName(catCopy.getName().toLowerCase());
+      catalogCache.put(cat.getName(), catCopy);
+      isCatalogCacheDirty.set(true);
+    } finally {
+      cacheLock.writeLock().unlock();
+    }
+  }
+
+  public void alterCatalogInCache(String catName, Catalog newCat) {
+    try {
+      cacheLock.writeLock().lock();
+      removeCatalogFromCache(catName);
+      addCatalogToCache(newCat.deepCopy());
+    } finally {
+      cacheLock.writeLock().unlock();
+    }
+  }
+
+  public void removeCatalogFromCache(String name) {
+    name = normalizeIdentifier(name);
+    try {
+      cacheLock.writeLock().lock();
+      // If db cache is not yet prewarmed, add this to a set which the prewarm thread can check
+      // so that the prewarm thread does not add it back
+      if (!isCatalogCachePrewarmed) {
+        catalogsDeletedDuringPrewarm.add(name);
+      }
+      if (catalogCache.remove(name) != null) {
+        isCatalogCacheDirty.set(true);
+      }
+    } finally {
+      cacheLock.writeLock().unlock();
+    }
+  }
+
+  public List<String> listCachedCatalogs() {
+    try {
+      cacheLock.readLock().lock();
+      return new ArrayList<>(catalogCache.keySet());
+    } finally {
+      cacheLock.readLock().unlock();
+    }
+  }
+
+  public boolean isCatalogCachePrewarmed() {
+    return isCatalogCachePrewarmed;
+  }
+
+  public Database getDatabaseFromCache(String catName, String name) {
     Database db = null;
     try {
       cacheLock.readLock().lock();
-      if (databaseCache.get(name) != null) {
-        db = databaseCache.get(name).deepCopy();
+      String key = CacheUtils.buildDbKey(catName, name);
+      if (databaseCache.get(key) != null) {
+        db = databaseCache.get(key).deepCopy();
       }
     } finally {
       cacheLock.readLock().unlock();
@@ -693,10 +801,11 @@ public class SharedCache {
         // 1. Don't add databases that were deleted while we were preparing list for prewarm
         // 2. Skip overwriting exisiting db object
         // (which is present because it was added after prewarm started)
-        if (databasesDeletedDuringPrewarm.contains(dbCopy.getName().toLowerCase())) {
+        String key = CacheUtils.buildDbKey(dbCopy.getCatalogName().toLowerCase(), dbCopy.getName().toLowerCase());
+        if (databasesDeletedDuringPrewarm.contains(key)) {
           continue;
         }
-        databaseCache.putIfAbsent(StringUtils.normalizeIdentifier(dbCopy.getName()), dbCopy);
+        databaseCache.putIfAbsent(key, dbCopy);
         databasesDeletedDuringPrewarm.clear();
         isDatabaseCachePrewarmed = true;
       } finally {
@@ -715,22 +824,24 @@ public class SharedCache {
       Database dbCopy = db.deepCopy();
       // ObjectStore also stores db name in lowercase
       dbCopy.setName(dbCopy.getName().toLowerCase());
-      databaseCache.put(StringUtils.normalizeIdentifier(dbCopy.getName()), dbCopy);
+      dbCopy.setCatalogName(dbCopy.getCatalogName().toLowerCase());
+      databaseCache.put(CacheUtils.buildDbKey(dbCopy.getCatalogName(), dbCopy.getName()), dbCopy);
       isDatabaseCacheDirty.set(true);
     } finally {
       cacheLock.writeLock().unlock();
     }
   }
 
-  public void removeDatabaseFromCache(String dbName) {
+  public void removeDatabaseFromCache(String catName, String dbName) {
     try {
       cacheLock.writeLock().lock();
       // If db cache is not yet prewarmed, add this to a set which the prewarm thread can check
       // so that the prewarm thread does not add it back
+      String key = CacheUtils.buildDbKey(catName, dbName);
       if (!isDatabaseCachePrewarmed) {
-        databasesDeletedDuringPrewarm.add(dbName.toLowerCase());
+        databasesDeletedDuringPrewarm.add(key);
       }
-      if (databaseCache.remove(dbName) != null) {
+      if (databaseCache.remove(key) != null) {
         isDatabaseCacheDirty.set(true);
       }
     } finally {
@@ -738,25 +849,31 @@ public class SharedCache {
     }
   }
 
-  public List<String> listCachedDatabases() {
+  public List<String> listCachedDatabases(String catName) {
     List<String> results = new ArrayList<>();
     try {
       cacheLock.readLock().lock();
-      results.addAll(databaseCache.keySet());
+      for (String pair : databaseCache.keySet()) {
+        String[] n = CacheUtils.splitDbName(pair);
+        if (catName.equals(n[0])) results.add(n[1]);
+      }
     } finally {
       cacheLock.readLock().unlock();
     }
     return results;
   }
 
-  public List<String> listCachedDatabases(String pattern) {
+  public List<String> listCachedDatabases(String catName, String pattern) {
     List<String> results = new ArrayList<>();
     try {
       cacheLock.readLock().lock();
-      for (String dbName : databaseCache.keySet()) {
-        dbName = StringUtils.normalizeIdentifier(dbName);
-        if (CacheUtils.matches(dbName, pattern)) {
-          results.add(dbName);
+      for (String pair : databaseCache.keySet()) {
+        String[] n = CacheUtils.splitDbName(pair);
+        if (catName.equals(n[0])) {
+          n[1] = StringUtils.normalizeIdentifier(n[1]);
+          if (CacheUtils.matches(n[1], pattern)) {
+            results.add(n[1]);
+          }
         }
       }
     } finally {
@@ -768,13 +885,11 @@ public class SharedCache {
   /**
    * Replaces the old db object with the new one.
    * This will add the new database to cache if it does not exist.
-   * @param dbName
-   * @param newDb
    */
-  public void alterDatabaseInCache(String dbName, Database newDb) {
+  public void alterDatabaseInCache(String catName, String dbName, Database newDb) {
     try {
       cacheLock.writeLock().lock();
-      removeDatabaseFromCache(dbName);
+      removeDatabaseFromCache(catName, dbName);
       addDatabaseToCache(newDb.deepCopy());
       isDatabaseCacheDirty.set(true);
     } finally {
@@ -810,14 +925,15 @@ public class SharedCache {
   public void populateTableInCache(Table table, ColumnStatistics tableColStats,
       List<Partition> partitions, List<ColumnStatistics> partitionColStats,
       AggrStats aggrStatsAllPartitions, AggrStats aggrStatsAllButDefaultPartition) {
+    String catName = StringUtils.normalizeIdentifier(table.getCatName());
     String dbName = StringUtils.normalizeIdentifier(table.getDbName());
     String tableName = StringUtils.normalizeIdentifier(table.getTableName());
     // Since we allow write operations on cache while prewarm is happening:
     // 1. Don't add tables that were deleted while we were preparing list for prewarm
-    if (tablesDeletedDuringPrewarm.contains(CacheUtils.buildTableCacheKey(dbName, tableName))) {
+    if (tablesDeletedDuringPrewarm.contains(CacheUtils.buildTableKey(catName, dbName, tableName))) {
       return;
     }
-    TableWrapper tblWrapper = createTableWrapper(dbName, tableName, table);
+    TableWrapper tblWrapper = createTableWrapper(catName, dbName, tableName, table);
     if (!table.isSetPartitionKeys() && (tableColStats != null)) {
       tblWrapper.updateTableColStats(tableColStats.getStatsObj());
     } else {
@@ -843,12 +959,14 @@ public class SharedCache {
       cacheLock.writeLock().lock();
       // 2. Skip overwriting exisiting table object
       // (which is present because it was added after prewarm started)
-      tableCache.putIfAbsent(CacheUtils.buildTableCacheKey(dbName, tableName), tblWrapper);
+      tableCache.putIfAbsent(CacheUtils.buildTableKey(catName, dbName, tableName), tblWrapper);
     } finally {
       cacheLock.writeLock().unlock();
     }
   }
 
+
+
   public void completeTableCachePrewarm() {
     try {
       cacheLock.writeLock().lock();
@@ -859,11 +977,11 @@ public class SharedCache {
     }
   }
 
-  public Table getTableFromCache(String dbName, String tableName) {
+  public Table getTableFromCache(String catName, String dbName, String tableName) {
     Table t = null;
     try {
       cacheLock.readLock().lock();
-      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableCacheKey(dbName, tableName));
+      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableKey(catName, dbName, tableName));
       if (tblWrapper != null) {
         t = CacheUtils.assemble(tblWrapper, this);
       }
@@ -873,11 +991,11 @@ public class SharedCache {
     return t;
   }
 
-  public TableWrapper addTableToCache(String dbName, String tblName, Table tbl) {
+  public TableWrapper addTableToCache(String catName, String dbName, String tblName, Table tbl) {
     try {
       cacheLock.writeLock().lock();
-      TableWrapper wrapper = createTableWrapper(dbName, tblName, tbl);
-      tableCache.put(CacheUtils.buildTableCacheKey(dbName, tblName), wrapper);
+      TableWrapper wrapper = createTableWrapper(catName, dbName, tblName, tbl);
+      tableCache.put(CacheUtils.buildTableKey(catName, dbName, tblName), wrapper);
       isTableCacheDirty.set(true);
       return wrapper;
     } finally {
@@ -885,14 +1003,15 @@ public class SharedCache {
     }
   }
 
-  private TableWrapper createTableWrapper(String dbName, String tblName, Table tbl) {
+  private TableWrapper createTableWrapper(String catName, String dbName, String tblName, Table tbl) {
     TableWrapper wrapper;
     Table tblCopy = tbl.deepCopy();
-    tblCopy.setDbName(StringUtils.normalizeIdentifier(dbName));
-    tblCopy.setTableName(StringUtils.normalizeIdentifier(tblName));
+    tblCopy.setCatName(normalizeIdentifier(catName));
+    tblCopy.setDbName(normalizeIdentifier(dbName));
+    tblCopy.setTableName(normalizeIdentifier(tblName));
     if (tblCopy.getPartitionKeys() != null) {
       for (FieldSchema fs : tblCopy.getPartitionKeys()) {
-        fs.setName(StringUtils.normalizeIdentifier(fs.getName()));
+        fs.setName(normalizeIdentifier(fs.getName()));
       }
     }
     if (tbl.getSd() != null) {
@@ -907,15 +1026,16 @@ public class SharedCache {
     return wrapper;
   }
 
-  public void removeTableFromCache(String dbName, String tblName) {
+
+  public void removeTableFromCache(String catName, String dbName, String tblName) {
     try {
       cacheLock.writeLock().lock();
       // If table cache is not yet prewarmed, add this to a set which the prewarm thread can check
       // so that the prewarm thread does not add it back
       if (!isTableCachePrewarmed) {
-        tablesDeletedDuringPrewarm.add(CacheUtils.buildTableCacheKey(dbName, tblName));
+        tablesDeletedDuringPrewarm.add(CacheUtils.buildTableKey(catName, dbName, tblName));
       }
-      TableWrapper tblWrapper = tableCache.remove(CacheUtils.buildTableCacheKey(dbName, tblName));
+      TableWrapper tblWrapper = tableCache.remove(CacheUtils.buildTableKey(catName, dbName, tblName));
       byte[] sdHash = tblWrapper.getSdHash();
       if (sdHash != null) {
         decrSd(sdHash);
@@ -926,15 +1046,15 @@ public class SharedCache {
     }
   }
 
-  public void alterTableInCache(String dbName, String tblName, Table newTable) {
+  public void alterTableInCache(String catName, String dbName, String tblName, Table newTable) {
     try {
       cacheLock.writeLock().lock();
-      TableWrapper tblWrapper = tableCache.remove(CacheUtils.buildTableCacheKey(dbName, tblName));
+      TableWrapper tblWrapper = tableCache.remove(CacheUtils.buildTableKey(catName, dbName, tblName));
       if (tblWrapper != null) {
         tblWrapper.updateTableObj(newTable, this);
         String newDbName = StringUtils.normalizeIdentifier(newTable.getDbName());
         String newTblName = StringUtils.normalizeIdentifier(newTable.getTableName());
-        tableCache.put(CacheUtils.buildTableCacheKey(newDbName, newTblName), tblWrapper);
+        tableCache.put(CacheUtils.buildTableKey(catName, newDbName, newTblName), tblWrapper);
         isTableCacheDirty.set(true);
       }
     } finally {
@@ -942,12 +1062,12 @@ public class SharedCache {
     }
   }
 
-  public List<Table> listCachedTables(String dbName) {
+  public List<Table> listCachedTables(String catName, String dbName) {
     List<Table> tables = new ArrayList<>();
     try {
       cacheLock.readLock().lock();
       for (TableWrapper wrapper : tableCache.values()) {
-        if (wrapper.getTable().getDbName().equals(dbName)) {
+        if (wrapper.sameDatabase(catName, dbName)) {
           tables.add(CacheUtils.assemble(wrapper, this));
         }
       }
@@ -957,12 +1077,12 @@ public class SharedCache {
     return tables;
   }
 
-  public List<String> listCachedTableNames(String dbName) {
+  public List<String> listCachedTableNames(String catName, String dbName) {
     List<String> tableNames = new ArrayList<>();
     try {
       cacheLock.readLock().lock();
       for (TableWrapper wrapper : tableCache.values()) {
-        if (wrapper.getTable().getDbName().equals(dbName)) {
+        if (wrapper.sameDatabase(catName, dbName)) {
           tableNames.add(StringUtils.normalizeIdentifier(wrapper.getTable().getTableName()));
         }
       }
@@ -972,13 +1092,13 @@ public class SharedCache {
     return tableNames;
   }
 
-  public List<String> listCachedTableNames(String dbName, String pattern, short maxTables) {
-    List<String> tableNames = new ArrayList<String>();
+  public List<String> listCachedTableNames(String catName, String dbName, String pattern, short maxTables) {
+    List<String> tableNames = new ArrayList<>();
     try {
       cacheLock.readLock().lock();
       int count = 0;
       for (TableWrapper wrapper : tableCache.values()) {
-        if ((wrapper.getTable().getDbName().equals(dbName))
+        if (wrapper.sameDatabase(catName, dbName)
             && CacheUtils.matches(wrapper.getTable().getTableName(), pattern)
             && (maxTables == -1 || count < maxTables)) {
           tableNames.add(StringUtils.normalizeIdentifier(wrapper.getTable().getTableName()));
@@ -991,12 +1111,12 @@ public class SharedCache {
     return tableNames;
   }
 
-  public List<String> listCachedTableNames(String dbName, String pattern, TableType tableType) {
-    List<String> tableNames = new ArrayList<String>();
+  public List<String> listCachedTableNames(String catName, String dbName, String pattern, TableType tableType) {
+    List<String> tableNames = new ArrayList<>();
     try {
       cacheLock.readLock().lock();
       for (TableWrapper wrapper : tableCache.values()) {
-        if ((wrapper.getTable().getDbName().equals(dbName))
+        if (wrapper.sameDatabase(catName, dbName)
             && CacheUtils.matches(wrapper.getTable().getTableName(), pattern)
             && wrapper.getTable().getTableType().equals(tableType.toString())) {
           tableNames.add(StringUtils.normalizeIdentifier(wrapper.getTable().getTableName()));
@@ -1008,23 +1128,23 @@ public class SharedCache {
     return tableNames;
   }
 
-  public void refreshTablesInCache(String dbName, List<Table> tables) {
+  public void refreshTablesInCache(String catName, String dbName, List<Table> tables) {
     try {
       cacheLock.writeLock().lock();
       if (isTableCacheDirty.compareAndSet(true, false)) {
         LOG.debug("Skipping table cache update; the table list we have is dirty.");
         return;
       }
-      Map<String, TableWrapper> newTableCache = new HashMap<String, TableWrapper>();
+      Map<String, TableWrapper> newTableCache = new HashMap<>();
       for (Table tbl : tables) {
         String tblName = StringUtils.normalizeIdentifier(tbl.getTableName());
-        TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableCacheKey(dbName, tblName));
+        TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableKey(catName, dbName, tblName));
         if (tblWrapper != null) {
           tblWrapper.updateTableObj(tbl, this);
         } else {
-          tblWrapper = createTableWrapper(dbName, tblName, tbl);
+          tblWrapper = createTableWrapper(catName, dbName, tblName, tbl);
         }
-        newTableCache.put(CacheUtils.buildTableCacheKey(dbName, tblName), tblWrapper);
+        newTableCache.put(CacheUtils.buildTableKey(catName, dbName, tblName), tblWrapper);
       }
       tableCache.clear();
       tableCache = newTableCache;
@@ -1033,12 +1153,12 @@ public class SharedCache {
     }
   }
 
-  public List<ColumnStatisticsObj> getTableColStatsFromCache(String dbName, String tblName,
-      List<String> colNames) {
-    List<ColumnStatisticsObj> colStatObjs = new ArrayList<ColumnStatisticsObj>();
+  public List<ColumnStatisticsObj> getTableColStatsFromCache(
+      String catName, String dbName, String tblName, List<String> colNames) {
+    List<ColumnStatisticsObj> colStatObjs = new ArrayList<>();
     try {
       cacheLock.readLock().lock();
-      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableCacheKey(dbName, tblName));
+      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableKey(catName, dbName, tblName));
       if (tblWrapper != null) {
         colStatObjs = tblWrapper.getCachedTableColStats(colNames);
       }
@@ -1048,10 +1168,10 @@ public class SharedCache {
     return colStatObjs;
   }
 
-  public void removeTableColStatsFromCache(String dbName, String tblName, String colName) {
+  public void removeTableColStatsFromCache(String catName, String dbName, String tblName, String colName) {
     try {
       cacheLock.readLock().lock();
-      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableCacheKey(dbName, tblName));
+      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableKey(catName, dbName, tblName));
       if (tblWrapper != null) {
         tblWrapper.removeTableColStats(colName);
       }
@@ -1060,11 +1180,11 @@ public class SharedCache {
     }
   }
 
-  public void updateTableColStatsInCache(String dbName, String tableName,
-      List<ColumnStatisticsObj> colStatsForTable) {
+  public void updateTableColStatsInCache(String catName, String dbName, String tableName,
+                                         List<ColumnStatisticsObj> colStatsForTable) {
     try {
       cacheLock.readLock().lock();
-      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableCacheKey(dbName, tableName));
+      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableKey(catName, dbName, tableName));
       if (tblWrapper != null) {
         tblWrapper.updateTableColStats(colStatsForTable);
       }
@@ -1073,11 +1193,11 @@ public class SharedCache {
     }
   }
 
-  public void refreshTableColStatsInCache(String dbName, String tableName,
-      List<ColumnStatisticsObj> colStatsForTable) {
+  public void refreshTableColStatsInCache(String catName, String dbName, String tableName,
+                                          List<ColumnStatisticsObj> colStatsForTable) {
     try {
       cacheLock.readLock().lock();
-      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableCacheKey(dbName, tableName));
+      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableKey(catName, dbName, tableName));
       if (tblWrapper != null) {
         tblWrapper.refreshTableColStats(colStatsForTable);
       }
@@ -1095,18 +1215,19 @@ public class SharedCache {
     }
   }
 
-  public List<TableMeta> getTableMeta(String dbNames, String tableNames,
-      List<String> tableTypes) {
+  public List<TableMeta> getTableMeta(String catName, String dbNames, String tableNames,
+                                      List<String> tableTypes) {
     List<TableMeta> tableMetas = new ArrayList<>();
     try {
       cacheLock.readLock().lock();
-      for (String dbName : listCachedDatabases()) {
+      for (String dbName : listCachedDatabases(catName)) {
         if (CacheUtils.matches(dbName, dbNames)) {
-          for (Table table : listCachedTables(dbName)) {
+          for (Table table : listCachedTables(catName, dbName)) {
             if (CacheUtils.matches(table.getTableName(), tableNames)) {
               if (tableTypes == null || tableTypes.contains(table.getTableType())) {
                 TableMeta metaData =
                     new TableMeta(dbName, table.getTableName(), table.getTableType());
+                metaData.setCatName(catName);
                 metaData.setComments(table.getParameters().get("comment"));
                 tableMetas.add(metaData);
               }
@@ -1120,10 +1241,10 @@ public class SharedCache {
     return tableMetas;
   }
 
-  public void addPartitionToCache(String dbName, String tblName, Partition part) {
+  public void addPartitionToCache(String catName, String dbName, String tblName, Partition part) {
     try {
       cacheLock.readLock().lock();
-      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableCacheKey(dbName, tblName));
+      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableKey(catName, dbName, tblName));
       if (tblWrapper != null) {
         tblWrapper.cachePartition(part, this);
       }
@@ -1132,10 +1253,10 @@ public class SharedCache {
     }
   }
 
-  public void addPartitionsToCache(String dbName, String tblName, List<Partition> parts) {
+  public void addPartitionsToCache(String catName, String dbName, String tblName, List<Partition> parts) {
     try {
       cacheLock.readLock().lock();
-      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableCacheKey(dbName, tblName));
+      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableKey(catName, dbName, tblName));
       if (tblWrapper != null) {
         tblWrapper.cachePartitions(parts, this);
       }
@@ -1144,12 +1265,12 @@ public class SharedCache {
     }
   }
 
-  public Partition getPartitionFromCache(String dbName, String tblName,
-      List<String> partVals) {
+  public Partition getPartitionFromCache(String catName, String dbName, String tblName,
+                                         List<String> partVals) {
     Partition part = null;
     try {
       cacheLock.readLock().lock();
-      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableCacheKey(dbName, tblName));
+      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableKey(catName, dbName, tblName));
       if (tblWrapper != null) {
         part = tblWrapper.getPartition(partVals, this);
       }
@@ -1159,11 +1280,11 @@ public class SharedCache {
     return part;
   }
 
-  public boolean existPartitionFromCache(String dbName, String tblName, List<String> partVals) {
+  public boolean existPartitionFromCache(String catName, String dbName, String tblName, List<String> partVals) {
     boolean existsPart = false;
     try {
       cacheLock.readLock().lock();
-      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableCacheKey(dbName, tblName));
+      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableKey(catName, dbName, tblName));
       if (tblWrapper != null) {
         existsPart = tblWrapper.containsPartition(partVals);
       }
@@ -1173,12 +1294,12 @@ public class SharedCache {
     return existsPart;
   }
 
-  public Partition removePartitionFromCache(String dbName, String tblName,
-      List<String> partVals) {
+  public Partition removePartitionFromCache(String catName, String dbName, String tblName,
+                                            List<String> partVals) {
     Partition part = null;
     try {
       cacheLock.readLock().lock();
-      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableCacheKey(dbName, tblName));
+      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableKey(catName, dbName, tblName));
       if (tblWrapper != null) {
         part = tblWrapper.removePartition(partVals, this);
       }
@@ -1188,11 +1309,11 @@ public class SharedCache {
     return part;
   }
 
-  public void removePartitionsFromCache(String dbName, String tblName,
-      List<List<String>> partVals) {
+  public void removePartitionsFromCache(String catName, String dbName, String tblName,
+                                        List<List<String>> partVals) {
     try {
       cacheLock.readLock().lock();
-      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableCacheKey(dbName, tblName));
+      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableKey(catName, dbName, tblName));
       if (tblWrapper != null) {
         tblWrapper.removePartitions(partVals, this);
       }
@@ -1201,11 +1322,11 @@ public class SharedCache {
     }
   }
 
-  public List<Partition> listCachedPartitions(String dbName, String tblName, int max) {
+  public List<Partition> listCachedPartitions(String catName, String dbName, String tblName, int max) {
     List<Partition> parts = new ArrayList<Partition>();
     try {
       cacheLock.readLock().lock();
-      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableCacheKey(dbName, tblName));
+      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableKey(catName, dbName, tblName));
       if (tblWrapper != null) {
         parts = tblWrapper.listPartitions(max, this);
       }
@@ -1215,11 +1336,11 @@ public class SharedCache {
     return parts;
   }
 
-  public void alterPartitionInCache(String dbName, String tblName, List<String> partVals,
-      Partition newPart) {
+  public void alterPartitionInCache(String catName, String dbName, String tblName, List<String> partVals,
+                                    Partition newPart) {
     try {
       cacheLock.readLock().lock();
-      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableCacheKey(dbName, tblName));
+      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableKey(catName, dbName, tblName));
       if (tblWrapper != null) {
         tblWrapper.alterPartition(partVals, newPart, this);
       }
@@ -1228,11 +1349,11 @@ public class SharedCache {
     }
   }
 
-  public void alterPartitionsInCache(String dbName, String tblName, List<List<String>> partValsList,
-      List<Partition> newParts) {
+  public void alterPartitionsInCache(String catName, String dbName, String tblName, List<List<String>> partValsList,
+                                     List<Partition> newParts) {
     try {
       cacheLock.readLock().lock();
-      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableCacheKey(dbName, tblName));
+      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableKey(catName, dbName, tblName));
       if (tblWrapper != null) {
         tblWrapper.alterPartitions(partValsList, newParts, this);
       }
@@ -1241,10 +1362,10 @@ public class SharedCache {
     }
   }
 
-  public void refreshPartitionsInCache(String dbName, String tblName, List<Partition> partitions) {
+  public void refreshPartitionsInCache(String catName, String dbName, String tblName, List<Partition> partitions) {
     try {
       cacheLock.readLock().lock();
-      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableCacheKey(dbName, tblName));
+      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableKey(catName, dbName, tblName));
       if (tblWrapper != null) {
         tblWrapper.refreshPartitions(partitions, this);
       }
@@ -1253,11 +1374,11 @@ public class SharedCache {
     }
   }
 
-  public void removePartitionColStatsFromCache(String dbName, String tblName,
-      List<String> partVals, String colName) {
+  public void removePartitionColStatsFromCache(String catName, String dbName, String tblName,
+                                               List<String> partVals, String colName) {
     try {
       cacheLock.readLock().lock();
-      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableCacheKey(dbName, tblName));
+      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableKey(catName, dbName, tblName));
       if (tblWrapper != null) {
         tblWrapper.removePartitionColStats(partVals, colName);
       }
@@ -1266,11 +1387,11 @@ public class SharedCache {
     }
   }
 
-  public void updatePartitionColStatsInCache(String dbName, String tableName,
-      List<String> partVals, List<ColumnStatisticsObj> colStatsObjs) {
+  public void updatePartitionColStatsInCache(String catName, String dbName, String tableName,
+                                             List<String> partVals, List<ColumnStatisticsObj> colStatsObjs) {
     try {
       cacheLock.readLock().lock();
-      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableCacheKey(dbName, tableName));
+      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableKey(catName, dbName, tableName));
       if (tblWrapper != null) {
         tblWrapper.updatePartitionColStats(partVals, colStatsObjs);
       }
@@ -1279,12 +1400,12 @@ public class SharedCache {
     }
   }
 
-  public ColumnStatisticsObj getPartitionColStatsFromCache(String dbName, String tblName,
-      List<String> partVal, String colName) {
+  public ColumnStatisticsObj getPartitionColStatsFromCache(String catName, String dbName, String tblName,
+                                                           List<String> partVal, String colName) {
     ColumnStatisticsObj colStatObj = null;
     try {
       cacheLock.readLock().lock();
-      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableCacheKey(dbName, tblName));
+      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableKey(catName, dbName, tblName));
       if (tblWrapper != null){
         colStatObj = tblWrapper.getPartitionColStats(partVal, colName);
       }
@@ -1294,11 +1415,11 @@ public class SharedCache {
     return colStatObj;
   }
 
-  public void refreshPartitionColStatsInCache(String dbName, String tblName,
-      List<ColumnStatistics> partitionColStats) {
+  public void refreshPartitionColStatsInCache(String catName, String dbName, String tblName,
+                                              List<ColumnStatistics> partitionColStats) {
     try {
       cacheLock.readLock().lock();
-      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableCacheKey(dbName, tblName));
+      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableKey(catName, dbName, tblName));
       if (tblWrapper != null) {
         tblWrapper.refreshPartitionColStats(partitionColStats);
       }
@@ -1307,11 +1428,11 @@ public class SharedCache {
     }
   }
 
-  public List<ColumnStatisticsObj> getAggrStatsFromCache(String dbName, String tblName,
-      List<String> colNames, StatsType statsType) {
+  public List<ColumnStatisticsObj> getAggrStatsFromCache(String catName, String dbName, String tblName,
+                                                         List<String> colNames, StatsType statsType) {
     try {
       cacheLock.readLock().lock();
-      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableCacheKey(dbName, tblName));
+      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableKey(catName, dbName, tblName));
       if (tblWrapper != null) {
         return tblWrapper.getAggrPartitionColStats(colNames, statsType);
       }
@@ -1321,11 +1442,11 @@ public class SharedCache {
     return null;
   }
 
-  public void addAggregateStatsToCache(String dbName, String tblName,
-      AggrStats aggrStatsAllPartitions, AggrStats aggrStatsAllButDefaultPartition) {
+  public void addAggregateStatsToCache(String catName, String dbName, String tblName,
+                                       AggrStats aggrStatsAllPartitions, AggrStats aggrStatsAllButDefaultPartition) {
     try {
       cacheLock.readLock().lock();
-      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableCacheKey(dbName, tblName));
+      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableKey(catName, dbName, tblName));
       if (tblWrapper != null){
         tblWrapper.cacheAggrPartitionColStats(aggrStatsAllPartitions,
             aggrStatsAllButDefaultPartition);
@@ -1335,11 +1456,11 @@ public class SharedCache {
     }
   }
 
-  public void refreshAggregateStatsInCache(String dbName, String tblName,
-      AggrStats aggrStatsAllPartitions, AggrStats aggrStatsAllButDefaultPartition) {
+  public void refreshAggregateStatsInCache(String catName, String dbName, String tblName,
+                                           AggrStats aggrStatsAllPartitions, AggrStats aggrStatsAllButDefaultPartition) {
     try {
       cacheLock.readLock().lock();
-      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableCacheKey(dbName, tblName));
+      TableWrapper tblWrapper = tableCache.get(CacheUtils.buildTableKey(catName, dbName, tblName));
       if (tblWrapper != null) {
         tblWrapper.refreshAggrPartitionColStats(aggrStatsAllPartitions,
             aggrStatsAllButDefaultPartition);
@@ -1390,6 +1511,16 @@ public class SharedCache {
     return sdCache;
   }
 
+  /**
+   * This resets the contents of the cataog cache so that we can re-fill it in another test.
+   */
+  void resetCatalogCache() {
+    isCatalogCachePrewarmed = false;
+    catalogCache.clear();
+    catalogsDeletedDuringPrewarm.clear();
+    isCatalogCacheDirty.set(false);
+  }
+
   public long getUpdateCount() {
     return cacheUpdateCount.get();
   }
@@ -1398,3 +1529,8 @@ public class SharedCache {
     cacheUpdateCount.incrementAndGet();
   }
 }
+
+
+
+
+

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/CatalogBuilder.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/CatalogBuilder.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/CatalogBuilder.java
new file mode 100644
index 0000000..be76d93
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/CatalogBuilder.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.client.builder;
+
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.Catalog;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.thrift.TException;
+
+public class CatalogBuilder {
+  private String name, description, location;
+
+  public CatalogBuilder setName(String name) {
+    this.name = name;
+    return this;
+  }
+
+  public CatalogBuilder setDescription(String description) {
+    this.description = description;
+    return this;
+  }
+
+  public CatalogBuilder setLocation(String location) {
+    this.location = location;
+    return this;
+  }
+
+  public Catalog build() throws MetaException {
+    if (name == null) throw new MetaException("You must name the catalog");
+    if (location == null) throw new MetaException("You must give the catalog a location");
+    Catalog catalog = new Catalog(name, location);
+    if (description != null) catalog.setDescription(description);
+    return catalog;
+  }
+
+  /**
+   * Build the catalog object and create it in the metastore.
+   * @param client metastore client
+   * @return new catalog object
+   * @throws TException thrown from the client
+   */
+  public Catalog create(IMetaStoreClient client) throws TException {
+    Catalog cat = build();
+    client.createCatalog(cat);
+    return cat;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/ConstraintBuilder.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/ConstraintBuilder.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/ConstraintBuilder.java
index 50e779a..2e32cbf 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/ConstraintBuilder.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/ConstraintBuilder.java
@@ -17,8 +17,15 @@
  */
 package org.apache.hadoop.hive.metastore.client.builder;
 
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+
+import java.util.ArrayList;
+import java.util.List;
 
 /**
  * Base builder for all types of constraints.  Database name, table name, and column name
@@ -26,28 +33,42 @@ import org.apache.hadoop.hive.metastore.api.Table;
  * @param <T> Type of builder extending this.
  */
 abstract class ConstraintBuilder<T> {
-  protected String dbName, tableName, columnName, constraintName;
-  protected int keySeq;
+  protected String catName, dbName, tableName, constraintName;
+  List<String> columns;
   protected boolean enable, validate, rely;
+  private int nextSeq;
   private T child;
 
   protected ConstraintBuilder() {
-    keySeq = 1;
+    nextSeq = 1;
     enable = true;
     validate = rely = false;
+    dbName = Warehouse.DEFAULT_DATABASE_NAME;
+    columns = new ArrayList<>();
   }
 
   protected void setChild(T child) {
     this.child = child;
   }
 
-  protected void checkBuildable(String defaultConstraintName) throws MetaException {
-    if (dbName == null || tableName == null || columnName == null) {
-      throw new MetaException("You must provide database name, table name, and column name");
+  protected void checkBuildable(String defaultConstraintName, Configuration conf)
+      throws MetaException {
+    if (tableName == null || columns.isEmpty()) {
+      throw new MetaException("You must provide table name and columns");
     }
     if (constraintName == null) {
-      constraintName = dbName + "_" + tableName + "_" + columnName + "_" + defaultConstraintName;
+      constraintName = tableName + "_" + defaultConstraintName;
     }
+    if (catName == null) catName = MetaStoreUtils.getDefaultCatalog(conf);
+  }
+
+  protected int getNextSeq() {
+    return nextSeq++;
+  }
+
+  public T setCatName(String catName) {
+    this.catName = catName;
+    return child;
   }
 
   public T setDbName(String dbName) {
@@ -60,14 +81,15 @@ abstract class ConstraintBuilder<T> {
     return child;
   }
 
-  public T setDbAndTableName(Table table) {
+  public T onTable(Table table) {
+    this.catName = table.getCatName();
     this.dbName = table.getDbName();
     this.tableName = table.getTableName();
     return child;
   }
 
-  public T setColumnName(String columnName) {
-    this.columnName = columnName;
+  public T addColumn(String columnName) {
+    this.columns.add(columnName);
     return child;
   }
 
@@ -76,11 +98,6 @@ abstract class ConstraintBuilder<T> {
     return child;
   }
 
-  public T setKeySeq(int keySeq) {
-    this.keySeq = keySeq;
-    return child;
-  }
-
   public T setEnable(boolean enable) {
     this.enable = enable;
     return child;

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/DatabaseBuilder.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/DatabaseBuilder.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/DatabaseBuilder.java
index 01693ec..f3d2182 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/DatabaseBuilder.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/DatabaseBuilder.java
@@ -17,6 +17,10 @@
  */
 package org.apache.hadoop.hive.metastore.client.builder;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.Warehouse;
+import org.apache.hadoop.hive.metastore.api.Catalog;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.PrincipalType;
@@ -33,11 +37,24 @@ import java.util.Map;
  * selects reasonable defaults.
  */
 public class DatabaseBuilder {
-  private String name, description, location;
+  private String name, description, location, catalogName;
   private Map<String, String> params = new HashMap<>();
   private String ownerName;
   private PrincipalType ownerType;
 
+  public DatabaseBuilder() {
+  }
+
+  public DatabaseBuilder setCatalogName(String catalogName) {
+    this.catalogName = catalogName;
+    return this;
+  }
+
+  public DatabaseBuilder setCatalogName(Catalog catalog) {
+    this.catalogName = catalog.getName();
+    return this;
+  }
+
   public DatabaseBuilder setName(String name) {
     this.name = name;
     return this;
@@ -73,11 +90,13 @@ public class DatabaseBuilder {
     return this;
   }
 
-  public Database build() throws MetaException {
+  public Database build(Configuration conf) throws MetaException {
     if (name == null) throw new MetaException("You must name the database");
+    if (catalogName == null) catalogName = MetaStoreUtils.getDefaultCatalog(conf);
     Database db = new Database(name, description, location, params);
+    db.setCatalogName(catalogName);
     try {
-      if (ownerName != null) ownerName = SecurityUtils.getUser();
+      if (ownerName == null) ownerName = SecurityUtils.getUser();
       db.setOwnerName(ownerName);
       if (ownerType == null) ownerType = PrincipalType.USER;
       db.setOwnerType(ownerType);
@@ -86,4 +105,18 @@ public class DatabaseBuilder {
       throw MetaStoreUtils.newMetaException(e);
     }
   }
+
+  /**
+   * Build the database, create it in the metastore, and then return the db object.
+   * @param client metastore client
+   * @param conf configuration file
+   * @return new database object
+   * @throws TException comes from {@link #build(Configuration)} or
+   * {@link IMetaStoreClient#createDatabase(Database)}.
+   */
+  public Database create(IMetaStoreClient client, Configuration conf) throws TException {
+    Database db = build(conf);
+    client.createDatabase(db);
+    return db;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/FunctionBuilder.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/FunctionBuilder.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/FunctionBuilder.java
index aa9b9f5..c4c09dc 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/FunctionBuilder.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/FunctionBuilder.java
@@ -18,6 +18,9 @@
 
 package org.apache.hadoop.hive.metastore.client.builder;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.Function;
 import org.apache.hadoop.hive.metastore.api.FunctionType;
@@ -26,6 +29,7 @@ import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.hadoop.hive.metastore.api.ResourceUri;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.utils.SecurityUtils;
+import org.apache.thrift.TException;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -35,7 +39,7 @@ import java.util.List;
  * Class for creating Thrift Function objects for tests, and API usage.
  */
 public class FunctionBuilder {
-  private String dbName = "default";
+  private String catName, dbName;
   private String funcName = null;
   private String className = null;
   private String owner = null;
@@ -49,7 +53,13 @@ public class FunctionBuilder {
     ownerType = PrincipalType.USER;
     createTime = (int) (System.currentTimeMillis() / 1000);
     funcType = FunctionType.JAVA;
-    resourceUris = new ArrayList<ResourceUri>();
+    resourceUris = new ArrayList<>();
+    dbName = Warehouse.DEFAULT_DATABASE_NAME;
+  }
+
+  public FunctionBuilder setCatName(String catName) {
+    this.catName = catName;
+    return this;
   }
 
   public FunctionBuilder setDbName(String dbName) {
@@ -57,8 +67,9 @@ public class FunctionBuilder {
     return this;
   }
 
-  public FunctionBuilder setDbName(Database db) {
+  public FunctionBuilder inDb(Database db) {
     this.dbName = db.getName();
+    this.catName = db.getCatalogName();
     return this;
   }
 
@@ -102,7 +113,7 @@ public class FunctionBuilder {
     return this;
   }
 
-  public Function build() throws MetaException {
+  public Function build(Configuration conf) throws MetaException {
     try {
       if (owner != null) {
         owner = SecurityUtils.getUser();
@@ -110,7 +121,23 @@ public class FunctionBuilder {
     } catch (IOException e) {
       throw MetaStoreUtils.newMetaException(e);
     }
-    return new Function(funcName, dbName, className, owner, ownerType, createTime, funcType,
+    if (catName == null) catName = MetaStoreUtils.getDefaultCatalog(conf);
+    Function f = new Function(funcName, dbName, className, owner, ownerType, createTime, funcType,
         resourceUris);
+    f.setCatName(catName);
+    return f;
+  }
+
+  /**
+   * Create the function object in the metastore and return it.
+   * @param client metastore client
+   * @param conf configuration
+   * @return new function object
+   * @throws TException if thrown by build or the client.
+   */
+  public Function create(IMetaStoreClient client, Configuration conf) throws TException {
+    Function f = build(conf);
+    client.createFunction(f);
+    return f;
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/ISchemaBuilder.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/ISchemaBuilder.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/ISchemaBuilder.java
index 32a84ac..f61a62c 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/ISchemaBuilder.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/ISchemaBuilder.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hive.metastore.client.builder;
 
 import org.apache.hadoop.hive.metastore.Warehouse;
+import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.ISchema;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.SchemaCompatibility;
@@ -27,7 +28,7 @@ import org.apache.hadoop.hive.metastore.api.SchemaValidation;
 public class ISchemaBuilder {
   private SchemaType schemaType; // required
   private String name; // required
-  private String dbName; // required
+  private String dbName, catName; // required
   private SchemaCompatibility compatibility; // required
   private SchemaValidation validationLevel; // required
   private boolean canEvolve; // required
@@ -39,6 +40,7 @@ public class ISchemaBuilder {
     validationLevel = SchemaValidation.ALL;
     canEvolve = true;
     dbName = Warehouse.DEFAULT_DATABASE_NAME;
+    catName = Warehouse.DEFAULT_CATALOG_NAME;
   }
 
   public ISchemaBuilder setSchemaType(SchemaType schemaType) {
@@ -56,6 +58,12 @@ public class ISchemaBuilder {
     return this;
   }
 
+  public ISchemaBuilder inDb(Database db) {
+    this.catName = db.getCatalogName();
+    this.dbName = db.getName();
+    return this;
+  }
+
   public ISchemaBuilder setCompatibility(SchemaCompatibility compatibility) {
     this.compatibility = compatibility;
     return this;
@@ -86,7 +94,7 @@ public class ISchemaBuilder {
       throw new MetaException("You must provide a schemaType and name");
     }
     ISchema iSchema =
-        new ISchema(schemaType, name, dbName, compatibility, validationLevel, canEvolve);
+        new ISchema(schemaType, name, catName, dbName, compatibility, validationLevel, canEvolve);
     if (schemaGroup != null) iSchema.setSchemaGroup(schemaGroup);
     if (description != null) iSchema.setDescription(description);
     return iSchema;

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/PartitionBuilder.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/PartitionBuilder.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/PartitionBuilder.java
index 38e5a8f..d6ee673 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/PartitionBuilder.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/PartitionBuilder.java
@@ -17,9 +17,14 @@
  */
 package org.apache.hadoop.hive.metastore.client.builder;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.thrift.TException;
 
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -31,7 +36,7 @@ import java.util.Map;
  * reference; 2. partition values; 3. whatever {@link StorageDescriptorBuilder} requires.
  */
 public class PartitionBuilder extends StorageDescriptorBuilder<PartitionBuilder> {
-  private String dbName, tableName;
+  private String catName, dbName, tableName;
   private int createTime, lastAccessTime;
   private Map<String, String> partParams;
   private List<String> values;
@@ -40,6 +45,7 @@ public class PartitionBuilder extends StorageDescriptorBuilder<PartitionBuilder>
     // Set some reasonable defaults
     partParams = new HashMap<>();
     createTime = lastAccessTime = (int)(System.currentTimeMillis() / 1000);
+    dbName = Warehouse.DEFAULT_DATABASE_NAME;
     super.setChild(this);
   }
 
@@ -53,9 +59,10 @@ public class PartitionBuilder extends StorageDescriptorBuilder<PartitionBuilder>
     return this;
   }
 
-  public PartitionBuilder fromTable(Table table) {
+  public PartitionBuilder inTable(Table table) {
     this.dbName = table.getDbName();
     this.tableName = table.getTableName();
+    this.catName = table.getCatName();
     setCols(table.getSd().getCols());
     return this;
   }
@@ -92,12 +99,21 @@ public class PartitionBuilder extends StorageDescriptorBuilder<PartitionBuilder>
     return this;
   }
 
-  public Partition build() throws MetaException {
-    if (dbName == null || tableName == null) {
-      throw new MetaException("database name and table name must be provided");
+  public Partition build(Configuration conf) throws MetaException {
+    if (tableName == null) {
+      throw new MetaException("table name must be provided");
     }
     if (values == null) throw new MetaException("You must provide partition values");
-    return new Partition(values, dbName, tableName, createTime, lastAccessTime, buildSd(),
+    if (catName == null) catName = MetaStoreUtils.getDefaultCatalog(conf);
+    Partition p = new Partition(values, dbName, tableName, createTime, lastAccessTime, buildSd(),
         partParams);
+    p.setCatName(catName);
+    return p;
+  }
+
+  public Partition addToTable(IMetaStoreClient client, Configuration conf) throws TException {
+    Partition p = build(conf);
+    client.add_partition(p);
+    return p;
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SQLCheckConstraintBuilder.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SQLCheckConstraintBuilder.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SQLCheckConstraintBuilder.java
new file mode 100644
index 0000000..ec99729
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SQLCheckConstraintBuilder.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.client.builder;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.SQLCheckConstraint;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class SQLCheckConstraintBuilder extends ConstraintBuilder<SQLCheckConstraintBuilder> {
+  private String checkExpression;
+
+  public SQLCheckConstraintBuilder() {
+    super.setChild(this);
+  }
+
+  public SQLCheckConstraintBuilder setCheckExpression(String checkExpression) {
+    this.checkExpression = checkExpression;
+    return this;
+  }
+
+  public List<SQLCheckConstraint> build(Configuration conf) throws MetaException {
+    if (checkExpression == null) {
+      throw new MetaException("check expression must be set");
+    }
+    checkBuildable("check_constraint", conf);
+    List<SQLCheckConstraint> cc = new ArrayList<>(columns.size());
+    for (String column : columns) {
+      cc.add(new SQLCheckConstraint(catName, dbName, tableName, column, checkExpression,
+          constraintName, enable, validate, rely));
+    }
+    return cc;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SQLDefaultConstraintBuilder.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SQLDefaultConstraintBuilder.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SQLDefaultConstraintBuilder.java
new file mode 100644
index 0000000..b24663d
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SQLDefaultConstraintBuilder.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore.client.builder;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class SQLDefaultConstraintBuilder extends ConstraintBuilder<SQLDefaultConstraintBuilder> {
+  private Object defaultVal;
+
+  public SQLDefaultConstraintBuilder() {
+    super.setChild(this);
+  }
+
+  public SQLDefaultConstraintBuilder setDefaultVal(Object defaultVal) {
+    this.defaultVal = defaultVal;
+    return this;
+  }
+
+  public List<SQLDefaultConstraint> build(Configuration conf) throws MetaException {
+    if (defaultVal == null) {
+      throw new MetaException("default value must be set");
+    }
+    checkBuildable("default_value", conf);
+    List<SQLDefaultConstraint> dv = new ArrayList<>(columns.size());
+    for (String column : columns) {
+      dv.add(new SQLDefaultConstraint(catName, dbName, tableName, column,
+          defaultVal.toString(), constraintName, enable, validate, rely));
+    }
+    return dv;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SQLForeignKeyBuilder.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SQLForeignKeyBuilder.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SQLForeignKeyBuilder.java
index a39319a..f5adda1 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SQLForeignKeyBuilder.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SQLForeignKeyBuilder.java
@@ -17,21 +17,30 @@
  */
 package org.apache.hadoop.hive.metastore.client.builder;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 
+import java.util.ArrayList;
+import java.util.List;
+
 /**
  * Builder for {@link SQLForeignKey}.  Requires what {@link ConstraintBuilder} requires, plus
  * primary key
  * database, table, column and name.
  */
 public class SQLForeignKeyBuilder extends ConstraintBuilder<SQLForeignKeyBuilder> {
-  private String pkDb, pkTable, pkColumn, pkName;
+  private String pkDb, pkTable, pkName;
+  private List<String> pkColumns;
   private int updateRule, deleteRule;
 
   public SQLForeignKeyBuilder() {
+    super.setChild(this);
     updateRule = deleteRule = 0;
+    pkColumns = new ArrayList<>();
+    pkDb = Warehouse.DEFAULT_DATABASE_NAME;
   }
 
   public SQLForeignKeyBuilder setPkDb(String pkDb) {
@@ -44,8 +53,8 @@ public class SQLForeignKeyBuilder extends ConstraintBuilder<SQLForeignKeyBuilder
     return this;
   }
 
-  public SQLForeignKeyBuilder setPkColumn(String pkColumn) {
-    this.pkColumn = pkColumn;
+  public SQLForeignKeyBuilder addPkColumn(String pkColumn) {
+    pkColumns.add(pkColumn);
     return this;
   }
 
@@ -54,11 +63,11 @@ public class SQLForeignKeyBuilder extends ConstraintBuilder<SQLForeignKeyBuilder
     return this;
   }
 
-  public SQLForeignKeyBuilder setPrimaryKey(SQLPrimaryKey pk) {
-    pkDb = pk.getTable_db();
-    pkTable = pk.getTable_name();
-    pkColumn = pk.getColumn_name();
-    pkName = pk.getPk_name();
+  public SQLForeignKeyBuilder fromPrimaryKey(List<SQLPrimaryKey> pk) {
+    pkDb = pk.get(0).getTable_db();
+    pkTable = pk.get(0).getTable_name();
+    for (SQLPrimaryKey pkcol : pk) pkColumns.add(pkcol.getColumn_name());
+    pkName = pk.get(0).getPk_name();
     return this;
   }
 
@@ -72,12 +81,23 @@ public class SQLForeignKeyBuilder extends ConstraintBuilder<SQLForeignKeyBuilder
     return this;
   }
 
-  public SQLForeignKey build() throws MetaException {
-    checkBuildable("foreign_key");
-    if (pkDb == null || pkTable == null || pkColumn == null || pkName == null) {
-      throw new MetaException("You must provide the primary key database, table, column, and name");
+  public List<SQLForeignKey> build(Configuration conf) throws MetaException {
+    checkBuildable("to_" + pkTable + "_foreign_key", conf);
+    if (pkTable == null || pkColumns.isEmpty() || pkName == null) {
+      throw new MetaException("You must provide the primary key table, columns, and name");
+    }
+    if (columns.size() != pkColumns.size()) {
+      throw new MetaException("The number of foreign columns must match the number of primary key" +
+          " columns");
+    }
+    List<SQLForeignKey> fk = new ArrayList<>(columns.size());
+    for (int i = 0; i < columns.size(); i++) {
+      SQLForeignKey keyCol = new SQLForeignKey(pkDb, pkTable, pkColumns.get(i), dbName, tableName,
+          columns.get(i), getNextSeq(), updateRule, deleteRule, constraintName, pkName, enable,
+          validate, rely);
+      keyCol.setCatName(catName);
+      fk.add(keyCol);
     }
-    return new SQLForeignKey(pkDb, pkTable, pkColumn, dbName, tableName, columnName, keySeq,
-        updateRule, deleteRule, constraintName, pkName, enable, validate, rely);
+    return fk;
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SQLNotNullConstraintBuilder.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SQLNotNullConstraintBuilder.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SQLNotNullConstraintBuilder.java
index 77d1e49..497032e 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SQLNotNullConstraintBuilder.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SQLNotNullConstraintBuilder.java
@@ -17,9 +17,13 @@
  */
 package org.apache.hadoop.hive.metastore.client.builder;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 
+import java.util.ArrayList;
+import java.util.List;
+
 /**
  * Builder for {@link SQLNotNullConstraint}.  Only requires what {@link ConstraintBuilder} requires.
  */
@@ -29,9 +33,20 @@ public class SQLNotNullConstraintBuilder extends ConstraintBuilder<SQLNotNullCon
     super.setChild(this);
   }
 
-  public SQLNotNullConstraint build() throws MetaException {
-    checkBuildable("not_null_constraint");
-    return new SQLNotNullConstraint(dbName, tableName, columnName, constraintName, enable,
-        validate, rely);
+  public SQLNotNullConstraintBuilder setColName(String colName) {
+    assert columns.isEmpty();
+    columns.add(colName);
+    return this;
+  }
+
+  public List<SQLNotNullConstraint> build(Configuration conf) throws MetaException {
+    checkBuildable("not_null_constraint", conf);
+    List<SQLNotNullConstraint> uc = new ArrayList<>(columns.size());
+    for (String column : columns) {
+      SQLNotNullConstraint c = new SQLNotNullConstraint(catName, dbName, tableName, columns.get(0),
+          constraintName, enable, validate, rely);
+      uc.add(c);
+    }
+    return uc;
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SQLPrimaryKeyBuilder.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SQLPrimaryKeyBuilder.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SQLPrimaryKeyBuilder.java
index 9000f86..40f74bd 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SQLPrimaryKeyBuilder.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SQLPrimaryKeyBuilder.java
@@ -17,9 +17,13 @@
  */
 package org.apache.hadoop.hive.metastore.client.builder;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 
+import java.util.ArrayList;
+import java.util.List;
+
 /**
  * Builder for {@link SQLPrimaryKey}.  Only requires what {@link ConstraintBuilder} requires.
  */
@@ -34,9 +38,15 @@ public class SQLPrimaryKeyBuilder extends ConstraintBuilder<SQLPrimaryKeyBuilder
     return setConstraintName(name);
   }
 
-  public SQLPrimaryKey build() throws MetaException {
-    checkBuildable("primary_key");
-    return new SQLPrimaryKey(dbName, tableName, columnName, keySeq, constraintName, enable,
-        validate, rely);
+  public List<SQLPrimaryKey> build(Configuration conf) throws MetaException {
+    checkBuildable("primary_key", conf);
+    List<SQLPrimaryKey> pk = new ArrayList<>(columns.size());
+    for (String colName : columns) {
+      SQLPrimaryKey keyCol = new SQLPrimaryKey(dbName, tableName, colName, getNextSeq(),
+          constraintName, enable, validate, rely);
+      keyCol.setCatName(catName);
+      pk.add(keyCol);
+    }
+    return pk;
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SQLUniqueConstraintBuilder.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SQLUniqueConstraintBuilder.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SQLUniqueConstraintBuilder.java
index 640e9d1..138ee15 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SQLUniqueConstraintBuilder.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SQLUniqueConstraintBuilder.java
@@ -17,9 +17,13 @@
  */
 package org.apache.hadoop.hive.metastore.client.builder;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
 
+import java.util.ArrayList;
+import java.util.List;
+
 /**
  * Builder for {@link SQLUniqueConstraint}.  Only requires what {@link ConstraintBuilder} requires.
  */
@@ -29,9 +33,14 @@ public class SQLUniqueConstraintBuilder extends ConstraintBuilder<SQLUniqueConst
     super.setChild(this);
   }
 
-  public SQLUniqueConstraint build() throws MetaException {
-    checkBuildable("unique_constraint");
-    return new SQLUniqueConstraint(dbName, tableName, columnName, keySeq, constraintName, enable,
-        validate, rely);
+  public List<SQLUniqueConstraint> build(Configuration conf) throws MetaException {
+    checkBuildable("unique_constraint", conf);
+    List<SQLUniqueConstraint> uc = new ArrayList<>(columns.size());
+    for (String column : columns) {
+      SQLUniqueConstraint c = new SQLUniqueConstraint(catName, dbName, tableName, column, getNextSeq(),
+          constraintName, enable, validate, rely);
+      uc.add(c);
+    }
+    return uc;
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SchemaVersionBuilder.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SchemaVersionBuilder.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SchemaVersionBuilder.java
index ceb0f49..521be3e 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SchemaVersionBuilder.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SchemaVersionBuilder.java
@@ -23,8 +23,11 @@ import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.SchemaVersion;
 import org.apache.hadoop.hive.metastore.api.SchemaVersionState;
 
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
+
 public class SchemaVersionBuilder extends SerdeAndColsBuilder<SchemaVersionBuilder> {
-  private String schemaName, dbName; // required
+  private String schemaName, dbName, catName; // required
   private int version; // required
   private long createdAt; // required
   private SchemaVersionState state; // optional
@@ -34,6 +37,8 @@ public class SchemaVersionBuilder extends SerdeAndColsBuilder<SchemaVersionBuild
   private String name; // optional
 
   public SchemaVersionBuilder() {
+    catName = DEFAULT_CATALOG_NAME;
+    dbName = DEFAULT_DATABASE_NAME;
     createdAt = System.currentTimeMillis() / 1000;
     version = -1;
     super.setChild(this);
@@ -50,6 +55,7 @@ public class SchemaVersionBuilder extends SerdeAndColsBuilder<SchemaVersionBuild
   }
 
   public SchemaVersionBuilder versionOf(ISchema schema) {
+    this.catName = schema.getCatName();
     this.dbName = schema.getDbName();
     this.schemaName = schema.getName();
     return this;
@@ -92,11 +98,11 @@ public class SchemaVersionBuilder extends SerdeAndColsBuilder<SchemaVersionBuild
   }
 
   public SchemaVersion build() throws MetaException {
-    if (schemaName == null || dbName == null || version < 0) {
-      throw new MetaException("You must provide the database name, schema name, and schema version");
+    if (schemaName == null || version < 0) {
+      throw new MetaException("You must provide the schema name, and schema version");
     }
     SchemaVersion schemaVersion =
-        new SchemaVersion(new ISchemaName(dbName, schemaName), version, createdAt, getCols());
+        new SchemaVersion(new ISchemaName(catName, dbName, schemaName), version, createdAt, getCols());
     if (state != null) schemaVersion.setState(state);
     if (description != null) schemaVersion.setDescription(description);
     if (schemaText != null) schemaVersion.setSchemaText(schemaText);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/TableBuilder.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/TableBuilder.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/TableBuilder.java
index 2b9f816..79ef7de 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/TableBuilder.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/TableBuilder.java
@@ -17,48 +17,69 @@
  */
 package org.apache.hadoop.hive.metastore.client.builder;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.metastore.TableType;
+import org.apache.hadoop.hive.metastore.Warehouse;
+import org.apache.hadoop.hive.metastore.api.BasicTxnInfo;
+import org.apache.hadoop.hive.metastore.api.CreationMetadata;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.utils.SecurityUtils;
+import org.apache.thrift.TException;
 
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 /**
  * Build a {@link Table}.  The database name and table name must be provided, plus whatever is
  * needed by the underlying {@link StorageDescriptorBuilder}.
  */
 public class TableBuilder extends StorageDescriptorBuilder<TableBuilder> {
-  private String dbName, tableName, owner, viewOriginalText, viewExpandedText, type;
+  private String catName, dbName, tableName, owner, viewOriginalText, viewExpandedText, type,
+      mvValidTxnList;
   private List<FieldSchema> partCols;
   private int createTime, lastAccessTime, retention;
   private Map<String, String> tableParams;
   private boolean rewriteEnabled, temporary;
+  private Set<String> mvReferencedTables;
+
 
   public TableBuilder() {
     // Set some reasonable defaults
+    dbName = Warehouse.DEFAULT_DATABASE_NAME;
     tableParams = new HashMap<>();
     createTime = lastAccessTime = (int)(System.currentTimeMillis() / 1000);
     retention = 0;
     partCols = new ArrayList<>();
     type = TableType.MANAGED_TABLE.name();
+    mvReferencedTables = new HashSet<>();
+    temporary = false;
     super.setChild(this);
   }
 
+  public TableBuilder setCatName(String catName) {
+    this.catName = catName;
+    return this;
+  }
+
   public TableBuilder setDbName(String dbName) {
     this.dbName = dbName;
     return this;
   }
 
-  public TableBuilder setDbName(Database db) {
+  public TableBuilder inDb(Database db) {
     this.dbName = db.getName();
+    this.catName = db.getCatalogName();
     return this;
   }
 
@@ -139,9 +160,19 @@ public class TableBuilder extends StorageDescriptorBuilder<TableBuilder> {
     return this;
   }
 
-  public Table build() throws MetaException {
-    if (dbName == null || tableName == null) {
-      throw new MetaException("You must set the database and table name");
+  public TableBuilder addMaterializedViewReferencedTable(String tableName) {
+    mvReferencedTables.add(tableName);
+    return this;
+  }
+
+  public TableBuilder setMaterializedViewValidTxnList(ValidTxnList validTxnList) {
+    mvValidTxnList = validTxnList.writeToString();
+    return this;
+  }
+
+  public Table build(Configuration conf) throws MetaException {
+    if (tableName == null) {
+      throw new MetaException("You must set the table name");
     }
     if (owner == null) {
       try {
@@ -150,15 +181,24 @@ public class TableBuilder extends StorageDescriptorBuilder<TableBuilder> {
         throw MetaStoreUtils.newMetaException(e);
       }
     }
+    if (catName == null) catName = MetaStoreUtils.getDefaultCatalog(conf);
     Table t = new Table(tableName, dbName, owner, createTime, lastAccessTime, retention, buildSd(),
         partCols, tableParams, viewOriginalText, viewExpandedText, type);
-    if (rewriteEnabled) {
-      t.setRewriteEnabled(true);
-    }
-    if (temporary) {
-      t.setTemporary(temporary);
+    if (rewriteEnabled) t.setRewriteEnabled(true);
+    if (temporary) t.setTemporary(temporary);
+    t.setCatName(catName);
+    if (!mvReferencedTables.isEmpty()) {
+      CreationMetadata cm = new CreationMetadata(catName, dbName, tableName, mvReferencedTables);
+      if (mvValidTxnList != null) cm.setValidTxnList(mvValidTxnList);
+      t.setCreationMetadata(cm);
     }
     return t;
   }
 
+  public Table create(IMetaStoreClient client, Configuration conf) throws TException {
+    Table t = build(conf);
+    client.createTable(t);
+    return t;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
index b8976ed..995137f 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
@@ -306,6 +306,12 @@ public class MetastoreConf {
     CAPABILITY_CHECK("metastore.client.capability.check",
         "hive.metastore.client.capability.check", true,
         "Whether to check client capabilities for potentially breaking API usage."),
+    CATALOG_DEFAULT("metastore.catalog.default", "metastore.catalog.default", "hive",
+        "The default catalog to use when a catalog is not specified.  Default is 'hive' (the " +
+            "default catalog)."),
+    CATALOGS_TO_CACHE("metastore.cached.rawstore.catalogs", "metastore.cached.rawstore.catalogs",
+        "hive", "Comma separated list of catalogs to cache in the CachedStore. Default is 'hive' " +
+        "(the default catalog).  Empty string means all catalogs will be cached."),
     CLIENT_CONNECT_RETRY_DELAY("metastore.client.connect.retry.delay",
         "hive.metastore.client.connect.retry.delay", 1, TimeUnit.SECONDS,
         "Number of seconds for the client to wait between consecutive connection attempts"),

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/CreateCatalogEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/CreateCatalogEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/CreateCatalogEvent.java
new file mode 100644
index 0000000..e667277
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/CreateCatalogEvent.java
@@ -0,0 +1,39 @@
+/*
+ * 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.events;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.Catalog;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class CreateCatalogEvent extends ListenerEvent {
+
+  private final Catalog cat;
+
+  public CreateCatalogEvent(boolean status, IHMSHandler handler, Catalog cat) {
+    super(status, handler);
+    this.cat = cat;
+  }
+
+  public Catalog getCatalog() {
+    return cat;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropCatalogEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropCatalogEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropCatalogEvent.java
new file mode 100644
index 0000000..67c6d51
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropCatalogEvent.java
@@ -0,0 +1,39 @@
+/*
+ * 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.events;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.Catalog;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class DropCatalogEvent extends ListenerEvent {
+
+  private final Catalog cat;
+
+  public DropCatalogEvent(boolean status, IHMSHandler handler, Catalog cat) {
+    super(status, handler);
+    this.cat = cat;
+  }
+
+  public Catalog getCatalog() {
+    return cat;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropConstraintEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropConstraintEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropConstraintEvent.java
index 4c5918f..ccd968b 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropConstraintEvent.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/DropConstraintEvent.java
@@ -26,17 +26,23 @@ import org.apache.hadoop.hive.metastore.IHMSHandler;
 @InterfaceStability.Stable
 public class DropConstraintEvent extends ListenerEvent {
 
+  private final String catName;
   private final String dbName;
   private final String tableName;
   private final String constraintName;
-  public DropConstraintEvent(String dbName, String tableName, String constraintName,
+  public DropConstraintEvent(String catName, String dbName, String tableName, String constraintName,
       boolean status, IHMSHandler handler) {
     super(status, handler);
+    this.catName = catName;
     this.dbName = dbName;
     this.tableName = tableName;
     this.constraintName = constraintName;
   }
 
+  public String getCatName() {
+    return catName;
+  }
+
   public String getDbName() {
     return dbName;
   }


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/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 d4bdcf1..0794cc5 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
@@ -19,6 +19,8 @@
 package org.apache.hadoop.hive.metastore;
 
 import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.prependCatalogToDbName;
 
 import java.io.IOException;
 import java.lang.reflect.Constructor;
@@ -70,6 +72,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.thrift.TApplicationException;
+import org.apache.thrift.TBase;
 import org.apache.thrift.TException;
 import org.apache.thrift.protocol.TBinaryProtocol;
 import org.apache.thrift.protocol.TCompactProtocol;
@@ -112,7 +115,6 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   private URI metastoreUris[];
   private final HiveMetaHookLoader hookLoader;
   protected final Configuration conf;  // Keep a copy of HiveConf so if Session conf changes, we may need to get a new HMS client.
-  protected boolean fastpath = false;
   private String tokenStrForm;
   private final boolean localMetaStore;
   private final MetaStoreFilterHook filterHook;
@@ -379,27 +381,14 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
     }
   }
 
-  /**
-   * @param dbname
-   * @param tbl_name
-   * @param new_tbl
-   * @throws InvalidOperationException
-   * @throws MetaException
-   * @throws TException
-   * @see
-   *   org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#alter_table(
-   *   java.lang.String, java.lang.String,
-   *   org.apache.hadoop.hive.metastore.api.Table)
-   */
   @Override
-  public void alter_table(String dbname, String tbl_name, Table new_tbl)
-      throws InvalidOperationException, MetaException, TException {
+  public void alter_table(String dbname, String tbl_name, Table new_tbl) throws TException {
     alter_table_with_environmentContext(dbname, tbl_name, new_tbl, null);
   }
 
   @Override
   public void alter_table(String defaultDatabaseName, String tblName, Table table,
-      boolean cascade) throws InvalidOperationException, MetaException, TException {
+                          boolean cascade) throws TException {
     EnvironmentContext environmentContext = new EnvironmentContext();
     if (cascade) {
       environmentContext.putToProperties(StatsSetupConst.CASCADE, StatsSetupConst.TRUE);
@@ -414,24 +403,28 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
     if (hook != null) {
       hook.preAlterTable(new_tbl, envContext);
     }
-    client.alter_table_with_environment_context(dbname, tbl_name, new_tbl, envContext);
+    client.alter_table_with_environment_context(prependCatalogToDbName(dbname, conf),
+        tbl_name, new_tbl, envContext);
   }
 
-  /**
-   * @param dbname
-   * @param name
-   * @param part_vals
-   * @param newPart
-   * @throws InvalidOperationException
-   * @throws MetaException
-   * @throws TException
-   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#rename_partition(
-   *      java.lang.String, java.lang.String, java.util.List, org.apache.hadoop.hive.metastore.api.Partition)
-   */
   @Override
-  public void renamePartition(final String dbname, final String name, final List<String> part_vals, final Partition newPart)
-      throws InvalidOperationException, MetaException, TException {
-    client.rename_partition(dbname, name, part_vals, newPart);
+  public void alter_table(String catName, String dbName, String tblName, Table newTable,
+                         EnvironmentContext envContext) throws TException {
+    client.alter_table_with_environment_context(prependCatalogToDbName(catName,
+        dbName, conf), tblName, newTable, envContext);
+  }
+
+  @Override
+  public void renamePartition(final String dbname, final String tableName, final List<String> part_vals,
+                              final Partition newPart) throws TException {
+    renamePartition(getDefaultCatalog(conf), dbname, tableName, part_vals, newPart);
+  }
+
+  @Override
+  public void renamePartition(String catName, String dbname, String tableName, List<String> part_vals,
+                              Partition newPart) throws TException {
+    client.rename_partition(prependCatalogToDbName(catName, dbname, conf), tableName, part_vals, newPart);
+
   }
 
   private void open() throws MetaException {
@@ -619,6 +612,28 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
     return client.getMetaConf(key);
   }
 
+  @Override
+  public void createCatalog(Catalog catalog) throws TException {
+    client.create_catalog(new CreateCatalogRequest(catalog));
+  }
+
+  @Override
+  public Catalog getCatalog(String catName) throws TException {
+    GetCatalogResponse rsp = client.get_catalog(new GetCatalogRequest(catName));
+    return rsp == null ? null : filterHook.filterCatalog(rsp.getCatalog());
+  }
+
+  @Override
+  public List<String> getCatalogs() throws TException {
+    GetCatalogsResponse rsp = client.get_catalogs();
+    return rsp == null ? null : filterHook.filterCatalogs(rsp.getNames());
+  }
+
+  @Override
+  public void dropCatalog(String catName) throws TException {
+    client.drop_catalog(new DropCatalogRequest(catName));
+  }
+
   /**
    * @param new_part
    * @return the added partition
@@ -635,8 +650,9 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
 
   public Partition add_partition(Partition new_part, EnvironmentContext envContext)
       throws TException {
+    if (!new_part.isSetCatName()) new_part.setCatName(getDefaultCatalog(conf));
     Partition p = client.add_partition_with_environment_context(new_part, envContext);
-    return fastpath ? p : deepCopy(p);
+    return deepCopy(p);
   }
 
   /**
@@ -649,6 +665,10 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
    */
   @Override
   public int add_partitions(List<Partition> new_parts) throws TException {
+    if (new_parts != null && !new_parts.isEmpty() && !new_parts.get(0).isSetCatName()) {
+      final String defaultCat = getDefaultCatalog(conf);
+      new_parts.forEach(p -> p.setCatName(defaultCat));
+    }
     return client.add_partitions(new_parts);
   }
 
@@ -661,6 +681,7 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
     Partition part = parts.get(0);
     AddPartitionsRequest req = new AddPartitionsRequest(
         part.getDbName(), part.getTableName(), parts, ifNotExists);
+    req.setCatName(part.isSetCatName() ? part.getCatName() : getDefaultCatalog(conf));
     req.setNeedResult(needResults);
     AddPartitionsResult result = client.add_partitions_req(req);
     return needResults ? filterHook.filterPartitions(result.getPartitions()) : null;
@@ -668,45 +689,43 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
 
   @Override
   public int add_partitions_pspec(PartitionSpecProxy partitionSpec) throws TException {
+    if (partitionSpec.getCatName() == null) partitionSpec.setCatName(getDefaultCatalog(conf));
     return client.add_partitions_pspec(partitionSpec.toPartitionSpec());
   }
 
-  /**
-   * @param table_name
-   * @param db_name
-   * @param part_vals
-   * @return the appended partition
-   * @throws InvalidObjectException
-   * @throws AlreadyExistsException
-   * @throws MetaException
-   * @throws TException
-   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#append_partition(java.lang.String,
-   *      java.lang.String, java.util.List)
-   */
   @Override
   public Partition appendPartition(String db_name, String table_name,
       List<String> part_vals) throws TException {
-    return appendPartition(db_name, table_name, part_vals, null);
-  }
-
-  public Partition appendPartition(String db_name, String table_name, List<String> part_vals,
-      EnvironmentContext envContext) throws TException {
-    Partition p = client.append_partition_with_environment_context(db_name, table_name,
-        part_vals, envContext);
-    return fastpath ? p : deepCopy(p);
+    return appendPartition(getDefaultCatalog(conf), db_name, table_name, part_vals);
   }
 
   @Override
   public Partition appendPartition(String dbName, String tableName, String partName)
       throws TException {
-    return appendPartition(dbName, tableName, partName, null);
+    return appendPartition(getDefaultCatalog(conf), dbName, tableName, partName);
   }
 
-  public Partition appendPartition(String dbName, String tableName, String partName,
-      EnvironmentContext envContext) throws TException {
-    Partition p = client.append_partition_by_name_with_environment_context(dbName, tableName,
-        partName, envContext);
-    return fastpath ? p : deepCopy(p);
+  @Override
+  public Partition appendPartition(String catName, String dbName, String tableName,
+                                   String name) throws TException {
+    Partition p = client.append_partition_by_name(prependCatalogToDbName(
+        catName, dbName, conf), tableName, name);
+    return deepCopy(p);
+  }
+
+  @Override
+  public Partition appendPartition(String catName, String dbName, String tableName,
+                                   List<String> partVals) throws TException {
+    Partition p = client.append_partition(prependCatalogToDbName(
+        catName, dbName, conf), tableName, partVals);
+    return deepCopy(p);
+  }
+
+  @Deprecated
+  public Partition appendPartition(String dbName, String tableName, List<String> partVals,
+                                   EnvironmentContext ec) throws TException {
+    return client.append_partition_with_environment_context(prependCatalogToDbName(dbName, conf),
+        tableName, partVals, ec).deepCopy();
   }
 
   /**
@@ -719,10 +738,17 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   @Override
   public Partition exchange_partition(Map<String, String> partitionSpecs,
       String sourceDb, String sourceTable, String destDb,
-      String destinationTableName) throws MetaException,
-      NoSuchObjectException, InvalidObjectException, TException {
-    return client.exchange_partition(partitionSpecs, sourceDb, sourceTable,
-        destDb, destinationTableName);
+      String destinationTableName) throws TException {
+    return exchange_partition(partitionSpecs, getDefaultCatalog(conf), sourceDb, sourceTable,
+        getDefaultCatalog(conf), destDb, destinationTableName);
+  }
+
+  @Override
+  public Partition exchange_partition(Map<String, String> partitionSpecs, String sourceCat,
+                                      String sourceDb, String sourceTable, String destCat,
+                                      String destDb, String destTableName) throws TException {
+    return client.exchange_partition(partitionSpecs, prependCatalogToDbName(sourceCat, sourceDb, conf),
+        sourceTable, prependCatalogToDbName(destCat, destDb, conf), destTableName);
   }
 
   /**
@@ -735,10 +761,17 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   @Override
   public List<Partition> exchange_partitions(Map<String, String> partitionSpecs,
       String sourceDb, String sourceTable, String destDb,
-      String destinationTableName) throws MetaException,
-      NoSuchObjectException, InvalidObjectException, TException {
-    return client.exchange_partitions(partitionSpecs, sourceDb, sourceTable,
-        destDb, destinationTableName);
+      String destinationTableName) throws TException {
+    return exchange_partitions(partitionSpecs, getDefaultCatalog(conf), sourceDb, sourceTable,
+        getDefaultCatalog(conf), destDb, destinationTableName);
+  }
+
+  @Override
+  public List<Partition> exchange_partitions(Map<String, String> partitionSpecs, String sourceCat,
+                                             String sourceDb, String sourceTable, String destCat,
+                                             String destDb, String destTableName) throws TException {
+    return client.exchange_partitions(partitionSpecs, prependCatalogToDbName(sourceCat, sourceDb, conf),
+        sourceTable, prependCatalogToDbName(destCat, destDb, conf), destTableName);
   }
 
   @Override
@@ -759,6 +792,7 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   @Override
   public void createDatabase(Database db)
       throws AlreadyExistsException, InvalidObjectException, MetaException, TException {
+    if (!db.isSetCatalogName()) db.setCatalogName(getDefaultCatalog(conf));
     client.create_database(db);
   }
 
@@ -777,6 +811,7 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
 
   public void createTable(Table tbl, EnvironmentContext envContext) throws AlreadyExistsException,
       InvalidObjectException, MetaException, NoSuchObjectException, TException {
+    if (!tbl.isSetCatName()) tbl.setCatName(getDefaultCatalog(conf));
     HiveMetaHook hook = getHook(tbl);
     if (hook != null) {
       hook.preCreateTable(tbl);
@@ -801,7 +836,6 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
     }
   }
 
-  @Override
   public void createTableWithConstraints(Table tbl,
     List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys,
     List<SQLUniqueConstraint> uniqueConstraints,
@@ -810,6 +844,17 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
     List<SQLCheckConstraint> checkConstraints)
         throws AlreadyExistsException, InvalidObjectException,
         MetaException, NoSuchObjectException, TException {
+
+    if (!tbl.isSetCatName()) {
+      String defaultCat = getDefaultCatalog(conf);
+      tbl.setCatName(defaultCat);
+      if (primaryKeys != null) primaryKeys.forEach(pk -> pk.setCatName(defaultCat));
+      if (foreignKeys != null) foreignKeys.forEach(fk -> fk.setCatName(defaultCat));
+      if (uniqueConstraints != null) uniqueConstraints.forEach(uc -> uc.setCatName(defaultCat));
+      if (notNullConstraints != null) notNullConstraints.forEach(nn -> nn.setCatName(defaultCat));
+      if (defaultConstraints != null) defaultConstraints.forEach(def -> def.setCatName(defaultCat));
+      if (checkConstraints != null) checkConstraints.forEach(cc -> cc.setCatName(defaultCat));
+    }
     HiveMetaHook hook = getHook(tbl);
     if (hook != null) {
       hook.preCreateTable(tbl);
@@ -831,44 +876,74 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   }
 
   @Override
-  public void dropConstraint(String dbName, String tableName, String constraintName) throws
-    NoSuchObjectException, MetaException, TException {
-    client.drop_constraint(new DropConstraintRequest(dbName, tableName, constraintName));
+  public void dropConstraint(String dbName, String tableName, String constraintName)
+      throws TException {
+    dropConstraint(getDefaultCatalog(conf), dbName, tableName, constraintName);
   }
 
   @Override
-  public void addPrimaryKey(List<SQLPrimaryKey> primaryKeyCols) throws
-    NoSuchObjectException, MetaException, TException {
+  public void dropConstraint(String catName, String dbName, String tableName, String constraintName)
+      throws TException {
+    DropConstraintRequest rqst = new DropConstraintRequest(dbName, tableName, constraintName);
+    rqst.setCatName(catName);
+    client.drop_constraint(rqst);
+  }
+
+  @Override
+  public void addPrimaryKey(List<SQLPrimaryKey> primaryKeyCols) throws TException {
+    if (!primaryKeyCols.isEmpty() && !primaryKeyCols.get(0).isSetCatName()) {
+      String defaultCat = getDefaultCatalog(conf);
+      primaryKeyCols.forEach(pk -> pk.setCatName(defaultCat));
+    }
     client.add_primary_key(new AddPrimaryKeyRequest(primaryKeyCols));
   }
 
   @Override
-  public void addForeignKey(List<SQLForeignKey> foreignKeyCols) throws
-    NoSuchObjectException, MetaException, TException {
+  public void addForeignKey(List<SQLForeignKey> foreignKeyCols) throws TException {
+    if (!foreignKeyCols.isEmpty() && !foreignKeyCols.get(0).isSetCatName()) {
+      String defaultCat = getDefaultCatalog(conf);
+      foreignKeyCols.forEach(fk -> fk.setCatName(defaultCat));
+    }
     client.add_foreign_key(new AddForeignKeyRequest(foreignKeyCols));
   }
 
   @Override
   public void addUniqueConstraint(List<SQLUniqueConstraint> uniqueConstraintCols) throws
     NoSuchObjectException, MetaException, TException {
+    if (!uniqueConstraintCols.isEmpty() && !uniqueConstraintCols.get(0).isSetCatName()) {
+      String defaultCat = getDefaultCatalog(conf);
+      uniqueConstraintCols.forEach(uc -> uc.setCatName(defaultCat));
+    }
     client.add_unique_constraint(new AddUniqueConstraintRequest(uniqueConstraintCols));
   }
 
   @Override
   public void addNotNullConstraint(List<SQLNotNullConstraint> notNullConstraintCols) throws
     NoSuchObjectException, MetaException, TException {
+    if (!notNullConstraintCols.isEmpty() && !notNullConstraintCols.get(0).isSetCatName()) {
+      String defaultCat = getDefaultCatalog(conf);
+      notNullConstraintCols.forEach(nn -> nn.setCatName(defaultCat));
+    }
     client.add_not_null_constraint(new AddNotNullConstraintRequest(notNullConstraintCols));
   }
 
   @Override
   public void addDefaultConstraint(List<SQLDefaultConstraint> defaultConstraints) throws
       NoSuchObjectException, MetaException, TException {
+    if (!defaultConstraints.isEmpty() && !defaultConstraints.get(0).isSetCatName()) {
+      String defaultCat = getDefaultCatalog(conf);
+      defaultConstraints.forEach(def -> def.setCatName(defaultCat));
+    }
     client.add_default_constraint(new AddDefaultConstraintRequest(defaultConstraints));
   }
 
   @Override
   public void addCheckConstraint(List<SQLCheckConstraint> checkConstraints) throws
       NoSuchObjectException, MetaException, TException {
+    if (!checkConstraints.isEmpty() && !checkConstraints.get(0).isSetCatName()) {
+      String defaultCat = getDefaultCatalog(conf);
+      checkConstraints.forEach(cc -> cc.setCatName(defaultCat));
+    }
     client.add_check_constraint(new AddCheckConstraintRequest(checkConstraints));
   }
 
@@ -897,20 +972,26 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   @Override
   public void dropDatabase(String name)
       throws NoSuchObjectException, InvalidOperationException, MetaException, TException {
-    dropDatabase(name, true, false, false);
+    dropDatabase(getDefaultCatalog(conf), name, true, false, false);
   }
 
   @Override
   public void dropDatabase(String name, boolean deleteData, boolean ignoreUnknownDb)
       throws NoSuchObjectException, InvalidOperationException, MetaException, TException {
-    dropDatabase(name, deleteData, ignoreUnknownDb, false);
+    dropDatabase(getDefaultCatalog(conf), name, deleteData, ignoreUnknownDb, false);
   }
 
   @Override
   public void dropDatabase(String name, boolean deleteData, boolean ignoreUnknownDb, boolean cascade)
       throws NoSuchObjectException, InvalidOperationException, MetaException, TException {
+    dropDatabase(getDefaultCatalog(conf), name, deleteData, ignoreUnknownDb, cascade);
+  }
+
+  public void dropDatabase(String catalogName, String dbName, boolean deleteData,
+                           boolean ignoreUnknownDb, boolean cascade)
+      throws NoSuchObjectException, InvalidOperationException, MetaException, TException {
     try {
-      getDatabase(name);
+      getDatabase(catalogName, dbName);
     } catch (NoSuchObjectException e) {
       if (!ignoreUnknownDb) {
         throw e;
@@ -919,45 +1000,30 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
     }
 
     if (cascade) {
-       List<String> tableList = getAllTables(name);
+       List<String> tableList = getAllTables(dbName);
        for (String table : tableList) {
          try {
            // Subclasses can override this step (for example, for temporary tables)
-           dropTable(name, table, deleteData, true);
+           dropTable(dbName, table, deleteData, true);
          } catch (UnsupportedOperationException e) {
            // Ignore Index tables, those will be dropped with parent tables
          }
         }
     }
-    client.drop_database(name, deleteData, cascade);
-  }
-
-  /**
-   * @param tbl_name
-   * @param db_name
-   * @param part_vals
-   * @return true or false
-   * @throws NoSuchObjectException
-   * @throws MetaException
-   * @throws TException
-   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#drop_partition(java.lang.String,
-   *      java.lang.String, java.util.List, boolean)
-   */
-  public boolean dropPartition(String db_name, String tbl_name,
-      List<String> part_vals) throws NoSuchObjectException, MetaException,
-      TException {
-    return dropPartition(db_name, tbl_name, part_vals, true, null);
+    client.drop_database(prependCatalogToDbName(catalogName, dbName, conf), deleteData, cascade);
   }
 
-  public boolean dropPartition(String db_name, String tbl_name, List<String> part_vals,
-      EnvironmentContext env_context) throws NoSuchObjectException, MetaException, TException {
-    return dropPartition(db_name, tbl_name, part_vals, true, env_context);
+  @Override
+  public boolean dropPartition(String dbName, String tableName, String partName, boolean deleteData)
+      throws TException {
+    return dropPartition(getDefaultCatalog(conf), dbName, tableName, partName, deleteData);
   }
 
   @Override
-  public boolean dropPartition(String dbName, String tableName, String partName, boolean deleteData)
-      throws NoSuchObjectException, MetaException, TException {
-    return dropPartition(dbName, tableName, partName, deleteData, null);
+  public boolean dropPartition(String catName, String db_name, String tbl_name, String name,
+                               boolean deleteData) throws TException {
+    return client.drop_partition_by_name_with_environment_context(prependCatalogToDbName(
+        catName, db_name, conf), tbl_name, name, deleteData, null);
   }
 
   private static EnvironmentContext getEnvironmentContextWithIfPurgeSet() {
@@ -966,54 +1032,57 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
     return new EnvironmentContext(warehouseOptions);
   }
 
-  /*
-  public boolean dropPartition(String dbName, String tableName, String partName, boolean deleteData, boolean ifPurge)
-      throws NoSuchObjectException, MetaException, TException {
+  // A bunch of these are in HiveMetaStoreClient but not IMetaStoreClient.  I have marked these
+  // as deprecated and not updated them for the catalogs.  If we really want to support them we
+  // should add them to IMetaStoreClient.
 
-    return dropPartition(dbName, tableName, partName, deleteData,
-                         ifPurge? getEnvironmentContextWithIfPurgeSet() : null);
+  @Deprecated
+  public boolean dropPartition(String db_name, String tbl_name, List<String> part_vals,
+      EnvironmentContext env_context) throws TException {
+    return client.drop_partition_with_environment_context(prependCatalogToDbName(db_name, conf),
+        tbl_name, part_vals, true, env_context);
   }
-  */
 
-  public boolean dropPartition(String dbName, String tableName, String partName, boolean deleteData,
-      EnvironmentContext envContext) throws NoSuchObjectException, MetaException, TException {
-    return client.drop_partition_by_name_with_environment_context(dbName, tableName, partName,
-        deleteData, envContext);
+  @Deprecated
+  public boolean dropPartition(String dbName, String tableName, String partName, boolean dropData,
+                               EnvironmentContext ec) throws TException {
+    return client.drop_partition_by_name_with_environment_context(prependCatalogToDbName(dbName, conf),
+        tableName, partName, dropData, ec);
+  }
+
+  @Deprecated
+  public boolean dropPartition(String dbName, String tableName, List<String> partVals)
+      throws TException {
+    return client.drop_partition(prependCatalogToDbName(dbName, conf), tableName, partVals, true);
   }
 
-  /**
-   * @param db_name
-   * @param tbl_name
-   * @param part_vals
-   * @param deleteData
-   *          delete the underlying data or just delete the table in metadata
-   * @return true or false
-   * @throws NoSuchObjectException
-   * @throws MetaException
-   * @throws TException
-   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#drop_partition(java.lang.String,
-   *      java.lang.String, java.util.List, boolean)
-   */
   @Override
   public boolean dropPartition(String db_name, String tbl_name,
-      List<String> part_vals, boolean deleteData) throws NoSuchObjectException,
-      MetaException, TException {
-    return dropPartition(db_name, tbl_name, part_vals, deleteData, null);
+      List<String> part_vals, boolean deleteData) throws TException {
+    return dropPartition(getDefaultCatalog(conf), db_name, tbl_name, part_vals,
+        PartitionDropOptions.instance().deleteData(deleteData));
+  }
+
+  @Override
+  public boolean dropPartition(String catName, String db_name, String tbl_name,
+                               List<String> part_vals, boolean deleteData) throws TException {
+    return dropPartition(catName, db_name, tbl_name, part_vals, PartitionDropOptions.instance()
+            .deleteData(deleteData));
   }
 
   @Override
   public boolean dropPartition(String db_name, String tbl_name,
-      List<String> part_vals, PartitionDropOptions options) throws TException {
+                               List<String> part_vals, PartitionDropOptions options) throws TException {
+    return dropPartition(getDefaultCatalog(conf), db_name, tbl_name, part_vals, options);
+  }
+
+  @Override
+  public boolean dropPartition(String catName, String db_name, String tbl_name,
+                               List<String> part_vals, PartitionDropOptions options)
+      throws TException {
     if (options == null) {
       options = PartitionDropOptions.instance();
     }
-    return dropPartition(db_name, tbl_name, part_vals, options.deleteData,
-                         options.purgeData? getEnvironmentContextWithIfPurgeSet() : null);
-  }
-
-  public boolean dropPartition(String db_name, String tbl_name, List<String> part_vals,
-      boolean deleteData, EnvironmentContext envContext) throws NoSuchObjectException,
-      MetaException, TException {
     if (part_vals != null) {
       for (String partVal : part_vals) {
         if (partVal == null) {
@@ -1021,32 +1090,17 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
         }
       }
     }
-    return client.drop_partition_with_environment_context(db_name, tbl_name, part_vals, deleteData,
-        envContext);
+    return client.drop_partition_with_environment_context(prependCatalogToDbName(
+        catName, db_name, conf), tbl_name, part_vals, options.deleteData,
+        options.purgeData ? getEnvironmentContextWithIfPurgeSet() : null);
   }
 
   @Override
   public List<Partition> dropPartitions(String dbName, String tblName,
-                                        List<ObjectPair<Integer, byte[]>> partExprs, PartitionDropOptions options)
+                                        List<ObjectPair<Integer, byte[]>> partExprs,
+                                        PartitionDropOptions options)
       throws TException {
-    RequestPartsSpec rps = new RequestPartsSpec();
-    List<DropPartitionsExpr> exprs = new ArrayList<>(partExprs.size());
-    for (ObjectPair<Integer, byte[]> partExpr : partExprs) {
-      DropPartitionsExpr dpe = new DropPartitionsExpr();
-      dpe.setExpr(partExpr.getSecond());
-      dpe.setPartArchiveLevel(partExpr.getFirst());
-      exprs.add(dpe);
-    }
-    rps.setExprs(exprs);
-    DropPartitionsRequest req = new DropPartitionsRequest(dbName, tblName, rps);
-    req.setDeleteData(options.deleteData);
-    req.setNeedResult(options.returnResults);
-    req.setIfExists(options.ifExists);
-    if (options.purgeData) {
-      LOG.info("Dropped partitions will be purged!");
-      req.setEnvironmentContext(getEnvironmentContextWithIfPurgeSet());
-    }
-    return client.drop_partitions_req(req).getPartitions();
+    return dropPartitions(getDefaultCatalog(conf), dbName, tblName, partExprs, options);
   }
 
   @Override
@@ -1054,7 +1108,7 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
       List<ObjectPair<Integer, byte[]>> partExprs, boolean deleteData,
       boolean ifExists, boolean needResult) throws NoSuchObjectException, MetaException, TException {
 
-    return dropPartitions(dbName, tblName, partExprs,
+    return dropPartitions(getDefaultCatalog(conf), dbName, tblName, partExprs,
                           PartitionDropOptions.instance()
                                               .deleteData(deleteData)
                                               .ifExists(ifExists)
@@ -1067,33 +1121,58 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
       List<ObjectPair<Integer, byte[]>> partExprs, boolean deleteData,
       boolean ifExists) throws NoSuchObjectException, MetaException, TException {
     // By default, we need the results from dropPartitions();
-    return dropPartitions(dbName, tblName, partExprs,
+    return dropPartitions(getDefaultCatalog(conf), dbName, tblName, partExprs,
                           PartitionDropOptions.instance()
                                               .deleteData(deleteData)
                                               .ifExists(ifExists));
   }
 
-  /**
-   * {@inheritDoc}
-   * @see #dropTable(String, String, boolean, boolean, EnvironmentContext)
-   */
+  @Override
+  public List<Partition> dropPartitions(String catName, String dbName, String tblName,
+                                        List<ObjectPair<Integer, byte[]>> partExprs,
+                                        PartitionDropOptions options) throws TException {
+    RequestPartsSpec rps = new RequestPartsSpec();
+    List<DropPartitionsExpr> exprs = new ArrayList<>(partExprs.size());
+    for (ObjectPair<Integer, byte[]> partExpr : partExprs) {
+      DropPartitionsExpr dpe = new DropPartitionsExpr();
+      dpe.setExpr(partExpr.getSecond());
+      dpe.setPartArchiveLevel(partExpr.getFirst());
+      exprs.add(dpe);
+    }
+    rps.setExprs(exprs);
+    DropPartitionsRequest req = new DropPartitionsRequest(dbName, tblName, rps);
+    req.setCatName(catName);
+    req.setDeleteData(options.deleteData);
+    req.setNeedResult(options.returnResults);
+    req.setIfExists(options.ifExists);
+    if (options.purgeData) {
+      LOG.info("Dropped partitions will be purged!");
+      req.setEnvironmentContext(getEnvironmentContextWithIfPurgeSet());
+    }
+    return client.drop_partitions_req(req).getPartitions();
+  }
+
   @Override
   public void dropTable(String dbname, String name, boolean deleteData,
       boolean ignoreUnknownTab) throws MetaException, TException,
       NoSuchObjectException, UnsupportedOperationException {
-    dropTable(dbname, name, deleteData, ignoreUnknownTab, null);
+    dropTable(getDefaultCatalog(conf), dbname, name, deleteData, ignoreUnknownTab, null);
   }
 
-  /**
-   * Drop the table and choose whether to save the data in the trash.
-   * @param ifPurge completely purge the table (skipping trash) while removing
-   *                data from warehouse
-   * @see #dropTable(String, String, boolean, boolean, EnvironmentContext)
-   */
   @Override
   public void dropTable(String dbname, String name, boolean deleteData,
-      boolean ignoreUnknownTab, boolean ifPurge)
-      throws MetaException, TException, NoSuchObjectException, UnsupportedOperationException {
+      boolean ignoreUnknownTab, boolean ifPurge) throws TException {
+    dropTable(getDefaultCatalog(conf), dbname, name, deleteData, ignoreUnknownTab, ifPurge);
+  }
+
+  @Override
+  public void dropTable(String dbname, String name) throws TException {
+    dropTable(getDefaultCatalog(conf), dbname, name, true, true, null);
+  }
+
+  @Override
+  public void dropTable(String catName, String dbName, String tableName, boolean deleteData,
+                        boolean ignoreUnknownTable, boolean ifPurge) throws TException {
     //build new environmentContext with ifPurge;
     EnvironmentContext envContext = null;
     if(ifPurge){
@@ -1102,32 +1181,17 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
       warehouseOptions.put("ifPurge", "TRUE");
       envContext = new EnvironmentContext(warehouseOptions);
     }
-    dropTable(dbname, name, deleteData, ignoreUnknownTab, envContext);
-  }
-
-  /** {@inheritDoc} */
-  @Override
-  @Deprecated
-  public void dropTable(String tableName, boolean deleteData)
-      throws MetaException, UnknownTableException, TException, NoSuchObjectException {
-    dropTable(DEFAULT_DATABASE_NAME, tableName, deleteData, false, null);
-  }
+    dropTable(catName, dbName, tableName, deleteData, ignoreUnknownTable, envContext);
 
-  /**
-   * @see #dropTable(String, String, boolean, boolean, EnvironmentContext)
-   */
-  @Override
-  public void dropTable(String dbname, String name)
-      throws NoSuchObjectException, MetaException, TException {
-    dropTable(dbname, name, true, true, null);
   }
 
   /**
    * Drop the table and choose whether to: delete the underlying table data;
    * throw if the table doesn't exist; save the data in the trash.
    *
-   * @param dbname
-   * @param name
+   * @param catName catalog name
+   * @param dbname database name
+   * @param name table name
    * @param deleteData
    *          delete the underlying data or just delete the table in metadata
    * @param ignoreUnknownTab
@@ -1145,12 +1209,12 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
    * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#drop_table(java.lang.String,
    *      java.lang.String, boolean)
    */
-  public void dropTable(String dbname, String name, boolean deleteData,
+  public void dropTable(String catName, String dbname, String name, boolean deleteData,
       boolean ignoreUnknownTab, EnvironmentContext envContext) throws MetaException, TException,
       NoSuchObjectException, UnsupportedOperationException {
     Table tbl;
     try {
-      tbl = getTable(dbname, name);
+      tbl = getTable(catName, dbname, name);
     } catch (NoSuchObjectException e) {
       if (!ignoreUnknownTab) {
         throw e;
@@ -1163,7 +1227,7 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
     }
     boolean success = false;
     try {
-      drop_table_with_environment_context(dbname, name, deleteData, envContext);
+      drop_table_with_environment_context(catName, dbname, name, deleteData, envContext);
       if (hook != null) {
         hook.commitDropTable(tbl, deleteData || (envContext != null && "TRUE".equals(envContext.getProperties().get("ifPurge"))));
       }
@@ -1179,21 +1243,15 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
     }
   }
 
-  /**
-   * Truncate the table/partitions in the DEFAULT database.
-   * @param dbName
-   *          The db to which the table to be truncate belongs to
-   * @param tableName
-   *          The table to truncate
-   * @param partNames
-   *          List of partitions to truncate. NULL will truncate the whole table/all partitions
-   * @throws MetaException
-   * @throws TException
-   *           Could not truncate table properly.
-   */
   @Override
-  public void truncateTable(String dbName, String tableName, List<String> partNames) throws MetaException, TException {
-    client.truncate_table(dbName, tableName, partNames);
+  public void truncateTable(String dbName, String tableName, List<String> partNames) throws TException {
+    truncateTable(getDefaultCatalog(conf), dbName, tableName, partNames);
+  }
+
+  @Override
+  public void truncateTable(String catName, String dbName, String tableName, List<String> partNames)
+      throws TException {
+    client.truncate_table(prependCatalogToDbName(catName, dbName, conf), tableName, partNames);
   }
 
   /**
@@ -1239,111 +1297,144 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
     return result;
   }
 
-  /** {@inheritDoc} */
   @Override
-  public List<String> getDatabases(String databasePattern)
-    throws MetaException {
-    try {
-      return filterHook.filterDatabases(client.get_databases(databasePattern));
-    } catch (Exception e) {
-      MetaStoreUtils.logAndThrowMetaException(e);
-    }
-    return null;
+  public List<String> getDatabases(String databasePattern) throws TException {
+    return getDatabases(getDefaultCatalog(conf), databasePattern);
   }
 
-  /** {@inheritDoc} */
   @Override
-  public List<String> getAllDatabases() throws MetaException {
-    try {
-      return filterHook.filterDatabases(client.get_all_databases());
-    } catch (Exception e) {
-      MetaStoreUtils.logAndThrowMetaException(e);
-    }
-    return null;
+  public List<String> getDatabases(String catName, String databasePattern) throws TException {
+    return filterHook.filterDatabases(client.get_databases(prependCatalogToDbName(
+        catName, databasePattern, conf)));
   }
 
-  /**
-   * @param tbl_name
-   * @param db_name
-   * @param max_parts
-   * @return list of partitions
-   * @throws NoSuchObjectException
-   * @throws MetaException
-   * @throws TException
-   */
   @Override
-  public List<Partition> listPartitions(String db_name, String tbl_name,
-      short max_parts) throws NoSuchObjectException, MetaException, TException {
-    List<Partition> parts = client.get_partitions(db_name, tbl_name, max_parts);
-    return fastpath ? parts : deepCopyPartitions(filterHook.filterPartitions(parts));
+  public List<String> getAllDatabases() throws TException {
+    return getAllDatabases(getDefaultCatalog(conf));
+  }
+
+  @Override
+  public List<String> getAllDatabases(String catName) throws TException {
+    return filterHook.filterDatabases(client.get_databases(prependCatalogToDbName(catName, null, conf)));
+  }
+
+  @Override
+  public List<Partition> listPartitions(String db_name, String tbl_name, short max_parts)
+      throws TException {
+    return listPartitions(getDefaultCatalog(conf), db_name, tbl_name, max_parts);
+  }
+
+  @Override
+  public List<Partition> listPartitions(String catName, String db_name, String tbl_name,
+                                        int max_parts) throws TException {
+    List<Partition> parts = client.get_partitions(prependCatalogToDbName(catName, db_name, conf),
+        tbl_name, shrinkMaxtoShort(max_parts));
+    return deepCopyPartitions(filterHook.filterPartitions(parts));
   }
 
   @Override
   public PartitionSpecProxy listPartitionSpecs(String dbName, String tableName, int maxParts) throws TException {
+    return listPartitionSpecs(getDefaultCatalog(conf), dbName, tableName, maxParts);
+  }
+
+  @Override
+  public PartitionSpecProxy listPartitionSpecs(String catName, String dbName, String tableName,
+                                               int maxParts) throws TException {
     return PartitionSpecProxy.Factory.get(filterHook.filterPartitionSpecs(
-        client.get_partitions_pspec(dbName, tableName, maxParts)));
+        client.get_partitions_pspec(prependCatalogToDbName(catName, dbName, conf), tableName, maxParts)));
   }
 
   @Override
   public List<Partition> listPartitions(String db_name, String tbl_name,
-      List<String> part_vals, short max_parts)
-      throws NoSuchObjectException, MetaException, TException {
-    List<Partition> parts = client.get_partitions_ps(db_name, tbl_name, part_vals, max_parts);
-    return fastpath ? parts : deepCopyPartitions(filterHook.filterPartitions(parts));
+                                        List<String> part_vals, short max_parts) throws TException {
+    return listPartitions(getDefaultCatalog(conf), db_name, tbl_name, part_vals, max_parts);
   }
 
   @Override
-  public List<Partition> listPartitionsWithAuthInfo(String db_name,
-      String tbl_name, short max_parts, String user_name, List<String> group_names)
-       throws NoSuchObjectException, MetaException, TException {
-    List<Partition> parts = client.get_partitions_with_auth(db_name, tbl_name, max_parts,
+  public List<Partition> listPartitions(String catName, String db_name, String tbl_name,
+                                        List<String> part_vals, int max_parts) throws TException {
+    List<Partition> parts = client.get_partitions_ps(prependCatalogToDbName(catName, db_name, conf),
+        tbl_name, part_vals, shrinkMaxtoShort(max_parts));
+    return deepCopyPartitions(filterHook.filterPartitions(parts));
+  }
+
+  @Override
+  public List<Partition> listPartitionsWithAuthInfo(String db_name, String tbl_name,
+                                                    short max_parts, String user_name,
+                                                    List<String> group_names) throws TException {
+    return listPartitionsWithAuthInfo(getDefaultCatalog(conf), db_name, tbl_name, max_parts, user_name,
+        group_names);
+  }
+
+  @Override
+  public List<Partition> listPartitionsWithAuthInfo(String catName, String dbName, String tableName,
+                                                    int maxParts, String userName,
+                                                    List<String> groupNames) throws TException {
+    List<Partition> parts = client.get_partitions_with_auth(prependCatalogToDbName(catName,
+        dbName, conf), tableName, shrinkMaxtoShort(maxParts), userName, groupNames);
+    return deepCopyPartitions(filterHook.filterPartitions(parts));
+  }
+
+  @Override
+  public List<Partition> listPartitionsWithAuthInfo(String db_name, String tbl_name,
+                                                    List<String> part_vals, short max_parts,
+                                                    String user_name, List<String> group_names)
+      throws TException {
+    return listPartitionsWithAuthInfo(getDefaultCatalog(conf), db_name, tbl_name, part_vals, max_parts,
         user_name, group_names);
-    return fastpath ? parts :deepCopyPartitions(filterHook.filterPartitions(parts));
   }
 
   @Override
-  public List<Partition> listPartitionsWithAuthInfo(String db_name,
-      String tbl_name, List<String> part_vals, short max_parts,
-      String user_name, List<String> group_names) throws NoSuchObjectException,
-      MetaException, TException {
-    List<Partition> parts = client.get_partitions_ps_with_auth(db_name,
-        tbl_name, part_vals, max_parts, user_name, group_names);
-    return fastpath ? parts : deepCopyPartitions(filterHook.filterPartitions(parts));
+  public List<Partition> listPartitionsWithAuthInfo(String catName, String dbName, String tableName,
+                                                    List<String> partialPvals, int maxParts,
+                                                    String userName, List<String> groupNames)
+      throws TException {
+    List<Partition> parts = client.get_partitions_ps_with_auth(prependCatalogToDbName(catName,
+        dbName, conf), tableName, partialPvals, shrinkMaxtoShort(maxParts), userName, groupNames);
+    return deepCopyPartitions(filterHook.filterPartitions(parts));
   }
 
-  /**
-   * Get list of partitions matching specified filter
-   * @param db_name the database name
-   * @param tbl_name the table name
-   * @param filter the filter string,
-   *    for example "part1 = \"p1_abc\" and part2 &lt;= "\p2_test\"". Filtering can
-   *    be done only on string partition keys.
-   * @param max_parts the maximum number of partitions to return,
-   *    all partitions are returned if -1 is passed
-   * @return list of partitions
-   * @throws MetaException
-   * @throws NoSuchObjectException
-   * @throws TException
-   */
   @Override
   public List<Partition> listPartitionsByFilter(String db_name, String tbl_name,
-      String filter, short max_parts) throws MetaException,
-         NoSuchObjectException, TException {
-    List<Partition> parts = client.get_partitions_by_filter(db_name, tbl_name, filter, max_parts);
-    return fastpath ? parts :deepCopyPartitions(filterHook.filterPartitions(parts));
+      String filter, short max_parts) throws TException {
+    return listPartitionsByFilter(getDefaultCatalog(conf), db_name, tbl_name, filter, max_parts);
+  }
+
+  @Override
+  public List<Partition> listPartitionsByFilter(String catName, String db_name, String tbl_name,
+                                                String filter, int max_parts) throws TException {
+    List<Partition> parts =client.get_partitions_by_filter(prependCatalogToDbName(
+        catName, db_name, conf), tbl_name, filter, shrinkMaxtoShort(max_parts));
+    return deepCopyPartitions(filterHook.filterPartitions(parts));
   }
 
   @Override
   public PartitionSpecProxy listPartitionSpecsByFilter(String db_name, String tbl_name,
-                                                       String filter, int max_parts) throws MetaException,
-         NoSuchObjectException, TException {
+                                                       String filter, int max_parts)
+      throws TException {
+    return listPartitionSpecsByFilter(getDefaultCatalog(conf), db_name, tbl_name, filter, max_parts);
+  }
+
+  @Override
+  public PartitionSpecProxy listPartitionSpecsByFilter(String catName, String db_name,
+                                                       String tbl_name, String filter,
+                                                       int max_parts) throws TException {
     return PartitionSpecProxy.Factory.get(filterHook.filterPartitionSpecs(
-        client.get_part_specs_by_filter(db_name, tbl_name, filter, max_parts)));
+        client.get_part_specs_by_filter(prependCatalogToDbName(catName, db_name, conf), tbl_name, filter,
+            max_parts)));
   }
 
   @Override
   public boolean listPartitionsByExpr(String db_name, String tbl_name, byte[] expr,
-      String default_partition_name, short max_parts, List<Partition> result)
+                                      String default_partition_name, short max_parts,
+                                      List<Partition> result) throws TException {
+    return listPartitionsByExpr(getDefaultCatalog(conf), db_name, tbl_name, expr,
+        default_partition_name, max_parts, result);
+  }
+
+  @Override
+  public boolean listPartitionsByExpr(String catName, String db_name, String tbl_name, byte[] expr,
+      String default_partition_name, int max_parts, List<Partition> result)
           throws TException {
     assert result != null;
     PartitionsByExprRequest req = new PartitionsByExprRequest(
@@ -1352,7 +1443,7 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
       req.setDefaultPartitionName(default_partition_name);
     }
     if (max_parts >= 0) {
-      req.setMaxParts(max_parts);
+      req.setMaxParts(shrinkMaxtoShort(max_parts));
     }
     PartitionsByExprResult r;
     try {
@@ -1366,132 +1457,138 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
       throw new IncompatibleMetastoreException(
           "Metastore doesn't support listPartitionsByExpr: " + te.getMessage());
     }
-    if (fastpath) {
-      result.addAll(r.getPartitions());
-    } else {
-      r.setPartitions(filterHook.filterPartitions(r.getPartitions()));
-      // TODO: in these methods, do we really need to deepcopy?
-      deepCopyPartitions(r.getPartitions(), result);
-    }
+    r.setPartitions(filterHook.filterPartitions(r.getPartitions()));
+    // TODO: in these methods, do we really need to deepcopy?
+    deepCopyPartitions(r.getPartitions(), result);
     return !r.isSetHasUnknownPartitions() || r.isHasUnknownPartitions(); // Assume the worst.
   }
 
-  /**
-   * @param name
-   * @return the database
-   * @throws NoSuchObjectException
-   * @throws MetaException
-   * @throws TException
-   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#get_database(java.lang.String)
-   */
   @Override
-  public Database getDatabase(String name) throws NoSuchObjectException,
-      MetaException, TException {
-    Database d = client.get_database(name);
-    return fastpath ? d :deepCopy(filterHook.filterDatabase(d));
+  public Database getDatabase(String name) throws TException {
+    return getDatabase(getDefaultCatalog(conf), name);
+  }
+
+  @Override
+  public Database getDatabase(String catalogName, String databaseName) throws TException {
+    Database d = client.get_database(prependCatalogToDbName(catalogName, databaseName, conf));
+    return deepCopy(filterHook.filterDatabase(d));
   }
 
-  /**
-   * @param tbl_name
-   * @param db_name
-   * @param part_vals
-   * @return the partition
-   * @throws MetaException
-   * @throws TException
-   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#get_partition(java.lang.String,
-   *      java.lang.String, java.util.List)
-   */
   @Override
-  public Partition getPartition(String db_name, String tbl_name,
-      List<String> part_vals) throws NoSuchObjectException, MetaException, TException {
-    Partition p = client.get_partition(db_name, tbl_name, part_vals);
-    return fastpath ? p : deepCopy(filterHook.filterPartition(p));
+  public Partition getPartition(String db_name, String tbl_name, List<String> part_vals)
+      throws TException {
+    return getPartition(getDefaultCatalog(conf), db_name, tbl_name, part_vals);
+  }
+
+  @Override
+  public Partition getPartition(String catName, String dbName, String tblName,
+                                List<String> partVals) throws TException {
+    Partition p = client.get_partition(prependCatalogToDbName(catName, dbName, conf), tblName, partVals);
+    return deepCopy(filterHook.filterPartition(p));
   }
 
   @Override
   public List<Partition> getPartitionsByNames(String db_name, String tbl_name,
-      List<String> part_names) throws NoSuchObjectException, MetaException, TException {
-    List<Partition> parts = client.get_partitions_by_names(db_name, tbl_name, part_names);
-    return fastpath ? parts : deepCopyPartitions(filterHook.filterPartitions(parts));
+      List<String> part_names) throws TException {
+    return getPartitionsByNames(getDefaultCatalog(conf), db_name, tbl_name, part_names);
+  }
+
+  @Override
+  public List<Partition> getPartitionsByNames(String catName, String db_name, String tbl_name,
+                                              List<String> part_names) throws TException {
+    List<Partition> parts =
+        client.get_partitions_by_names(prependCatalogToDbName(catName, db_name, conf), tbl_name, part_names);
+    return deepCopyPartitions(filterHook.filterPartitions(parts));
   }
 
   @Override
   public PartitionValuesResponse listPartitionValues(PartitionValuesRequest request)
       throws MetaException, TException, NoSuchObjectException {
+    if (!request.isSetCatName()) request.setCatName(getDefaultCatalog(conf));
     return client.get_partition_values(request);
   }
 
   @Override
   public Partition getPartitionWithAuthInfo(String db_name, String tbl_name,
       List<String> part_vals, String user_name, List<String> group_names)
-      throws MetaException, UnknownTableException, NoSuchObjectException,
-      TException {
-    Partition p = client.get_partition_with_auth(db_name, tbl_name, part_vals, user_name,
-        group_names);
-    return fastpath ? p : deepCopy(filterHook.filterPartition(p));
+      throws TException {
+    return getPartitionWithAuthInfo(getDefaultCatalog(conf), db_name, tbl_name, part_vals,
+        user_name, group_names);
+  }
+
+  @Override
+  public Partition getPartitionWithAuthInfo(String catName, String dbName, String tableName,
+                                            List<String> pvals, String userName,
+                                            List<String> groupNames) throws TException {
+    Partition p = client.get_partition_with_auth(prependCatalogToDbName(catName, dbName, conf), tableName,
+        pvals, userName, groupNames);
+    return deepCopy(filterHook.filterPartition(p));
   }
 
-  /**
-   * @param name
-   * @param dbname
-   * @return the table
-   * @throws NoSuchObjectException
-   * @throws MetaException
-   * @throws TException
-   * @throws NoSuchObjectException
-   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#get_table(java.lang.String,
-   *      java.lang.String)
-   */
   @Override
-  public Table getTable(String dbname, String name) throws MetaException,
-      TException, NoSuchObjectException {
-    GetTableRequest req = new GetTableRequest(dbname, name);
+  public Table getTable(String dbname, String name) throws TException {
+    return getTable(getDefaultCatalog(conf), dbname, name);
+  }
+
+  @Override
+  public Table getTable(String catName, String dbName, String tableName) throws TException {
+    GetTableRequest req = new GetTableRequest(dbName, tableName);
+    req.setCatName(catName);
     req.setCapabilities(version);
     Table t = client.get_table_req(req).getTable();
-    return fastpath ? t : deepCopy(filterHook.filterTable(t));
+    return deepCopy(filterHook.filterTable(t));
   }
 
-  /** {@inheritDoc} */
   @Override
-  @Deprecated
-  public Table getTable(String tableName) throws MetaException, TException,
-      NoSuchObjectException {
-    Table t = getTable(DEFAULT_DATABASE_NAME, tableName);
-    return fastpath ? t : filterHook.filterTable(t);
+  public List<Table> getTableObjectsByName(String dbName, List<String> tableNames)
+      throws TException {
+    return getTableObjectsByName(getDefaultCatalog(conf), dbName, tableNames);
   }
 
-  /** {@inheritDoc} */
   @Override
-  public List<Table> getTableObjectsByName(String dbName, List<String> tableNames)
-      throws MetaException, InvalidOperationException, UnknownDBException, TException {
+  public List<Table> getTableObjectsByName(String catName, String dbName,
+                                           List<String> tableNames) throws TException {
     GetTablesRequest req = new GetTablesRequest(dbName);
+    req.setCatName(catName);
     req.setTblNames(tableNames);
     req.setCapabilities(version);
     List<Table> tabs = client.get_table_objects_by_name_req(req).getTables();
-    return fastpath ? tabs : deepCopyTables(filterHook.filterTables(tabs));
+    return deepCopyTables(filterHook.filterTables(tabs));
   }
 
-  /** {@inheritDoc} */
   @Override
   public Map<String, Materialization> getMaterializationsInvalidationInfo(String dbName, List<String> viewNames)
       throws MetaException, InvalidOperationException, UnknownDBException, TException {
     return client.get_materialization_invalidation_info(
-        dbName, filterHook.filterTableNames(dbName, viewNames));
+        dbName, filterHook.filterTableNames(getDefaultCatalog(conf), dbName, viewNames));
   }
 
-  /** {@inheritDoc} */
   @Override
   public void updateCreationMetadata(String dbName, String tableName, CreationMetadata cm)
       throws MetaException, InvalidOperationException, UnknownDBException, TException {
-    client.update_creation_metadata(dbName, tableName, cm);
+    client.update_creation_metadata(getDefaultCatalog(conf), dbName, tableName, cm);
+  }
+
+  @Override
+  public void updateCreationMetadata(String catName, String dbName, String tableName,
+                                     CreationMetadata cm) throws MetaException, TException {
+    client.update_creation_metadata(catName, dbName, tableName, cm);
+
   }
 
   /** {@inheritDoc} */
   @Override
   public List<String> listTableNamesByFilter(String dbName, String filter, short maxTables)
-      throws MetaException, TException, InvalidOperationException, UnknownDBException {
-    return filterHook.filterTableNames(dbName,
-        client.get_table_names_by_filter(dbName, filter, maxTables));
+      throws TException {
+    return listTableNamesByFilter(getDefaultCatalog(conf), dbName, filter, maxTables);
+  }
+
+  @Override
+  public List<String> listTableNamesByFilter(String catName, String dbName, String filter,
+                                             int maxTables) throws TException {
+    return filterHook.filterTableNames(catName, dbName,
+        client.get_table_names_by_filter(prependCatalogToDbName(catName, dbName, conf), filter,
+            shrinkMaxtoShort(maxTables)));
   }
 
   /**
@@ -1506,34 +1603,52 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
     return deepCopy(client.get_type(name));
   }
 
-  /** {@inheritDoc} */
   @Override
   public List<String> getTables(String dbname, String tablePattern) throws MetaException {
     try {
-      return filterHook.filterTableNames(dbname, client.get_tables(dbname, tablePattern));
+      return getTables(getDefaultCatalog(conf), dbname, tablePattern);
     } catch (Exception e) {
       MetaStoreUtils.logAndThrowMetaException(e);
     }
     return null;
   }
 
-  /** {@inheritDoc} */
+  @Override
+  public List<String> getTables(String catName, String dbName, String tablePattern)
+      throws TException {
+    return filterHook.filterTableNames(catName, dbName,
+        client.get_tables(prependCatalogToDbName(catName, dbName, conf), tablePattern));
+  }
+
   @Override
   public List<String> getTables(String dbname, String tablePattern, TableType tableType) throws MetaException {
     try {
-      return filterHook.filterTableNames(dbname,
-          client.get_tables_by_type(dbname, tablePattern, tableType.toString()));
+      return getTables(getDefaultCatalog(conf), dbname, tablePattern, tableType);
     } catch (Exception e) {
       MetaStoreUtils.logAndThrowMetaException(e);
     }
     return null;
   }
 
-  /** {@inheritDoc} */
   @Override
-  public List<String> getMaterializedViewsForRewriting(String dbname) throws MetaException {
+  public List<String> getTables(String catName, String dbName, String tablePattern,
+                                TableType tableType) throws TException {
+    return filterHook.filterTableNames(catName, dbName,
+        client.get_tables_by_type(prependCatalogToDbName(catName, dbName, conf), tablePattern,
+            tableType.toString()));
+  }
+
+  @Override
+  public List<String> getMaterializedViewsForRewriting(String dbName) throws TException {
+    return getMaterializedViewsForRewriting(getDefaultCatalog(conf), dbName);
+  }
+
+  @Override
+  public List<String> getMaterializedViewsForRewriting(String catName, String dbname)
+      throws MetaException {
     try {
-      return filterHook.filterTableNames(dbname, client.get_materialized_views_for_rewriting(dbname));
+      return filterHook.filterTableNames(catName, dbname,
+          client.get_materialized_views_for_rewriting(prependCatalogToDbName(catName, dbname, conf)));
     } catch (Exception e) {
       MetaStoreUtils.logAndThrowMetaException(e);
     }
@@ -1544,38 +1659,24 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   public List<TableMeta> getTableMeta(String dbPatterns, String tablePatterns, List<String> tableTypes)
       throws MetaException {
     try {
-      return filterNames(client.get_table_meta(dbPatterns, tablePatterns, tableTypes));
+      return getTableMeta(getDefaultCatalog(conf), dbPatterns, tablePatterns, tableTypes);
     } catch (Exception e) {
       MetaStoreUtils.logAndThrowMetaException(e);
     }
     return null;
   }
 
-  private List<TableMeta> filterNames(List<TableMeta> metas) throws MetaException {
-    Map<String, TableMeta> sources = new LinkedHashMap<>();
-    Map<String, List<String>> dbTables = new LinkedHashMap<>();
-    for (TableMeta meta : metas) {
-      sources.put(meta.getDbName() + "." + meta.getTableName(), meta);
-      List<String> tables = dbTables.get(meta.getDbName());
-      if (tables == null) {
-        dbTables.put(meta.getDbName(), tables = new ArrayList<>());
-      }
-      tables.add(meta.getTableName());
-    }
-    List<TableMeta> filtered = new ArrayList<>();
-    for (Map.Entry<String, List<String>> entry : dbTables.entrySet()) {
-      for (String table : filterHook.filterTableNames(entry.getKey(), entry.getValue())) {
-        filtered.add(sources.get(entry.getKey() + "." + table));
-      }
-    }
-    return filtered;
+  @Override
+  public List<TableMeta> getTableMeta(String catName, String dbPatterns, String tablePatterns,
+                                      List<String> tableTypes) throws TException {
+    return filterHook.filterTableMetas(client.get_table_meta(prependCatalogToDbName(
+        catName, dbPatterns, conf), tablePatterns, tableTypes));
   }
 
-  /** {@inheritDoc} */
   @Override
   public List<String> getAllTables(String dbname) throws MetaException {
     try {
-      return filterHook.filterTableNames(dbname, client.get_all_tables(dbname));
+      return getAllTables(getDefaultCatalog(conf), dbname);
     } catch (Exception e) {
       MetaStoreUtils.logAndThrowMetaException(e);
     }
@@ -1583,10 +1684,21 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   }
 
   @Override
-  public boolean tableExists(String databaseName, String tableName) throws MetaException,
-      TException, UnknownDBException {
+  public List<String> getAllTables(String catName, String dbName) throws TException {
+    return filterHook.filterTableNames(catName, dbName, client.get_all_tables(
+        prependCatalogToDbName(catName, dbName, conf)));
+  }
+
+  @Override
+  public boolean tableExists(String databaseName, String tableName) throws TException {
+    return tableExists(getDefaultCatalog(conf), databaseName, tableName);
+  }
+
+  @Override
+  public boolean tableExists(String catName, String dbName, String tableName) throws TException {
     try {
-      GetTableRequest req = new GetTableRequest(databaseName, tableName);
+      GetTableRequest req = new GetTableRequest(dbName, tableName);
+      req.setCatName(catName);
       req.setCapabilities(version);
       return filterHook.filterTable(client.get_table_req(req).getTable()) != null;
     } catch (NoSuchObjectException e) {
@@ -1594,156 +1706,167 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
     }
   }
 
-  /** {@inheritDoc} */
   @Override
-  @Deprecated
-  public boolean tableExists(String tableName) throws MetaException,
-      TException, UnknownDBException {
-    return tableExists(DEFAULT_DATABASE_NAME, tableName);
+  public List<String> listPartitionNames(String dbName, String tblName,
+      short max) throws NoSuchObjectException, MetaException, TException {
+    return listPartitionNames(getDefaultCatalog(conf), dbName, tblName, max);
   }
 
   @Override
-  public List<String> listPartitionNames(String dbName, String tblName,
-      short max) throws NoSuchObjectException, MetaException, TException {
-    return filterHook.filterPartitionNames(dbName, tblName,
-        client.get_partition_names(dbName, tblName, max));
+  public List<String> listPartitionNames(String catName, String dbName, String tableName,
+                                         int maxParts) throws TException {
+    return filterHook.filterPartitionNames(catName, dbName, tableName,
+        client.get_partition_names(prependCatalogToDbName(catName, dbName, conf), tableName, shrinkMaxtoShort(maxParts)));
   }
 
   @Override
   public List<String> listPartitionNames(String db_name, String tbl_name,
-      List<String> part_vals, short max_parts)
-      throws MetaException, TException, NoSuchObjectException {
-    return filterHook.filterPartitionNames(db_name, tbl_name,
-        client.get_partition_names_ps(db_name, tbl_name, part_vals, max_parts));
+      List<String> part_vals, short max_parts) throws TException {
+    return listPartitionNames(getDefaultCatalog(conf), db_name, tbl_name, part_vals, max_parts);
+  }
+
+  @Override
+  public List<String> listPartitionNames(String catName, String db_name, String tbl_name,
+                                         List<String> part_vals, int max_parts) throws TException {
+    return filterHook.filterPartitionNames(catName, db_name, tbl_name,
+        client.get_partition_names_ps(prependCatalogToDbName(catName, db_name, conf), tbl_name,
+            part_vals, shrinkMaxtoShort(max_parts)));
   }
 
-  /**
-   * Get number of partitions matching specified filter
-   * @param db_name the database name
-   * @param tbl_name the table name
-   * @param filter the filter string,
-   *    for example "part1 = \"p1_abc\" and part2 &lt;= "\p2_test\"". Filtering can
-   *    be done only on string partition keys.
-   * @return number of partitions
-   * @throws MetaException
-   * @throws NoSuchObjectException
-   * @throws TException
-   */
   @Override
   public int getNumPartitionsByFilter(String db_name, String tbl_name,
-                                      String filter) throws MetaException,
-          NoSuchObjectException, TException {
-    return client.get_num_partitions_by_filter(db_name, tbl_name, filter);
+                                      String filter) throws TException {
+    return getNumPartitionsByFilter(getDefaultCatalog(conf), db_name, tbl_name, filter);
+  }
+
+  @Override
+  public int getNumPartitionsByFilter(String catName, String dbName, String tableName,
+                                      String filter) throws TException {
+    return client.get_num_partitions_by_filter(prependCatalogToDbName(catName, dbName, conf), tableName,
+        filter);
   }
 
   @Override
   public void alter_partition(String dbName, String tblName, Partition newPart)
       throws InvalidOperationException, MetaException, TException {
-    client.alter_partition_with_environment_context(dbName, tblName, newPart, null);
+    alter_partition(getDefaultCatalog(conf), dbName, tblName, newPart, null);
   }
 
   @Override
   public void alter_partition(String dbName, String tblName, Partition newPart, EnvironmentContext environmentContext)
       throws InvalidOperationException, MetaException, TException {
-    client.alter_partition_with_environment_context(dbName, tblName, newPart, environmentContext);
+    alter_partition(getDefaultCatalog(conf), dbName, tblName, newPart, environmentContext);
+  }
+
+  @Override
+  public void alter_partition(String catName, String dbName, String tblName, Partition newPart,
+                              EnvironmentContext environmentContext) throws TException {
+    client.alter_partition_with_environment_context(prependCatalogToDbName(catName, dbName, conf), tblName,
+        newPart, environmentContext);
   }
 
   @Override
   public void alter_partitions(String dbName, String tblName, List<Partition> newParts)
-      throws InvalidOperationException, MetaException, TException {
-    client.alter_partitions_with_environment_context(dbName, tblName, newParts, null);
+      throws TException {
+    alter_partitions(getDefaultCatalog(conf), dbName, tblName, newParts, null);
   }
 
   @Override
-  public void alter_partitions(String dbName, String tblName, List<Partition> newParts, EnvironmentContext environmentContext)
-  throws InvalidOperationException, MetaException, TException {
-    client.alter_partitions_with_environment_context(dbName, tblName, newParts, environmentContext);
-}
+  public void alter_partitions(String dbName, String tblName, List<Partition> newParts,
+                               EnvironmentContext environmentContext) throws TException {
+    alter_partitions(getDefaultCatalog(conf), dbName, tblName, newParts, environmentContext);
+  }
 
   @Override
-  public void alterDatabase(String dbName, Database db)
-      throws MetaException, NoSuchObjectException, TException {
-    client.alter_database(dbName, db);
+  public void alter_partitions(String catName, String dbName, String tblName,
+                               List<Partition> newParts,
+                               EnvironmentContext environmentContext) throws TException {
+    client.alter_partitions_with_environment_context(prependCatalogToDbName(catName, dbName, conf),
+        tblName, newParts, environmentContext);
   }
-  /**
-   * @param db
-   * @param tableName
-   * @throws UnknownTableException
-   * @throws UnknownDBException
-   * @throws MetaException
-   * @throws TException
-   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#get_fields(java.lang.String,
-   *      java.lang.String)
-   */
+
   @Override
-  public List<FieldSchema> getFields(String db, String tableName)
-      throws MetaException, TException, UnknownTableException,
-      UnknownDBException {
-    List<FieldSchema> fields = client.get_fields(db, tableName);
-    return fastpath ? fields : deepCopyFieldSchemas(fields);
+  public void alterDatabase(String dbName, Database db) throws TException {
+    alterDatabase(getDefaultCatalog(conf), dbName, db);
   }
 
   @Override
-  public List<SQLPrimaryKey> getPrimaryKeys(PrimaryKeysRequest req)
-    throws MetaException, NoSuchObjectException, TException {
+  public void alterDatabase(String catName, String dbName, Database newDb) throws TException {
+    client.alter_database(prependCatalogToDbName(catName, dbName, conf), newDb);
+  }
+
+  @Override
+  public List<FieldSchema> getFields(String db, String tableName) throws TException {
+    return getFields(getDefaultCatalog(conf), db, tableName);
+  }
+
+  @Override
+  public List<FieldSchema> getFields(String catName, String db, String tableName)
+      throws TException {
+    List<FieldSchema> fields = client.get_fields(prependCatalogToDbName(catName, db, conf), tableName);
+    return deepCopyFieldSchemas(fields);
+  }
+
+  @Override
+  public List<SQLPrimaryKey> getPrimaryKeys(PrimaryKeysRequest req) throws TException {
+    if (!req.isSetCatName()) req.setCatName(getDefaultCatalog(conf));
     return client.get_primary_keys(req).getPrimaryKeys();
   }
 
   @Override
   public List<SQLForeignKey> getForeignKeys(ForeignKeysRequest req) throws MetaException,
     NoSuchObjectException, TException {
+    if (!req.isSetCatName()) req.setCatName(getDefaultCatalog(conf));
     return client.get_foreign_keys(req).getForeignKeys();
   }
 
   @Override
   public List<SQLUniqueConstraint> getUniqueConstraints(UniqueConstraintsRequest req)
     throws MetaException, NoSuchObjectException, TException {
+    if (!req.isSetCatName()) req.setCatName(getDefaultCatalog(conf));
     return client.get_unique_constraints(req).getUniqueConstraints();
   }
 
   @Override
   public List<SQLNotNullConstraint> getNotNullConstraints(NotNullConstraintsRequest req)
     throws MetaException, NoSuchObjectException, TException {
+    if (!req.isSetCatName()) req.setCatName(getDefaultCatalog(conf));
     return client.get_not_null_constraints(req).getNotNullConstraints();
   }
 
   @Override
   public List<SQLDefaultConstraint> getDefaultConstraints(DefaultConstraintsRequest req)
       throws MetaException, NoSuchObjectException, TException {
+    if (!req.isSetCatName()) req.setCatName(getDefaultCatalog(conf));
     return client.get_default_constraints(req).getDefaultConstraints();
   }
 
   @Override
   public List<SQLCheckConstraint> getCheckConstraints(CheckConstraintsRequest req)
       throws MetaException, NoSuchObjectException, TException {
+    if (!req.isSetCatName()) req.setCatName(getDefaultCatalog(conf));
     return client.get_check_constraints(req).getCheckConstraints();
   }
 
   /** {@inheritDoc} */
   @Override
-  @Deprecated
-  //use setPartitionColumnStatistics instead
-  public boolean updateTableColumnStatistics(ColumnStatistics statsObj)
-    throws NoSuchObjectException, InvalidObjectException, MetaException, TException,
-    InvalidInputException{
+  public boolean updateTableColumnStatistics(ColumnStatistics statsObj) throws TException {
+    if (!statsObj.getStatsDesc().isSetCatName()) statsObj.getStatsDesc().setCatName(getDefaultCatalog(conf));
     return client.update_table_column_statistics(statsObj);
   }
 
-  /** {@inheritDoc} */
   @Override
-  @Deprecated
-  //use setPartitionColumnStatistics instead
-  public boolean updatePartitionColumnStatistics(ColumnStatistics statsObj)
-    throws NoSuchObjectException, InvalidObjectException, MetaException, TException,
-    InvalidInputException{
+  public boolean updatePartitionColumnStatistics(ColumnStatistics statsObj) throws TException {
+    if (!statsObj.getStatsDesc().isSetCatName()) statsObj.getStatsDesc().setCatName(getDefaultCatalog(conf));
     return client.update_partition_column_statistics(statsObj);
   }
 
-  /** {@inheritDoc} */
   @Override
-  public boolean setPartitionColumnStatistics(SetPartitionsStatsRequest request)
-    throws NoSuchObjectException, InvalidObjectException, MetaException, TException,
-    InvalidInputException{
+  public boolean setPartitionColumnStatistics(SetPartitionsStatsRequest request) throws TException {
+    String defaultCat = getDefaultCatalog(conf);
+    for (ColumnStatistics stats : request.getColStats()) {
+      if (!stats.getStatsDesc().isSetCatName()) stats.getStatsDesc().setCatName(defaultCat);
+    }
     return client.set_aggr_stats_for(request);
   }
 
@@ -1757,66 +1880,84 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
     }
   }
 
-  /** {@inheritDoc} */
   @Override
   public List<ColumnStatisticsObj> getTableColumnStatistics(String dbName, String tableName,
-      List<String> colNames) throws NoSuchObjectException, MetaException, TException,
-      InvalidInputException, InvalidObjectException {
-    return client.get_table_statistics_req(
-        new TableStatsRequest(dbName, tableName, colNames)).getTableStats();
+      List<String> colNames) throws TException {
+    return getTableColumnStatistics(getDefaultCatalog(conf), dbName, tableName, colNames);
+  }
+
+  @Override
+  public List<ColumnStatisticsObj> getTableColumnStatistics(String catName, String dbName,
+                                                            String tableName,
+                                                            List<String> colNames) throws TException {
+    TableStatsRequest rqst = new TableStatsRequest(dbName, tableName, colNames);
+    rqst.setCatName(catName);
+    return client.get_table_statistics_req(rqst).getTableStats();
   }
 
-  /** {@inheritDoc} */
   @Override
   public Map<String, List<ColumnStatisticsObj>> getPartitionColumnStatistics(
       String dbName, String tableName, List<String> partNames, List<String> colNames)
-          throws NoSuchObjectException, MetaException, TException {
-    return client.get_partitions_statistics_req(
-        new PartitionsStatsRequest(dbName, tableName, colNames, partNames)).getPartStats();
+          throws TException {
+    return getPartitionColumnStatistics(getDefaultCatalog(conf), dbName, tableName, partNames, colNames);
+  }
+
+  @Override
+  public Map<String, List<ColumnStatisticsObj>> getPartitionColumnStatistics(
+      String catName, String dbName, String tableName, List<String> partNames,
+      List<String> colNames) throws TException {
+    PartitionsStatsRequest rqst = new PartitionsStatsRequest(dbName, tableName, colNames,
+        partNames);
+    rqst.setCatName(catName);
+    return client.get_partitions_statistics_req(rqst).getPartStats();
   }
 
-  /** {@inheritDoc} */
   @Override
   public boolean deletePartitionColumnStatistics(String dbName, String tableName, String partName,
-    String colName) throws NoSuchObjectException, InvalidObjectException, MetaException,
-    TException, InvalidInputException
-  {
-    return client.delete_partition_column_statistics(dbName, tableName, partName, colName);
+    String colName) throws TException {
+    return deletePartitionColumnStatistics(getDefaultCatalog(conf), dbName, tableName, partName,
+        colName);
+  }
+
+  @Override
+  public boolean deletePartitionColumnStatistics(String catName, String dbName, String tableName,
+                                                 String partName, String colName)
+      throws TException {
+    return client.delete_partition_column_statistics(prependCatalogToDbName(catName, dbName, conf),
+        tableName, partName, colName);
   }
 
-  /** {@inheritDoc} */
   @Override
   public boolean deleteTableColumnStatistics(String dbName, String tableName, String colName)
-    throws NoSuchObjectException, InvalidObjectException, MetaException, TException,
-    InvalidInputException
-  {
-    return client.delete_table_column_statistics(dbName, tableName, colName);
+    throws TException {
+    return deleteTableColumnStatistics(getDefaultCatalog(conf), dbName, tableName, colName);
+  }
+
+  @Override
+  public boolean deleteTableColumnStatistics(String catName, String dbName, String tableName,
+                                             String colName) throws TException {
+    return client.delete_table_column_statistics(prependCatalogToDbName(catName, dbName, conf),
+        tableName, colName);
+  }
+
+  @Override
+  public List<FieldSchema> getSchema(String db, String tableName) throws TException {
+    return getSchema(getDefaultCatalog(conf), db, tableName);
   }
 
-  /**
-   * @param db
-   * @param tableName
-   * @throws UnknownTableException
-   * @throws UnknownDBException
-   * @throws MetaException
-   * @throws TException
-   * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#get_schema(java.lang.String,
-   *      java.lang.String)
-   */
   @Override
-  public List<FieldSchema> getSchema(String db, String tableName)
-      throws MetaException, TException, UnknownTableException,
-      UnknownDBException {
-      EnvironmentContext envCxt = null;
-      String addedJars = MetastoreConf.getVar(conf, ConfVars.ADDED_JARS);
-      if(org.apache.commons.lang.StringUtils.isNotBlank(addedJars)) {
-         Map<String, String> props = new HashMap<String, String>();
-         props.put("hive.added.jars.path", addedJars);
-         envCxt = new EnvironmentContext(props);
-       }
+  public List<FieldSchema> getSchema(String catName, String db, String tableName) throws TException {
+    EnvironmentContext envCxt = null;
+    String addedJars = MetastoreConf.getVar(conf, ConfVars.ADDED_JARS);
+    if(org.apache.commons.lang.StringUtils.isNotBlank(addedJars)) {
+      Map<String, String> props = new HashMap<>();
+      props.put("hive.added.jars.path", addedJars);
+      envCxt = new EnvironmentContext(props);
+    }
 
-    List<FieldSchema> fields = client.get_schema_with_environment_context(db, tableName, envCxt);
-    return fastpath ? fields : deepCopyFieldSchemas(fields);
+    List<FieldSchema> fields = client.get_schema_with_environment_context(prependCatalogToDbName(
+        catName, db, conf), tableName, envCxt);
+    return deepCopyFieldSchemas(fields);
   }
 
   @Override
@@ -1826,10 +1967,16 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   }
 
   @Override
-  public Partition getPartition(String db, String tableName, String partName)
-      throws MetaException, TException, UnknownTableException, NoSuchObjectException {
-    Partition p = client.get_partition_by_name(db, tableName, partName);
-    return fastpath ? p : deepCopy(filterHook.filterPartition(p));
+  public Partition getPartition(String db, String tableName, String partName) throws TException {
+    return getPartition(getDefaultCatalog(conf), db, tableName, partName);
+  }
+
+  @Override
+  public Partition getPartition(String catName, String dbName, String tblName, String name)
+      throws TException {
+    Partition p = client.get_partition_by_name(prependCatalogToDbName(catName, dbName, conf), tblName,
+        name);
+    return deepCopy(filterHook.filterPartition(p));
   }
 
   public Partition appendPartitionByName(String dbName, String tableName, String partName)
@@ -1842,7 +1989,7 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
       MetaException, TException {
     Partition p = client.append_partition_by_name_with_environment_context(dbName, tableName,
         partName, envContext);
-    return fastpath ? p : deepCopy(p);
+    return deepCopy(p);
   }
 
   public boolean dropPartitionByName(String dbName, String tableName, String partName,
@@ -2030,6 +2177,10 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   @Override
   public boolean grant_privileges(PrivilegeBag privileges)
       throws MetaException, TException {
+    String defaultCat = getDefaultCatalog(conf);
+    for (HiveObjectPrivilege priv : privileges.getPrivileges()) {
+      if (!priv.getHiveObject().isSetCatName()) priv.getHiveObject().setCatName(defaultCat);
+    }
     GrantRevokePrivilegeRequest req = new GrantRevokePrivilegeRequest();
     req.setRequestType(GrantRevokeType.GRANT);
     req.setPrivileges(privileges);
@@ -2059,6 +2210,10 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   @Override
   public boolean revoke_privileges(PrivilegeBag privileges, boolean grantOption) throws MetaException,
       TException {
+    String defaultCat = getDefaultCatalog(conf);
+    for (HiveObjectPrivilege priv : privileges.getPrivileges()) {
+      if (!priv.getHiveObject().isSetCatName()) priv.getHiveObject().setCatName(defaultCat);
+    }
     GrantRevokePrivilegeRequest req = new GrantRevokePrivilegeRequest();
     req.setRequestType(GrantRevokeType.REVOKE);
     req.setPrivileges(privileges);
@@ -2074,6 +2229,7 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   public PrincipalPrivilegeSet get_privilege_set(HiveObjectRef hiveObject,
       String userName, List<String> groupNames) throws MetaException,
       TException {
+    if (!hiveObject.isSetCatName()) hiveObject.setCatName(getDefaultCatalog(conf));
     return client.get_privilege_set(hiveObject, userName, groupNames);
   }
 
@@ -2081,6 +2237,7 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   public List<HiveObjectPrivilege> list_privileges(String principalName,
       PrincipalType principalType, HiveObjectRef hiveObject)
       throws MetaException, TException {
+    if (!hiveObject.isSetCatName()) hiveObject.setCatName(getDefaultCatalog(conf));
     return client.list_privileges(principalName, principalType, hiveObject);
   }
 
@@ -2396,12 +2553,14 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   @Override
   public NotificationEventsCountResponse getNotificationEventsCount(NotificationEventsCountRequest rqst)
           throws TException {
+    if (!rqst.isSetCatName()) rqst.setCatName(getDefaultCatalog(conf));
     return client.get_notification_events_count(rqst);
   }
 
   @InterfaceAudience.LimitedPrivate({"Apache Hive, HCatalog"})
   @Override
   public FireEventResponse fireListenerEvent(FireEventRequest rqst) throws TException {
+    if (!rqst.isSetCatName()) rqst.setCatName(getDefaultCatalog(conf));
     return client.fire_listener_event(rqst);
   }
 
@@ -2442,60 +2601,83 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
 
   @Override
   public void markPartitionForEvent(String db_name, String tbl_name, Map<String,String> partKVs, PartitionEventType eventType)
-      throws MetaException, TException, NoSuchObjectException, UnknownDBException,
-      UnknownTableException,
-      InvalidPartitionException, UnknownPartitionException {
-    assert db_name != null;
-    assert tbl_name != null;
-    assert partKVs != null;
-    client.markPartitionForEvent(db_name, tbl_name, partKVs, eventType);
+      throws TException {
+    markPartitionForEvent(getDefaultCatalog(conf), db_name, tbl_name, partKVs, eventType);
+  }
+
+  @Override
+  public void markPartitionForEvent(String catName, String db_name, String tbl_name,
+                                    Map<String, String> partKVs,
+                                    PartitionEventType eventType) throws TException {
+    client.markPartitionForEvent(prependCatalogToDbName(catName, db_name, conf), tbl_name, partKVs,
+        eventType);
+
   }
 
   @Override
   public boolean isPartitionMarkedForEvent(String db_name, String tbl_name, Map<String,String> partKVs, PartitionEventType eventType)
-      throws MetaException, NoSuchObjectException, UnknownTableException, UnknownDBException, TException,
-      InvalidPartitionException, UnknownPartitionException {
-    assert db_name != null;
-    assert tbl_name != null;
-    assert partKVs != null;
-    return client.isPartitionMarkedForEvent(db_name, tbl_name, partKVs, eventType);
+      throws TException {
+    return isPartitionMarkedForEvent(getDefaultCatalog(conf), db_name, tbl_name, partKVs, eventType);
   }
 
   @Override
-  public void createFunction(Function func) throws InvalidObjectException,
-      MetaException, TException {
+  public boolean isPartitionMarkedForEvent(String catName, String db_name, String tbl_name,
+                                           Map<String, String> partKVs,
+                                           PartitionEventType eventType) throws TException {
+    return client.isPartitionMarkedForEvent(prependCatalogToDbName(catName, db_name, conf), tbl_name,
+        partKVs, eventType);
+  }
+
+  @Override
+  public void createFunction(Function func) throws TException {
+    if (!func.isSetCatName()) func.setCatName(getDefaultCatalog(conf));
     client.create_function(func);
   }
 
   @Override
   public void alterFunction(String dbName, String funcName, Function newFunction)
-      throws InvalidObjectException, MetaException, TException {
-    client.alter_function(dbName, funcName, newFunction);
+      throws TException {
+    alterFunction(getDefaultCatalog(conf), dbName, funcName, newFunction);
   }
 
   @Override
-  public void dropFunction(String dbName, String funcName)
-      throws MetaException, NoSuchObjectException, InvalidObjectException,
-      InvalidInputException, TException {
-    client.drop_function(dbName, funcName);
+  public void alterFunction(String catName, String dbName, String funcName,
+                            Function newFunction) throws TException {
+    client.alter_function(prependCatalogToDbName(catName, dbName, conf), funcName, newFunction);
   }
 
   @Override
-  public Function getFunction(String dbName, String funcName)
-      throws MetaException, TException {
-    Function f = client.get_function(dbName, funcName);
-    return fastpath ? f : deepCopy(f);
+  public void dropFunction(String dbName, String funcName) throws TException {
+    dropFunction(getDefaultCatalog(conf), dbName, funcName);
   }
 
   @Override
-  public List<String> getFunctions(String dbName, String pattern)
-      throws MetaException, TException {
-    return client.get_functions(dbName, pattern);
+  public void dropFunction(String catName, String dbName, String funcName) throws TException {
+    client.drop_function(prependCatalogToDbName(catName, dbName, conf), funcName);
+  }
+
+  @Override
+  public Function getFunction(String dbName, String funcName) throws TException {
+    return getFunction(getDefaultCatalog(conf), dbName, funcName);
+  }
+
+  @Override
+  public Function getFunction(String catName, String dbName, String funcName) throws TException {
+    return deepCopy(client.get_function(prependCatalogToDbName(catName, dbName, conf), funcName));
+  }
+
+  @Override
+  public List<String> getFunctions(String dbName, String pattern) throws TException {
+    return getFunctions(getDefaultCatalog(conf), dbName, pattern);
+  }
+
+  @Override
+  public List<String> getFunctions(String catName, String dbName, String pattern) throws TException {
+    return client.get_functions(prependCatalogToDbName(catName, dbName, conf), pattern);
   }
 
   @Override
-  public GetAllFunctionsResponse getAllFunctions()
-          throws MetaException, TException {
+  public GetAllFunctionsResponse getAllFunctions() throws TException {
     return client.get_all_functions();
   }
 
@@ -2505,20 +2687,27 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
     client.create_table_with_environment_context(tbl, envContext);
   }
 
-  protected void drop_table_with_environment_context(String dbname, String name,
-      boolean deleteData, EnvironmentContext envContext) throws MetaException, TException,
-      NoSuchObjectExcep

<TRUNCATED>

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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableValidWriteIds.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableValidWriteIds.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableValidWriteIds.java
index 893454e..e0defbd 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableValidWriteIds.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableValidWriteIds.java
@@ -708,13 +708,13 @@ import org.slf4j.LoggerFactory;
           case 3: // INVALID_WRITE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list586 = iprot.readListBegin();
-                struct.invalidWriteIds = new ArrayList<Long>(_list586.size);
-                long _elem587;
-                for (int _i588 = 0; _i588 < _list586.size; ++_i588)
+                org.apache.thrift.protocol.TList _list594 = iprot.readListBegin();
+                struct.invalidWriteIds = new ArrayList<Long>(_list594.size);
+                long _elem595;
+                for (int _i596 = 0; _i596 < _list594.size; ++_i596)
                 {
-                  _elem587 = iprot.readI64();
-                  struct.invalidWriteIds.add(_elem587);
+                  _elem595 = iprot.readI64();
+                  struct.invalidWriteIds.add(_elem595);
                 }
                 iprot.readListEnd();
               }
@@ -764,9 +764,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(INVALID_WRITE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.invalidWriteIds.size()));
-          for (long _iter589 : struct.invalidWriteIds)
+          for (long _iter597 : struct.invalidWriteIds)
           {
-            oprot.writeI64(_iter589);
+            oprot.writeI64(_iter597);
           }
           oprot.writeListEnd();
         }
@@ -803,9 +803,9 @@ import org.slf4j.LoggerFactory;
       oprot.writeI64(struct.writeIdHighWaterMark);
       {
         oprot.writeI32(struct.invalidWriteIds.size());
-        for (long _iter590 : struct.invalidWriteIds)
+        for (long _iter598 : struct.invalidWriteIds)
         {
-          oprot.writeI64(_iter590);
+          oprot.writeI64(_iter598);
         }
       }
       oprot.writeBinary(struct.abortedBits);
@@ -827,13 +827,13 @@ import org.slf4j.LoggerFactory;
       struct.writeIdHighWaterMark = iprot.readI64();
       struct.setWriteIdHighWaterMarkIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list591 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.invalidWriteIds = new ArrayList<Long>(_list591.size);
-        long _elem592;
-        for (int _i593 = 0; _i593 < _list591.size; ++_i593)
+        org.apache.thrift.protocol.TList _list599 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.invalidWriteIds = new ArrayList<Long>(_list599.size);
+        long _elem600;
+        for (int _i601 = 0; _i601 < _list599.size; ++_i601)
         {
-          _elem592 = iprot.readI64();
-          struct.invalidWriteIds.add(_elem592);
+          _elem600 = iprot.readI64();
+          struct.invalidWriteIds.add(_elem600);
         }
       }
       struct.setInvalidWriteIdsIsSet(true);


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SerDeInfo.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SerDeInfo.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SerDeInfo.java
index a7aba9f..71957f7 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SerDeInfo.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SerDeInfo.java
@@ -859,15 +859,15 @@ import org.slf4j.LoggerFactory;
           case 3: // PARAMETERS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map104 = iprot.readMapBegin();
-                struct.parameters = new HashMap<String,String>(2*_map104.size);
-                String _key105;
-                String _val106;
-                for (int _i107 = 0; _i107 < _map104.size; ++_i107)
+                org.apache.thrift.protocol.TMap _map112 = iprot.readMapBegin();
+                struct.parameters = new HashMap<String,String>(2*_map112.size);
+                String _key113;
+                String _val114;
+                for (int _i115 = 0; _i115 < _map112.size; ++_i115)
                 {
-                  _key105 = iprot.readString();
-                  _val106 = iprot.readString();
-                  struct.parameters.put(_key105, _val106);
+                  _key113 = iprot.readString();
+                  _val114 = iprot.readString();
+                  struct.parameters.put(_key113, _val114);
                 }
                 iprot.readMapEnd();
               }
@@ -935,10 +935,10 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PARAMETERS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.parameters.size()));
-          for (Map.Entry<String, String> _iter108 : struct.parameters.entrySet())
+          for (Map.Entry<String, String> _iter116 : struct.parameters.entrySet())
           {
-            oprot.writeString(_iter108.getKey());
-            oprot.writeString(_iter108.getValue());
+            oprot.writeString(_iter116.getKey());
+            oprot.writeString(_iter116.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -1021,10 +1021,10 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetParameters()) {
         {
           oprot.writeI32(struct.parameters.size());
-          for (Map.Entry<String, String> _iter109 : struct.parameters.entrySet())
+          for (Map.Entry<String, String> _iter117 : struct.parameters.entrySet())
           {
-            oprot.writeString(_iter109.getKey());
-            oprot.writeString(_iter109.getValue());
+            oprot.writeString(_iter117.getKey());
+            oprot.writeString(_iter117.getValue());
           }
         }
       }
@@ -1056,15 +1056,15 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TMap _map110 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.parameters = new HashMap<String,String>(2*_map110.size);
-          String _key111;
-          String _val112;
-          for (int _i113 = 0; _i113 < _map110.size; ++_i113)
+          org.apache.thrift.protocol.TMap _map118 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.parameters = new HashMap<String,String>(2*_map118.size);
+          String _key119;
+          String _val120;
+          for (int _i121 = 0; _i121 < _map118.size; ++_i121)
           {
-            _key111 = iprot.readString();
-            _val112 = iprot.readString();
-            struct.parameters.put(_key111, _val112);
+            _key119 = iprot.readString();
+            _val120 = iprot.readString();
+            struct.parameters.put(_key119, _val120);
           }
         }
         struct.parameters = org.apache.hadoop.hive.metastore.utils.StringUtils.intern(struct.parameters); struct.setParametersIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SetPartitionsStatsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SetPartitionsStatsRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SetPartitionsStatsRequest.java
index dd95409..a0ae84e 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SetPartitionsStatsRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SetPartitionsStatsRequest.java
@@ -435,14 +435,14 @@ import org.slf4j.LoggerFactory;
           case 1: // COL_STATS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list276 = iprot.readListBegin();
-                struct.colStats = new ArrayList<ColumnStatistics>(_list276.size);
-                ColumnStatistics _elem277;
-                for (int _i278 = 0; _i278 < _list276.size; ++_i278)
+                org.apache.thrift.protocol.TList _list284 = iprot.readListBegin();
+                struct.colStats = new ArrayList<ColumnStatistics>(_list284.size);
+                ColumnStatistics _elem285;
+                for (int _i286 = 0; _i286 < _list284.size; ++_i286)
                 {
-                  _elem277 = new ColumnStatistics();
-                  _elem277.read(iprot);
-                  struct.colStats.add(_elem277);
+                  _elem285 = new ColumnStatistics();
+                  _elem285.read(iprot);
+                  struct.colStats.add(_elem285);
                 }
                 iprot.readListEnd();
               }
@@ -476,9 +476,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(COL_STATS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.colStats.size()));
-          for (ColumnStatistics _iter279 : struct.colStats)
+          for (ColumnStatistics _iter287 : struct.colStats)
           {
-            _iter279.write(oprot);
+            _iter287.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -508,9 +508,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.colStats.size());
-        for (ColumnStatistics _iter280 : struct.colStats)
+        for (ColumnStatistics _iter288 : struct.colStats)
         {
-          _iter280.write(oprot);
+          _iter288.write(oprot);
         }
       }
       BitSet optionals = new BitSet();
@@ -527,14 +527,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, SetPartitionsStatsRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list281 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.colStats = new ArrayList<ColumnStatistics>(_list281.size);
-        ColumnStatistics _elem282;
-        for (int _i283 = 0; _i283 < _list281.size; ++_i283)
+        org.apache.thrift.protocol.TList _list289 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.colStats = new ArrayList<ColumnStatistics>(_list289.size);
+        ColumnStatistics _elem290;
+        for (int _i291 = 0; _i291 < _list289.size; ++_i291)
         {
-          _elem282 = new ColumnStatistics();
-          _elem282.read(iprot);
-          struct.colStats.add(_elem282);
+          _elem290 = new ColumnStatistics();
+          _elem290.read(iprot);
+          struct.colStats.add(_elem290);
         }
       }
       struct.setColStatsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
index dd1366b..35d6d24 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // COMPACTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list660 = iprot.readListBegin();
-                struct.compacts = new ArrayList<ShowCompactResponseElement>(_list660.size);
-                ShowCompactResponseElement _elem661;
-                for (int _i662 = 0; _i662 < _list660.size; ++_i662)
+                org.apache.thrift.protocol.TList _list668 = iprot.readListBegin();
+                struct.compacts = new ArrayList<ShowCompactResponseElement>(_list668.size);
+                ShowCompactResponseElement _elem669;
+                for (int _i670 = 0; _i670 < _list668.size; ++_i670)
                 {
-                  _elem661 = new ShowCompactResponseElement();
-                  _elem661.read(iprot);
-                  struct.compacts.add(_elem661);
+                  _elem669 = new ShowCompactResponseElement();
+                  _elem669.read(iprot);
+                  struct.compacts.add(_elem669);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(COMPACTS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.compacts.size()));
-          for (ShowCompactResponseElement _iter663 : struct.compacts)
+          for (ShowCompactResponseElement _iter671 : struct.compacts)
           {
-            _iter663.write(oprot);
+            _iter671.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.compacts.size());
-        for (ShowCompactResponseElement _iter664 : struct.compacts)
+        for (ShowCompactResponseElement _iter672 : struct.compacts)
         {
-          _iter664.write(oprot);
+          _iter672.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, ShowCompactResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list665 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.compacts = new ArrayList<ShowCompactResponseElement>(_list665.size);
-        ShowCompactResponseElement _elem666;
-        for (int _i667 = 0; _i667 < _list665.size; ++_i667)
+        org.apache.thrift.protocol.TList _list673 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.compacts = new ArrayList<ShowCompactResponseElement>(_list673.size);
+        ShowCompactResponseElement _elem674;
+        for (int _i675 = 0; _i675 < _list673.size; ++_i675)
         {
-          _elem666 = new ShowCompactResponseElement();
-          _elem666.read(iprot);
-          struct.compacts.add(_elem666);
+          _elem674 = new ShowCompactResponseElement();
+          _elem674.read(iprot);
+          struct.compacts.add(_elem674);
         }
       }
       struct.setCompactsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
index 941f756..c8fd20e 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
@@ -350,14 +350,14 @@ import org.slf4j.LoggerFactory;
           case 1: // LOCKS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list626 = iprot.readListBegin();
-                struct.locks = new ArrayList<ShowLocksResponseElement>(_list626.size);
-                ShowLocksResponseElement _elem627;
-                for (int _i628 = 0; _i628 < _list626.size; ++_i628)
+                org.apache.thrift.protocol.TList _list634 = iprot.readListBegin();
+                struct.locks = new ArrayList<ShowLocksResponseElement>(_list634.size);
+                ShowLocksResponseElement _elem635;
+                for (int _i636 = 0; _i636 < _list634.size; ++_i636)
                 {
-                  _elem627 = new ShowLocksResponseElement();
-                  _elem627.read(iprot);
-                  struct.locks.add(_elem627);
+                  _elem635 = new ShowLocksResponseElement();
+                  _elem635.read(iprot);
+                  struct.locks.add(_elem635);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(LOCKS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.locks.size()));
-          for (ShowLocksResponseElement _iter629 : struct.locks)
+          for (ShowLocksResponseElement _iter637 : struct.locks)
           {
-            _iter629.write(oprot);
+            _iter637.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -416,9 +416,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetLocks()) {
         {
           oprot.writeI32(struct.locks.size());
-          for (ShowLocksResponseElement _iter630 : struct.locks)
+          for (ShowLocksResponseElement _iter638 : struct.locks)
           {
-            _iter630.write(oprot);
+            _iter638.write(oprot);
           }
         }
       }
@@ -430,14 +430,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list631 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.locks = new ArrayList<ShowLocksResponseElement>(_list631.size);
-          ShowLocksResponseElement _elem632;
-          for (int _i633 = 0; _i633 < _list631.size; ++_i633)
+          org.apache.thrift.protocol.TList _list639 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.locks = new ArrayList<ShowLocksResponseElement>(_list639.size);
+          ShowLocksResponseElement _elem640;
+          for (int _i641 = 0; _i641 < _list639.size; ++_i641)
           {
-            _elem632 = new ShowLocksResponseElement();
-            _elem632.read(iprot);
-            struct.locks.add(_elem632);
+            _elem640 = new ShowLocksResponseElement();
+            _elem640.read(iprot);
+            struct.locks.add(_elem640);
           }
         }
         struct.setLocksIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SkewedInfo.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SkewedInfo.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SkewedInfo.java
index a391f94..89fe49a 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SkewedInfo.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SkewedInfo.java
@@ -557,13 +557,13 @@ import org.slf4j.LoggerFactory;
           case 1: // SKEWED_COL_NAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list114 = iprot.readListBegin();
-                struct.skewedColNames = new ArrayList<String>(_list114.size);
-                String _elem115;
-                for (int _i116 = 0; _i116 < _list114.size; ++_i116)
+                org.apache.thrift.protocol.TList _list122 = iprot.readListBegin();
+                struct.skewedColNames = new ArrayList<String>(_list122.size);
+                String _elem123;
+                for (int _i124 = 0; _i124 < _list122.size; ++_i124)
                 {
-                  _elem115 = iprot.readString();
-                  struct.skewedColNames.add(_elem115);
+                  _elem123 = iprot.readString();
+                  struct.skewedColNames.add(_elem123);
                 }
                 iprot.readListEnd();
               }
@@ -575,23 +575,23 @@ import org.slf4j.LoggerFactory;
           case 2: // SKEWED_COL_VALUES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list117 = iprot.readListBegin();
-                struct.skewedColValues = new ArrayList<List<String>>(_list117.size);
-                List<String> _elem118;
-                for (int _i119 = 0; _i119 < _list117.size; ++_i119)
+                org.apache.thrift.protocol.TList _list125 = iprot.readListBegin();
+                struct.skewedColValues = new ArrayList<List<String>>(_list125.size);
+                List<String> _elem126;
+                for (int _i127 = 0; _i127 < _list125.size; ++_i127)
                 {
                   {
-                    org.apache.thrift.protocol.TList _list120 = iprot.readListBegin();
-                    _elem118 = new ArrayList<String>(_list120.size);
-                    String _elem121;
-                    for (int _i122 = 0; _i122 < _list120.size; ++_i122)
+                    org.apache.thrift.protocol.TList _list128 = iprot.readListBegin();
+                    _elem126 = new ArrayList<String>(_list128.size);
+                    String _elem129;
+                    for (int _i130 = 0; _i130 < _list128.size; ++_i130)
                     {
-                      _elem121 = iprot.readString();
-                      _elem118.add(_elem121);
+                      _elem129 = iprot.readString();
+                      _elem126.add(_elem129);
                     }
                     iprot.readListEnd();
                   }
-                  struct.skewedColValues.add(_elem118);
+                  struct.skewedColValues.add(_elem126);
                 }
                 iprot.readListEnd();
               }
@@ -603,25 +603,25 @@ import org.slf4j.LoggerFactory;
           case 3: // SKEWED_COL_VALUE_LOCATION_MAPS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map123 = iprot.readMapBegin();
-                struct.skewedColValueLocationMaps = new HashMap<List<String>,String>(2*_map123.size);
-                List<String> _key124;
-                String _val125;
-                for (int _i126 = 0; _i126 < _map123.size; ++_i126)
+                org.apache.thrift.protocol.TMap _map131 = iprot.readMapBegin();
+                struct.skewedColValueLocationMaps = new HashMap<List<String>,String>(2*_map131.size);
+                List<String> _key132;
+                String _val133;
+                for (int _i134 = 0; _i134 < _map131.size; ++_i134)
                 {
                   {
-                    org.apache.thrift.protocol.TList _list127 = iprot.readListBegin();
-                    _key124 = new ArrayList<String>(_list127.size);
-                    String _elem128;
-                    for (int _i129 = 0; _i129 < _list127.size; ++_i129)
+                    org.apache.thrift.protocol.TList _list135 = iprot.readListBegin();
+                    _key132 = new ArrayList<String>(_list135.size);
+                    String _elem136;
+                    for (int _i137 = 0; _i137 < _list135.size; ++_i137)
                     {
-                      _elem128 = iprot.readString();
-                      _key124.add(_elem128);
+                      _elem136 = iprot.readString();
+                      _key132.add(_elem136);
                     }
                     iprot.readListEnd();
                   }
-                  _val125 = iprot.readString();
-                  struct.skewedColValueLocationMaps.put(_key124, _val125);
+                  _val133 = iprot.readString();
+                  struct.skewedColValueLocationMaps.put(_key132, _val133);
                 }
                 iprot.readMapEnd();
               }
@@ -647,9 +647,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(SKEWED_COL_NAMES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.skewedColNames.size()));
-          for (String _iter130 : struct.skewedColNames)
+          for (String _iter138 : struct.skewedColNames)
           {
-            oprot.writeString(_iter130);
+            oprot.writeString(_iter138);
           }
           oprot.writeListEnd();
         }
@@ -659,13 +659,13 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(SKEWED_COL_VALUES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.LIST, struct.skewedColValues.size()));
-          for (List<String> _iter131 : struct.skewedColValues)
+          for (List<String> _iter139 : struct.skewedColValues)
           {
             {
-              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, _iter131.size()));
-              for (String _iter132 : _iter131)
+              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, _iter139.size()));
+              for (String _iter140 : _iter139)
               {
-                oprot.writeString(_iter132);
+                oprot.writeString(_iter140);
               }
               oprot.writeListEnd();
             }
@@ -678,17 +678,17 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(SKEWED_COL_VALUE_LOCATION_MAPS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.STRING, struct.skewedColValueLocationMaps.size()));
-          for (Map.Entry<List<String>, String> _iter133 : struct.skewedColValueLocationMaps.entrySet())
+          for (Map.Entry<List<String>, String> _iter141 : struct.skewedColValueLocationMaps.entrySet())
           {
             {
-              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, _iter133.getKey().size()));
-              for (String _iter134 : _iter133.getKey())
+              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, _iter141.getKey().size()));
+              for (String _iter142 : _iter141.getKey())
               {
-                oprot.writeString(_iter134);
+                oprot.writeString(_iter142);
               }
               oprot.writeListEnd();
             }
-            oprot.writeString(_iter133.getValue());
+            oprot.writeString(_iter141.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -725,22 +725,22 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetSkewedColNames()) {
         {
           oprot.writeI32(struct.skewedColNames.size());
-          for (String _iter135 : struct.skewedColNames)
+          for (String _iter143 : struct.skewedColNames)
           {
-            oprot.writeString(_iter135);
+            oprot.writeString(_iter143);
           }
         }
       }
       if (struct.isSetSkewedColValues()) {
         {
           oprot.writeI32(struct.skewedColValues.size());
-          for (List<String> _iter136 : struct.skewedColValues)
+          for (List<String> _iter144 : struct.skewedColValues)
           {
             {
-              oprot.writeI32(_iter136.size());
-              for (String _iter137 : _iter136)
+              oprot.writeI32(_iter144.size());
+              for (String _iter145 : _iter144)
               {
-                oprot.writeString(_iter137);
+                oprot.writeString(_iter145);
               }
             }
           }
@@ -749,16 +749,16 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetSkewedColValueLocationMaps()) {
         {
           oprot.writeI32(struct.skewedColValueLocationMaps.size());
-          for (Map.Entry<List<String>, String> _iter138 : struct.skewedColValueLocationMaps.entrySet())
+          for (Map.Entry<List<String>, String> _iter146 : struct.skewedColValueLocationMaps.entrySet())
           {
             {
-              oprot.writeI32(_iter138.getKey().size());
-              for (String _iter139 : _iter138.getKey())
+              oprot.writeI32(_iter146.getKey().size());
+              for (String _iter147 : _iter146.getKey())
               {
-                oprot.writeString(_iter139);
+                oprot.writeString(_iter147);
               }
             }
-            oprot.writeString(_iter138.getValue());
+            oprot.writeString(_iter146.getValue());
           }
         }
       }
@@ -770,59 +770,59 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(3);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list140 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.skewedColNames = new ArrayList<String>(_list140.size);
-          String _elem141;
-          for (int _i142 = 0; _i142 < _list140.size; ++_i142)
+          org.apache.thrift.protocol.TList _list148 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.skewedColNames = new ArrayList<String>(_list148.size);
+          String _elem149;
+          for (int _i150 = 0; _i150 < _list148.size; ++_i150)
           {
-            _elem141 = iprot.readString();
-            struct.skewedColNames.add(_elem141);
+            _elem149 = iprot.readString();
+            struct.skewedColNames.add(_elem149);
           }
         }
         struct.setSkewedColNamesIsSet(true);
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TList _list143 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.LIST, iprot.readI32());
-          struct.skewedColValues = new ArrayList<List<String>>(_list143.size);
-          List<String> _elem144;
-          for (int _i145 = 0; _i145 < _list143.size; ++_i145)
+          org.apache.thrift.protocol.TList _list151 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.LIST, iprot.readI32());
+          struct.skewedColValues = new ArrayList<List<String>>(_list151.size);
+          List<String> _elem152;
+          for (int _i153 = 0; _i153 < _list151.size; ++_i153)
           {
             {
-              org.apache.thrift.protocol.TList _list146 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-              _elem144 = new ArrayList<String>(_list146.size);
-              String _elem147;
-              for (int _i148 = 0; _i148 < _list146.size; ++_i148)
+              org.apache.thrift.protocol.TList _list154 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+              _elem152 = new ArrayList<String>(_list154.size);
+              String _elem155;
+              for (int _i156 = 0; _i156 < _list154.size; ++_i156)
               {
-                _elem147 = iprot.readString();
-                _elem144.add(_elem147);
+                _elem155 = iprot.readString();
+                _elem152.add(_elem155);
               }
             }
-            struct.skewedColValues.add(_elem144);
+            struct.skewedColValues.add(_elem152);
           }
         }
         struct.setSkewedColValuesIsSet(true);
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TMap _map149 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.skewedColValueLocationMaps = new HashMap<List<String>,String>(2*_map149.size);
-          List<String> _key150;
-          String _val151;
-          for (int _i152 = 0; _i152 < _map149.size; ++_i152)
+          org.apache.thrift.protocol.TMap _map157 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.skewedColValueLocationMaps = new HashMap<List<String>,String>(2*_map157.size);
+          List<String> _key158;
+          String _val159;
+          for (int _i160 = 0; _i160 < _map157.size; ++_i160)
           {
             {
-              org.apache.thrift.protocol.TList _list153 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-              _key150 = new ArrayList<String>(_list153.size);
-              String _elem154;
-              for (int _i155 = 0; _i155 < _list153.size; ++_i155)
+              org.apache.thrift.protocol.TList _list161 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+              _key158 = new ArrayList<String>(_list161.size);
+              String _elem162;
+              for (int _i163 = 0; _i163 < _list161.size; ++_i163)
               {
-                _elem154 = iprot.readString();
-                _key150.add(_elem154);
+                _elem162 = iprot.readString();
+                _key158.add(_elem162);
               }
             }
-            _val151 = iprot.readString();
-            struct.skewedColValueLocationMaps.put(_key150, _val151);
+            _val159 = iprot.readString();
+            struct.skewedColValueLocationMaps.put(_key158, _val159);
           }
         }
         struct.setSkewedColValueLocationMapsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/StorageDescriptor.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/StorageDescriptor.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/StorageDescriptor.java
index 385a015..00e6041 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/StorageDescriptor.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/StorageDescriptor.java
@@ -1290,14 +1290,14 @@ import org.slf4j.LoggerFactory;
           case 1: // COLS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list156 = iprot.readListBegin();
-                struct.cols = new ArrayList<FieldSchema>(_list156.size);
-                FieldSchema _elem157;
-                for (int _i158 = 0; _i158 < _list156.size; ++_i158)
+                org.apache.thrift.protocol.TList _list164 = iprot.readListBegin();
+                struct.cols = new ArrayList<FieldSchema>(_list164.size);
+                FieldSchema _elem165;
+                for (int _i166 = 0; _i166 < _list164.size; ++_i166)
                 {
-                  _elem157 = new FieldSchema();
-                  _elem157.read(iprot);
-                  struct.cols.add(_elem157);
+                  _elem165 = new FieldSchema();
+                  _elem165.read(iprot);
+                  struct.cols.add(_elem165);
                 }
                 iprot.readListEnd();
               }
@@ -1358,13 +1358,13 @@ import org.slf4j.LoggerFactory;
           case 8: // BUCKET_COLS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list159 = iprot.readListBegin();
-                struct.bucketCols = new ArrayList<String>(_list159.size);
-                String _elem160;
-                for (int _i161 = 0; _i161 < _list159.size; ++_i161)
+                org.apache.thrift.protocol.TList _list167 = iprot.readListBegin();
+                struct.bucketCols = new ArrayList<String>(_list167.size);
+                String _elem168;
+                for (int _i169 = 0; _i169 < _list167.size; ++_i169)
                 {
-                  _elem160 = iprot.readString();
-                  struct.bucketCols.add(_elem160);
+                  _elem168 = iprot.readString();
+                  struct.bucketCols.add(_elem168);
                 }
                 iprot.readListEnd();
               }
@@ -1376,14 +1376,14 @@ import org.slf4j.LoggerFactory;
           case 9: // SORT_COLS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list162 = iprot.readListBegin();
-                struct.sortCols = new ArrayList<Order>(_list162.size);
-                Order _elem163;
-                for (int _i164 = 0; _i164 < _list162.size; ++_i164)
+                org.apache.thrift.protocol.TList _list170 = iprot.readListBegin();
+                struct.sortCols = new ArrayList<Order>(_list170.size);
+                Order _elem171;
+                for (int _i172 = 0; _i172 < _list170.size; ++_i172)
                 {
-                  _elem163 = new Order();
-                  _elem163.read(iprot);
-                  struct.sortCols.add(_elem163);
+                  _elem171 = new Order();
+                  _elem171.read(iprot);
+                  struct.sortCols.add(_elem171);
                 }
                 iprot.readListEnd();
               }
@@ -1395,15 +1395,15 @@ import org.slf4j.LoggerFactory;
           case 10: // PARAMETERS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map165 = iprot.readMapBegin();
-                struct.parameters = new HashMap<String,String>(2*_map165.size);
-                String _key166;
-                String _val167;
-                for (int _i168 = 0; _i168 < _map165.size; ++_i168)
+                org.apache.thrift.protocol.TMap _map173 = iprot.readMapBegin();
+                struct.parameters = new HashMap<String,String>(2*_map173.size);
+                String _key174;
+                String _val175;
+                for (int _i176 = 0; _i176 < _map173.size; ++_i176)
                 {
-                  _key166 = iprot.readString();
-                  _val167 = iprot.readString();
-                  struct.parameters.put(_key166, _val167);
+                  _key174 = iprot.readString();
+                  _val175 = iprot.readString();
+                  struct.parameters.put(_key174, _val175);
                 }
                 iprot.readMapEnd();
               }
@@ -1446,9 +1446,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(COLS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.cols.size()));
-          for (FieldSchema _iter169 : struct.cols)
+          for (FieldSchema _iter177 : struct.cols)
           {
-            _iter169.write(oprot);
+            _iter177.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -1484,9 +1484,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(BUCKET_COLS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.bucketCols.size()));
-          for (String _iter170 : struct.bucketCols)
+          for (String _iter178 : struct.bucketCols)
           {
-            oprot.writeString(_iter170);
+            oprot.writeString(_iter178);
           }
           oprot.writeListEnd();
         }
@@ -1496,9 +1496,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(SORT_COLS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.sortCols.size()));
-          for (Order _iter171 : struct.sortCols)
+          for (Order _iter179 : struct.sortCols)
           {
-            _iter171.write(oprot);
+            _iter179.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -1508,10 +1508,10 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PARAMETERS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.parameters.size()));
-          for (Map.Entry<String, String> _iter172 : struct.parameters.entrySet())
+          for (Map.Entry<String, String> _iter180 : struct.parameters.entrySet())
           {
-            oprot.writeString(_iter172.getKey());
-            oprot.writeString(_iter172.getValue());
+            oprot.writeString(_iter180.getKey());
+            oprot.writeString(_iter180.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -1587,9 +1587,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetCols()) {
         {
           oprot.writeI32(struct.cols.size());
-          for (FieldSchema _iter173 : struct.cols)
+          for (FieldSchema _iter181 : struct.cols)
           {
-            _iter173.write(oprot);
+            _iter181.write(oprot);
           }
         }
       }
@@ -1614,28 +1614,28 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetBucketCols()) {
         {
           oprot.writeI32(struct.bucketCols.size());
-          for (String _iter174 : struct.bucketCols)
+          for (String _iter182 : struct.bucketCols)
           {
-            oprot.writeString(_iter174);
+            oprot.writeString(_iter182);
           }
         }
       }
       if (struct.isSetSortCols()) {
         {
           oprot.writeI32(struct.sortCols.size());
-          for (Order _iter175 : struct.sortCols)
+          for (Order _iter183 : struct.sortCols)
           {
-            _iter175.write(oprot);
+            _iter183.write(oprot);
           }
         }
       }
       if (struct.isSetParameters()) {
         {
           oprot.writeI32(struct.parameters.size());
-          for (Map.Entry<String, String> _iter176 : struct.parameters.entrySet())
+          for (Map.Entry<String, String> _iter184 : struct.parameters.entrySet())
           {
-            oprot.writeString(_iter176.getKey());
-            oprot.writeString(_iter176.getValue());
+            oprot.writeString(_iter184.getKey());
+            oprot.writeString(_iter184.getValue());
           }
         }
       }
@@ -1653,14 +1653,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(12);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list177 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.cols = new ArrayList<FieldSchema>(_list177.size);
-          FieldSchema _elem178;
-          for (int _i179 = 0; _i179 < _list177.size; ++_i179)
+          org.apache.thrift.protocol.TList _list185 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.cols = new ArrayList<FieldSchema>(_list185.size);
+          FieldSchema _elem186;
+          for (int _i187 = 0; _i187 < _list185.size; ++_i187)
           {
-            _elem178 = new FieldSchema();
-            _elem178.read(iprot);
-            struct.cols.add(_elem178);
+            _elem186 = new FieldSchema();
+            _elem186.read(iprot);
+            struct.cols.add(_elem186);
           }
         }
         struct.setColsIsSet(true);
@@ -1692,42 +1692,42 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(7)) {
         {
-          org.apache.thrift.protocol.TList _list180 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.bucketCols = new ArrayList<String>(_list180.size);
-          String _elem181;
-          for (int _i182 = 0; _i182 < _list180.size; ++_i182)
+          org.apache.thrift.protocol.TList _list188 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.bucketCols = new ArrayList<String>(_list188.size);
+          String _elem189;
+          for (int _i190 = 0; _i190 < _list188.size; ++_i190)
           {
-            _elem181 = iprot.readString();
-            struct.bucketCols.add(_elem181);
+            _elem189 = iprot.readString();
+            struct.bucketCols.add(_elem189);
           }
         }
         struct.setBucketColsIsSet(true);
       }
       if (incoming.get(8)) {
         {
-          org.apache.thrift.protocol.TList _list183 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.sortCols = new ArrayList<Order>(_list183.size);
-          Order _elem184;
-          for (int _i185 = 0; _i185 < _list183.size; ++_i185)
+          org.apache.thrift.protocol.TList _list191 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.sortCols = new ArrayList<Order>(_list191.size);
+          Order _elem192;
+          for (int _i193 = 0; _i193 < _list191.size; ++_i193)
           {
-            _elem184 = new Order();
-            _elem184.read(iprot);
-            struct.sortCols.add(_elem184);
+            _elem192 = new Order();
+            _elem192.read(iprot);
+            struct.sortCols.add(_elem192);
           }
         }
         struct.setSortColsIsSet(true);
       }
       if (incoming.get(9)) {
         {
-          org.apache.thrift.protocol.TMap _map186 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.parameters = new HashMap<String,String>(2*_map186.size);
-          String _key187;
-          String _val188;
-          for (int _i189 = 0; _i189 < _map186.size; ++_i189)
+          org.apache.thrift.protocol.TMap _map194 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.parameters = new HashMap<String,String>(2*_map194.size);
+          String _key195;
+          String _val196;
+          for (int _i197 = 0; _i197 < _map194.size; ++_i197)
           {
-            _key187 = iprot.readString();
-            _val188 = iprot.readString();
-            struct.parameters.put(_key187, _val188);
+            _key195 = iprot.readString();
+            _val196 = iprot.readString();
+            struct.parameters.put(_key195, _val196);
           }
         }
         struct.parameters = org.apache.hadoop.hive.metastore.utils.StringUtils.intern(struct.parameters); struct.setParametersIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java
index a132e5e..8dfec98 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Table.java
@@ -54,6 +54,7 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField TEMPORARY_FIELD_DESC = new org.apache.thrift.protocol.TField("temporary", org.apache.thrift.protocol.TType.BOOL, (short)14);
   private static final org.apache.thrift.protocol.TField REWRITE_ENABLED_FIELD_DESC = new org.apache.thrift.protocol.TField("rewriteEnabled", org.apache.thrift.protocol.TType.BOOL, (short)15);
   private static final org.apache.thrift.protocol.TField CREATION_METADATA_FIELD_DESC = new org.apache.thrift.protocol.TField("creationMetadata", org.apache.thrift.protocol.TType.STRUCT, (short)16);
+  private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)17);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -77,6 +78,7 @@ import org.slf4j.LoggerFactory;
   private boolean temporary; // optional
   private boolean rewriteEnabled; // optional
   private CreationMetadata creationMetadata; // optional
+  private String catName; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -95,7 +97,8 @@ import org.slf4j.LoggerFactory;
     PRIVILEGES((short)13, "privileges"),
     TEMPORARY((short)14, "temporary"),
     REWRITE_ENABLED((short)15, "rewriteEnabled"),
-    CREATION_METADATA((short)16, "creationMetadata");
+    CREATION_METADATA((short)16, "creationMetadata"),
+    CAT_NAME((short)17, "catName");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -142,6 +145,8 @@ import org.slf4j.LoggerFactory;
           return REWRITE_ENABLED;
         case 16: // CREATION_METADATA
           return CREATION_METADATA;
+        case 17: // CAT_NAME
+          return CAT_NAME;
         default:
           return null;
       }
@@ -188,7 +193,7 @@ import org.slf4j.LoggerFactory;
   private static final int __TEMPORARY_ISSET_ID = 3;
   private static final int __REWRITEENABLED_ISSET_ID = 4;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.PRIVILEGES,_Fields.TEMPORARY,_Fields.REWRITE_ENABLED,_Fields.CREATION_METADATA};
+  private static final _Fields optionals[] = {_Fields.PRIVILEGES,_Fields.TEMPORARY,_Fields.REWRITE_ENABLED,_Fields.CREATION_METADATA,_Fields.CAT_NAME};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -227,6 +232,8 @@ import org.slf4j.LoggerFactory;
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
     tmpMap.put(_Fields.CREATION_METADATA, new org.apache.thrift.meta_data.FieldMetaData("creationMetadata", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT        , "CreationMetadata")));
+    tmpMap.put(_Fields.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(Table.class, metaDataMap);
   }
@@ -316,6 +323,9 @@ import org.slf4j.LoggerFactory;
     if (other.isSetCreationMetadata()) {
       this.creationMetadata = other.creationMetadata;
     }
+    if (other.isSetCatName()) {
+      this.catName = other.catName;
+    }
   }
 
   public Table deepCopy() {
@@ -345,6 +355,7 @@ import org.slf4j.LoggerFactory;
     setRewriteEnabledIsSet(false);
     this.rewriteEnabled = false;
     this.creationMetadata = null;
+    this.catName = null;
   }
 
   public String getTableName() {
@@ -736,6 +747,29 @@ import org.slf4j.LoggerFactory;
     }
   }
 
+  public String getCatName() {
+    return this.catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
+  public void unsetCatName() {
+    this.catName = null;
+  }
+
+  /** Returns true if field catName is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatName() {
+    return this.catName != null;
+  }
+
+  public void setCatNameIsSet(boolean value) {
+    if (!value) {
+      this.catName = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case TABLE_NAME:
@@ -866,6 +900,14 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case CAT_NAME:
+      if (value == null) {
+        unsetCatName();
+      } else {
+        setCatName((String)value);
+      }
+      break;
+
     }
   }
 
@@ -919,6 +961,9 @@ import org.slf4j.LoggerFactory;
     case CREATION_METADATA:
       return getCreationMetadata();
 
+    case CAT_NAME:
+      return getCatName();
+
     }
     throw new IllegalStateException();
   }
@@ -962,6 +1007,8 @@ import org.slf4j.LoggerFactory;
       return isSetRewriteEnabled();
     case CREATION_METADATA:
       return isSetCreationMetadata();
+    case CAT_NAME:
+      return isSetCatName();
     }
     throw new IllegalStateException();
   }
@@ -1123,6 +1170,15 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_catName = true && this.isSetCatName();
+    boolean that_present_catName = true && that.isSetCatName();
+    if (this_present_catName || that_present_catName) {
+      if (!(this_present_catName && that_present_catName))
+        return false;
+      if (!this.catName.equals(that.catName))
+        return false;
+    }
+
     return true;
   }
 
@@ -1210,6 +1266,11 @@ import org.slf4j.LoggerFactory;
     if (present_creationMetadata)
       list.add(creationMetadata);
 
+    boolean present_catName = true && (isSetCatName());
+    list.add(present_catName);
+    if (present_catName)
+      list.add(catName);
+
     return list.hashCode();
   }
 
@@ -1381,6 +1442,16 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetCatName()).compareTo(other.isSetCatName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catName, other.catName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -1516,6 +1587,16 @@ import org.slf4j.LoggerFactory;
       }
       first = false;
     }
+    if (isSetCatName()) {
+      if (!first) sb.append(", ");
+      sb.append("catName:");
+      if (this.catName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.catName);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -1627,14 +1708,14 @@ import org.slf4j.LoggerFactory;
           case 8: // PARTITION_KEYS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list190 = iprot.readListBegin();
-                struct.partitionKeys = new ArrayList<FieldSchema>(_list190.size);
-                FieldSchema _elem191;
-                for (int _i192 = 0; _i192 < _list190.size; ++_i192)
+                org.apache.thrift.protocol.TList _list198 = iprot.readListBegin();
+                struct.partitionKeys = new ArrayList<FieldSchema>(_list198.size);
+                FieldSchema _elem199;
+                for (int _i200 = 0; _i200 < _list198.size; ++_i200)
                 {
-                  _elem191 = new FieldSchema();
-                  _elem191.read(iprot);
-                  struct.partitionKeys.add(_elem191);
+                  _elem199 = new FieldSchema();
+                  _elem199.read(iprot);
+                  struct.partitionKeys.add(_elem199);
                 }
                 iprot.readListEnd();
               }
@@ -1646,15 +1727,15 @@ import org.slf4j.LoggerFactory;
           case 9: // PARAMETERS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map193 = iprot.readMapBegin();
-                struct.parameters = new HashMap<String,String>(2*_map193.size);
-                String _key194;
-                String _val195;
-                for (int _i196 = 0; _i196 < _map193.size; ++_i196)
+                org.apache.thrift.protocol.TMap _map201 = iprot.readMapBegin();
+                struct.parameters = new HashMap<String,String>(2*_map201.size);
+                String _key202;
+                String _val203;
+                for (int _i204 = 0; _i204 < _map201.size; ++_i204)
                 {
-                  _key194 = iprot.readString();
-                  _val195 = iprot.readString();
-                  struct.parameters.put(_key194, _val195);
+                  _key202 = iprot.readString();
+                  _val203 = iprot.readString();
+                  struct.parameters.put(_key202, _val203);
                 }
                 iprot.readMapEnd();
               }
@@ -1721,6 +1802,14 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 17: // CAT_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.catName = iprot.readString();
+              struct.setCatNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -1767,9 +1856,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PARTITION_KEYS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitionKeys.size()));
-          for (FieldSchema _iter197 : struct.partitionKeys)
+          for (FieldSchema _iter205 : struct.partitionKeys)
           {
-            _iter197.write(oprot);
+            _iter205.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -1779,10 +1868,10 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PARAMETERS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.parameters.size()));
-          for (Map.Entry<String, String> _iter198 : struct.parameters.entrySet())
+          for (Map.Entry<String, String> _iter206 : struct.parameters.entrySet())
           {
-            oprot.writeString(_iter198.getKey());
-            oprot.writeString(_iter198.getValue());
+            oprot.writeString(_iter206.getKey());
+            oprot.writeString(_iter206.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -1827,6 +1916,13 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldEnd();
         }
       }
+      if (struct.catName != null) {
+        if (struct.isSetCatName()) {
+          oprot.writeFieldBegin(CAT_NAME_FIELD_DESC);
+          oprot.writeString(struct.catName);
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -1893,7 +1989,10 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetCreationMetadata()) {
         optionals.set(15);
       }
-      oprot.writeBitSet(optionals, 16);
+      if (struct.isSetCatName()) {
+        optionals.set(16);
+      }
+      oprot.writeBitSet(optionals, 17);
       if (struct.isSetTableName()) {
         oprot.writeString(struct.tableName);
       }
@@ -1918,19 +2017,19 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetPartitionKeys()) {
         {
           oprot.writeI32(struct.partitionKeys.size());
-          for (FieldSchema _iter199 : struct.partitionKeys)
+          for (FieldSchema _iter207 : struct.partitionKeys)
           {
-            _iter199.write(oprot);
+            _iter207.write(oprot);
           }
         }
       }
       if (struct.isSetParameters()) {
         {
           oprot.writeI32(struct.parameters.size());
-          for (Map.Entry<String, String> _iter200 : struct.parameters.entrySet())
+          for (Map.Entry<String, String> _iter208 : struct.parameters.entrySet())
           {
-            oprot.writeString(_iter200.getKey());
-            oprot.writeString(_iter200.getValue());
+            oprot.writeString(_iter208.getKey());
+            oprot.writeString(_iter208.getValue());
           }
         }
       }
@@ -1955,12 +2054,15 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetCreationMetadata()) {
         struct.creationMetadata.write(oprot);
       }
+      if (struct.isSetCatName()) {
+        oprot.writeString(struct.catName);
+      }
     }
 
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, Table struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
-      BitSet incoming = iprot.readBitSet(16);
+      BitSet incoming = iprot.readBitSet(17);
       if (incoming.get(0)) {
         struct.tableName = iprot.readString();
         struct.setTableNameIsSet(true);
@@ -1992,29 +2094,29 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(7)) {
         {
-          org.apache.thrift.protocol.TList _list201 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.partitionKeys = new ArrayList<FieldSchema>(_list201.size);
-          FieldSchema _elem202;
-          for (int _i203 = 0; _i203 < _list201.size; ++_i203)
+          org.apache.thrift.protocol.TList _list209 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.partitionKeys = new ArrayList<FieldSchema>(_list209.size);
+          FieldSchema _elem210;
+          for (int _i211 = 0; _i211 < _list209.size; ++_i211)
           {
-            _elem202 = new FieldSchema();
-            _elem202.read(iprot);
-            struct.partitionKeys.add(_elem202);
+            _elem210 = new FieldSchema();
+            _elem210.read(iprot);
+            struct.partitionKeys.add(_elem210);
           }
         }
         struct.setPartitionKeysIsSet(true);
       }
       if (incoming.get(8)) {
         {
-          org.apache.thrift.protocol.TMap _map204 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.parameters = new HashMap<String,String>(2*_map204.size);
-          String _key205;
-          String _val206;
-          for (int _i207 = 0; _i207 < _map204.size; ++_i207)
+          org.apache.thrift.protocol.TMap _map212 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.parameters = new HashMap<String,String>(2*_map212.size);
+          String _key213;
+          String _val214;
+          for (int _i215 = 0; _i215 < _map212.size; ++_i215)
           {
-            _key205 = iprot.readString();
-            _val206 = iprot.readString();
-            struct.parameters.put(_key205, _val206);
+            _key213 = iprot.readString();
+            _val214 = iprot.readString();
+            struct.parameters.put(_key213, _val214);
           }
         }
         struct.setParametersIsSet(true);
@@ -2049,6 +2151,10 @@ import org.slf4j.LoggerFactory;
         struct.creationMetadata.read(iprot);
         struct.setCreationMetadataIsSet(true);
       }
+      if (incoming.get(16)) {
+        struct.catName = iprot.readString();
+        struct.setCatNameIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableMeta.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableMeta.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableMeta.java
index cadbaaa..9e20f6f 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableMeta.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableMeta.java
@@ -42,6 +42,7 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)2);
   private static final org.apache.thrift.protocol.TField TABLE_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("tableType", org.apache.thrift.protocol.TType.STRING, (short)3);
   private static final org.apache.thrift.protocol.TField COMMENTS_FIELD_DESC = new org.apache.thrift.protocol.TField("comments", org.apache.thrift.protocol.TType.STRING, (short)4);
+  private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)5);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -53,13 +54,15 @@ import org.slf4j.LoggerFactory;
   private String tableName; // required
   private String tableType; // required
   private String comments; // optional
+  private String catName; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
     DB_NAME((short)1, "dbName"),
     TABLE_NAME((short)2, "tableName"),
     TABLE_TYPE((short)3, "tableType"),
-    COMMENTS((short)4, "comments");
+    COMMENTS((short)4, "comments"),
+    CAT_NAME((short)5, "catName");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -82,6 +85,8 @@ import org.slf4j.LoggerFactory;
           return TABLE_TYPE;
         case 4: // COMMENTS
           return COMMENTS;
+        case 5: // CAT_NAME
+          return CAT_NAME;
         default:
           return null;
       }
@@ -122,7 +127,7 @@ import org.slf4j.LoggerFactory;
   }
 
   // isset id assignments
-  private static final _Fields optionals[] = {_Fields.COMMENTS};
+  private static final _Fields optionals[] = {_Fields.COMMENTS,_Fields.CAT_NAME};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -134,6 +139,8 @@ import org.slf4j.LoggerFactory;
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.COMMENTS, new org.apache.thrift.meta_data.FieldMetaData("comments", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TableMeta.class, metaDataMap);
   }
@@ -168,6 +175,9 @@ import org.slf4j.LoggerFactory;
     if (other.isSetComments()) {
       this.comments = other.comments;
     }
+    if (other.isSetCatName()) {
+      this.catName = other.catName;
+    }
   }
 
   public TableMeta deepCopy() {
@@ -180,6 +190,7 @@ import org.slf4j.LoggerFactory;
     this.tableName = null;
     this.tableType = null;
     this.comments = null;
+    this.catName = null;
   }
 
   public String getDbName() {
@@ -274,6 +285,29 @@ import org.slf4j.LoggerFactory;
     }
   }
 
+  public String getCatName() {
+    return this.catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
+  public void unsetCatName() {
+    this.catName = null;
+  }
+
+  /** Returns true if field catName is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatName() {
+    return this.catName != null;
+  }
+
+  public void setCatNameIsSet(boolean value) {
+    if (!value) {
+      this.catName = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case DB_NAME:
@@ -308,6 +342,14 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case CAT_NAME:
+      if (value == null) {
+        unsetCatName();
+      } else {
+        setCatName((String)value);
+      }
+      break;
+
     }
   }
 
@@ -325,6 +367,9 @@ import org.slf4j.LoggerFactory;
     case COMMENTS:
       return getComments();
 
+    case CAT_NAME:
+      return getCatName();
+
     }
     throw new IllegalStateException();
   }
@@ -344,6 +389,8 @@ import org.slf4j.LoggerFactory;
       return isSetTableType();
     case COMMENTS:
       return isSetComments();
+    case CAT_NAME:
+      return isSetCatName();
     }
     throw new IllegalStateException();
   }
@@ -397,6 +444,15 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_catName = true && this.isSetCatName();
+    boolean that_present_catName = true && that.isSetCatName();
+    if (this_present_catName || that_present_catName) {
+      if (!(this_present_catName && that_present_catName))
+        return false;
+      if (!this.catName.equals(that.catName))
+        return false;
+    }
+
     return true;
   }
 
@@ -424,6 +480,11 @@ import org.slf4j.LoggerFactory;
     if (present_comments)
       list.add(comments);
 
+    boolean present_catName = true && (isSetCatName());
+    list.add(present_catName);
+    if (present_catName)
+      list.add(catName);
+
     return list.hashCode();
   }
 
@@ -475,6 +536,16 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetCatName()).compareTo(other.isSetCatName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catName, other.catName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -528,6 +599,16 @@ import org.slf4j.LoggerFactory;
       }
       first = false;
     }
+    if (isSetCatName()) {
+      if (!first) sb.append(", ");
+      sb.append("catName:");
+      if (this.catName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.catName);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -615,6 +696,14 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 5: // CAT_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.catName = iprot.readString();
+              struct.setCatNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -650,6 +739,13 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldEnd();
         }
       }
+      if (struct.catName != null) {
+        if (struct.isSetCatName()) {
+          oprot.writeFieldBegin(CAT_NAME_FIELD_DESC);
+          oprot.writeString(struct.catName);
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -674,10 +770,16 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetComments()) {
         optionals.set(0);
       }
-      oprot.writeBitSet(optionals, 1);
+      if (struct.isSetCatName()) {
+        optionals.set(1);
+      }
+      oprot.writeBitSet(optionals, 2);
       if (struct.isSetComments()) {
         oprot.writeString(struct.comments);
       }
+      if (struct.isSetCatName()) {
+        oprot.writeString(struct.catName);
+      }
     }
 
     @Override
@@ -689,11 +791,15 @@ import org.slf4j.LoggerFactory;
       struct.setTableNameIsSet(true);
       struct.tableType = iprot.readString();
       struct.setTableTypeIsSet(true);
-      BitSet incoming = iprot.readBitSet(1);
+      BitSet incoming = iprot.readBitSet(2);
       if (incoming.get(0)) {
         struct.comments = iprot.readString();
         struct.setCommentsIsSet(true);
       }
+      if (incoming.get(1)) {
+        struct.catName = iprot.readString();
+        struct.setCatNameIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsRequest.java
index 6c7c9af..a663a64 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsRequest.java
@@ -41,6 +41,7 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbName", org.apache.thrift.protocol.TType.STRING, (short)1);
   private static final org.apache.thrift.protocol.TField TBL_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tblName", org.apache.thrift.protocol.TType.STRING, (short)2);
   private static final org.apache.thrift.protocol.TField COL_NAMES_FIELD_DESC = new org.apache.thrift.protocol.TField("colNames", org.apache.thrift.protocol.TType.LIST, (short)3);
+  private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)4);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -51,12 +52,14 @@ import org.slf4j.LoggerFactory;
   private String dbName; // required
   private String tblName; // required
   private List<String> colNames; // required
+  private String catName; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
     DB_NAME((short)1, "dbName"),
     TBL_NAME((short)2, "tblName"),
-    COL_NAMES((short)3, "colNames");
+    COL_NAMES((short)3, "colNames"),
+    CAT_NAME((short)4, "catName");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -77,6 +80,8 @@ import org.slf4j.LoggerFactory;
           return TBL_NAME;
         case 3: // COL_NAMES
           return COL_NAMES;
+        case 4: // CAT_NAME
+          return CAT_NAME;
         default:
           return null;
       }
@@ -117,6 +122,7 @@ import org.slf4j.LoggerFactory;
   }
 
   // isset id assignments
+  private static final _Fields optionals[] = {_Fields.CAT_NAME};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -127,6 +133,8 @@ import org.slf4j.LoggerFactory;
     tmpMap.put(_Fields.COL_NAMES, new org.apache.thrift.meta_data.FieldMetaData("colNames", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
             new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    tmpMap.put(_Fields.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TableStatsRequest.class, metaDataMap);
   }
@@ -159,6 +167,9 @@ import org.slf4j.LoggerFactory;
       List<String> __this__colNames = new ArrayList<String>(other.colNames);
       this.colNames = __this__colNames;
     }
+    if (other.isSetCatName()) {
+      this.catName = other.catName;
+    }
   }
 
   public TableStatsRequest deepCopy() {
@@ -170,6 +181,7 @@ import org.slf4j.LoggerFactory;
     this.dbName = null;
     this.tblName = null;
     this.colNames = null;
+    this.catName = null;
   }
 
   public String getDbName() {
@@ -256,6 +268,29 @@ import org.slf4j.LoggerFactory;
     }
   }
 
+  public String getCatName() {
+    return this.catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
+  public void unsetCatName() {
+    this.catName = null;
+  }
+
+  /** Returns true if field catName is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatName() {
+    return this.catName != null;
+  }
+
+  public void setCatNameIsSet(boolean value) {
+    if (!value) {
+      this.catName = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case DB_NAME:
@@ -282,6 +317,14 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case CAT_NAME:
+      if (value == null) {
+        unsetCatName();
+      } else {
+        setCatName((String)value);
+      }
+      break;
+
     }
   }
 
@@ -296,6 +339,9 @@ import org.slf4j.LoggerFactory;
     case COL_NAMES:
       return getColNames();
 
+    case CAT_NAME:
+      return getCatName();
+
     }
     throw new IllegalStateException();
   }
@@ -313,6 +359,8 @@ import org.slf4j.LoggerFactory;
       return isSetTblName();
     case COL_NAMES:
       return isSetColNames();
+    case CAT_NAME:
+      return isSetCatName();
     }
     throw new IllegalStateException();
   }
@@ -357,6 +405,15 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_catName = true && this.isSetCatName();
+    boolean that_present_catName = true && that.isSetCatName();
+    if (this_present_catName || that_present_catName) {
+      if (!(this_present_catName && that_present_catName))
+        return false;
+      if (!this.catName.equals(that.catName))
+        return false;
+    }
+
     return true;
   }
 
@@ -379,6 +436,11 @@ import org.slf4j.LoggerFactory;
     if (present_colNames)
       list.add(colNames);
 
+    boolean present_catName = true && (isSetCatName());
+    list.add(present_catName);
+    if (present_catName)
+      list.add(catName);
+
     return list.hashCode();
   }
 
@@ -420,6 +482,16 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetCatName()).compareTo(other.isSetCatName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catName, other.catName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -463,6 +535,16 @@ import org.slf4j.LoggerFactory;
       sb.append(this.colNames);
     }
     first = false;
+    if (isSetCatName()) {
+      if (!first) sb.append(", ");
+      sb.append("catName:");
+      if (this.catName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.catName);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -537,13 +619,13 @@ import org.slf4j.LoggerFactory;
           case 3: // COL_NAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list442 = iprot.readListBegin();
-                struct.colNames = new ArrayList<String>(_list442.size);
-                String _elem443;
-                for (int _i444 = 0; _i444 < _list442.size; ++_i444)
+                org.apache.thrift.protocol.TList _list450 = iprot.readListBegin();
+                struct.colNames = new ArrayList<String>(_list450.size);
+                String _elem451;
+                for (int _i452 = 0; _i452 < _list450.size; ++_i452)
                 {
-                  _elem443 = iprot.readString();
-                  struct.colNames.add(_elem443);
+                  _elem451 = iprot.readString();
+                  struct.colNames.add(_elem451);
                 }
                 iprot.readListEnd();
               }
@@ -552,6 +634,14 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 4: // CAT_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.catName = iprot.readString();
+              struct.setCatNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -579,14 +669,21 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(COL_NAMES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.colNames.size()));
-          for (String _iter445 : struct.colNames)
+          for (String _iter453 : struct.colNames)
           {
-            oprot.writeString(_iter445);
+            oprot.writeString(_iter453);
           }
           oprot.writeListEnd();
         }
         oprot.writeFieldEnd();
       }
+      if (struct.catName != null) {
+        if (struct.isSetCatName()) {
+          oprot.writeFieldBegin(CAT_NAME_FIELD_DESC);
+          oprot.writeString(struct.catName);
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -608,11 +705,19 @@ import org.slf4j.LoggerFactory;
       oprot.writeString(struct.tblName);
       {
         oprot.writeI32(struct.colNames.size());
-        for (String _iter446 : struct.colNames)
+        for (String _iter454 : struct.colNames)
         {
-          oprot.writeString(_iter446);
+          oprot.writeString(_iter454);
         }
       }
+      BitSet optionals = new BitSet();
+      if (struct.isSetCatName()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.isSetCatName()) {
+        oprot.writeString(struct.catName);
+      }
     }
 
     @Override
@@ -623,16 +728,21 @@ import org.slf4j.LoggerFactory;
       struct.tblName = iprot.readString();
       struct.setTblNameIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list447 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.colNames = new ArrayList<String>(_list447.size);
-        String _elem448;
-        for (int _i449 = 0; _i449 < _list447.size; ++_i449)
+        org.apache.thrift.protocol.TList _list455 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.colNames = new ArrayList<String>(_list455.size);
+        String _elem456;
+        for (int _i457 = 0; _i457 < _list455.size; ++_i457)
         {
-          _elem448 = iprot.readString();
-          struct.colNames.add(_elem448);
+          _elem456 = iprot.readString();
+          struct.colNames.add(_elem456);
         }
       }
       struct.setColNamesIsSet(true);
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.catName = iprot.readString();
+        struct.setCatNameIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsResult.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsResult.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsResult.java
index 789f91c..dff7d5c 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsResult.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableStatsResult.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // TABLE_STATS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list416 = iprot.readListBegin();
-                struct.tableStats = new ArrayList<ColumnStatisticsObj>(_list416.size);
-                ColumnStatisticsObj _elem417;
-                for (int _i418 = 0; _i418 < _list416.size; ++_i418)
+                org.apache.thrift.protocol.TList _list424 = iprot.readListBegin();
+                struct.tableStats = new ArrayList<ColumnStatisticsObj>(_list424.size);
+                ColumnStatisticsObj _elem425;
+                for (int _i426 = 0; _i426 < _list424.size; ++_i426)
                 {
-                  _elem417 = new ColumnStatisticsObj();
-                  _elem417.read(iprot);
-                  struct.tableStats.add(_elem417);
+                  _elem425 = new ColumnStatisticsObj();
+                  _elem425.read(iprot);
+                  struct.tableStats.add(_elem425);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TABLE_STATS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.tableStats.size()));
-          for (ColumnStatisticsObj _iter419 : struct.tableStats)
+          for (ColumnStatisticsObj _iter427 : struct.tableStats)
           {
-            _iter419.write(oprot);
+            _iter427.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.tableStats.size());
-        for (ColumnStatisticsObj _iter420 : struct.tableStats)
+        for (ColumnStatisticsObj _iter428 : struct.tableStats)
         {
-          _iter420.write(oprot);
+          _iter428.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, TableStatsResult struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list421 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.tableStats = new ArrayList<ColumnStatisticsObj>(_list421.size);
-        ColumnStatisticsObj _elem422;
-        for (int _i423 = 0; _i423 < _list421.size; ++_i423)
+        org.apache.thrift.protocol.TList _list429 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.tableStats = new ArrayList<ColumnStatisticsObj>(_list429.size);
+        ColumnStatisticsObj _elem430;
+        for (int _i431 = 0; _i431 < _list429.size; ++_i431)
         {
-          _elem422 = new ColumnStatisticsObj();
-          _elem422.read(iprot);
-          struct.tableStats.add(_elem422);
+          _elem430 = new ColumnStatisticsObj();
+          _elem430.read(iprot);
+          struct.tableStats.add(_elem430);
         }
       }
       struct.setTableStatsIsSet(true);


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/InjectableBehaviourObjectStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/InjectableBehaviourObjectStore.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/InjectableBehaviourObjectStore.java
index d7a40b6..fdb0dc4 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/InjectableBehaviourObjectStore.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/InjectableBehaviourObjectStore.java
@@ -88,13 +88,13 @@ public class InjectableBehaviourObjectStore extends ObjectStore {
 
   // ObjectStore methods to be overridden with injected behavior
   @Override
-  public Table getTable(String dbName, String tableName) throws MetaException {
-    return getTableModifier.apply(super.getTable(dbName, tableName));
+  public Table getTable(String catName, String dbName, String tableName) throws MetaException {
+    return getTableModifier.apply(super.getTable(catName, dbName, tableName));
   }
 
   @Override
-  public List<String> listPartitionNames(String dbName, String tableName, short max) throws MetaException {
-    return listPartitionNamesModifier.apply(super.listPartitionNames(dbName, tableName, max));
+  public List<String> listPartitionNames(String catName, String dbName, String tableName, short max) throws MetaException {
+    return listPartitionNamesModifier.apply(super.listPartitionNames(catName, dbName, tableName, max));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/MetaStoreTestUtils.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/MetaStoreTestUtils.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/MetaStoreTestUtils.java
index 60fcb86..1d12cf9 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/MetaStoreTestUtils.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/MetaStoreTestUtils.java
@@ -17,11 +17,13 @@
  */
 package org.apache.hadoop.hive.metastore;
 
+import java.io.File;
 import java.io.IOException;
 import java.net.ConnectException;
 import java.net.InetSocketAddress;
 import java.net.ServerSocket;
 import java.net.Socket;
+import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
@@ -30,9 +32,12 @@ import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
 import org.apache.hadoop.hive.metastore.events.EventCleanerTask;
 import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
+import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
+
 public class MetaStoreTestUtils {
   private static final Logger LOG = LoggerFactory.getLogger(MetaStoreTestUtils.class);
   public static final int RETRY_COUNT = 10;
@@ -220,4 +225,28 @@ public class MetaStoreTestUtils {
           DefaultPartitionExpressionProxy.class, PartitionExpressionProxy.class);
     }
   }
+
+
+  public static String getTestWarehouseDir(String name) {
+    File dir = new File(System.getProperty("java.io.tmpdir"), name);
+    dir.deleteOnExit();
+    return dir.getAbsolutePath();
+  }
+
+  /**
+   * There is no cascade option for dropping a catalog for security reasons.  But this in
+   * inconvenient in tests, so this method does it.
+   * @param client metastore client
+   * @param catName catalog to drop, cannot be the default catalog
+   * @throws TException from underlying client calls
+   */
+  public static void dropCatalogCascade(IMetaStoreClient client, String catName) throws TException {
+    if (catName != null && !catName.equals(DEFAULT_CATALOG_NAME)) {
+      List<String> databases = client.getAllDatabases(catName);
+      for (String db : databases) {
+        client.dropDatabase(catName, db, true, false, true);
+      }
+      client.dropCatalog(catName);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestAggregateStatsCache.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestAggregateStatsCache.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestAggregateStatsCache.java
index b95f1f2..75ab4e0 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestAggregateStatsCache.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestAggregateStatsCache.java
@@ -41,6 +41,8 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
+
 @Category(MetastoreUnitTest.class)
 public class TestAggregateStatsCache {
   static String DB_NAME = "db";
@@ -117,11 +119,11 @@ public class TestAggregateStatsCache {
 
   @Test
   public void testCacheKey() {
-    Key k1 = new Key("db", "tbl1", "col");
-    Key k2 = new Key("db", "tbl1", "col");
+    Key k1 = new Key("cat", "db", "tbl1", "col");
+    Key k2 = new Key("cat", "db", "tbl1", "col");
     // k1 equals k2
     Assert.assertEquals(k1, k2);
-    Key k3 = new Key("db", "tbl2", "col");
+    Key k3 = new Key("cat", "db", "tbl2", "col");
     // k1 not equals k3
     Assert.assertNotEquals(k1, k3);
   }
@@ -140,16 +142,16 @@ public class TestAggregateStatsCache {
     ColumnStatisticsObj aggrColStats =
         getDummyLongColStat(colName, highVal, lowVal, numDVs, numNulls);
     // Now add to cache the dummy colstats for these 10 partitions
-    cache.add(DB_NAME, tblName, colName, 10, aggrColStats, bloomFilter);
+    cache.add(DEFAULT_CATALOG_NAME, DB_NAME, tblName, colName, 10, aggrColStats, bloomFilter);
     // Now get from cache
-    AggrColStats aggrStatsCached = cache.get(DB_NAME, tblName, colName, partNames);
+    AggrColStats aggrStatsCached = cache.get(DEFAULT_CATALOG_NAME, DB_NAME, tblName, colName, partNames);
     Assert.assertNotNull(aggrStatsCached);
 
     ColumnStatisticsObj aggrColStatsCached = aggrStatsCached.getColStats();
     Assert.assertEquals(aggrColStats, aggrColStatsCached);
 
     // Now get a non-existant entry
-    aggrStatsCached = cache.get("dbNotThere", tblName, colName, partNames);
+    aggrStatsCached = cache.get(DEFAULT_CATALOG_NAME, "dbNotThere", tblName, colName, partNames);
     Assert.assertNull(aggrStatsCached);
   }
 
@@ -167,25 +169,25 @@ public class TestAggregateStatsCache {
     ColumnStatisticsObj aggrColStats =
         getDummyLongColStat(colName, highVal, lowVal, numDVs, numNulls);
     // Now add to cache
-    cache.add(DB_NAME, tblName, colName, 10, aggrColStats, bloomFilter);
+    cache.add(DEFAULT_CATALOG_NAME, DB_NAME, tblName, colName, 10, aggrColStats, bloomFilter);
 
     // Now prepare partnames with only 5 partitions: [tab1part1...tab1part5]
     partNames = preparePartNames(tables.get(0), 1, 5);
     // This get should fail because its variance ((10-5)/5) is way past MAX_VARIANCE (0.5)
-    AggrColStats aggrStatsCached = cache.get(DB_NAME, tblName, colName, partNames);
+    AggrColStats aggrStatsCached = cache.get(DEFAULT_CATALOG_NAME, DB_NAME, tblName, colName, partNames);
     Assert.assertNull(aggrStatsCached);
 
     // Now prepare partnames with 10 partitions: [tab1part11...tab1part20], but with no overlap
     partNames = preparePartNames(tables.get(0), 11, 20);
     // This get should fail because its variance ((10-0)/10) is way past MAX_VARIANCE (0.5)
-    aggrStatsCached = cache.get(DB_NAME, tblName, colName, partNames);
+    aggrStatsCached = cache.get(DEFAULT_CATALOG_NAME, DB_NAME, tblName, colName, partNames);
     Assert.assertNull(aggrStatsCached);
 
     // Now prepare partnames with 9 partitions: [tab1part1...tab1part8], which are contained in the
     // object that we added to the cache
     partNames = preparePartNames(tables.get(0), 1, 8);
     // This get should succeed because its variance ((10-9)/9) is within past MAX_VARIANCE (0.5)
-    aggrStatsCached = cache.get(DB_NAME, tblName, colName, partNames);
+    aggrStatsCached = cache.get(DEFAULT_CATALOG_NAME, DB_NAME, tblName, colName, partNames);
     Assert.assertNotNull(aggrStatsCached);
     ColumnStatisticsObj aggrColStatsCached = aggrStatsCached.getColStats();
     Assert.assertEquals(aggrColStats, aggrColStatsCached);
@@ -206,13 +208,13 @@ public class TestAggregateStatsCache {
     ColumnStatisticsObj aggrColStats =
         getDummyLongColStat(colName, highVal, lowVal, numDVs, numNulls);
     // Now add to cache
-    cache.add(DB_NAME, tblName, colName, 10, aggrColStats, bloomFilter);
+    cache.add(DEFAULT_CATALOG_NAME, DB_NAME, tblName, colName, 10, aggrColStats, bloomFilter);
 
     // Sleep for 3 seconds
     Thread.sleep(3000);
 
     // Get should fail now (since TTL is 2s) and we've snoozed for 3 seconds
-    AggrColStats aggrStatsCached = cache.get(DB_NAME, tblName, colName, partNames);
+    AggrColStats aggrStatsCached = cache.get(DEFAULT_CATALOG_NAME, DB_NAME, tblName, colName, partNames);
     Assert.assertNull(aggrStatsCached);
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/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
new file mode 100644
index 0000000..dfe05e9
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestCatalogNonDefaultClient.java
@@ -0,0 +1,74 @@
+/*
+ * 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 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;
+
+/**
+ * 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 {
+
+  final private String catName = "non_default_catalog";
+  private String catLocation;
+
+  @After
+  public void dropCatalog() throws TException {
+    MetaStoreTestUtils.dropCatalogCascade(client, catName);
+  }
+
+  @Override
+  protected IMetaStoreClient getClient() throws Exception {
+
+    Configuration svrConf = new Configuration(conf);
+    int port = MetaStoreTestUtils.startMetaStoreWithRetry(HadoopThriftAuthBridge.getBridge(),
+        svrConf);
+    // Only set the default catalog on the client.
+    MetastoreConf.setVar(conf, MetastoreConf.ConfVars.THRIFT_URIS, "thrift://localhost:" + port);
+    MetastoreConf.setVar(conf, MetastoreConf.ConfVars.CATALOG_DEFAULT, catName);
+    IMetaStoreClient client = new HiveMetaStoreClient(conf);
+    assert !client.isLocalMetaStore();
+    // Don't make any calls but catalog calls until the catalog has been created, as we just told
+    // the client to direct all calls to a catalog that does not yet exist.
+    catLocation = MetaStoreTestUtils.getTestWarehouseDir(catName);
+    Catalog cat = new CatalogBuilder()
+        .setName(catName)
+        .setLocation(catLocation)
+        .build();
+    client.createCatalog(cat);
+    return client;
+  }
+
+  @Override
+  protected String expectedCatalog() {
+    return catName;
+  }
+
+  @Override
+  protected String expectedBaseDir() throws MetaException {
+    return catLocation;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/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
new file mode 100644
index 0000000..13c8723
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestCatalogNonDefaultSvr.java
@@ -0,0 +1,70 @@
+/*
+ * 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 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;
+
+/**
+ * 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 {
+
+  final private String catName = "non_default_svr_catalog";
+  private String catLocation;
+  private IMetaStoreClient catalogCapableClient;
+
+  @After
+  public void dropCatalog() throws TException {
+    MetaStoreTestUtils.dropCatalogCascade(catalogCapableClient, catName);
+    catalogCapableClient.close();
+  }
+
+  @Override
+  protected IMetaStoreClient getClient() throws Exception {
+    // Separate client to create the catalog
+    catalogCapableClient = new HiveMetaStoreClient(conf);
+    catLocation = MetaStoreTestUtils.getTestWarehouseDir(catName);
+    Catalog cat = new CatalogBuilder()
+        .setName(catName)
+        .setLocation(catLocation)
+        .build();
+    catalogCapableClient.createCatalog(cat);
+    catalogCapableClient.close();
+
+    MetastoreConf.setVar(conf, MetastoreConf.ConfVars.CATALOG_DEFAULT, catName);
+    return new HiveMetaStoreClientPreCatalog(conf);
+  }
+
+  @Override
+  protected String expectedCatalog() {
+    return catName;
+  }
+
+  @Override
+  protected String expectedBaseDir() throws MetaException {
+    return catLocation;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/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
new file mode 100644
index 0000000..bb57b85
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestCatalogOldClient.java
@@ -0,0 +1,44 @@
+/*
+ * 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 org.apache.hadoop.hive.metastore.api.MetaException;
+
+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 {
+
+  @Override
+  protected IMetaStoreClient getClient() throws MetaException {
+    return new HiveMetaStoreClientPreCatalog(conf);
+  }
+
+  @Override
+  protected String expectedCatalog() {
+    return DEFAULT_CATALOG_NAME;
+  }
+
+  @Override
+  protected String expectedBaseDir() throws MetaException {
+    return new Warehouse(conf).getWhRoot().toUri().getPath();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestFilterHooks.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestFilterHooks.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestFilterHooks.java
index ea5dd3c..7dc69bc 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestFilterHooks.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestFilterHooks.java
@@ -75,11 +75,12 @@ public class TestFilterHooks {
     }
 
     @Override
-    public List<String> filterTableNames(String dbName, List<String> tableList) throws MetaException {
+    public List<String> filterTableNames(String catName, String dbName, List<String> tableList)
+        throws MetaException {
       if (blockResults) {
         return new ArrayList<>();
       }
-      return super.filterTableNames(dbName, tableList);
+      return super.filterTableNames(catName, dbName, tableList);
     }
 
     @Override
@@ -124,12 +125,12 @@ public class TestFilterHooks {
     }
 
     @Override
-    public List<String> filterPartitionNames(String dbName, String tblName,
+    public List<String> filterPartitionNames(String catName, String dbName, String tblName,
         List<String> partitionNames) throws MetaException {
       if (blockResults) {
         return new ArrayList<>();
       }
-      return super.filterPartitionNames(dbName, tblName, partitionNames);
+      return super.filterPartitionNames(catName, dbName, tblName, partitionNames);
     }
 
   }
@@ -159,36 +160,32 @@ public class TestFilterHooks {
     msc.dropDatabase(DBNAME2, true, true, true);
     Database db1 = new DatabaseBuilder()
         .setName(DBNAME1)
-        .build();
-    msc.createDatabase(db1);
+        .setCatalogName(Warehouse.DEFAULT_CATALOG_NAME)
+        .create(msc, conf);
     Database db2 = new DatabaseBuilder()
         .setName(DBNAME2)
-        .build();
-    msc.createDatabase(db2);
-    Table tab1 = new TableBuilder()
+        .setCatalogName(Warehouse.DEFAULT_CATALOG_NAME)
+        .create(msc, conf);
+    new TableBuilder()
         .setDbName(DBNAME1)
         .setTableName(TAB1)
         .addCol("id", "int")
         .addCol("name", "string")
-        .build();
-    msc.createTable(tab1);
+        .create(msc, conf);
     Table tab2 = new TableBuilder()
         .setDbName(DBNAME1)
         .setTableName(TAB2)
         .addCol("id", "int")
         .addPartCol("name", "string")
-        .build();
-    msc.createTable(tab2);
-    Partition part1 = new PartitionBuilder()
-        .fromTable(tab2)
+        .create(msc, conf);
+    new PartitionBuilder()
+        .inTable(tab2)
         .addValue("value1")
-        .build();
-    msc.add_partition(part1);
-    Partition part2 = new PartitionBuilder()
-        .fromTable(tab2)
+        .addToTable(msc, conf);
+    new PartitionBuilder()
+        .inTable(tab2)
         .addValue("value2")
-        .build();
-    msc.add_partition(part2);
+        .addToTable(msc, conf);
   }
 
   @AfterClass

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveAlterHandler.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveAlterHandler.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveAlterHandler.java
index ba8c1a0..adc82b0 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveAlterHandler.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveAlterHandler.java
@@ -18,17 +18,24 @@
 
 package org.apache.hadoop.hive.metastore;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.annotation.MetastoreUnitTest;
 import org.apache.hadoop.hive.metastore.api.*;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.mockito.Mockito;
 
 import java.util.Arrays;
 
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+
 @Category(MetastoreUnitTest.class)
 public class TestHiveAlterHandler {
 
+  private Configuration conf = MetastoreConf.newMetastoreConf();
+
   @Test
   public void testAlterTableAddColNotUpdateStats() throws MetaException, InvalidObjectException, NoSuchObjectException {
     FieldSchema col1 = new FieldSchema("col1", "string", "col1 comment");
@@ -50,8 +57,9 @@ public class TestHiveAlterHandler {
 
     RawStore msdb = Mockito.mock(RawStore.class);
     Mockito.doThrow(new RuntimeException("shouldn't be called")).when(msdb).getTableColumnStatistics(
-        oldTable.getDbName(), oldTable.getTableName(), Arrays.asList("col1", "col2", "col3"));
+        getDefaultCatalog(conf), oldTable.getDbName(), oldTable.getTableName(), Arrays.asList("col1", "col2", "col3"));
     HiveAlterHandler handler = new HiveAlterHandler();
+    handler.setConf(conf);
     handler.alterTableUpdateTableColumnStats(msdb, oldTable, newTable);
   }
 
@@ -76,9 +84,10 @@ public class TestHiveAlterHandler {
 
     RawStore msdb = Mockito.mock(RawStore.class);
     HiveAlterHandler handler = new HiveAlterHandler();
+    handler.setConf(conf);
     handler.alterTableUpdateTableColumnStats(msdb, oldTable, newTable);
     Mockito.verify(msdb, Mockito.times(1)).getTableColumnStatistics(
-        oldTable.getDbName(), oldTable.getTableName(), Arrays.asList("col1", "col2", "col3", "col4")
+        getDefaultCatalog(conf), oldTable.getDbName(), oldTable.getTableName(), Arrays.asList("col1", "col2", "col3", "col4")
     );
   }
 
@@ -103,8 +112,9 @@ public class TestHiveAlterHandler {
 
     RawStore msdb = Mockito.mock(RawStore.class);
     Mockito.doThrow(new RuntimeException("shouldn't be called")).when(msdb).getTableColumnStatistics(
-        oldTable.getDbName(), oldTable.getTableName(), Arrays.asList("col1", "col2", "col3", "col4"));
+        getDefaultCatalog(conf), oldTable.getDbName(), oldTable.getTableName(), Arrays.asList("col1", "col2", "col3", "col4"));
     HiveAlterHandler handler = new HiveAlterHandler();
+    handler.setConf(conf);
     handler.alterTableUpdateTableColumnStats(msdb, oldTable, newTable);
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
index 7091c5b..9a56c1c 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
@@ -178,10 +178,10 @@ public abstract class TestHiveMetaStore {
 
       client.dropTable(dbName, tblName);
       silentDropDatabase(dbName);
-      Database db = new Database();
-      db.setName(dbName);
-      client.createDatabase(db);
-      db = client.getDatabase(dbName);
+      new DatabaseBuilder()
+          .setName(dbName)
+          .create(client, conf);
+      Database db = client.getDatabase(dbName);
       Path dbPath = new Path(db.getLocationUri());
       FileSystem fs = FileSystem.get(dbPath.toUri(), conf);
 
@@ -209,9 +209,7 @@ public abstract class TestHiveMetaStore {
           .setSkewedColValueLocationMaps(Collections.singletonMap(skewedColValue, "location1"))
           .addPartCol("ds", ColumnType.STRING_TYPE_NAME)
           .addPartCol("hr", ColumnType.STRING_TYPE_NAME)
-          .build();
-
-      client.createTable(tbl);
+          .create(client, conf);
 
       if (isThriftClient) {
         // the createTable() above does not update the location in the 'tbl'
@@ -709,19 +707,17 @@ public abstract class TestHiveMetaStore {
 
     client.dropTable(dbName, tblName);
     silentDropDatabase(dbName);
-    Database db = new Database();
-    db.setName(dbName);
-    db.setDescription("Alter Partition Test database");
-    client.createDatabase(db);
+    new DatabaseBuilder()
+        .setName(dbName)
+        .setDescription("Alter Partition Test database")
+        .create(client, conf);
 
     Table tbl = new TableBuilder()
         .setDbName(dbName)
         .setTableName(tblName)
         .addCol("name", ColumnType.STRING_TYPE_NAME)
         .addCol("income", ColumnType.INT_TYPE_NAME)
-        .build();
-
-    client.createTable(tbl);
+        .create(client, conf);
 
     if (isThriftClient) {
       // the createTable() above does not update the location in the 'tbl'
@@ -804,10 +800,10 @@ public abstract class TestHiveMetaStore {
 
       client.dropTable(dbName, tblName);
       silentDropDatabase(dbName);
-      Database db = new Database();
-      db.setName(dbName);
-      db.setDescription("Alter Partition Test database");
-      client.createDatabase(db);
+      new DatabaseBuilder()
+          .setName(dbName)
+          .setDescription("Alter Partition Test database")
+          .create(client, conf);
 
       Table tbl = new TableBuilder()
           .setDbName(dbName)
@@ -819,9 +815,7 @@ public abstract class TestHiveMetaStore {
           .addSerdeParam(ColumnType.SERIALIZATION_FORMAT, "1")
           .addPartCol("ds", ColumnType.STRING_TYPE_NAME)
           .addPartCol("hr", ColumnType.INT_TYPE_NAME)
-          .build();
-
-      client.createTable(tbl);
+          .create(client, conf);
 
       if (isThriftClient) {
         // the createTable() above does not update the location in the 'tbl'
@@ -884,10 +878,10 @@ public abstract class TestHiveMetaStore {
 
       client.dropTable(dbName, tblName);
       silentDropDatabase(dbName);
-      Database db = new Database();
-      db.setName(dbName);
-      db.setDescription("Rename Partition Test database");
-      client.createDatabase(db);
+      new DatabaseBuilder()
+          .setName(dbName)
+          .setDescription("Rename Partition Test database")
+          .create(client, conf);
 
       Table tbl = new TableBuilder()
           .setDbName(dbName)
@@ -896,9 +890,7 @@ public abstract class TestHiveMetaStore {
           .addCol("income", ColumnType.INT_TYPE_NAME)
           .addPartCol("ds", ColumnType.STRING_TYPE_NAME)
           .addPartCol("hr", ColumnType.INT_TYPE_NAME)
-          .build();
-
-      client.createTable(tbl);
+          .create(client, conf);
 
       if (isThriftClient) {
         // the createTable() above does not update the location in the 'tbl'
@@ -988,7 +980,7 @@ public abstract class TestHiveMetaStore {
       Database db = new DatabaseBuilder()
           .setName(TEST_DB1_NAME)
           .setOwnerName(SecurityUtils.getUser())
-          .build();
+          .build(conf);
       Assert.assertEquals(SecurityUtils.getUser(), db.getOwnerName());
       client.createDatabase(db);
 
@@ -1000,9 +992,10 @@ public abstract class TestHiveMetaStore {
           warehouse.getDatabasePath(db).toString(), db.getLocationUri());
       assertEquals(db.getOwnerName(), SecurityUtils.getUser());
       assertEquals(db.getOwnerType(), PrincipalType.USER);
-      Database db2 = new Database();
-      db2.setName(TEST_DB2_NAME);
-      client.createDatabase(db2);
+      assertEquals(Warehouse.DEFAULT_CATALOG_NAME, db.getCatalogName());
+      Database db2 = new DatabaseBuilder()
+          .setName(TEST_DB2_NAME)
+          .create(client, conf);
 
       db2 = client.getDatabase(TEST_DB2_NAME);
 
@@ -1041,15 +1034,16 @@ public abstract class TestHiveMetaStore {
 
     silentDropDatabase(TEST_DB1_NAME);
 
-    Database db = new Database();
-    db.setName(TEST_DB1_NAME);
     String dbLocation =
       MetastoreConf.getVar(conf, ConfVars.WAREHOUSE) + "/test/_testDB_create_";
     FileSystem fs = FileSystem.get(new Path(dbLocation).toUri(), conf);
     fs.mkdirs(
               new Path(MetastoreConf.getVar(conf, ConfVars.WAREHOUSE) + "/test"),
               new FsPermission((short) 0));
-    db.setLocationUri(dbLocation);
+    Database db = new DatabaseBuilder()
+        .setName(TEST_DB1_NAME)
+        .setLocation(dbLocation)
+        .build(conf);
 
 
     boolean createFailed = false;
@@ -1081,14 +1075,14 @@ public abstract class TestHiveMetaStore {
       // clear up any existing databases
       silentDropDatabase(TEST_DB1_NAME);
 
-      Database db = new Database();
-      db.setName(TEST_DB1_NAME);
       String dbLocation =
           MetastoreConf.getVar(conf, ConfVars.WAREHOUSE) + "/_testDB_create_";
-      db.setLocationUri(dbLocation);
-      client.createDatabase(db);
+      new DatabaseBuilder()
+          .setName(TEST_DB1_NAME)
+          .setLocation(dbLocation)
+          .create(client, conf);
 
-      db = client.getDatabase(TEST_DB1_NAME);
+      Database db = client.getDatabase(TEST_DB1_NAME);
 
       assertEquals("name of returned db is different from that of inserted db",
           TEST_DB1_NAME, db.getName());
@@ -1106,14 +1100,15 @@ public abstract class TestHiveMetaStore {
       }
       assertTrue("Database " + TEST_DB1_NAME + " exists ", objectNotExist);
 
-      db = new Database();
-      db.setName(TEST_DB1_NAME);
       dbLocation =
           MetastoreConf.getVar(conf, ConfVars.WAREHOUSE) + "/_testDB_file_";
       FileSystem fs = FileSystem.get(new Path(dbLocation).toUri(), conf);
       fs.createNewFile(new Path(dbLocation));
       fs.deleteOnExit(new Path(dbLocation));
-      db.setLocationUri(dbLocation);
+      db = new DatabaseBuilder()
+          .setName(TEST_DB1_NAME)
+          .setLocation(dbLocation)
+          .build(conf);
 
       boolean createFailed = false;
       try {
@@ -1247,9 +1242,9 @@ public abstract class TestHiveMetaStore {
       client.dropTable(dbName, tblName);
       silentDropDatabase(dbName);
 
-      Database db = new Database();
-      db.setName(dbName);
-      client.createDatabase(db);
+      new DatabaseBuilder()
+          .setName(dbName)
+          .create(client, conf);
 
       client.dropType(typeName);
       Type typ1 = new Type();
@@ -1268,9 +1263,7 @@ public abstract class TestHiveMetaStore {
           .setNumBuckets(1)
           .addBucketCol("name")
           .addStorageDescriptorParam("test_param_1", "Use this for comments etc")
-          .build();
-
-      client.createTable(tbl);
+          .create(client, conf);
 
       if (isThriftClient) {
         // the createTable() above does not update the location in the 'tbl'
@@ -1397,7 +1390,8 @@ public abstract class TestHiveMetaStore {
         udbe = e;
       }
       assertNotNull(udbe);
-      assertTrue("DB not found", udbe.getMessage().contains("not find database db_that_doesnt_exist"));
+      assertTrue("DB not found",
+          udbe.getMessage().contains("not find database hive.db_that_doesnt_exist"));
 
       udbe = null;
       try {
@@ -1498,9 +1492,9 @@ public abstract class TestHiveMetaStore {
 
     try {
       cleanUp(dbName, tblName, typeName);
-      Database db = new Database();
-      db.setName(dbName);
-      client.createDatabase(db);
+      new DatabaseBuilder()
+          .setName(dbName)
+          .create(client, conf);
       createTableForTestFilter(dbName,tblName, tblOwner, lastAccessed, true);
 
       // Create a ColumnStatistics Obj
@@ -1658,17 +1652,16 @@ public abstract class TestHiveMetaStore {
     client.dropTable(dbName, tblName);
     silentDropDatabase(dbName);
 
-    Database db = new Database();
-    db.setName(dbName);
-    client.createDatabase(db);
+    new DatabaseBuilder()
+        .setName(dbName)
+        .create(client, conf);
 
     Table tbl = new TableBuilder()
         .setDbName(dbName)
         .setTableName(tblName)
         .addCol("name", ColumnType.STRING_TYPE_NAME, "")
         .setSerdeLib("no.such.class")
-        .build();
-    client.createTable(tbl);
+        .create(client, conf);
 
     client.getSchema(dbName, tblName);
   }
@@ -1683,9 +1676,9 @@ public abstract class TestHiveMetaStore {
       client.dropTable(dbName, tblName);
       silentDropDatabase(dbName);
 
-      Database db = new Database();
-      db.setName(dbName);
-      client.createDatabase(db);
+      new DatabaseBuilder()
+          .setName(dbName)
+          .create(client, conf);
 
       ArrayList<FieldSchema> invCols = new ArrayList<>(2);
       invCols.add(new FieldSchema("n-ame", ColumnType.STRING_TYPE_NAME, ""));
@@ -1695,7 +1688,7 @@ public abstract class TestHiveMetaStore {
           .setDbName(dbName)
           .setTableName(invTblName)
           .setCols(invCols)
-          .build();
+          .build(conf);
 
       boolean failed = false;
       try {
@@ -1834,9 +1827,9 @@ public abstract class TestHiveMetaStore {
     try {
       client.dropTable(dbName, tblName);
       silentDropDatabase(dbName);
-      Database db = new Database();
-      db.setName(dbName);
-      client.createDatabase(db);
+      new DatabaseBuilder()
+          .setName(dbName)
+          .create(client, conf);
 
       client.dropType(typeName);
       Type typ1 = new Type();
@@ -1857,9 +1850,7 @@ public abstract class TestHiveMetaStore {
           .setNumBuckets(1)
           .addBucketCol("name")
           .addStorageDescriptorParam("test_param_1","Use this for comments etc")
-          .build();
-
-      client.createTable(tbl);
+          .create(client, conf);
 
       Table tbl2 = client.getTable(dbName, tblName);
       assertEquals(tbl2.getDbName(), dbName);
@@ -1920,22 +1911,21 @@ public abstract class TestHiveMetaStore {
     try {
       silentDropDatabase(dbName);
 
-      Database db = new Database();
-      db.setName(dbName);
       String dbLocation =
           MetastoreConf.getVar(conf, ConfVars.WAREHOUSE) + "_testDB_table_create_";
-      db.setLocationUri(dbLocation);
-      client.createDatabase(db);
-      db = client.getDatabase(dbName);
+      new DatabaseBuilder()
+          .setName(dbName)
+          .setLocation(dbLocation)
+          .create(client, conf);
+      Database db = client.getDatabase(dbName);
 
       Table tbl = new TableBuilder()
           .setDbName(dbName)
           .setTableName(tblName_1)
           .addCol("name", ColumnType.STRING_TYPE_NAME)
           .addCol("income", ColumnType.INT_TYPE_NAME)
-          .build();
+          .create(client, conf);
 
-      client.createTable(tbl);
       tbl = client.getTable(dbName, tblName_1);
 
       Path path = new Path(tbl.getSd().getLocation());
@@ -2014,9 +2004,9 @@ public abstract class TestHiveMetaStore {
 
     silentDropDatabase(dbName);
 
-    Database db = new Database();
-    db.setName(dbName);
-    client.createDatabase(db);
+    new DatabaseBuilder()
+        .setName(dbName)
+        .create(client, conf);
 
     Table tbl = new TableBuilder()
         .setDbName(dbName)
@@ -2026,8 +2016,7 @@ public abstract class TestHiveMetaStore {
         .addPartCol("p1", ColumnType.STRING_TYPE_NAME)
         .addPartCol("p2", ColumnType.STRING_TYPE_NAME)
         .addPartCol("p3", ColumnType.INT_TYPE_NAME)
-        .build();
-    client.createTable(tbl);
+        .create(client, conf);
 
     tbl = client.getTable(dbName, tblName);
 
@@ -2188,9 +2177,9 @@ public abstract class TestHiveMetaStore {
 
       silentDropDatabase(dbName);
 
-      Database db = new Database();
-      db.setName(dbName);
-      client.createDatabase(db);
+      new DatabaseBuilder()
+          .setName(dbName)
+          .create(client, conf);
 
       Table tbl = new TableBuilder()
           .setDbName(dbName)
@@ -2198,8 +2187,7 @@ public abstract class TestHiveMetaStore {
           .addCol("c1", ColumnType.STRING_TYPE_NAME)
           .addCol("c2", ColumnType.INT_TYPE_NAME)
           .addPartCol("p1", ColumnType.STRING_TYPE_NAME)
-          .build();
-      client.createTable(tbl);
+          .create(client, conf);
 
       tbl = client.getTable(dbName, tblName);
 
@@ -2249,9 +2237,8 @@ public abstract class TestHiveMetaStore {
           .addCol("c2", ColumnType.INT_TYPE_NAME)
           .addPartCol("p1", ColumnType.STRING_TYPE_NAME)
           .addPartCol("p2", ColumnType.STRING_TYPE_NAME)
-          .build();
+          .create(client, conf);
 
-      client.createTable(tbl);
       tbl = client.getTable(dbName, tblName);
 
       add_partition(client, tbl, vals, "part1");
@@ -2334,10 +2321,10 @@ public abstract class TestHiveMetaStore {
       client.dropTable(dbName, tableName2);
       client.dropTable(dbName, tableName3);
       silentDropDatabase(dbName);
-      Database db = new Database();
-      db.setName(dbName);
-      db.setDescription("Alter Partition Test database");
-      client.createDatabase(db);
+      new DatabaseBuilder()
+          .setName(dbName)
+          .setDescription("Alter Partition Test database")
+          .create(client, conf);
 
       Table table1 = createTableForTestFilter(dbName,tableName1, owner1, lastAccessTime1, true);
       Table table2 = createTableForTestFilter(dbName,tableName2, owner2, lastAccessTime2, true);
@@ -2475,8 +2462,7 @@ public abstract class TestHiveMetaStore {
         .setTableParams(tableParams)
         .setOwner(owner)
         .setLastAccessTime(lastAccessTime)
-        .build();
-    client.createTable(tbl);
+        .create(client, conf);
 
     if (isThriftClient) {
       // the createTable() above does not update the location in the 'tbl'
@@ -2508,8 +2494,7 @@ public abstract class TestHiveMetaStore {
           .setTableName(tblName)
           .addCol("c1", ColumnType.STRING_TYPE_NAME)
           .addCol("c2", ColumnType.INT_TYPE_NAME)
-          .build();
-      client.createTable(tbl1);
+          .create(client, conf);
 
       // get the table from the client, verify the name is correct
       Table tbl2 = client.getTable(dbName, tblName);
@@ -2692,10 +2677,9 @@ public abstract class TestHiveMetaStore {
 
   private Database createDb(String dbName) throws Exception {
     if(null == dbName) { return null; }
-    Database db = new Database();
-    db.setName(dbName);
-    client.createDatabase(db);
-    return db;
+    return new DatabaseBuilder()
+        .setName(dbName)
+        .create(client, conf);
   }
 
   private Type createType(String typeName, Map<String, String> fields) throws Throwable {
@@ -2717,13 +2701,12 @@ public abstract class TestHiveMetaStore {
    */
 
   private void createTable(String dbName, String tableName) throws TException {
-    Table t = new TableBuilder()
+    new TableBuilder()
         .setDbName(dbName)
         .setTableName(tableName)
         .addCol("foo", "string")
         .addCol("bar", "string")
-        .build();
-    client.createTable(t);
+        .create(client, conf);
   }
 
   private List<Partition> createPartitions(String dbName, Table tbl,
@@ -2765,8 +2748,7 @@ public abstract class TestHiveMetaStore {
         .addCol("income", ColumnType.INT_TYPE_NAME)
         .addPartCol("ds", ColumnType.STRING_TYPE_NAME)
         .addPartCol("hr", ColumnType.STRING_TYPE_NAME)
-        .build();
-    client.createTable(tbl);
+        .create(client, conf);
 
     if (isThriftClient) {
       // the createTable() above does not update the location in the 'tbl'
@@ -2797,12 +2779,12 @@ public abstract class TestHiveMetaStore {
     final String role1 = "role1";
 
     silentDropDatabase(dbName);
-    Database db = new Database();
-    db.setName(dbName);
-    db.setOwnerName(user1);
-    db.setOwnerType(PrincipalType.USER);
+    Database db = new DatabaseBuilder()
+        .setName(dbName)
+        .setOwnerName(user1)
+        .setOwnerType(PrincipalType.USER)
+        .create(client, conf);
 
-    client.createDatabase(db);
     checkDbOwnerType(dbName, user1, PrincipalType.USER);
 
     db.setOwnerName(user2);
@@ -2827,9 +2809,9 @@ public abstract class TestHiveMetaStore {
     // Setup
     silentDropDatabase(dbName);
 
-    Database db = new Database();
-    db.setName(dbName);
-    client.createDatabase(db);
+    new DatabaseBuilder()
+        .setName(dbName)
+        .create(client, conf);
     for (String tableName : tableNames) {
       createTable(dbName, tableName);
     }
@@ -2853,12 +2835,12 @@ public abstract class TestHiveMetaStore {
     String defaultUri = MetastoreConf.getVar(conf, ConfVars.WAREHOUSE) + "/default_location.db";
     String newUri = MetastoreConf.getVar(conf, ConfVars.WAREHOUSE) + "/new_location.db";
 
-    Database db = new Database();
-    db.setName(dbName);
-    db.setLocationUri(defaultUri);
-    client.createDatabase(db);
+    new DatabaseBuilder()
+        .setName(dbName)
+        .setLocation(defaultUri)
+        .create(client, conf);
 
-    db = client.getDatabase(dbName);
+    Database db = client.getDatabase(dbName);
 
     assertEquals("Incorrect default location of the database",
         warehouse.getDnsPath(new Path(defaultUri)).toString(), db.getLocationUri());
@@ -2981,19 +2963,18 @@ public abstract class TestHiveMetaStore {
 
       client.dropTable(dbName, tblName);
       silentDropDatabase(dbName);
-      Database db = new Database();
-      db.setName(dbName);
-      db.setDescription("Validate Table Columns test");
-      client.createDatabase(db);
+      new DatabaseBuilder()
+          .setName(dbName)
+          .setDescription("Validate Table Columns test")
+          .create(client, conf);
 
       Table tbl = new TableBuilder()
           .setDbName(dbName)
           .setTableName(tblName)
           .addCol("name", ColumnType.STRING_TYPE_NAME)
           .addCol("income", ColumnType.INT_TYPE_NAME)
-          .build();
+          .create(client, conf);
 
-      client.createTable(tbl);
       if (isThriftClient) {
         tbl = client.getTable(dbName, tblName);
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStorePartitionSpecs.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStorePartitionSpecs.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStorePartitionSpecs.java
index b2d1d5a..df83171 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStorePartitionSpecs.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStorePartitionSpecs.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hive.metastore.api.PartitionSpec;
 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.client.builder.DatabaseBuilder;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
 import org.apache.hadoop.hive.metastore.partition.spec.CompositePartitionSpecProxy;
@@ -121,11 +122,9 @@ public class TestHiveMetaStorePartitionSpecs {
                       true    // Cascade.
                       );
 
-    hmsc.createDatabase(new Database(dbName,
-                                     "",    // Description.
-                                     null,  // Location.
-                                     null   // Parameters.
-                       ));
+    new DatabaseBuilder()
+        .setName(dbName)
+        .create(hmsc, conf);
   }
 
   // Get partition-path. For grid='XYZ', place the partition outside the table-path.

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreSchemaMethods.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreSchemaMethods.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreSchemaMethods.java
index 1b30090..3d48c5f 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreSchemaMethods.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreSchemaMethods.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.metastore;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
+import org.apache.hadoop.hive.metastore.api.Catalog;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.FindSchemasByColsResp;
@@ -36,6 +37,7 @@ import org.apache.hadoop.hive.metastore.api.SchemaVersionDescriptor;
 import org.apache.hadoop.hive.metastore.api.SchemaVersionState;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.SerdeType;
+import org.apache.hadoop.hive.metastore.client.builder.CatalogBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.ISchemaBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.SchemaVersionBuilder;
@@ -64,6 +66,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
 import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
 
 // This does the testing using a remote metastore, as that finds more issues in thrift
@@ -74,11 +77,12 @@ public class TestHiveMetaStoreSchemaMethods {
   private static Map<PreEventContext.PreEventType, Integer> preEvents;
 
   private static IMetaStoreClient client;
+  private static Configuration conf;
 
 
   @BeforeClass
   public static void startMetastore() throws Exception {
-    Configuration conf = MetastoreConf.newMetastoreConf();
+    conf = MetastoreConf.newMetastoreConf();
     MetaStoreTestUtils.setConfForStandloneMode(conf);
     MetastoreConf.setClass(conf, ConfVars.EVENT_LISTENERS, SchemaEventListener.class,
         MetaStoreEventListener.class);
@@ -101,7 +105,7 @@ public class TestHiveMetaStoreSchemaMethods {
 
   @Test(expected = NoSuchObjectException.class)
   public void getNonExistentSchema() throws TException {
-    client.getISchema(DEFAULT_DATABASE_NAME, "no.such.schema");
+    client.getISchema(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, "no.such.schema");
   }
 
   @Test
@@ -124,11 +128,13 @@ public class TestHiveMetaStoreSchemaMethods {
     Assert.assertEquals(1, (int)events.get(EventMessage.EventType.CREATE_ISCHEMA));
     Assert.assertEquals(1, (int)transactionalEvents.get(EventMessage.EventType.CREATE_ISCHEMA));
 
-    schema = client.getISchema(DEFAULT_DATABASE_NAME, schemaName);
+    schema = client.getISchema(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, schemaName);
     Assert.assertEquals(1, (int)preEvents.get(PreEventContext.PreEventType.READ_ISCHEMA));
 
     Assert.assertEquals(SchemaType.AVRO, schema.getSchemaType());
     Assert.assertEquals(schemaName, schema.getName());
+    Assert.assertEquals(DEFAULT_CATALOG_NAME, schema.getCatName());
+    Assert.assertEquals(DEFAULT_DATABASE_NAME, schema.getDbName());
     Assert.assertEquals(SchemaCompatibility.FORWARD, schema.getCompatibility());
     Assert.assertEquals(SchemaValidation.LATEST, schema.getValidationLevel());
     Assert.assertFalse(schema.isCanEvolve());
@@ -142,12 +148,12 @@ public class TestHiveMetaStoreSchemaMethods {
     schema.setCanEvolve(true);
     schema.setSchemaGroup(schemaGroup);
     schema.setDescription(description);
-    client.alterISchema(DEFAULT_DATABASE_NAME, schemaName, schema);
+    client.alterISchema(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, schemaName, schema);
     Assert.assertEquals(1, (int)preEvents.get(PreEventContext.PreEventType.ALTER_ISCHEMA));
     Assert.assertEquals(1, (int)events.get(EventMessage.EventType.ALTER_ISCHEMA));
     Assert.assertEquals(1, (int)transactionalEvents.get(EventMessage.EventType.ALTER_ISCHEMA));
 
-    schema = client.getISchema(DEFAULT_DATABASE_NAME, schemaName);
+    schema = client.getISchema(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, schemaName);
     Assert.assertEquals(2, (int)preEvents.get(PreEventContext.PreEventType.READ_ISCHEMA));
 
     Assert.assertEquals(SchemaType.AVRO, schema.getSchemaType());
@@ -158,12 +164,12 @@ public class TestHiveMetaStoreSchemaMethods {
     Assert.assertEquals(schemaGroup, schema.getSchemaGroup());
     Assert.assertEquals(description, schema.getDescription());
 
-    client.dropISchema(DEFAULT_DATABASE_NAME, schemaName);
+    client.dropISchema(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, schemaName);
     Assert.assertEquals(1, (int)preEvents.get(PreEventContext.PreEventType.DROP_ISCHEMA));
     Assert.assertEquals(1, (int)events.get(EventMessage.EventType.DROP_ISCHEMA));
     Assert.assertEquals(1, (int)transactionalEvents.get(EventMessage.EventType.DROP_ISCHEMA));
     try {
-      client.getISchema(DEFAULT_DATABASE_NAME, schemaName);
+      client.getISchema(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, schemaName);
       Assert.fail();
     } catch (NoSuchObjectException e) {
       // all good
@@ -172,11 +178,18 @@ public class TestHiveMetaStoreSchemaMethods {
 
   @Test
   public void iSchemaOtherDatabase() throws TException {
+    String catName = "other_cat";
+    Catalog cat = new CatalogBuilder()
+        .setName(catName)
+        .setLocation(MetaStoreTestUtils.getTestWarehouseDir(catName))
+        .build();
+    client.createCatalog(cat);
+
     String dbName = "other_db";
     Database db = new DatabaseBuilder()
         .setName(dbName)
-        .build();
-    client.createDatabase(db);
+        .setCatalogName(catName)
+        .create(client, conf);
 
     String schemaName = uniqueSchemaName();
     String schemaGroup = "group1";
@@ -184,7 +197,7 @@ public class TestHiveMetaStoreSchemaMethods {
     ISchema schema = new ISchemaBuilder()
         .setSchemaType(SchemaType.AVRO)
         .setName(schemaName)
-        .setDbName(dbName)
+        .inDb(db)
         .setCompatibility(SchemaCompatibility.FORWARD)
         .setValidationLevel(SchemaValidation.LATEST)
         .setCanEvolve(false)
@@ -193,10 +206,11 @@ public class TestHiveMetaStoreSchemaMethods {
         .build();
     client.createISchema(schema);
 
-    schema = client.getISchema(dbName, schemaName);
+    schema = client.getISchema(catName, dbName, schemaName);
 
     Assert.assertEquals(SchemaType.AVRO, schema.getSchemaType());
     Assert.assertEquals(schemaName, schema.getName());
+    Assert.assertEquals(catName, schema.getCatName());
     Assert.assertEquals(dbName, schema.getDbName());
     Assert.assertEquals(SchemaCompatibility.FORWARD, schema.getCompatibility());
     Assert.assertEquals(SchemaValidation.LATEST, schema.getValidationLevel());
@@ -211,12 +225,13 @@ public class TestHiveMetaStoreSchemaMethods {
     schema.setCanEvolve(true);
     schema.setSchemaGroup(schemaGroup);
     schema.setDescription(description);
-    client.alterISchema(dbName, schemaName, schema);
+    client.alterISchema(catName, dbName, schemaName, schema);
 
-    schema = client.getISchema(dbName, schemaName);
+    schema = client.getISchema(catName, dbName, schemaName);
 
     Assert.assertEquals(SchemaType.AVRO, schema.getSchemaType());
     Assert.assertEquals(schemaName, schema.getName());
+    Assert.assertEquals(catName, schema.getCatName());
     Assert.assertEquals(dbName, schema.getDbName());
     Assert.assertEquals(SchemaCompatibility.BOTH, schema.getCompatibility());
     Assert.assertEquals(SchemaValidation.ALL, schema.getValidationLevel());
@@ -224,9 +239,9 @@ public class TestHiveMetaStoreSchemaMethods {
     Assert.assertEquals(schemaGroup, schema.getSchemaGroup());
     Assert.assertEquals(description, schema.getDescription());
 
-    client.dropISchema(dbName, schemaName);
+    client.dropISchema(catName, dbName, schemaName);
     try {
-      client.getISchema(dbName, schemaName);
+      client.getISchema(catName, dbName, schemaName);
       Assert.fail();
     } catch (NoSuchObjectException e) {
       // all good
@@ -252,7 +267,6 @@ public class TestHiveMetaStoreSchemaMethods {
         .build();
     client.createISchema(schema);
 
-    schema = client.getISchema(DEFAULT_DATABASE_NAME, schemaName);
     Assert.assertNotNull(schema);
 
     Assert.assertEquals(SchemaType.HIVE, schema.getSchemaType());
@@ -273,19 +287,18 @@ public class TestHiveMetaStoreSchemaMethods {
         .setName(schemaName)
         .setDescription("a new description")
         .build();
-    client.alterISchema(DEFAULT_DATABASE_NAME, schemaName, schema);
+    client.alterISchema(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, schemaName, schema);
   }
 
   @Test(expected = NoSuchObjectException.class)
   public void dropNonExistentSchema() throws TException {
-    client.dropISchema(DEFAULT_DATABASE_NAME, "no_such_schema");
+    client.dropISchema(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, "no_such_schema");
   }
 
   @Test(expected = NoSuchObjectException.class)
   public void createVersionOfNonExistentSchema() throws TException {
     SchemaVersion schemaVersion = new SchemaVersionBuilder()
         .setSchemaName("noSchemaOfThisNameExists")
-        .setDbName(DEFAULT_DATABASE_NAME)
         .setVersion(1)
         .addCol("a", ColumnType.STRING_TYPE_NAME)
         .build();
@@ -333,10 +346,11 @@ public class TestHiveMetaStoreSchemaMethods {
     Assert.assertEquals(1, (int)events.get(EventMessage.EventType.ADD_SCHEMA_VERSION));
     Assert.assertEquals(1, (int)transactionalEvents.get(EventMessage.EventType.ADD_SCHEMA_VERSION));
 
-    schemaVersion = client.getSchemaVersion(DEFAULT_DATABASE_NAME, schemaName, version);
+    schemaVersion = client.getSchemaVersion(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, schemaName, version);
     Assert.assertNotNull(schemaVersion);
     Assert.assertEquals(schemaName, schemaVersion.getSchema().getSchemaName());
     Assert.assertEquals(DEFAULT_DATABASE_NAME, schemaVersion.getSchema().getDbName());
+    Assert.assertEquals(DEFAULT_CATALOG_NAME, schemaVersion.getSchema().getCatName());
     Assert.assertEquals(version, schemaVersion.getVersion());
     Assert.assertEquals(creationTime, schemaVersion.getCreatedAt());
     Assert.assertEquals(SchemaVersionState.INITIATED, schemaVersion.getState());
@@ -357,12 +371,12 @@ public class TestHiveMetaStoreSchemaMethods {
     Assert.assertEquals(ColumnType.FLOAT_TYPE_NAME, cols.get(1).getType());
     Assert.assertEquals(1, (int)preEvents.get(PreEventContext.PreEventType.READ_SCHEMA_VERSION));
 
-    client.dropSchemaVersion(DEFAULT_DATABASE_NAME, schemaName, version);
+    client.dropSchemaVersion(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, schemaName, version);
     Assert.assertEquals(1, (int)preEvents.get(PreEventContext.PreEventType.DROP_SCHEMA_VERSION));
     Assert.assertEquals(1, (int)events.get(EventMessage.EventType.DROP_SCHEMA_VERSION));
     Assert.assertEquals(1, (int)transactionalEvents.get(EventMessage.EventType.DROP_SCHEMA_VERSION));
     try {
-      client.getSchemaVersion(DEFAULT_DATABASE_NAME, schemaName, version);
+      client.getSchemaVersion(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, schemaName, version);
       Assert.fail();
     } catch (NoSuchObjectException e) {
       // all good
@@ -371,17 +385,24 @@ public class TestHiveMetaStoreSchemaMethods {
 
   @Test
   public void addSchemaVersionOtherDb() throws TException {
+    String catName = "other_cat_for_schema_version";
+    Catalog cat = new CatalogBuilder()
+        .setName(catName)
+        .setLocation(MetaStoreTestUtils.getTestWarehouseDir(catName))
+        .build();
+    client.createCatalog(cat);
+
     String dbName = "other_db_for_schema_version";
     Database db = new DatabaseBuilder()
         .setName(dbName)
-        .build();
-    client.createDatabase(db);
+        .setCatalogName(catName)
+        .create(client, conf);
 
     String schemaName = uniqueSchemaName();
     int version = 1;
 
     ISchema schema = new ISchemaBuilder()
-        .setDbName(dbName)
+        .inDb(db)
         .setSchemaType(SchemaType.AVRO)
         .setName(schemaName)
         .build();
@@ -414,10 +435,11 @@ public class TestHiveMetaStoreSchemaMethods {
         .build();
     client.addSchemaVersion(schemaVersion);
 
-    schemaVersion = client.getSchemaVersion(dbName, schemaName, version);
+    schemaVersion = client.getSchemaVersion(catName, dbName, schemaName, version);
     Assert.assertNotNull(schemaVersion);
     Assert.assertEquals(schemaName, schemaVersion.getSchema().getSchemaName());
     Assert.assertEquals(dbName, schemaVersion.getSchema().getDbName());
+    Assert.assertEquals(catName, schemaVersion.getSchema().getCatName());
     Assert.assertEquals(version, schemaVersion.getVersion());
     Assert.assertEquals(creationTime, schemaVersion.getCreatedAt());
     Assert.assertEquals(SchemaVersionState.INITIATED, schemaVersion.getState());
@@ -438,9 +460,9 @@ public class TestHiveMetaStoreSchemaMethods {
     Assert.assertEquals(ColumnType.FLOAT_TYPE_NAME, cols.get(1).getType());
     Assert.assertEquals(1, (int)preEvents.get(PreEventContext.PreEventType.READ_SCHEMA_VERSION));
 
-    client.dropSchemaVersion(dbName, schemaName, version);
+    client.dropSchemaVersion(catName, dbName, schemaName, version);
     try {
-      client.getSchemaVersion(dbName, schemaName, version);
+      client.getSchemaVersion(catName, dbName, schemaName, version);
       Assert.fail();
     } catch (NoSuchObjectException e) {
       // all good
@@ -484,7 +506,7 @@ public class TestHiveMetaStoreSchemaMethods {
     Assert.assertEquals(3, (int)events.get(EventMessage.EventType.ADD_SCHEMA_VERSION));
     Assert.assertEquals(3, (int)transactionalEvents.get(EventMessage.EventType.ADD_SCHEMA_VERSION));
 
-    schemaVersion = client.getSchemaLatestVersion(DEFAULT_DATABASE_NAME, schemaName);
+    schemaVersion = client.getSchemaLatestVersion(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, schemaName);
     Assert.assertEquals(3, schemaVersion.getVersion());
     Assert.assertEquals(3, schemaVersion.getColsSize());
     List<FieldSchema> cols = schemaVersion.getCols();
@@ -497,7 +519,7 @@ public class TestHiveMetaStoreSchemaMethods {
     Assert.assertEquals(ColumnType.TIMESTAMP_TYPE_NAME, cols.get(2).getType());
     Assert.assertEquals(1, (int)preEvents.get(PreEventContext.PreEventType.READ_SCHEMA_VERSION));
 
-    List<SchemaVersion> versions = client.getSchemaAllVersions(DEFAULT_DATABASE_NAME, schemaName);
+    List<SchemaVersion> versions = client.getSchemaAllVersions(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, schemaName);
     Assert.assertEquals(2, (int)preEvents.get(PreEventContext.PreEventType.READ_SCHEMA_VERSION));
     Assert.assertEquals(3, versions.size());
     versions.sort(Comparator.comparingInt(SchemaVersion::getVersion));
@@ -534,7 +556,7 @@ public class TestHiveMetaStoreSchemaMethods {
         .setName(schemaName)
         .build();
     client.createISchema(schema);
-    client.getSchemaVersion(DEFAULT_DATABASE_NAME, schemaName, 1);
+    client.getSchemaVersion(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, schemaName, 1);
   }
 
   @Test(expected = NoSuchObjectException.class)
@@ -545,7 +567,18 @@ public class TestHiveMetaStoreSchemaMethods {
         .setName(schemaName)
         .build();
     client.createISchema(schema);
-    client.getSchemaVersion("bogus", schemaName, 1);
+    client.getSchemaVersion(DEFAULT_CATALOG_NAME, "bogus", schemaName, 1);
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void schemaVersionBogusCatalog() throws TException {
+    String schemaName = uniqueSchemaName();
+    ISchema schema = new ISchemaBuilder()
+        .setSchemaType(SchemaType.AVRO)
+        .setName(schemaName)
+        .build();
+    client.createISchema(schema);
+    client.getSchemaVersion("bogus", DEFAULT_DATABASE_NAME, schemaName, 1);
   }
 
   @Test(expected = NoSuchObjectException.class)
@@ -566,7 +599,7 @@ public class TestHiveMetaStoreSchemaMethods {
         .build();
     client.addSchemaVersion(schemaVersion);
 
-    client.getSchemaVersion(DEFAULT_DATABASE_NAME, schemaName, 2);
+    client.getSchemaVersion(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, schemaName, 2);
   }
 
   @Test(expected = NoSuchObjectException.class)
@@ -577,12 +610,12 @@ public class TestHiveMetaStoreSchemaMethods {
         .setName(schemaName)
         .build();
     client.createISchema(schema);
-    client.getSchemaLatestVersion(DEFAULT_DATABASE_NAME, schemaName);
+    client.getSchemaLatestVersion(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, schemaName);
   }
 
   @Test(expected = NoSuchObjectException.class)
   public void getLatestSchemaNoSuchSchema() throws TException {
-    client.getSchemaLatestVersion(DEFAULT_DATABASE_NAME, "no.such.schema.with.this.name");
+    client.getSchemaLatestVersion(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, "no.such.schema.with.this.name");
   }
 
   @Test(expected = NoSuchObjectException.class)
@@ -593,7 +626,18 @@ public class TestHiveMetaStoreSchemaMethods {
         .setName(schemaName)
         .build();
     client.createISchema(schema);
-    client.getSchemaLatestVersion("bogus", schemaName);
+    client.getSchemaLatestVersion(DEFAULT_CATALOG_NAME, "bogus", schemaName);
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void latestSchemaVersionBogusCatalog() throws TException {
+    String schemaName = uniqueSchemaName();
+    ISchema schema = new ISchemaBuilder()
+        .setSchemaType(SchemaType.AVRO)
+        .setName(schemaName)
+        .build();
+    client.createISchema(schema);
+    client.getSchemaLatestVersion("bogus", DEFAULT_DATABASE_NAME, schemaName);
   }
 
   @Test(expected = NoSuchObjectException.class)
@@ -604,12 +648,12 @@ public class TestHiveMetaStoreSchemaMethods {
         .setName(schemaName)
         .build();
     client.createISchema(schema);
-    client.getSchemaAllVersions(DEFAULT_DATABASE_NAME, schemaName);
+    client.getSchemaAllVersions(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, schemaName);
   }
 
   @Test(expected = NoSuchObjectException.class)
   public void getAllSchemaNoSuchSchema() throws TException {
-    client.getSchemaAllVersions(DEFAULT_DATABASE_NAME, "no.such.schema.with.this.name");
+    client.getSchemaAllVersions(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, "no.such.schema.with.this.name");
   }
 
   @Test(expected = NoSuchObjectException.class)
@@ -620,7 +664,18 @@ public class TestHiveMetaStoreSchemaMethods {
         .setName(schemaName)
         .build();
     client.createISchema(schema);
-    client.getSchemaAllVersions("bogus", schemaName);
+    client.getSchemaAllVersions(DEFAULT_CATALOG_NAME, "bogus", schemaName);
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void allSchemaVersionBogusCatalog() throws TException {
+    String schemaName = uniqueSchemaName();
+    ISchema schema = new ISchemaBuilder()
+        .setSchemaType(SchemaType.AVRO)
+        .setName(schemaName)
+        .build();
+    client.createISchema(schema);
+    client.getSchemaAllVersions("bogus", DEFAULT_DATABASE_NAME, schemaName);
   }
 
   @Test(expected = AlreadyExistsException.class)
@@ -648,7 +703,7 @@ public class TestHiveMetaStoreSchemaMethods {
   @Test(expected = NoSuchObjectException.class)
   public void mapSerDeNoSuchSchema() throws TException {
     SerDeInfo serDeInfo = new SerDeInfo(uniqueSerdeName(), "lib", Collections.emptyMap());
-    client.mapSchemaVersionToSerde(DEFAULT_DATABASE_NAME, uniqueSchemaName(), 1, serDeInfo.getName());
+    client.mapSchemaVersionToSerde(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, uniqueSchemaName(), 1, serDeInfo.getName());
   }
 
   @Test(expected = NoSuchObjectException.class)
@@ -659,7 +714,7 @@ public class TestHiveMetaStoreSchemaMethods {
         .setName(uniqueSchemaName())
         .build();
     client.createISchema(schema);
-    client.mapSchemaVersionToSerde(DEFAULT_DATABASE_NAME, schema.getName(), 3, serDeInfo.getName());
+    client.mapSchemaVersionToSerde(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, schema.getName(), 3, serDeInfo.getName());
   }
 
   @Test(expected = NoSuchObjectException.class)
@@ -676,7 +731,7 @@ public class TestHiveMetaStoreSchemaMethods {
         .addCol("x", ColumnType.BOOLEAN_TYPE_NAME)
         .build();
     client.addSchemaVersion(schemaVersion);
-    client.mapSchemaVersionToSerde(DEFAULT_DATABASE_NAME, schema.getName(), schemaVersion.getVersion(), uniqueSerdeName());
+    client.mapSchemaVersionToSerde(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, schema.getName(), schemaVersion.getVersion(), uniqueSerdeName());
   }
 
   @Test
@@ -698,8 +753,8 @@ public class TestHiveMetaStoreSchemaMethods {
     SerDeInfo serDeInfo = new SerDeInfo(uniqueSerdeName(), "lib", Collections.emptyMap());
     client.addSerDe(serDeInfo);
 
-    client.mapSchemaVersionToSerde(DEFAULT_DATABASE_NAME, schema.getName(), schemaVersion.getVersion(), serDeInfo.getName());
-    schemaVersion = client.getSchemaVersion(DEFAULT_DATABASE_NAME, schema.getName(), schemaVersion.getVersion());
+    client.mapSchemaVersionToSerde(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, schema.getName(), schemaVersion.getVersion(), serDeInfo.getName());
+    schemaVersion = client.getSchemaVersion(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, schema.getName(), schemaVersion.getVersion());
     Assert.assertEquals(serDeInfo.getName(), schemaVersion.getSerDe().getName());
 
     // Create schema with a serde, then remap it
@@ -713,27 +768,34 @@ public class TestHiveMetaStoreSchemaMethods {
         .build();
     client.addSchemaVersion(schemaVersion);
 
-    schemaVersion = client.getSchemaVersion(DEFAULT_DATABASE_NAME, schema.getName(), 2);
+    schemaVersion = client.getSchemaVersion(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, schema.getName(), 2);
     Assert.assertEquals(serDeName, schemaVersion.getSerDe().getName());
 
     serDeInfo = new SerDeInfo(uniqueSerdeName(), "y", Collections.emptyMap());
     client.addSerDe(serDeInfo);
-    client.mapSchemaVersionToSerde(DEFAULT_DATABASE_NAME, schema.getName(), 2, serDeInfo.getName());
-    schemaVersion = client.getSchemaVersion(DEFAULT_DATABASE_NAME, schema.getName(), 2);
+    client.mapSchemaVersionToSerde(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, schema.getName(), 2, serDeInfo.getName());
+    schemaVersion = client.getSchemaVersion(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, schema.getName(), 2);
     Assert.assertEquals(serDeInfo.getName(), schemaVersion.getSerDe().getName());
   }
 
   @Test
   public void mapSerdeToSchemaVersionOtherDb() throws TException {
+    String catName = "other_cat_for_map_to";
+    Catalog cat = new CatalogBuilder()
+        .setName(catName)
+        .setLocation(MetaStoreTestUtils.getTestWarehouseDir(catName))
+        .build();
+    client.createCatalog(cat);
+
     String dbName = "map_other_db";
     Database db = new DatabaseBuilder()
         .setName(dbName)
-        .build();
-    client.createDatabase(db);
+        .setCatalogName(catName)
+        .create(client, conf);
 
     ISchema schema = new ISchemaBuilder()
         .setSchemaType(SchemaType.AVRO)
-        .setDbName(dbName)
+        .inDb(db)
         .setName(uniqueSchemaName())
         .build();
     client.createISchema(schema);
@@ -749,8 +811,8 @@ public class TestHiveMetaStoreSchemaMethods {
     SerDeInfo serDeInfo = new SerDeInfo(uniqueSerdeName(), "lib", Collections.emptyMap());
     client.addSerDe(serDeInfo);
 
-    client.mapSchemaVersionToSerde(dbName, schema.getName(), schemaVersion.getVersion(), serDeInfo.getName());
-    schemaVersion = client.getSchemaVersion(dbName, schema.getName(), schemaVersion.getVersion());
+    client.mapSchemaVersionToSerde(catName, dbName, schema.getName(), schemaVersion.getVersion(), serDeInfo.getName());
+    schemaVersion = client.getSchemaVersion(catName, dbName, schema.getName(), schemaVersion.getVersion());
     Assert.assertEquals(serDeInfo.getName(), schemaVersion.getSerDe().getName());
 
     // Create schema with a serde, then remap it
@@ -764,13 +826,13 @@ public class TestHiveMetaStoreSchemaMethods {
         .build();
     client.addSchemaVersion(schemaVersion);
 
-    schemaVersion = client.getSchemaVersion(dbName, schema.getName(), 2);
+    schemaVersion = client.getSchemaVersion(catName, dbName, schema.getName(), 2);
     Assert.assertEquals(serDeName, schemaVersion.getSerDe().getName());
 
     serDeInfo = new SerDeInfo(uniqueSerdeName(), "y", Collections.emptyMap());
     client.addSerDe(serDeInfo);
-    client.mapSchemaVersionToSerde(dbName, schema.getName(), 2, serDeInfo.getName());
-    schemaVersion = client.getSchemaVersion(dbName, schema.getName(), 2);
+    client.mapSchemaVersionToSerde(catName, dbName, schema.getName(), 2, serDeInfo.getName());
+    schemaVersion = client.getSchemaVersion(catName, dbName, schema.getName(), 2);
     Assert.assertEquals(serDeInfo.getName(), schemaVersion.getSerDe().getName());
 
   }
@@ -811,7 +873,7 @@ public class TestHiveMetaStoreSchemaMethods {
 
   @Test(expected = NoSuchObjectException.class)
   public void setVersionStateNoSuchSchema() throws TException {
-    client.setSchemaVersionState(DEFAULT_DATABASE_NAME, "no.such.schema", 1, SchemaVersionState.INITIATED);
+    client.setSchemaVersionState(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, "no.such.schema", 1, SchemaVersionState.INITIATED);
   }
 
   @Test(expected = NoSuchObjectException.class)
@@ -822,7 +884,7 @@ public class TestHiveMetaStoreSchemaMethods {
         .setName(schemaName)
         .build();
     client.createISchema(schema);
-    client.setSchemaVersionState(DEFAULT_DATABASE_NAME, schemaName, 1, SchemaVersionState.INITIATED);
+    client.setSchemaVersionState(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, schemaName, 1, SchemaVersionState.INITIATED);
   }
 
   @Test
@@ -841,37 +903,44 @@ public class TestHiveMetaStoreSchemaMethods {
         .build();
     client.addSchemaVersion(schemaVersion);
 
-    schemaVersion = client.getSchemaVersion(DEFAULT_DATABASE_NAME, schemaName, 1);
+    schemaVersion = client.getSchemaVersion(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, schemaName, 1);
     Assert.assertNull(schemaVersion.getState());
 
-    client.setSchemaVersionState(DEFAULT_DATABASE_NAME, schemaName, 1, SchemaVersionState.INITIATED);
+    client.setSchemaVersionState(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, schemaName, 1, SchemaVersionState.INITIATED);
     Assert.assertEquals(1, (int)preEvents.get(PreEventContext.PreEventType.ALTER_SCHEMA_VERSION));
     Assert.assertEquals(1, (int)events.get(EventMessage.EventType.ALTER_SCHEMA_VERSION));
     Assert.assertEquals(1, (int)transactionalEvents.get(EventMessage.EventType.ALTER_SCHEMA_VERSION));
-    schemaVersion = client.getSchemaVersion(DEFAULT_DATABASE_NAME, schemaName, 1);
+    schemaVersion = client.getSchemaVersion(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, schemaName, 1);
     Assert.assertEquals(SchemaVersionState.INITIATED, schemaVersion.getState());
 
-    client.setSchemaVersionState(DEFAULT_DATABASE_NAME, schemaName, 1, SchemaVersionState.REVIEWED);
+    client.setSchemaVersionState(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, schemaName, 1, SchemaVersionState.REVIEWED);
     Assert.assertEquals(2, (int)preEvents.get(PreEventContext.PreEventType.ALTER_SCHEMA_VERSION));
     Assert.assertEquals(2, (int)events.get(EventMessage.EventType.ALTER_SCHEMA_VERSION));
     Assert.assertEquals(2, (int)transactionalEvents.get(EventMessage.EventType.ALTER_SCHEMA_VERSION));
-    schemaVersion = client.getSchemaVersion(DEFAULT_DATABASE_NAME, schemaName, 1);
+    schemaVersion = client.getSchemaVersion(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, schemaName, 1);
     Assert.assertEquals(SchemaVersionState.REVIEWED, schemaVersion.getState());
   }
 
   @Test
   public void setVersionStateOtherDb() throws TException {
+    String catName = "other_cat_for_set_version";
+    Catalog cat = new CatalogBuilder()
+        .setName(catName)
+        .setLocation(MetaStoreTestUtils.getTestWarehouseDir(catName))
+        .build();
+    client.createCatalog(cat);
+
     String dbName = "other_db_set_state";
     Database db = new DatabaseBuilder()
         .setName(dbName)
-        .build();
-    client.createDatabase(db);
+        .setCatalogName(catName)
+        .create(client, conf);
 
     String schemaName = uniqueSchemaName();
     ISchema schema = new ISchemaBuilder()
         .setSchemaType(SchemaType.AVRO)
         .setName(schemaName)
-        .setDbName(dbName)
+        .inDb(db)
         .build();
     client.createISchema(schema);
 
@@ -882,27 +951,27 @@ public class TestHiveMetaStoreSchemaMethods {
         .build();
     client.addSchemaVersion(schemaVersion);
 
-    schemaVersion = client.getSchemaVersion(dbName, schemaName, 1);
+    schemaVersion = client.getSchemaVersion(catName, dbName, schemaName, 1);
     Assert.assertNull(schemaVersion.getState());
 
-    client.setSchemaVersionState(dbName, schemaName, 1, SchemaVersionState.INITIATED);
+    client.setSchemaVersionState(catName, dbName, schemaName, 1, SchemaVersionState.INITIATED);
     Assert.assertEquals(1, (int)preEvents.get(PreEventContext.PreEventType.ALTER_SCHEMA_VERSION));
     Assert.assertEquals(1, (int)events.get(EventMessage.EventType.ALTER_SCHEMA_VERSION));
     Assert.assertEquals(1, (int)transactionalEvents.get(EventMessage.EventType.ALTER_SCHEMA_VERSION));
-    schemaVersion = client.getSchemaVersion(dbName, schemaName, 1);
+    schemaVersion = client.getSchemaVersion(catName, dbName, schemaName, 1);
     Assert.assertEquals(SchemaVersionState.INITIATED, schemaVersion.getState());
 
-    client.setSchemaVersionState(dbName, schemaName, 1, SchemaVersionState.REVIEWED);
+    client.setSchemaVersionState(catName, dbName, schemaName, 1, SchemaVersionState.REVIEWED);
     Assert.assertEquals(2, (int)preEvents.get(PreEventContext.PreEventType.ALTER_SCHEMA_VERSION));
     Assert.assertEquals(2, (int)events.get(EventMessage.EventType.ALTER_SCHEMA_VERSION));
     Assert.assertEquals(2, (int)transactionalEvents.get(EventMessage.EventType.ALTER_SCHEMA_VERSION));
-    schemaVersion = client.getSchemaVersion(dbName, schemaName, 1);
+    schemaVersion = client.getSchemaVersion(catName, dbName, schemaName, 1);
     Assert.assertEquals(SchemaVersionState.REVIEWED, schemaVersion.getState());
   }
 
   @Test(expected = NoSuchObjectException.class)
   public void dropNonExistentSchemaVersion() throws TException {
-    client.dropSchemaVersion(DEFAULT_DATABASE_NAME, "ther is no schema named this", 23);
+    client.dropSchemaVersion(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, "ther is no schema named this", 23);
   }
 
   @Test
@@ -910,8 +979,7 @@ public class TestHiveMetaStoreSchemaMethods {
     String dbName = "schema_query_db";
     Database db = new DatabaseBuilder()
         .setName(dbName)
-        .build();
-    client.createDatabase(db);
+        .create(client, conf);
 
     String schemaName1 = uniqueSchemaName();
     ISchema schema1 = new ISchemaBuilder()

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreTimeout.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreTimeout.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreTimeout.java
index 42df9c2..1560d05 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreTimeout.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreTimeout.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
 import org.apache.hadoop.util.StringUtils;
@@ -75,13 +76,9 @@ public class TestHiveMetaStoreTimeout {
     String dbName = "db";
     client.dropDatabase(dbName, true, true);
 
-    Database db = new Database();
-    db.setName(dbName);
-    try {
-      client.createDatabase(db);
-    } catch (MetaException e) {
-      Assert.fail("should not throw timeout exception: " + e.getMessage());
-    }
+    new DatabaseBuilder()
+        .setName(dbName)
+        .create(client, conf);
 
     client.dropDatabase(dbName, true, true);
   }
@@ -93,8 +90,9 @@ public class TestHiveMetaStoreTimeout {
     String dbName = "db";
     client.dropDatabase(dbName, true, true);
 
-    Database db = new Database();
-    db.setName(dbName);
+    Database db = new DatabaseBuilder()
+        .setName(dbName)
+        .build(conf);
     try {
       client.createDatabase(db);
       Assert.fail("should throw timeout exception.");
@@ -114,8 +112,9 @@ public class TestHiveMetaStoreTimeout {
 
     // no timeout before reset
     client.dropDatabase(dbName, true, true);
-    Database db = new Database();
-    db.setName(dbName);
+    Database db = new DatabaseBuilder()
+        .setName(dbName)
+        .build(conf);
     try {
       client.createDatabase(db);
     } catch (MetaException e) {

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreWithEnvironmentContext.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreWithEnvironmentContext.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreWithEnvironmentContext.java
index d53a606..38b3f6e 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreWithEnvironmentContext.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreWithEnvironmentContext.java
@@ -46,6 +46,7 @@ import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
 import static org.junit.Assert.assertEquals;
 
 /**
@@ -85,6 +86,7 @@ public class TestHiveMetaStoreWithEnvironmentContext {
     envContext = new EnvironmentContext(envProperties);
 
     db.setName(dbName);
+    db.setCatalogName(DEFAULT_CATALOG_NAME);
 
     table = new TableBuilder()
         .setDbName(dbName)
@@ -93,13 +95,13 @@ public class TestHiveMetaStoreWithEnvironmentContext {
         .addPartCol("b", "string")
         .addCol("a", "string")
         .addCol("b", "string")
-        .build();
+        .build(conf);
 
 
     partition = new PartitionBuilder()
-        .fromTable(table)
+        .inTable(table)
         .addValue("2011")
-        .build();
+        .build(conf);
 
     DummyListener.notifyList.clear();
   }
@@ -171,7 +173,7 @@ public class TestHiveMetaStoreWithEnvironmentContext {
     assert dropPartByNameEvent.getStatus();
     assertEquals(envContext, dropPartByNameEvent.getEnvironmentContext());
 
-    msc.dropTable(dbName, tblName, true, false, envContext);
+    msc.dropTable(DEFAULT_CATALOG_NAME, dbName, tblName, true, false, envContext);
     listSize++;
     assertEquals(notifyList.size(), listSize);
     DropTableEvent dropTblEvent = (DropTableEvent)notifyList.get(listSize-1);

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMarkPartition.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMarkPartition.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMarkPartition.java
index b477088..00fae25 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMarkPartition.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMarkPartition.java
@@ -63,8 +63,7 @@ public class TestMarkPartition {
     msc.dropDatabase(dbName, true, true, true);
     Database db = new DatabaseBuilder()
         .setName(dbName)
-        .build();
-    msc.createDatabase(db);
+        .create(msc, conf);
 
     final String tableName = "tmptbl";
     msc.dropTable(dbName, tableName, true, true);
@@ -73,13 +72,12 @@ public class TestMarkPartition {
         .setTableName(tableName)
         .addCol("a", "string")
         .addPartCol("b", "string")
-        .build();
-    msc.createTable(table);
+        .create(msc, conf);
 
     Partition part = new PartitionBuilder()
-        .fromTable(table)
+        .inTable(table)
         .addValue("2011")
-        .build();
+        .build(conf);
     msc.add_partition(part);
     Map<String,String> kvs = new HashMap<>();
     kvs.put("b", "'2011'");

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEndFunctionListener.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEndFunctionListener.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEndFunctionListener.java
index 1a720fb..b919eef 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEndFunctionListener.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEndFunctionListener.java
@@ -72,8 +72,8 @@ public class TestMetaStoreEndFunctionListener {
 
     Database db = new DatabaseBuilder()
         .setName(dbName)
-        .build();
-    msc.createDatabase(db);
+        .setCatalogName(Warehouse.DEFAULT_CATALOG_NAME)
+        .create(msc, conf);
 
     try {
       msc.getDatabase("UnknownDB");
@@ -91,13 +91,12 @@ public class TestMetaStoreEndFunctionListener {
     assertEquals(context.getInputTableName(), null);
 
     String unknownTable = "UnknownTable";
-    Table table = new TableBuilder()
-        .setDbName(db)
+    new TableBuilder()
+        .inDb(db)
         .setTableName(tblName)
         .addCol("a", "string")
         .addPartCol("b", "string")
-        .build();
-    msc.createTable(table);
+        .create(msc, conf);
     try {
       msc.getTable(dbName, unknownTable);
     } catch (Exception e1) {

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java
index fb7f940..fb4a761 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java
@@ -201,13 +201,12 @@ public class TestMetaStoreEventListener {
     assertEquals(notifyList.size(), listSize);
     assertEquals(preNotifyList.size(), listSize);
 
-    Database db = new DatabaseBuilder()
+    new DatabaseBuilder()
         .setName(dbName)
-        .build();
-    msc.createDatabase(db);
+        .create(msc, conf);
     listSize++;
     PreCreateDatabaseEvent preDbEvent = (PreCreateDatabaseEvent)(preNotifyList.get(preNotifyList.size() - 1));
-    db = msc.getDatabase(dbName);
+    Database db = msc.getDatabase(dbName);
     assertEquals(listSize, notifyList.size());
     assertEquals(listSize + 1, preNotifyList.size());
     validateCreateDb(db, preDbEvent.getDatabase());
@@ -217,12 +216,11 @@ public class TestMetaStoreEventListener {
     validateCreateDb(db, dbEvent.getDatabase());
 
     Table table = new TableBuilder()
-        .setDbName(db)
+        .inDb(db)
         .setTableName(tblName)
         .addCol("a", "string")
         .addPartCol("b", "string")
-        .build();
-    msc.createTable(table);
+        .create(msc, conf);
     PreCreateTableEvent preTblEvent = (PreCreateTableEvent)(preNotifyList.get(preNotifyList.size() - 1));
     listSize++;
     Table tbl = msc.getTable(dbName, tblName);
@@ -234,18 +232,17 @@ public class TestMetaStoreEventListener {
     validateCreateTable(tbl, tblEvent.getTable());
 
 
-    Partition part = new PartitionBuilder()
-        .fromTable(table)
+    new PartitionBuilder()
+        .inTable(table)
         .addValue("2011")
-        .build();
-    msc.add_partition(part);
+        .addToTable(msc, conf);
     listSize++;
     assertEquals(notifyList.size(), listSize);
     PreAddPartitionEvent prePartEvent = (PreAddPartitionEvent)(preNotifyList.get(preNotifyList.size() - 1));
 
     AddPartitionEvent partEvent = (AddPartitionEvent)(notifyList.get(listSize-1));
     Assert.assertTrue(partEvent.getStatus());
-    part = msc.getPartition("hive2038", "tmptbl", "b=2011");
+    Partition part = msc.getPartition("hive2038", "tmptbl", "b=2011");
     Partition partAdded = partEvent.getPartitionIterator().next();
     validateAddPartition(part, partAdded);
     validateTableInAddPartition(tbl, partEvent.getTable());


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/InsertEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/InsertEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/InsertEvent.java
index b963f78..aa014e9 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/InsertEvent.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/InsertEvent.java
@@ -55,12 +55,13 @@ public class InsertEvent extends ListenerEvent {
    * @param status status of insert, true = success, false = failure
    * @param handler handler that is firing the event
    */
-  public InsertEvent(String db, String table, List<String> partVals,
+  public InsertEvent(String catName, String db, String table, List<String> partVals,
       InsertEventRequestData insertData, boolean status, IHMSHandler handler) throws MetaException,
       NoSuchObjectException {
     super(status, handler);
 
     GetTableRequest req = new GetTableRequest(db, table);
+    req.setCatName(catName);
     // TODO MS-SPLIT Switch this back once HiveMetaStoreClient is moved.
     //req.setCapabilities(HiveMetaStoreClient.TEST_VERSION);
     req.setCapabilities(new ClientCapabilities(
@@ -68,7 +69,8 @@ public class InsertEvent extends ListenerEvent {
     try {
       this.tableObj = handler.get_table_req(req).getTable();
       if (partVals != null) {
-        this.ptnObj = handler.get_partition(db, table, partVals);
+        this.ptnObj = handler.get_partition(MetaStoreUtils.prependNotNullCatToDbName(catName, db),
+            table, partVals);
       } else {
         this.ptnObj = null;
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreCreateCatalogEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreCreateCatalogEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreCreateCatalogEvent.java
new file mode 100644
index 0000000..96aa22c
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreCreateCatalogEvent.java
@@ -0,0 +1,39 @@
+/*
+ * 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.events;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.Catalog;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class PreCreateCatalogEvent extends PreEventContext {
+
+  private final Catalog cat;
+
+  public PreCreateCatalogEvent(IHMSHandler handler, Catalog cat) {
+    super(PreEventType.CREATE_CATALOG, handler);
+    this.cat = cat;
+  }
+
+  public Catalog getCatalog() {
+    return cat;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropCatalogEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropCatalogEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropCatalogEvent.java
new file mode 100644
index 0000000..0e01ccd
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreDropCatalogEvent.java
@@ -0,0 +1,39 @@
+/*
+ * 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.events;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.Catalog;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class PreDropCatalogEvent extends PreEventContext {
+
+  private final Catalog cat;
+
+  public PreDropCatalogEvent(IHMSHandler handler, Catalog cat) {
+    super(PreEventType.DROP_CATALOG, handler);
+    this.cat = cat;
+  }
+
+  public Catalog getCatalog() {
+    return cat;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java
index 7ddb8fe..b45a537 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java
@@ -50,7 +50,10 @@ public abstract class PreEventContext {
     ALTER_SCHEMA_VERSION,
     DROP_SCHEMA_VERSION,
     READ_ISCHEMA,
-    READ_SCHEMA_VERSION
+    READ_SCHEMA_VERSION,
+    CREATE_CATALOG,
+    DROP_CATALOG,
+    READ_CATALOG
   }
 
   private final PreEventType eventType;

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreLoadPartitionDoneEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreLoadPartitionDoneEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreLoadPartitionDoneEvent.java
index 999ec31..a380301 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreLoadPartitionDoneEvent.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreLoadPartitionDoneEvent.java
@@ -28,18 +28,24 @@ import java.util.Map;
 @InterfaceStability.Stable
 public class PreLoadPartitionDoneEvent extends PreEventContext {
 
+  private final String catName;
   private final String dbName;
   private final String tableName;
   private final Map<String,String> partSpec;
 
-  public PreLoadPartitionDoneEvent(String dbName, String tableName,
+  public PreLoadPartitionDoneEvent(String catName, String dbName, String tableName,
       Map<String, String> partSpec, IHMSHandler handler) {
     super(PreEventType.LOAD_PARTITION_DONE, handler);
+    this.catName = catName;
     this.dbName = dbName;
     this.tableName = tableName;
     this.partSpec = partSpec;
   }
 
+  public String getCatName() {
+    return catName;
+  }
+
   public String getDbName() {
     return dbName;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreReadCatalogEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreReadCatalogEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreReadCatalogEvent.java
new file mode 100644
index 0000000..3f1afdf
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/PreReadCatalogEvent.java
@@ -0,0 +1,39 @@
+/*
+ * 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.events;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.IHMSHandler;
+import org.apache.hadoop.hive.metastore.api.Catalog;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class PreReadCatalogEvent extends PreEventContext {
+
+  private final Catalog cat;
+
+  public PreReadCatalogEvent(IHMSHandler handler, Catalog cat) {
+    super(PreEventType.READ_CATALOG, handler);
+    this.cat = cat;
+  }
+
+  public Catalog getCatalog() {
+    return cat;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/CreateCatalogMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/CreateCatalogMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/CreateCatalogMessage.java
new file mode 100644
index 0000000..cbb0f4e
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/CreateCatalogMessage.java
@@ -0,0 +1,25 @@
+/*
+ * 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.messaging;
+
+public abstract class CreateCatalogMessage extends EventMessage {
+
+  protected CreateCatalogMessage() {
+    super(EventType.CREATE_CATALOG);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/DropCatalogMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/DropCatalogMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/DropCatalogMessage.java
new file mode 100644
index 0000000..0e731ce
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/DropCatalogMessage.java
@@ -0,0 +1,25 @@
+/*
+ * 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.messaging;
+
+public abstract class DropCatalogMessage extends EventMessage {
+
+  protected DropCatalogMessage() {
+    super(EventType.DROP_CATALOG);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/EventMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/EventMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/EventMessage.java
index 8578d4a..3cbfa55 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/EventMessage.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/EventMessage.java
@@ -53,7 +53,9 @@ public abstract class EventMessage {
     DROP_ISCHEMA(MessageFactory.DROP_ISCHEMA_EVENT),
     ADD_SCHEMA_VERSION(MessageFactory.ADD_SCHEMA_VERSION_EVENT),
     ALTER_SCHEMA_VERSION(MessageFactory.ALTER_SCHEMA_VERSION_EVENT),
-    DROP_SCHEMA_VERSION(MessageFactory.DROP_SCHEMA_VERSION_EVENT);
+    DROP_SCHEMA_VERSION(MessageFactory.DROP_SCHEMA_VERSION_EVENT),
+    CREATE_CATALOG(MessageFactory.CREATE_CATALOG_EVENT),
+    DROP_CATALOG(MessageFactory.DROP_CATALOG_EVENT);
 
     private String typeString;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java
index 5976c48..ab93f82 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java
@@ -20,6 +20,7 @@
 package org.apache.hadoop.hive.metastore.messaging;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.Catalog;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.Function;
 import org.apache.hadoop.hive.metastore.api.MetaException;
@@ -65,6 +66,8 @@ public abstract class MessageFactory {
   public static final String ADD_SCHEMA_VERSION_EVENT = "ADD_SCHEMA_VERSION";
   public static final String ALTER_SCHEMA_VERSION_EVENT = "ALTER_SCHEMA_VERSION";
   public static final String DROP_SCHEMA_VERSION_EVENT = "DROP_SCHEMA_VERSION";
+  public static final String CREATE_CATALOG_EVENT = "CREATE_CATALOG";
+  public static final String DROP_CATALOG_EVENT = "DROP_CATALOG";
 
 
   private static MessageFactory instance = null;
@@ -276,4 +279,8 @@ public abstract class MessageFactory {
    */
   public abstract DropConstraintMessage buildDropConstraintMessage(String dbName, String tableName,
       String constraintName);
+
+  public abstract CreateCatalogMessage buildCreateCatalogMessage(Catalog catalog);
+
+  public abstract DropCatalogMessage buildDropCatalogMessage(Catalog catalog);
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateCatalogMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateCatalogMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateCatalogMessage.java
new file mode 100644
index 0000000..8a26764
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCreateCatalogMessage.java
@@ -0,0 +1,79 @@
+/*
+ * 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.messaging.json;
+
+import org.apache.hadoop.hive.metastore.messaging.CreateCatalogMessage;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+public class JSONCreateCatalogMessage extends CreateCatalogMessage {
+
+  @JsonProperty
+  String server, servicePrincipal, catalog;
+
+  @JsonProperty
+  Long timestamp;
+
+  /**
+   * Required for Jackson
+   */
+  public JSONCreateCatalogMessage() {
+
+  }
+
+  public JSONCreateCatalogMessage(String server, String servicePrincipal, String catalog,
+                                  Long timestamp) {
+    this.server = server;
+    this.servicePrincipal = servicePrincipal;
+    this.catalog = catalog;
+    this.timestamp = timestamp;
+  }
+
+  @Override
+  public String getDB() {
+    return null;
+  }
+
+  @Override
+  public String getServer() {
+    return server;
+  }
+
+  @Override
+  public String getServicePrincipal() {
+    return servicePrincipal;
+  }
+
+  public String getCatalog() {
+    return catalog;
+  }
+
+  @Override
+  public Long getTimestamp() {
+    return timestamp;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return JSONMessageDeserializer.mapper.writeValueAsString(this);
+    }
+    catch (Exception exception) {
+      throw new IllegalArgumentException("Could not serialize: ", exception);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropCatalogMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropCatalogMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropCatalogMessage.java
new file mode 100644
index 0000000..58e95f4
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONDropCatalogMessage.java
@@ -0,0 +1,66 @@
+/*
+ * 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.messaging.json;
+
+import org.apache.hadoop.hive.metastore.messaging.DropCatalogMessage;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+public class JSONDropCatalogMessage extends DropCatalogMessage {
+
+  @JsonProperty
+  String server, servicePrincipal, catalog;
+
+  @JsonProperty
+  Long timestamp;
+
+  public JSONDropCatalogMessage() {
+
+  }
+
+  public JSONDropCatalogMessage(String server, String servicePrincipal, String catalog,
+                                Long timestamp) {
+    this.server = server;
+    this.servicePrincipal = servicePrincipal;
+    this.catalog = catalog;
+    this.timestamp = timestamp;
+  }
+
+  @Override
+  public String getDB() {
+    return null;
+  }
+
+  @Override
+  public String getServer() {
+    return server;
+  }
+
+  @Override
+  public String getServicePrincipal() {
+    return servicePrincipal;
+  }
+
+  public String getCatalog() {
+    return catalog;
+  }
+
+  @Override
+  public Long getTimestamp() {
+    return timestamp;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java
index 4f03a27..0fc5387 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java
@@ -28,6 +28,7 @@ import javax.annotation.Nullable;
 
 import com.google.common.collect.Iterables;
 
+import org.apache.hadoop.hive.metastore.api.Catalog;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.Function;
 import org.apache.hadoop.hive.metastore.api.NotificationEvent;
@@ -45,9 +46,11 @@ import org.apache.hadoop.hive.metastore.messaging.AddUniqueConstraintMessage;
 import org.apache.hadoop.hive.metastore.messaging.AlterDatabaseMessage;
 import org.apache.hadoop.hive.metastore.messaging.AlterPartitionMessage;
 import org.apache.hadoop.hive.metastore.messaging.AlterTableMessage;
+import org.apache.hadoop.hive.metastore.messaging.CreateCatalogMessage;
 import org.apache.hadoop.hive.metastore.messaging.CreateDatabaseMessage;
 import org.apache.hadoop.hive.metastore.messaging.CreateFunctionMessage;
 import org.apache.hadoop.hive.metastore.messaging.CreateTableMessage;
+import org.apache.hadoop.hive.metastore.messaging.DropCatalogMessage;
 import org.apache.hadoop.hive.metastore.messaging.DropConstraintMessage;
 import org.apache.hadoop.hive.metastore.messaging.DropDatabaseMessage;
 import org.apache.hadoop.hive.metastore.messaging.DropFunctionMessage;
@@ -190,6 +193,16 @@ public class JSONMessageFactory extends MessageFactory {
         constraintName, now());
   }
 
+  @Override
+  public CreateCatalogMessage buildCreateCatalogMessage(Catalog catalog) {
+    return new JSONCreateCatalogMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, catalog.getName(), now());
+  }
+
+  @Override
+  public DropCatalogMessage buildDropCatalogMessage(Catalog catalog) {
+    return new JSONDropCatalogMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, catalog.getName(), now());
+  }
+
   private long now() {
     return System.currentTimeMillis() / 1000;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MCatalog.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MCatalog.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MCatalog.java
new file mode 100644
index 0000000..e82cb43
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MCatalog.java
@@ -0,0 +1,58 @@
+/*
+ * 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.model;
+
+public class MCatalog {
+  private String name;
+  private String description;
+  private String locationUri;
+
+  public MCatalog() {
+
+  }
+
+  public MCatalog(String name, String description, String locationUri) {
+    this.name = name;
+    this.description = description;
+    this.locationUri = locationUri;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  public String getDescription() {
+    return description;
+  }
+
+  public void setDescription(String description) {
+    this.description = description;
+  }
+
+  public String getLocationUri() {
+    return locationUri;
+  }
+
+  public void setLocationUri(String locationUri) {
+    this.locationUri = locationUri;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MCreationMetadata.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MCreationMetadata.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MCreationMetadata.java
index 1133cb1..66b5d48 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MCreationMetadata.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MCreationMetadata.java
@@ -27,6 +27,7 @@ import java.util.Set;
  */
 public class MCreationMetadata {
 
+  private String catalogName;
   private String dbName;
   private String tblName;
   private Set<MTable> tables;
@@ -35,8 +36,9 @@ public class MCreationMetadata {
   public MCreationMetadata() {
   }
 
-  public MCreationMetadata(String dbName, String tblName,
+  public MCreationMetadata(String catName, String dbName, String tblName,
       Set<MTable> tables, String txnList) {
+    this.catalogName = catName;
     this.dbName = dbName;
     this.tblName = tblName;
     this.tables = tables;
@@ -59,6 +61,14 @@ public class MCreationMetadata {
     this.txnList = txnList;
   }
 
+  public String getCatalogName() {
+    return catalogName;
+  }
+
+  public void setCatalogName(String catName) {
+    this.catalogName = catName;
+  }
+
   public String getDbName() {
     return dbName;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MDatabase.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MDatabase.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MDatabase.java
index e8034ce..fa30330 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MDatabase.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MDatabase.java
@@ -34,6 +34,7 @@ public class MDatabase {
   private Map<String, String> parameters;
   private String ownerName;
   private String ownerType;
+  private String catalogName;
 
   /**
    * Default construction to keep jpox/jdo happy
@@ -46,12 +47,13 @@ public class MDatabase {
    * @param locationUri Location of the database in the warehouse
    * @param description Comment describing the database
    */
-  public MDatabase(String name, String locationUri, String description,
+  public MDatabase(String catalogName, String name, String locationUri, String description,
       Map<String, String> parameters) {
     this.name = name;
     this.locationUri = locationUri;
     this.description = description;
     this.parameters = parameters;
+    this.catalogName = catalogName;
   }
 
   /**
@@ -125,4 +127,12 @@ public class MDatabase {
   public void setOwnerType(String ownerType) {
     this.ownerType = ownerType;
   }
+
+  public String getCatalogName() {
+    return catalogName;
+  }
+
+  public void setCatalogName(String catalogName) {
+    this.catalogName = catalogName;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MNotificationLog.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MNotificationLog.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MNotificationLog.java
index 1b1f7fd..60914ae 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MNotificationLog.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MNotificationLog.java
@@ -22,6 +22,7 @@ public class MNotificationLog {
   private long eventId; // This is not the datanucleus id, but the id assigned by the sequence
   private int eventTime;
   private String eventType;
+  private String catalogName;
   private String dbName;
   private String tableName;
   private String message;
@@ -30,10 +31,11 @@ public class MNotificationLog {
   public MNotificationLog() {
   }
 
-  public MNotificationLog(int eventId, String eventType, String dbName, String tableName,
+  public MNotificationLog(int eventId, String eventType, String catName, String dbName, String tableName,
                           String message) {
     this.eventId = eventId;
     this.eventType = eventType;
+    this.catalogName = catName;
     this.dbName = dbName;
     this.tableName = tableName;
     this.message = message;
@@ -72,6 +74,14 @@ public class MNotificationLog {
     this.dbName = dbName;
   }
 
+  public String getCatalogName() {
+    return catalogName;
+  }
+
+  public void setCatalogName(String catName) {
+    this.catalogName = catName;
+  }
+
   public String getTableName() {
     return tableName;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MPartitionColumnStatistics.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MPartitionColumnStatistics.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MPartitionColumnStatistics.java
index f7ef6fc..50d9c5b 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MPartitionColumnStatistics.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MPartitionColumnStatistics.java
@@ -34,6 +34,7 @@ public class MPartitionColumnStatistics {
 
   private MPartition partition;
 
+  private String catName;
   private String dbName;
   private String tableName;
   private String partitionName;
@@ -137,6 +138,14 @@ public class MPartitionColumnStatistics {
     this.dbName = dbName;
   }
 
+  public String getCatName() {
+    return catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
   public MPartition getPartition() {
     return partition;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MPartitionEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MPartitionEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MPartitionEvent.java
index 50c5045..d0cc51a 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MPartitionEvent.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MPartitionEvent.java
@@ -21,6 +21,8 @@ package org.apache.hadoop.hive.metastore.model;
 
 public class MPartitionEvent {
 
+  private String catalogName;
+
   private String dbName;
 
   private String tblName;
@@ -31,8 +33,9 @@ public class MPartitionEvent {
 
   private int eventType;
 
-  public MPartitionEvent(String dbName, String tblName, String partitionName, int eventType) {
+  public MPartitionEvent(String catName, String dbName, String tblName, String partitionName, int eventType) {
     super();
+    this.catalogName = catName;
     this.dbName = dbName;
     this.tblName = tblName;
     this.partName = partitionName;
@@ -42,6 +45,10 @@ public class MPartitionEvent {
 
   public MPartitionEvent() {}
 
+  public void setCatalogName(String catName) {
+    this.catalogName = catName;
+  }
+
   /**
    * @param dbName the dbName to set
    */

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MTableColumnStatistics.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MTableColumnStatistics.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MTableColumnStatistics.java
index ec61317..731cd6f 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MTableColumnStatistics.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MTableColumnStatistics.java
@@ -33,6 +33,7 @@ package org.apache.hadoop.hive.metastore.model;
 public class MTableColumnStatistics {
 
   private MTable table;
+  private String catName;
   private String dbName;
   private String tableName;
   private String colName;
@@ -151,6 +152,14 @@ public class MTableColumnStatistics {
     this.dbName = dbName;
   }
 
+  public String getCatName() {
+    return catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
   public void setBooleanStats(Long numTrues, Long numFalses, Long numNulls) {
     this.numTrues = numTrues;
     this.numFalses = numFalses;

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/CompositePartitionSpecProxy.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/CompositePartitionSpecProxy.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/CompositePartitionSpecProxy.java
index e34335d..92813b9 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/CompositePartitionSpecProxy.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/CompositePartitionSpecProxy.java
@@ -26,11 +26,14 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
+
 /**
  * Implementation of PartitionSpecProxy that composes a list of PartitionSpecProxy.
  */
 public class CompositePartitionSpecProxy extends PartitionSpecProxy {
 
+  private String catName;
   private String dbName;
   private String tableName;
   private List<PartitionSpec> partitionSpecs;
@@ -40,10 +43,12 @@ public class CompositePartitionSpecProxy extends PartitionSpecProxy {
   protected CompositePartitionSpecProxy(List<PartitionSpec> partitionSpecs) {
     this.partitionSpecs = partitionSpecs;
     if (partitionSpecs.isEmpty()) {
+      catName = null;
       dbName = null;
       tableName = null;
     }
     else {
+      catName = partitionSpecs.get(0).getCatName();
       dbName = partitionSpecs.get(0).getDbName();
       tableName = partitionSpecs.get(0).getTableName();
       this.partitionSpecProxies = new ArrayList<>(partitionSpecs.size());
@@ -57,7 +62,15 @@ public class CompositePartitionSpecProxy extends PartitionSpecProxy {
     assert isValid() : "Invalid CompositePartitionSpecProxy!";
   }
 
+  @Deprecated
   protected CompositePartitionSpecProxy(String dbName, String tableName, List<PartitionSpec> partitionSpecs) {
+    this(DEFAULT_CATALOG_NAME, dbName, tableName, partitionSpecs);
+
+  }
+
+  protected CompositePartitionSpecProxy(String catName, String dbName, String tableName,
+                                        List<PartitionSpec> partitionSpecs) {
+    this.catName = catName;
     this.dbName = dbName;
     this.tableName = tableName;
     this.partitionSpecs = partitionSpecs;
@@ -146,6 +159,11 @@ public class CompositePartitionSpecProxy extends PartitionSpecProxy {
     }
 
     @Override
+    public String getCatName() {
+      return composite.getCatName();
+    }
+
+    @Override
     public String getDbName() {
       return composite.dbName;
     }
@@ -182,6 +200,15 @@ public class CompositePartitionSpecProxy extends PartitionSpecProxy {
   }
 
   @Override
+  public void setCatName(String catName) {
+    this.catName = catName;
+    for (PartitionSpecProxy partSpecProxy : partitionSpecProxies) {
+      partSpecProxy.setCatName(catName);
+    }
+
+  }
+
+  @Override
   public void setDbName(String dbName) {
     this.dbName = dbName;
     for (PartitionSpecProxy partSpecProxy : partitionSpecProxies) {
@@ -198,6 +225,11 @@ public class CompositePartitionSpecProxy extends PartitionSpecProxy {
   }
 
   @Override
+  public String getCatName() {
+    return catName;
+  }
+
+  @Override
   public String getDbName() {
     return dbName;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionListComposingSpecProxy.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionListComposingSpecProxy.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionListComposingSpecProxy.java
index 7b0550b..6bd29d0 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionListComposingSpecProxy.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionListComposingSpecProxy.java
@@ -40,6 +40,11 @@ public class PartitionListComposingSpecProxy extends PartitionSpecProxy {
   }
 
   @Override
+  public String getCatName() {
+    return partitionSpec.getCatName();
+  }
+
+  @Override
   public String getDbName() {
     return partitionSpec.getDbName();
   }
@@ -65,6 +70,14 @@ public class PartitionListComposingSpecProxy extends PartitionSpecProxy {
   }
 
   @Override
+  public void setCatName(String catName) {
+    partitionSpec.setCatName(catName);
+    for (Partition partition : partitionSpec.getPartitionList().getPartitions()) {
+      partition.setCatName(catName);
+    }
+  }
+
+  @Override
   public void setDbName(String dbName) {
     partitionSpec.setDbName(dbName);
     for (Partition partition : partitionSpec.getPartitionList().getPartitions()) {
@@ -118,6 +131,11 @@ public class PartitionListComposingSpecProxy extends PartitionSpecProxy {
     }
 
     @Override
+    public String getCatName() {
+      return partitionSpecProxy.getCatName();
+    }
+
+    @Override
     public String getDbName() {
       return partitionSpecProxy.getDbName();
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionSpecProxy.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionSpecProxy.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionSpecProxy.java
index 2640a24..ff2dea1 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionSpecProxy.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionSpecProxy.java
@@ -37,6 +37,12 @@ public abstract class PartitionSpecProxy {
   public abstract int size();
 
   /**
+   * Set catalog name.
+   * @param catName catalog name.
+   */
+  public abstract void setCatName(String catName);
+
+  /**
    * Setter for name of the DB.
    * @param dbName The name of the DB.
    */
@@ -49,6 +55,12 @@ public abstract class PartitionSpecProxy {
   public abstract void setTableName(String tableName);
 
   /**
+   * Get catalog name.
+   * @return catalog name.
+   */
+  public abstract String getCatName();
+
+  /**
    * Getter for name of the DB.
    * @return The name of the DB.
    */
@@ -131,6 +143,12 @@ public abstract class PartitionSpecProxy {
     Partition getCurrent();
 
     /**
+     * Get the catalog name.
+     * @return catalog name.
+     */
+    String getCatName();
+
+    /**
      * Getter for the name of the DB.
      * @return Name of the DB.
      */
@@ -184,6 +202,7 @@ public abstract class PartitionSpecProxy {
     public SimplePartitionWrapperIterator(Partition partition) {this.partition = partition;}
 
     @Override public Partition getCurrent() { return partition; }
+    @Override public String getCatName() { return partition.getCatName(); }
     @Override public String getDbName() { return partition.getDbName(); }
     @Override public String getTableName() { return partition.getTableName(); }
     @Override public Map<String, String> getParameters() { return partition.getParameters(); }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionSpecWithSharedSDProxy.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionSpecWithSharedSDProxy.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionSpecWithSharedSDProxy.java
index 36b05f7..61e00ea 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionSpecWithSharedSDProxy.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/partition/spec/PartitionSpecWithSharedSDProxy.java
@@ -49,6 +49,11 @@ public class PartitionSpecWithSharedSDProxy extends PartitionSpecProxy {
   }
 
   @Override
+  public void setCatName(String catName) {
+    partitionSpec.setCatName(catName);
+  }
+
+  @Override
   public void setDbName(String dbName) {
     partitionSpec.setDbName(dbName);
   }
@@ -59,6 +64,11 @@ public class PartitionSpecWithSharedSDProxy extends PartitionSpecProxy {
   }
 
   @Override
+  public String getCatName() {
+    return partitionSpec.getCatName();
+  }
+
+  @Override
   public String getDbName() {
     return partitionSpec.getDbName();
   }
@@ -121,7 +131,7 @@ public class PartitionSpecWithSharedSDProxy extends PartitionSpecProxy {
       StorageDescriptor partSD = new StorageDescriptor(pSpec.getSd());
       partSD.setLocation(partSD.getLocation() + partWithoutSD.getRelativePath());
 
-      return new Partition(
+      Partition p = new Partition(
           partWithoutSD.getValues(),
           partitionSpecWithSharedSDProxy.partitionSpec.getDbName(),
           partitionSpecWithSharedSDProxy.partitionSpec.getTableName(),
@@ -130,6 +140,13 @@ public class PartitionSpecWithSharedSDProxy extends PartitionSpecProxy {
           partSD,
           partWithoutSD.getParameters()
       );
+      p.setCatName(partitionSpecWithSharedSDProxy.partitionSpec.getCatName());
+      return p;
+    }
+
+    @Override
+    public String getCatName() {
+      return partitionSpecWithSharedSDProxy.partitionSpec.getCatName();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SmokeTest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SmokeTest.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SmokeTest.java
index 7f4d9b0..9cdf271 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SmokeTest.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SmokeTest.java
@@ -46,6 +46,8 @@ public class SmokeTest {
   private static final String tableName = "internal_smoke_test_table";
   private static final String partValue = "internal_smoke_test_val1";
 
+  private static Configuration conf;
+
   private SmokeTest() {
 
   }
@@ -63,25 +65,22 @@ public class SmokeTest {
     Database db = new DatabaseBuilder()
         .setName(dbName)
         .setLocation(dbDir.getAbsolutePath())
-        .build();
-    client.createDatabase(db);
+        .create(client, conf);
 
     LOG.info("Going to create table " + tableName);
     Table table = new TableBuilder()
-        .setDbName(db)
+        .inDb(db)
         .setTableName(tableName)
         .addCol("col1", ColumnType.INT_TYPE_NAME)
         .addCol("col2", ColumnType.TIMESTAMP_TYPE_NAME)
         .addPartCol("pcol1", ColumnType.STRING_TYPE_NAME)
-        .build();
-    client.createTable(table);
+        .create(client, conf);
 
     LOG.info("Going to create partition with value " + partValue);
     Partition part = new PartitionBuilder()
-        .fromTable(table)
+        .inTable(table)
         .addValue("val1")
-        .build();
-    client.add_partition(part);
+        .addToTable(client, conf);
 
     LOG.info("Going to list the partitions");
     List<Partition> parts = client.listPartitions(dbName, tableName, (short)-1);
@@ -96,7 +95,7 @@ public class SmokeTest {
 
   public static void main(String[] args) throws Exception {
     SmokeTest test = new SmokeTest();
-    Configuration conf = MetastoreConf.newMetastoreConf();
+    conf = MetastoreConf.newMetastoreConf();
     IMetaStoreClient client = new HiveMetaStoreClient(conf);
     test.runTest(client);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java
index b477ce5..8ea6051 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java
@@ -118,6 +118,28 @@ public class MetaStoreUtils {
   private static final Charset ENCODING = StandardCharsets.UTF_8;
   private static final Logger LOG = LoggerFactory.getLogger(MetaStoreUtils.class);
 
+  // The following two are public for any external users who wish to use them.
+  /**
+   * This character is used to mark a database name as having a catalog name prepended.  This
+   * marker should be placed first in the String to make it easy to determine that this has both
+   * a catalog and a database name.  @ is chosen as it is not used in regular expressions.  This
+   * is only intended for use when making old Thrift calls that do not support catalog names.
+   */
+  public static final char CATALOG_DB_THRIFT_NAME_MARKER = '@';
+
+  /**
+   * This String is used to seaprate the catalog name from the database name.  This should only
+   * be used in Strings that are prepended with {@link #CATALOG_DB_THRIFT_NAME_MARKER}.  # is
+   * chosen because it is not used in regular expressions.  this is only intended for use when
+   * making old Thrift calls that do not support catalog names.
+   */
+  public static final String CATALOG_DB_SEPARATOR = "#";
+
+  /**
+   * Mark a database as being empty (as distinct from null).
+   */
+  public static final String DB_EMPTY_MARKER = "!";
+
   // Right now we only support one special character '/'.
   // More special characters can be added accordingly in the future.
   // NOTE:
@@ -217,7 +239,7 @@ public class MetaStoreUtils {
 
   // Given a list of partStats, this function will give you an aggr stats
   public static List<ColumnStatisticsObj> aggrPartitionStats(List<ColumnStatistics> partStats,
-      String dbName, String tableName, List<String> partNames, List<String> colNames,
+      String catName, String dbName, String tableName, List<String> partNames, List<String> colNames,
       boolean areAllPartsFound, boolean useDensityFunctionForNDVEstimation, double ndvTuner)
       throws MetaException {
     Map<ColumnStatsAggregator, List<ColStatsObjWithSourceInfo>> colStatsMap =
@@ -237,12 +259,12 @@ public class MetaStoreUtils {
               new ArrayList<ColStatsObjWithSourceInfo>());
         }
         colStatsMap.get(aliasToAggregator.get(obj.getColName()))
-            .add(new ColStatsObjWithSourceInfo(obj, dbName, tableName, partName));
+            .add(new ColStatsObjWithSourceInfo(obj, catName, dbName, tableName, partName));
       }
     }
     if (colStatsMap.size() < 1) {
-      LOG.debug("No stats data found for: dbName= {},  tblName= {}, partNames= {}, colNames= {}",
-          dbName, tableName, partNames, colNames);
+      LOG.debug("No stats data found for: tblName= {}, partNames= {}, colNames= {}",
+          Warehouse.getCatalogQualifiedTableName(catName, dbName, tableName), partNames, colNames);
       return new ArrayList<ColumnStatisticsObj>();
     }
     return aggrPartitionStats(colStatsMap, partNames, areAllPartsFound,
@@ -1622,13 +1644,15 @@ public class MetaStoreUtils {
   // ColumnStatisticsObj with info about its db, table, partition (if table is partitioned)
   public static class ColStatsObjWithSourceInfo {
     private final ColumnStatisticsObj colStatsObj;
+    private final String catName;
     private final String dbName;
     private final String tblName;
     private final String partName;
 
-    public ColStatsObjWithSourceInfo(ColumnStatisticsObj colStatsObj, String dbName, String tblName,
+    public ColStatsObjWithSourceInfo(ColumnStatisticsObj colStatsObj, String catName, String dbName, String tblName,
         String partName) {
       this.colStatsObj = colStatsObj;
+      this.catName = catName;
       this.dbName = dbName;
       this.tblName = tblName;
       this.partName = partName;
@@ -1638,6 +1662,10 @@ public class MetaStoreUtils {
       return colStatsObj;
     }
 
+    public String getCatName() {
+      return catName;
+    }
+
     public String getDbName() {
       return dbName;
     }
@@ -1650,4 +1678,109 @@ public class MetaStoreUtils {
       return partName;
     }
   }
+
+  private static boolean hasCatalogName(String dbName) {
+    return dbName != null && dbName.length() > 0 &&
+        dbName.charAt(0) == CATALOG_DB_THRIFT_NAME_MARKER;
+  }
+
+  /**
+   * Given a catalog name and database name cram them together into one string.  This method can
+   * be used if you do not know the catalog name, in which case the default catalog will be
+   * retrieved from the conf object.  The resulting string can be parsed apart again via
+   * {@link #parseDbName(String, Configuration)}.
+   * @param catalogName catalog name, can be null if no known.
+   * @param dbName database name, can be null or empty.
+   * @param conf configuration object, used to determine default catalog if catalogName is null
+   * @return one string that contains both.
+   */
+  public static String prependCatalogToDbName(@Nullable String catalogName, @Nullable String dbName,
+                                              Configuration conf) {
+    if (catalogName == null) catalogName = getDefaultCatalog(conf);
+    StringBuilder buf = new StringBuilder()
+        .append(CATALOG_DB_THRIFT_NAME_MARKER)
+        .append(catalogName)
+        .append(CATALOG_DB_SEPARATOR);
+    if (dbName != null) {
+      if (dbName.isEmpty()) buf.append(DB_EMPTY_MARKER);
+      else buf.append(dbName);
+    }
+    return buf.toString();
+  }
+
+  /**
+   * Given a catalog name and database name, cram them together into one string.  These can be
+   * parsed apart again via {@link #parseDbName(String, Configuration)}.
+   * @param catalogName catalog name.  This cannot be null.  If this might be null use
+   *                    {@link #prependCatalogToDbName(String, String, Configuration)} instead.
+   * @param dbName database name.
+   * @return one string that contains both.
+   */
+  public static String prependNotNullCatToDbName(String catalogName, String dbName) {
+    assert catalogName != null;
+    return prependCatalogToDbName(catalogName, dbName, null);
+  }
+
+  /**
+   * Prepend the default 'hive' catalog onto the database name.
+   * @param dbName database name
+   * @param conf configuration object, used to determine default catalog
+   * @return one string with the 'hive' catalog name prepended.
+   */
+  public static String prependCatalogToDbName(String dbName, Configuration conf) {
+    return prependCatalogToDbName(null, dbName, conf);
+  }
+
+  private final static String[] nullCatalogAndDatabase = {null, null};
+
+  /**
+   * Parse the catalog name out of the database name.  If no catalog name is present then the
+   * default catalog (as set in configuration file) will be assumed.
+   * @param dbName name of the database.  This may or may not contain the catalog name.
+   * @param conf configuration object, used to determine the default catalog if it is not present
+   *            in the database name.
+   * @return an array of two elements, the first being the catalog name, the second the database
+   * name.
+   * @throws MetaException if the name is not either just a database name or a catalog plus
+   * database name with the proper delimiters.
+   */
+  public static String[] parseDbName(String dbName, Configuration conf) throws MetaException {
+    if (dbName == null) return nullCatalogAndDatabase;
+    if (hasCatalogName(dbName)) {
+      if (dbName.endsWith(CATALOG_DB_SEPARATOR)) {
+        // This means the DB name is null
+        return new String[] {dbName.substring(1, dbName.length() - 1), null};
+      } else if (dbName.endsWith(DB_EMPTY_MARKER)) {
+        // This means the DB name is empty
+        return new String[] {dbName.substring(1, dbName.length() - DB_EMPTY_MARKER.length() - 1), ""};
+      }
+      String[] names = dbName.substring(1).split(CATALOG_DB_SEPARATOR, 2);
+      if (names.length != 2) {
+        throw new MetaException(dbName + " is prepended with the catalog marker but does not " +
+            "appear to have a catalog name in it");
+      }
+      return names;
+    } else {
+      return new String[] {getDefaultCatalog(conf), dbName};
+    }
+  }
+
+  /**
+   * Position in the array returned by {@link #parseDbName} that has the catalog name.
+   */
+  public static final int CAT_NAME = 0;
+  /**
+   * Position in the array returned by {@link #parseDbName} that has the database name.
+   */
+  public static final int DB_NAME = 1;
+
+  public static String getDefaultCatalog(Configuration conf) {
+    if (conf == null) {
+      LOG.warn("Configuration is null, so going with default catalog.");
+      return Warehouse.DEFAULT_CATALOG_NAME;
+    }
+    String catName = MetastoreConf.getVar(conf, MetastoreConf.ConfVars.CATALOG_DEFAULT);
+    if (catName == null || "".equals(catName)) catName = Warehouse.DEFAULT_CATALOG_NAME;
+    return catName;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/resources/package.jdo
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/resources/package.jdo b/standalone-metastore/src/main/resources/package.jdo
index 7612509..8d5ae5d 100644
--- a/standalone-metastore/src/main/resources/package.jdo
+++ b/standalone-metastore/src/main/resources/package.jdo
@@ -31,9 +31,15 @@
       <datastore-identity>
         <column name="DB_ID"/>
       </datastore-identity>
+      <index name="UniqueDatabase" unique="true">
+        <column name="NAME"/>
+        <column name="CTLG_NAME"/>
+      </index>
       <field name="name">  
         <column name="NAME" length="128" jdbc-type="VARCHAR"/>
-        <index name="UniqueDatabase" unique="true"/>
+      </field>
+      <field name="catalogName">
+        <column name="CTLG_NAME" length="256" jdbc-type="VARCHAR"/>
       </field>
       <field name="description">
         <column name="DESC" length="4000" jdbc-type="VARCHAR" allows-null="true"/>
@@ -61,6 +67,22 @@
       </field>
     </class>
 
+    <class name="MCatalog" identity-type="datastore" table="CTLGS" detachable="true">
+      <datastore-identity>
+        <column name="CTLG_ID"/>
+      </datastore-identity>
+      <field name="name">
+        <column name="NAME" length="256" jdbc-type="VARCHAR"/>
+        <index name="UniqueCatalog" unique="true"/>
+      </field>
+      <field name="description">
+        <column name="DESC" length="4000" jdbc-type="VARCHAR" allows-null="true"/>
+      </field>
+      <field name="locationUri">
+        <column name="LOCATION_URI" length="4000" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
+    </class>
+
     <class name="MFieldSchema" embedded-only="true" table="TYPE_FIELDS" detachable="true">
       <field name="name">
         <column name="FNAME" length="767" jdbc-type="VARCHAR"/>
@@ -191,6 +213,9 @@
       <datastore-identity>
         <column name="MV_CREATION_METADATA_ID"/>
       </datastore-identity>
+      <field name="catalogName">
+        <column name="CAT_NAME" length="256" jdbc-type="VARCHAR"/>
+      </field>
       <field name="dbName">
         <column name="DB_NAME" length="128" jdbc-type="VARCHAR"/>
       </field>
@@ -827,6 +852,9 @@
         <column name="PART_NAME_ID"/>
       </datastore-identity>
       
+      <field name="catalogName">  
+        <column name="CAT_NAME" length="256" jdbc-type="VARCHAR"/>
+      </field>
       <field name="dbName">  
         <column name="DB_NAME" length="128" jdbc-type="VARCHAR"/>
       </field>
@@ -874,6 +902,9 @@
         <column name="CS_ID"/>
       </datastore-identity>
 
+      <field name ="catName">
+        <column name="CAT_NAME" length="256" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
       <field name ="dbName">
         <column name="DB_NAME" length="128" jdbc-type="VARCHAR" allows-null="false"/>
       </field>
@@ -938,6 +969,9 @@
         <column name="CS_ID"/>
       </datastore-identity>
 
+      <field name ="catName">
+        <column name="CAT_NAME" length="256" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
       <field name ="dbName">
         <column name="DB_NAME" length="128" jdbc-type="VARCHAR" allows-null="false"/>
       </field>
@@ -1092,6 +1126,9 @@
       <field name="eventType">
         <column name="EVENT_TYPE" length="32" jdbc-type="VARCHAR" allows-null="false"/>
       </field>
+      <field name="catalogName">
+        <column name="CAT_NAME" length="256" jdbc-type="VARCHAR" allows-null="true"/>
+      </field>
       <field name="dbName">
         <column name="DB_NAME" length="128" jdbc-type="VARCHAR" allows-null="true"/>
       </field>

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/sql/derby/hive-schema-3.0.0.derby.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/derby/hive-schema-3.0.0.derby.sql b/standalone-metastore/src/main/sql/derby/hive-schema-3.0.0.derby.sql
index de9688d..0003048 100644
--- a/standalone-metastore/src/main/sql/derby/hive-schema-3.0.0.derby.sql
+++ b/standalone-metastore/src/main/sql/derby/hive-schema-3.0.0.derby.sql
@@ -15,8 +15,15 @@ CREATE FUNCTION "APP"."NUCLEUS_MATCHES" (TEXT VARCHAR(8000),PATTERN VARCHAR(8000
 -- ----------------------------------------------
 -- DDL Statements for tables
 -- ----------------------------------------------
-
-CREATE TABLE "APP"."DBS" ("DB_ID" BIGINT NOT NULL, "DESC" VARCHAR(4000), "DB_LOCATION_URI" VARCHAR(4000) NOT NULL, "NAME" VARCHAR(128), "OWNER_NAME" VARCHAR(128), "OWNER_TYPE" VARCHAR(10));
+CREATE TABLE "APP"."DBS" (
+  "DB_ID" BIGINT NOT NULL,
+  "DESC" VARCHAR(4000),
+  "DB_LOCATION_URI" VARCHAR(4000) NOT NULL,
+  "NAME" VARCHAR(128),
+  "OWNER_NAME" VARCHAR(128),
+  "OWNER_TYPE" VARCHAR(10),
+  "CTLG_NAME" VARCHAR(256) NOT NULL
+);
 
 CREATE TABLE "APP"."TBL_PRIVS" ("TBL_GRANT_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "GRANT_OPTION" SMALLINT NOT NULL, "GRANTOR" VARCHAR(128), "GRANTOR_TYPE" VARCHAR(128), "PRINCIPAL_NAME" VARCHAR(128), "PRINCIPAL_TYPE" VARCHAR(128), "TBL_PRIV" VARCHAR(128), "TBL_ID" BIGINT);
 
@@ -54,7 +61,15 @@ CREATE TABLE "APP"."GLOBAL_PRIVS" ("USER_GRANT_ID" BIGINT NOT NULL, "CREATE_TIME
 
 CREATE TABLE "APP"."PARTITION_PARAMS" ("PART_ID" BIGINT NOT NULL, "PARAM_KEY" VARCHAR(256) NOT NULL, "PARAM_VALUE" VARCHAR(4000));
 
-CREATE TABLE "APP"."PARTITION_EVENTS" ("PART_NAME_ID" BIGINT NOT NULL, "DB_NAME" VARCHAR(128), "EVENT_TIME" BIGINT NOT NULL, "EVENT_TYPE" INTEGER NOT NULL, "PARTITION_NAME" VARCHAR(767), "TBL_NAME" VARCHAR(256));
+CREATE TABLE "APP"."PARTITION_EVENTS" (
+    "PART_NAME_ID" BIGINT NOT NULL,
+    "CAT_NAME" VARCHAR(256),
+    "DB_NAME" VARCHAR(128),
+    "EVENT_TIME" BIGINT NOT NULL,
+    "EVENT_TYPE" INTEGER NOT NULL,
+    "PARTITION_NAME" VARCHAR(767),
+    "TBL_NAME" VARCHAR(256)
+);
 
 CREATE TABLE "APP"."COLUMNS" ("SD_ID" BIGINT NOT NULL, "COMMENT" VARCHAR(256), "COLUMN_NAME" VARCHAR(128) NOT NULL, "TYPE_NAME" VARCHAR(4000) NOT NULL, "INTEGER_IDX" INTEGER NOT NULL);
 
@@ -70,7 +85,29 @@ CREATE TABLE "APP"."SDS" ("SD_ID" BIGINT NOT NULL, "INPUT_FORMAT" VARCHAR(4000),
 
 CREATE TABLE "APP"."SEQUENCE_TABLE" ("SEQUENCE_NAME" VARCHAR(256) NOT NULL, "NEXT_VAL" BIGINT NOT NULL);
 
-CREATE TABLE "APP"."TAB_COL_STATS"("DB_NAME" VARCHAR(128) NOT NULL,"TABLE_NAME" VARCHAR(256) NOT NULL, "COLUMN_NAME" VARCHAR(767) NOT NULL, "COLUMN_TYPE" VARCHAR(128) NOT NULL, "LONG_LOW_VALUE" BIGINT, "LONG_HIGH_VALUE" BIGINT, "DOUBLE_LOW_VALUE" DOUBLE, "DOUBLE_HIGH_VALUE" DOUBLE, "BIG_DECIMAL_LOW_VALUE" VARCHAR(4000),"BIG_DECIMAL_HIGH_VALUE" VARCHAR(4000),"NUM_DISTINCTS" BIGINT, "NUM_NULLS" BIGINT NOT NULL, "AVG_COL_LEN" DOUBLE, "MAX_COL_LEN" BIGINT, "NUM_TRUES" BIGINT, "NUM_FALSES" BIGINT, "LAST_ANALYZED" BIGINT, "CS_ID" BIGINT NOT NULL, "TBL_ID" BIGINT NOT NULL, "BIT_VECTOR" BLOB);
+CREATE TABLE "APP"."TAB_COL_STATS"(
+    "CAT_NAME" VARCHAR(256) NOT NULL,
+    "DB_NAME" VARCHAR(128) NOT NULL,
+    "TABLE_NAME" VARCHAR(256) NOT NULL,
+    "COLUMN_NAME" VARCHAR(767) NOT NULL,
+    "COLUMN_TYPE" VARCHAR(128) NOT NULL,
+    "LONG_LOW_VALUE" BIGINT,
+    "LONG_HIGH_VALUE" BIGINT,
+    "DOUBLE_LOW_VALUE" DOUBLE,
+    "DOUBLE_HIGH_VALUE" DOUBLE,
+    "BIG_DECIMAL_LOW_VALUE" VARCHAR(4000),
+    "BIG_DECIMAL_HIGH_VALUE" VARCHAR(4000),
+    "NUM_DISTINCTS" BIGINT,
+    "NUM_NULLS" BIGINT NOT NULL,
+    "AVG_COL_LEN" DOUBLE,
+    "MAX_COL_LEN" BIGINT,
+    "NUM_TRUES" BIGINT,
+    "NUM_FALSES" BIGINT,
+    "LAST_ANALYZED" BIGINT,
+    "CS_ID" BIGINT NOT NULL,
+    "TBL_ID" BIGINT NOT NULL,
+    "BIT_VECTOR" BLOB
+);
 
 CREATE TABLE "APP"."TABLE_PARAMS" ("TBL_ID" BIGINT NOT NULL, "PARAM_KEY" VARCHAR(256) NOT NULL, "PARAM_VALUE" CLOB);
 
@@ -96,7 +133,30 @@ CREATE TABLE "APP"."MASTER_KEYS" ("KEY_ID" INTEGER NOT NULL generated always as
 
 CREATE TABLE "APP"."DELEGATION_TOKENS" ( "TOKEN_IDENT" VARCHAR(767) NOT NULL, "TOKEN" VARCHAR(767));
 
-CREATE TABLE "APP"."PART_COL_STATS"("DB_NAME" VARCHAR(128) NOT NULL,"TABLE_NAME" VARCHAR(256) NOT NULL, "PARTITION_NAME" VARCHAR(767) NOT NULL, "COLUMN_NAME" VARCHAR(767) NOT NULL, "COLUMN_TYPE" VARCHAR(128) NOT NULL, "LONG_LOW_VALUE" BIGINT, "LONG_HIGH_VALUE" BIGINT, "DOUBLE_LOW_VALUE" DOUBLE, "DOUBLE_HIGH_VALUE" DOUBLE, "BIG_DECIMAL_LOW_VALUE" VARCHAR(4000), "BIG_DECIMAL_HIGH_VALUE" VARCHAR(4000),"NUM_DISTINCTS" BIGINT, "BIT_VECTOR" BLOB, "NUM_NULLS" BIGINT NOT NULL, "AVG_COL_LEN" DOUBLE, "MAX_COL_LEN" BIGINT, "NUM_TRUES" BIGINT, "NUM_FALSES" BIGINT, "LAST_ANALYZED" BIGINT, "CS_ID" BIGINT NOT NULL, "PART_ID" BIGINT NOT NULL);
+CREATE TABLE "APP"."PART_COL_STATS"(
+    "CAT_NAME" VARCHAR(256) NOT NULL,
+    "DB_NAME" VARCHAR(128) NOT NULL,
+    "TABLE_NAME" VARCHAR(256) NOT NULL,
+    "PARTITION_NAME" VARCHAR(767) NOT NULL,
+    "COLUMN_NAME" VARCHAR(767) NOT NULL,
+    "COLUMN_TYPE" VARCHAR(128) NOT NULL,
+    "LONG_LOW_VALUE" BIGINT,
+    "LONG_HIGH_VALUE" BIGINT,
+    "DOUBLE_LOW_VALUE" DOUBLE,
+    "DOUBLE_HIGH_VALUE" DOUBLE,
+    "BIG_DECIMAL_LOW_VALUE" VARCHAR(4000),
+    "BIG_DECIMAL_HIGH_VALUE" VARCHAR(4000),
+    "NUM_DISTINCTS" BIGINT,
+    "BIT_VECTOR" BLOB,
+    "NUM_NULLS" BIGINT NOT NULL,
+    "AVG_COL_LEN" DOUBLE,
+    "MAX_COL_LEN" BIGINT,
+    "NUM_TRUES" BIGINT,
+    "NUM_FALSES" BIGINT,
+    "LAST_ANALYZED" BIGINT,
+    "CS_ID" BIGINT NOT NULL,
+    "PART_ID" BIGINT NOT NULL
+);
 
 CREATE TABLE "APP"."VERSION" ("VER_ID" BIGINT NOT NULL, "SCHEMA_VERSION" VARCHAR(127) NOT NULL, "VERSION_COMMENT" VARCHAR(255));
 
@@ -104,7 +164,17 @@ CREATE TABLE "APP"."FUNCS" ("FUNC_ID" BIGINT NOT NULL, "CLASS_NAME" VARCHAR(4000
 
 CREATE TABLE "APP"."FUNC_RU" ("FUNC_ID" BIGINT NOT NULL, "RESOURCE_TYPE" INTEGER NOT NULL, "RESOURCE_URI" VARCHAR(4000), "INTEGER_IDX" INTEGER NOT NULL);
 
-CREATE TABLE "APP"."NOTIFICATION_LOG" ("NL_ID" BIGINT NOT NULL, "DB_NAME" VARCHAR(128), "EVENT_ID" BIGINT NOT NULL, "EVENT_TIME" INTEGER NOT NULL, "EVENT_TYPE" VARCHAR(32) NOT NULL, "MESSAGE" CLOB, "TBL_NAME" VARCHAR(256), "MESSAGE_FORMAT" VARCHAR(16));
+CREATE TABLE "APP"."NOTIFICATION_LOG" (
+    "NL_ID" BIGINT NOT NULL,
+    "CAT_NAME" VARCHAR(256),
+    "DB_NAME" VARCHAR(128),
+    "EVENT_ID" BIGINT NOT NULL,
+    "EVENT_TIME" INTEGER NOT NULL,
+    "EVENT_TYPE" VARCHAR(32) NOT NULL,
+    "MESSAGE" CLOB,
+    "TBL_NAME" VARCHAR(256),
+    "MESSAGE_FORMAT" VARCHAR(16)
+);
 
 CREATE TABLE "APP"."NOTIFICATION_SEQUENCE" ("NNI_ID" BIGINT NOT NULL, "NEXT_EVENT_ID" BIGINT NOT NULL);
 
@@ -124,6 +194,7 @@ CREATE TABLE "APP"."WM_MAPPING" (MAPPING_ID BIGINT NOT NULL, RP_ID BIGINT NOT NU
 
 CREATE TABLE "APP"."MV_CREATION_METADATA" (
   "MV_CREATION_METADATA_ID" BIGINT NOT NULL,
+  "CAT_NAME" VARCHAR(256) NOT NULL,
   "DB_NAME" VARCHAR(128) NOT NULL,
   "TBL_NAME" VARCHAR(256) NOT NULL,
   "TXN_LIST" CLOB
@@ -134,6 +205,12 @@ CREATE TABLE "APP"."MV_TABLES_USED" (
   "TBL_ID" BIGINT NOT NULL
 );
 
+CREATE TABLE "APP"."CTLGS" (
+    "CTLG_ID" BIGINT NOT NULL,
+    "NAME" VARCHAR(256) UNIQUE,
+    "DESC" VARCHAR(4000),
+    "LOCATION_URI" VARCHAR(4000) NOT NULL);
+
 -- ----------------------------------------------
 -- DML Statements
 -- ----------------------------------------------
@@ -150,7 +227,7 @@ CREATE INDEX "APP"."TABLECOLUMNPRIVILEGEINDEX" ON "APP"."TBL_COL_PRIVS" ("TBL_ID
 
 CREATE UNIQUE INDEX "APP"."DBPRIVILEGEINDEX" ON "APP"."DB_PRIVS" ("DB_ID", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "DB_PRIV", "GRANTOR", "GRANTOR_TYPE");
 
-CREATE INDEX "APP"."PCS_STATS_IDX" ON "APP"."PART_COL_STATS" ("DB_NAME","TABLE_NAME","COLUMN_NAME","PARTITION_NAME");
+CREATE INDEX "APP"."PCS_STATS_IDX" ON "APP"."PART_COL_STATS" ("CAT_NAME", "DB_NAME","TABLE_NAME","COLUMN_NAME","PARTITION_NAME");
 
 CREATE INDEX "APP"."PARTPRIVILEGEINDEX" ON "APP"."PART_PRIVS" ("PART_ID", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "PART_PRIV", "GRANTOR", "GRANTOR_TYPE");
 
@@ -160,7 +237,7 @@ CREATE INDEX "APP"."TABLEPRIVILEGEINDEX" ON "APP"."TBL_PRIVS" ("TBL_ID", "PRINCI
 
 CREATE UNIQUE INDEX "APP"."UNIQUETABLE" ON "APP"."TBLS" ("TBL_NAME", "DB_ID");
 
-CREATE UNIQUE INDEX "APP"."UNIQUE_DATABASE" ON "APP"."DBS" ("NAME");
+CREATE UNIQUE INDEX "APP"."UNIQUE_DATABASE" ON "APP"."DBS" ("NAME", "CTLG_NAME");
 
 CREATE UNIQUE INDEX "APP"."USERROLEMAPINDEX" ON "APP"."ROLE_MAP" ("PRINCIPAL_NAME", "ROLE_ID", "GRANTOR", "GRANTOR_TYPE");
 
@@ -192,6 +269,9 @@ CREATE UNIQUE INDEX "APP"."UNIQUE_WM_MAPPING" ON "APP"."WM_MAPPING" ("RP_ID", "E
 
 CREATE UNIQUE INDEX "APP"."MV_UNIQUE_TABLE" ON "APP"."MV_CREATION_METADATA" ("TBL_NAME", "DB_NAME");
 
+CREATE UNIQUE INDEX "APP"."UNIQUE_CATALOG" ON "APP"."CTLGS" ("NAME");
+
+
 -- ----------------------------------------------
 -- DDL Statements for keys
 -- ----------------------------------------------
@@ -289,6 +369,9 @@ ALTER TABLE "APP"."METASTORE_DB_PROPERTIES" ADD CONSTRAINT "PROPERTY_KEY_PK" PRI
 
 ALTER TABLE "APP"."MV_CREATION_METADATA" ADD CONSTRAINT "MV_CREATION_METADATA_PK" PRIMARY KEY ("MV_CREATION_METADATA_ID");
 
+ALTER TABLE "APP"."CTLGS" ADD CONSTRAINT "CTLG_PK" PRIMARY KEY ("CTLG_ID");
+
+
 -- foreign
 ALTER TABLE "APP"."IDXS" ADD CONSTRAINT "IDXS_FK1" FOREIGN KEY ("ORIG_TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION;
 
@@ -322,6 +405,8 @@ ALTER TABLE "APP"."TBLS" ADD CONSTRAINT "TBLS_FK2" FOREIGN KEY ("SD_ID") REFEREN
 
 ALTER TABLE "APP"."TBLS" ADD CONSTRAINT "TBLS_FK1" FOREIGN KEY ("DB_ID") REFERENCES "APP"."DBS" ("DB_ID") ON DELETE NO ACTION ON UPDATE NO ACTION;
 
+ALTER TABLE "APP"."DBS" ADD CONSTRAINT "DBS_FK1" FOREIGN KEY ("CTLG_NAME") REFERENCES "APP"."CTLGS" ("NAME") ON DELETE NO ACTION ON UPDATE NO ACTION;
+
 ALTER TABLE "APP"."SD_PARAMS" ADD CONSTRAINT "SD_PARAMS_FK1" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION;
 
 ALTER TABLE "APP"."DATABASE_PARAMS" ADD CONSTRAINT "DATABASE_PARAMS_FK1" FOREIGN KEY ("DB_ID") REFERENCES "APP"."DBS" ("DB_ID") ON DELETE NO ACTION ON UPDATE NO ACTION;
@@ -394,6 +479,8 @@ ALTER TABLE "APP"."MV_TABLES_USED" ADD CONSTRAINT "MV_TABLES_USED_FK1" FOREIGN K
 
 ALTER TABLE "APP"."MV_TABLES_USED" ADD CONSTRAINT "MV_TABLES_USED_FK2" FOREIGN KEY ("TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION;
 
+ALTER TABLE "APP"."DBS" ADD CONSTRAINT "DBS_CTLG_FK" FOREIGN KEY ("CTLG_NAME") REFERENCES "APP"."CTLGS" ("NAME") ON DELETE NO ACTION ON UPDATE NO ACTION;
+
 -- ----------------------------------------------
 -- DDL Statements for checks
 -- ----------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql b/standalone-metastore/src/main/sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql
index 8925fa8..6aa2e82 100644
--- a/standalone-metastore/src/main/sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql
+++ b/standalone-metastore/src/main/sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql
@@ -81,6 +81,7 @@ CREATE UNIQUE INDEX "APP"."UNIQUE_SCHEMA_VERSION" ON "APP"."SCHEMA_VERSION" ("SC
 -- create mv_creation_metadata table
 CREATE TABLE "APP"."MV_CREATION_METADATA" (
   "MV_CREATION_METADATA_ID" BIGINT NOT NULL,
+  "CAT_NAME" VARCHAR(256) NOT NULL,
   "DB_NAME" VARCHAR(128) NOT NULL,
   "TBL_NAME" VARCHAR(256) NOT NULL,
   "TXN_LIST" CLOB
@@ -160,5 +161,62 @@ ALTER TABLE "APP"."KEY_CONSTRAINTS" ADD COLUMN "DEFAULT_VALUE" VARCHAR(400);
 
 ALTER TABLE "APP"."HIVE_LOCKS" ALTER COLUMN "HL_TXNID" NOT NULL;
 
+-- Create new Catalog table
+-- HIVE-18755, add catalogs
+-- new catalogs table
+CREATE TABLE "APP"."CTLGS" (
+    "CTLG_ID" BIGINT NOT NULL,
+    "NAME" VARCHAR(256) UNIQUE,
+    "DESC" VARCHAR(4000),
+    "LOCATION_URI" VARCHAR(4000) NOT NULL);
+
+ALTER TABLE "APP"."CTLGS" ADD CONSTRAINT "CTLGS_PK" PRIMARY KEY ("CTLG_ID");
+
+-- Insert a default value.  The location is TBD.  Hive will fix this when it starts
+INSERT INTO "APP"."CTLGS" VALUES (1, 'hive', 'Default catalog for Hive', 'TBD');
+
+-- Drop the unique index on DBS
+DROP INDEX "APP"."UNIQUE_DATABASE";
+
+-- Add the new column to the DBS table, can't put in the not null constraint yet
+ALTER TABLE "APP"."DBS" ADD COLUMN "CTLG_NAME" VARCHAR(256);
+
+-- Update all records in the DBS table to point to the Hive catalog
+UPDATE "APP"."DBS" 
+  SET "CTLG_NAME" = 'hive';
+
+-- Add the not null constraint
+ALTER TABLE "APP"."DBS" ALTER COLUMN "CTLG_NAME" NOT NULL;
+
+-- Put back the unique index 
+CREATE UNIQUE INDEX "APP"."UNIQUE_DATABASE" ON "APP"."DBS" ("NAME", "CTLG_NAME");
+
+-- Add the foreign key
+ALTER TABLE "APP"."DBS" ADD CONSTRAINT "DBS_FK1" FOREIGN KEY ("CTLG_NAME") REFERENCES "APP"."CTLGS" ("NAME") ON DELETE NO ACTION ON UPDATE NO ACTION;
+
+-- Add columns to table stats and part stats
+ALTER TABLE "APP"."TAB_COL_STATS" ADD COLUMN "CAT_NAME" VARCHAR(256);
+ALTER TABLE "APP"."PART_COL_STATS" ADD COLUMN "CAT_NAME" VARCHAR(256);
+
+-- Set the existing column names to Hive
+UPDATE "APP"."TAB_COL_STATS"
+  SET "CAT_NAME" = 'hive';
+UPDATE "APP"."PART_COL_STATS"
+  SET "CAT_NAME" = 'hive';
+
+-- Add the not null constraint
+ALTER TABLE "APP"."TAB_COL_STATS" ALTER COLUMN "CAT_NAME" NOT NULL;
+ALTER TABLE "APP"."PART_COL_STATS" ALTER COLUMN "CAT_NAME" NOT NULL;
+
+-- Rebuild the index for Part col stats.  No such index for table stats, which seems weird
+DROP INDEX "APP"."PCS_STATS_IDX";
+CREATE INDEX "APP"."PCS_STATS_IDX" ON "APP"."PART_COL_STATS" ("CAT_NAME", "DB_NAME","TABLE_NAME","COLUMN_NAME","PARTITION_NAME");
+
+-- Add column to partition events
+ALTER TABLE "APP"."PARTITION_EVENTS" ADD COLUMN "CAT_NAME" VARCHAR(256);
+
+-- Add column to notification log
+ALTER TABLE "APP"."NOTIFICATION_LOG" ADD COLUMN "CAT_NAME" VARCHAR(256);
+
 -- This needs to be the last thing done.  Insert any changes above this line.
 UPDATE "APP".VERSION SET SCHEMA_VERSION='3.0.0', VERSION_COMMENT='Hive release version 3.0.0' where VER_ID=1;

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/sql/mssql/hive-schema-3.0.0.mssql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/mssql/hive-schema-3.0.0.mssql.sql b/standalone-metastore/src/main/sql/mssql/hive-schema-3.0.0.mssql.sql
index 68237ec..77afd60 100644
--- a/standalone-metastore/src/main/sql/mssql/hive-schema-3.0.0.mssql.sql
+++ b/standalone-metastore/src/main/sql/mssql/hive-schema-3.0.0.mssql.sql
@@ -93,12 +93,13 @@ CREATE TABLE PART_COL_STATS
     NUM_TRUES bigint NULL,
     PART_ID bigint NULL,
     PARTITION_NAME nvarchar(767) NOT NULL,
-    "TABLE_NAME" nvarchar(256) NOT NULL
+    "TABLE_NAME" nvarchar(256) NOT NULL,
+    "CAT_NAME" nvarchar(256) NOT NULL
 );
 
 ALTER TABLE PART_COL_STATS ADD CONSTRAINT PART_COL_STATS_PK PRIMARY KEY (CS_ID);
 
-CREATE INDEX PCS_STATS_IDX ON PART_COL_STATS (DB_NAME,TABLE_NAME,COLUMN_NAME,PARTITION_NAME);
+CREATE INDEX PCS_STATS_IDX ON PART_COL_STATS (CAT_NAME, DB_NAME,TABLE_NAME,COLUMN_NAME,PARTITION_NAME);
 
 -- Table PART_PRIVS for classes [org.apache.hadoop.hive.metastore.model.MPartitionPrivilege]
 CREATE TABLE PART_PRIVS
@@ -236,7 +237,8 @@ CREATE TABLE TAB_COL_STATS
     NUM_NULLS bigint NOT NULL,
     NUM_TRUES bigint NULL,
     TBL_ID bigint NULL,
-    "TABLE_NAME" nvarchar(256) NOT NULL
+    "TABLE_NAME" nvarchar(256) NOT NULL,
+    "CAT_NAME" nvarchar(256) NOT NULL
 );
 
 ALTER TABLE TAB_COL_STATS ADD CONSTRAINT TAB_COL_STATS_PK PRIMARY KEY (CS_ID);
@@ -276,7 +278,8 @@ CREATE TABLE DBS
     DB_LOCATION_URI nvarchar(4000) NOT NULL,
     "NAME" nvarchar(128) NULL,
     OWNER_NAME nvarchar(128) NULL,
-    OWNER_TYPE nvarchar(10) NULL
+    OWNER_TYPE nvarchar(10) NULL,
+    CTLG_NAME nvarchar(256)
 );
 
 ALTER TABLE DBS ADD CONSTRAINT DBS_PK PRIMARY KEY (DB_ID);
@@ -374,6 +377,7 @@ ALTER TABLE TBLS ADD CONSTRAINT TBLS_PK PRIMARY KEY (TBL_ID);
 CREATE TABLE MV_CREATION_METADATA
 (
     MV_CREATION_METADATA_ID bigint NOT NULL,
+    CAT_NAME nvarchar(256) NOT NULL,
     DB_NAME nvarchar(128) NOT NULL,
     TBL_NAME nvarchar(256) NOT NULL,
     TXN_LIST text NULL
@@ -382,6 +386,7 @@ CREATE TABLE MV_CREATION_METADATA
 ALTER TABLE MV_CREATION_METADATA ADD CONSTRAINT MV_CREATION_METADATA_PK PRIMARY KEY (MV_CREATION_METADATA_ID);
 CREATE INDEX MV_UNIQUE_TABLE ON MV_CREATION_METADATA (TBL_NAME,DB_NAME);
 
+
 CREATE TABLE MV_TABLES_USED
 (
     MV_CREATION_METADATA_ID bigint NOT NULL,
@@ -411,6 +416,7 @@ ALTER TABLE SDS ADD CONSTRAINT SDS_PK PRIMARY KEY (SD_ID);
 CREATE TABLE PARTITION_EVENTS
 (
     PART_NAME_ID bigint NOT NULL,
+    CAT_NAME nvarchar(256) NULL,
     DB_NAME nvarchar(128) NULL,
     EVENT_TIME bigint NOT NULL,
     EVENT_TYPE int NOT NULL,
@@ -604,6 +610,7 @@ CREATE TABLE NOTIFICATION_LOG
     EVENT_ID bigint NOT NULL,
     EVENT_TIME int NOT NULL,
     EVENT_TYPE nvarchar(32) NOT NULL,
+    CAT_NAME nvarchar(128) NULL,
     DB_NAME nvarchar(128) NULL,
     TBL_NAME nvarchar(256) NULL,
     MESSAGE_FORMAT nvarchar(16),
@@ -677,6 +684,15 @@ CREATE TABLE WM_MAPPING
 
 ALTER TABLE WM_MAPPING ADD CONSTRAINT WM_MAPPING_PK PRIMARY KEY (MAPPING_ID);
 
+CREATE TABLE CTLGS (
+      CTLG_ID bigint primary key,
+      "NAME" nvarchar(256),
+      "DESC" nvarchar(4000),
+      LOCATION_URI nvarchar(4000) not null
+);
+
+CREATE UNIQUE INDEX UNIQUE_CTLG ON CTLGS ("NAME");
+
 -- Constraints for table MASTER_KEYS for class(es) [org.apache.hadoop.hive.metastore.model.MMasterKey]
 
 -- Constraints for table IDXS for class(es) [org.apache.hadoop.hive.metastore.model.MIndex]
@@ -770,7 +786,7 @@ CREATE INDEX TABLEPRIVILEGEINDEX ON TBL_PRIVS (TBL_ID,PRINCIPAL_NAME,PRINCIPAL_T
 
 
 -- Constraints for table DBS for class(es) [org.apache.hadoop.hive.metastore.model.MDatabase]
-CREATE UNIQUE INDEX UNIQUEDATABASE ON DBS ("NAME");
+CREATE UNIQUE INDEX UNIQUEDATABASE ON DBS ("NAME", "CTLG_NAME");
 
 
 -- Constraints for table TBL_COL_PRIVS for class(es) [org.apache.hadoop.hive.metastore.model.MTableColumnPrivilege]
@@ -958,6 +974,7 @@ CREATE UNIQUE INDEX UNIQUE_WM_MAPPING ON WM_MAPPING (RP_ID, ENTITY_TYPE, ENTITY_
 
 ALTER TABLE WM_MAPPING ADD CONSTRAINT WM_MAPPING_FK1 FOREIGN KEY (RP_ID) REFERENCES WM_RESOURCEPLAN (RP_ID);
 
+ALTER TABLE DBS ADD CONSTRAINT "DBS_FK1" FOREIGN KEY ("CTLG_NAME") REFERENCES CTLGS ("NAME");
 -- -----------------------------------------------------------------------------------------------------------------------------------------------
 -- Transaction and Lock Tables
 -- These are not part of package jdo, so if you are going to regenerate this file you need to manually add the following section back to the file.

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/sql/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql b/standalone-metastore/src/main/sql/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql
index c5041b3..b780329 100644
--- a/standalone-metastore/src/main/sql/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql
+++ b/standalone-metastore/src/main/sql/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql
@@ -140,6 +140,7 @@ CREATE TABLE "SCHEMA_VERSION" (
 CREATE TABLE MV_CREATION_METADATA
 (
     MV_CREATION_METADATA_ID bigint NOT NULL,
+    CAT_NAME nvarchar(256) NOT NULL,
     DB_NAME nvarchar(128) NOT NULL,
     TBL_NAME nvarchar(256) NOT NULL,
     TXN_LIST text NULL
@@ -212,6 +213,64 @@ ALTER TABLE COMPLETED_TXN_COMPONENTS ADD CTC_WRITEID bigint;
 
 ALTER TABLE HIVE_LOCKS ALTER COLUMN HL_TXNID bigint NOT NULL;
 
+-- HIVE-18755, add catalogs
+-- new catalog table
+CREATE TABLE CTLGS (
+      CTLG_ID bigint primary key,
+      "NAME" nvarchar(256),
+      "DESC" nvarchar(4000),
+      LOCATION_URI nvarchar(4000) not null
+);
+
+-- Create unique index on CTLGS.NAME
+CREATE UNIQUE INDEX UNIQUE_CTLG ON CTLGS ("NAME");
+
+-- Insert a default value.  The location is TBD.  Hive will fix this when it starts
+INSERT INTO CTLGS VALUES (1, 'hive', 'Default catalog for Hive', 'TBD');
+
+-- Drop the unique index on DBS
+DROP INDEX UNIQUEDATABASE ON DBS;
+
+-- Add the new column to the DBS table, can't put in the not null constraint yet
+ALTER TABLE DBS ADD CTLG_NAME nvarchar(256);
+
+-- Update all records in the DBS table to point to the Hive catalog
+UPDATE DBS
+  SET "CTLG_NAME" = 'hive';
+
+-- Add the not null constraint
+ALTER TABLE DBS ALTER COLUMN CTLG_NAME nvarchar(256) NOT NULL;
+
+-- Put back the unique index
+CREATE UNIQUE INDEX UNIQUEDATABASE ON DBS ("NAME", "CTLG_NAME");
+
+-- Add the foreign key
+ALTER TABLE DBS ADD CONSTRAINT "DBS_FK1" FOREIGN KEY ("CTLG_NAME") REFERENCES CTLGS ("NAME");
+
+-- Add columns to table stats and part stats
+ALTER TABLE TAB_COL_STATS ADD CAT_NAME nvarchar(256);
+ALTER TABLE PART_COL_STATS ADD CAT_NAME nvarchar(256);
+
+-- Set the existing column names to Hive
+UPDATE TAB_COL_STATS
+  SET CAT_NAME = 'hive';
+UPDATE PART_COL_STATS
+  SET CAT_NAME = 'hive';
+
+-- Add the not null constraint
+ALTER TABLE TAB_COL_STATS ALTER COLUMN CAT_NAME nvarchar(256) NOT NULL;
+ALTER TABLE PART_COL_STATS ALTER COLUMN CAT_NAME nvarchar(256) NOT NULL;
+
+-- Rebuild the index for Part col stats.  No such index for table stats, which seems weird
+DROP INDEX PCS_STATS_IDX ON PART_COL_STATS;
+CREATE INDEX PCS_STATS_IDX ON PART_COL_STATS (CAT_NAME, DB_NAME, TABLE_NAME, COLUMN_NAME, PARTITION_NAME);
+
+-- Add columns to partition events
+ALTER TABLE PARTITION_EVENTS ADD CAT_NAME nvarchar(256);
+
+-- Add columns to notification log
+ALTER TABLE NOTIFICATION_LOG ADD CAT_NAME nvarchar(256);
+
 -- These lines need to be last.  Insert any changes above.
 UPDATE VERSION SET SCHEMA_VERSION='3.0.0', VERSION_COMMENT='Hive release version 3.0.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.3.0 to 3.0.0' AS MESSAGE;

http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/main/sql/mysql/hive-schema-3.0.0.mysql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/mysql/hive-schema-3.0.0.mysql.sql b/standalone-metastore/src/main/sql/mysql/hive-schema-3.0.0.mysql.sql
index 3e2db2a..adbe129 100644
--- a/standalone-metastore/src/main/sql/mysql/hive-schema-3.0.0.mysql.sql
+++ b/standalone-metastore/src/main/sql/mysql/hive-schema-3.0.0.mysql.sql
@@ -77,6 +77,15 @@ CREATE TABLE IF NOT EXISTS `DATABASE_PARAMS` (
 ) ENGINE=InnoDB DEFAULT CHARSET=latin1;
 /*!40101 SET character_set_client = @saved_cs_client */;
 
+CREATE TABLE `CTLGS` (
+    `CTLG_ID` BIGINT PRIMARY KEY,
+    `NAME` VARCHAR(256),
+    `DESC` VARCHAR(4000),
+    `LOCATION_URI` VARCHAR(4000) NOT NULL,
+    UNIQUE KEY `UNIQUE_CATALOG` (`NAME`)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
+
 --
 -- Table structure for table `DBS`
 --
@@ -90,8 +99,10 @@ CREATE TABLE IF NOT EXISTS `DBS` (
   `NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
   `OWNER_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
   `OWNER_TYPE` varchar(10) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+  `CTLG_NAME` varchar(256) NOT NULL,
   PRIMARY KEY (`DB_ID`),
-  UNIQUE KEY `UNIQUE_DATABASE` (`NAME`)
+  UNIQUE KEY `UNIQUE_DATABASE` (`NAME`, `CTLG_NAME`),
+  CONSTRAINT `CTLG_FK1` FOREIGN KEY (`CTLG_NAME`) REFERENCES `CTLGS` (`NAME`)
 ) ENGINE=InnoDB DEFAULT CHARSET=latin1;
 /*!40101 SET character_set_client = @saved_cs_client */;
 
@@ -228,6 +239,7 @@ CREATE TABLE IF NOT EXISTS `PARTITIONS` (
 /*!40101 SET character_set_client = utf8 */;
 CREATE TABLE IF NOT EXISTS `PARTITION_EVENTS` (
   `PART_NAME_ID` bigint(20) NOT NULL,
+  `CAT_NAME` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
   `DB_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
   `EVENT_TIME` bigint(20) NOT NULL,
   `EVENT_TYPE` int(11) NOT NULL,
@@ -581,6 +593,7 @@ CREATE TABLE IF NOT EXISTS `TABLE_PARAMS` (
 /*!40101 SET character_set_client = utf8 */;
 CREATE TABLE IF NOT EXISTS `MV_CREATION_METADATA` (
   `MV_CREATION_METADATA_ID` bigint(20) NOT NULL,
+  `CAT_NAME` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
   `DB_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
   `TBL_NAME` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
   `TXN_LIST` TEXT DEFAULT NULL,
@@ -684,6 +697,7 @@ CREATE TABLE IF NOT EXISTS `TBL_PRIVS` (
 --
 CREATE TABLE IF NOT EXISTS `TAB_COL_STATS` (
  `CS_ID` bigint(20) NOT NULL,
+ `CAT_NAME` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
  `DB_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
  `TABLE_NAME` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
  `COLUMN_NAME` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
@@ -712,6 +726,7 @@ CREATE TABLE IF NOT EXISTS `TAB_COL_STATS` (
 --
 CREATE TABLE IF NOT EXISTS `PART_COL_STATS` (
  `CS_ID` bigint(20) NOT NULL,
+ `CAT_NAME` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
  `DB_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
  `TABLE_NAME` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
  `PARTITION_NAME` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
@@ -736,7 +751,7 @@ CREATE TABLE IF NOT EXISTS `PART_COL_STATS` (
   CONSTRAINT `PART_COL_STATS_FK` FOREIGN KEY (`PART_ID`) REFERENCES `PARTITIONS` (`PART_ID`)
 ) ENGINE=InnoDB DEFAULT CHARSET=latin1;
 
-CREATE INDEX PCS_STATS_IDX ON PART_COL_STATS (DB_NAME,TABLE_NAME,COLUMN_NAME,PARTITION_NAME) USING BTREE;
+CREATE INDEX PCS_STATS_IDX ON PART_COL_STATS (CAT_NAME, DB_NAME,TABLE_NAME,COLUMN_NAME,PARTITION_NAME) USING BTREE;
 
 --
 -- Table structure for table `TYPES`
@@ -833,6 +848,7 @@ CREATE TABLE IF NOT EXISTS `NOTIFICATION_LOG`
     `EVENT_ID` BIGINT(20) NOT NULL,
     `EVENT_TIME` INT(11) NOT NULL,
     `EVENT_TYPE` varchar(32) NOT NULL,
+    `CAT_NAME` varchar(256),
     `DB_NAME` varchar(128),
     `TBL_NAME` varchar(256),
     `MESSAGE` longtext,


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ba8a99e1/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php b/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
index 32c8cb7..49f71b5 100644
--- a/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
+++ b/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
@@ -511,6 +511,10 @@ class SQLPrimaryKey {
    * @var bool
    */
   public $rely_cstr = null;
+  /**
+   * @var string
+   */
+  public $catName = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -547,6 +551,10 @@ class SQLPrimaryKey {
           'var' => 'rely_cstr',
           'type' => TType::BOOL,
           ),
+        9 => array(
+          'var' => 'catName',
+          'type' => TType::STRING,
+          ),
         );
     }
     if (is_array($vals)) {
@@ -574,6 +582,9 @@ class SQLPrimaryKey {
       if (isset($vals['rely_cstr'])) {
         $this->rely_cstr = $vals['rely_cstr'];
       }
+      if (isset($vals['catName'])) {
+        $this->catName = $vals['catName'];
+      }
     }
   }
 
@@ -652,6 +663,13 @@ class SQLPrimaryKey {
             $xfer += $input->skip($ftype);
           }
           break;
+        case 9:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->catName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
         default:
           $xfer += $input->skip($ftype);
           break;
@@ -705,6 +723,11 @@ class SQLPrimaryKey {
       $xfer += $output->writeBool($this->rely_cstr);
       $xfer += $output->writeFieldEnd();
     }
+    if ($this->catName !== null) {
+      $xfer += $output->writeFieldBegin('catName', TType::STRING, 9);
+      $xfer += $output->writeString($this->catName);
+      $xfer += $output->writeFieldEnd();
+    }
     $xfer += $output->writeFieldStop();
     $xfer += $output->writeStructEnd();
     return $xfer;
@@ -771,6 +794,10 @@ class SQLForeignKey {
    * @var bool
    */
   public $rely_cstr = null;
+  /**
+   * @var string
+   */
+  public $catName = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -831,6 +858,10 @@ class SQLForeignKey {
           'var' => 'rely_cstr',
           'type' => TType::BOOL,
           ),
+        15 => array(
+          'var' => 'catName',
+          'type' => TType::STRING,
+          ),
         );
     }
     if (is_array($vals)) {
@@ -876,6 +907,9 @@ class SQLForeignKey {
       if (isset($vals['rely_cstr'])) {
         $this->rely_cstr = $vals['rely_cstr'];
       }
+      if (isset($vals['catName'])) {
+        $this->catName = $vals['catName'];
+      }
     }
   }
 
@@ -996,6 +1030,13 @@ class SQLForeignKey {
             $xfer += $input->skip($ftype);
           }
           break;
+        case 15:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->catName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
         default:
           $xfer += $input->skip($ftype);
           break;
@@ -1079,6 +1120,11 @@ class SQLForeignKey {
       $xfer += $output->writeBool($this->rely_cstr);
       $xfer += $output->writeFieldEnd();
     }
+    if ($this->catName !== null) {
+      $xfer += $output->writeFieldBegin('catName', TType::STRING, 15);
+      $xfer += $output->writeString($this->catName);
+      $xfer += $output->writeFieldEnd();
+    }
     $xfer += $output->writeFieldStop();
     $xfer += $output->writeStructEnd();
     return $xfer;
@@ -1092,6 +1138,10 @@ class SQLUniqueConstraint {
   /**
    * @var string
    */
+  public $catName = null;
+  /**
+   * @var string
+   */
   public $table_db = null;
   /**
    * @var string
@@ -1126,40 +1176,47 @@ class SQLUniqueConstraint {
     if (!isset(self::$_TSPEC)) {
       self::$_TSPEC = array(
         1 => array(
-          'var' => 'table_db',
+          'var' => 'catName',
           'type' => TType::STRING,
           ),
         2 => array(
-          'var' => 'table_name',
+          'var' => 'table_db',
           'type' => TType::STRING,
           ),
         3 => array(
-          'var' => 'column_name',
+          'var' => 'table_name',
           'type' => TType::STRING,
           ),
         4 => array(
+          'var' => 'column_name',
+          'type' => TType::STRING,
+          ),
+        5 => array(
           'var' => 'key_seq',
           'type' => TType::I32,
           ),
-        5 => array(
+        6 => array(
           'var' => 'uk_name',
           'type' => TType::STRING,
           ),
-        6 => array(
+        7 => array(
           'var' => 'enable_cstr',
           'type' => TType::BOOL,
           ),
-        7 => array(
+        8 => array(
           'var' => 'validate_cstr',
           'type' => TType::BOOL,
           ),
-        8 => array(
+        9 => array(
           'var' => 'rely_cstr',
           'type' => TType::BOOL,
           ),
         );
     }
     if (is_array($vals)) {
+      if (isset($vals['catName'])) {
+        $this->catName = $vals['catName'];
+      }
       if (isset($vals['table_db'])) {
         $this->table_db = $vals['table_db'];
       }
@@ -1208,54 +1265,61 @@ class SQLUniqueConstraint {
       {
         case 1:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->table_db);
+            $xfer += $input->readString($this->catName);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 2:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->table_name);
+            $xfer += $input->readString($this->table_db);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 3:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->column_name);
+            $xfer += $input->readString($this->table_name);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 4:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->column_name);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 5:
           if ($ftype == TType::I32) {
             $xfer += $input->readI32($this->key_seq);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 5:
+        case 6:
           if ($ftype == TType::STRING) {
             $xfer += $input->readString($this->uk_name);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 6:
+        case 7:
           if ($ftype == TType::BOOL) {
             $xfer += $input->readBool($this->enable_cstr);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 7:
+        case 8:
           if ($ftype == TType::BOOL) {
             $xfer += $input->readBool($this->validate_cstr);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 8:
+        case 9:
           if ($ftype == TType::BOOL) {
             $xfer += $input->readBool($this->rely_cstr);
           } else {
@@ -1275,43 +1339,48 @@ class SQLUniqueConstraint {
   public function write($output) {
     $xfer = 0;
     $xfer += $output->writeStructBegin('SQLUniqueConstraint');
+    if ($this->catName !== null) {
+      $xfer += $output->writeFieldBegin('catName', TType::STRING, 1);
+      $xfer += $output->writeString($this->catName);
+      $xfer += $output->writeFieldEnd();
+    }
     if ($this->table_db !== null) {
-      $xfer += $output->writeFieldBegin('table_db', TType::STRING, 1);
+      $xfer += $output->writeFieldBegin('table_db', TType::STRING, 2);
       $xfer += $output->writeString($this->table_db);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->table_name !== null) {
-      $xfer += $output->writeFieldBegin('table_name', TType::STRING, 2);
+      $xfer += $output->writeFieldBegin('table_name', TType::STRING, 3);
       $xfer += $output->writeString($this->table_name);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->column_name !== null) {
-      $xfer += $output->writeFieldBegin('column_name', TType::STRING, 3);
+      $xfer += $output->writeFieldBegin('column_name', TType::STRING, 4);
       $xfer += $output->writeString($this->column_name);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->key_seq !== null) {
-      $xfer += $output->writeFieldBegin('key_seq', TType::I32, 4);
+      $xfer += $output->writeFieldBegin('key_seq', TType::I32, 5);
       $xfer += $output->writeI32($this->key_seq);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->uk_name !== null) {
-      $xfer += $output->writeFieldBegin('uk_name', TType::STRING, 5);
+      $xfer += $output->writeFieldBegin('uk_name', TType::STRING, 6);
       $xfer += $output->writeString($this->uk_name);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->enable_cstr !== null) {
-      $xfer += $output->writeFieldBegin('enable_cstr', TType::BOOL, 6);
+      $xfer += $output->writeFieldBegin('enable_cstr', TType::BOOL, 7);
       $xfer += $output->writeBool($this->enable_cstr);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->validate_cstr !== null) {
-      $xfer += $output->writeFieldBegin('validate_cstr', TType::BOOL, 7);
+      $xfer += $output->writeFieldBegin('validate_cstr', TType::BOOL, 8);
       $xfer += $output->writeBool($this->validate_cstr);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->rely_cstr !== null) {
-      $xfer += $output->writeFieldBegin('rely_cstr', TType::BOOL, 8);
+      $xfer += $output->writeFieldBegin('rely_cstr', TType::BOOL, 9);
       $xfer += $output->writeBool($this->rely_cstr);
       $xfer += $output->writeFieldEnd();
     }
@@ -1328,6 +1397,10 @@ class SQLNotNullConstraint {
   /**
    * @var string
    */
+  public $catName = null;
+  /**
+   * @var string
+   */
   public $table_db = null;
   /**
    * @var string
@@ -1358,36 +1431,43 @@ class SQLNotNullConstraint {
     if (!isset(self::$_TSPEC)) {
       self::$_TSPEC = array(
         1 => array(
-          'var' => 'table_db',
+          'var' => 'catName',
           'type' => TType::STRING,
           ),
         2 => array(
-          'var' => 'table_name',
+          'var' => 'table_db',
           'type' => TType::STRING,
           ),
         3 => array(
-          'var' => 'column_name',
+          'var' => 'table_name',
           'type' => TType::STRING,
           ),
         4 => array(
-          'var' => 'nn_name',
+          'var' => 'column_name',
           'type' => TType::STRING,
           ),
         5 => array(
+          'var' => 'nn_name',
+          'type' => TType::STRING,
+          ),
+        6 => array(
           'var' => 'enable_cstr',
           'type' => TType::BOOL,
           ),
-        6 => array(
+        7 => array(
           'var' => 'validate_cstr',
           'type' => TType::BOOL,
           ),
-        7 => array(
+        8 => array(
           'var' => 'rely_cstr',
           'type' => TType::BOOL,
           ),
         );
     }
     if (is_array($vals)) {
+      if (isset($vals['catName'])) {
+        $this->catName = $vals['catName'];
+      }
       if (isset($vals['table_db'])) {
         $this->table_db = $vals['table_db'];
       }
@@ -1433,47 +1513,54 @@ class SQLNotNullConstraint {
       {
         case 1:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->table_db);
+            $xfer += $input->readString($this->catName);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 2:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->table_name);
+            $xfer += $input->readString($this->table_db);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 3:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->column_name);
+            $xfer += $input->readString($this->table_name);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 4:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->nn_name);
+            $xfer += $input->readString($this->column_name);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 5:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->nn_name);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 6:
           if ($ftype == TType::BOOL) {
             $xfer += $input->readBool($this->enable_cstr);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 6:
+        case 7:
           if ($ftype == TType::BOOL) {
             $xfer += $input->readBool($this->validate_cstr);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 7:
+        case 8:
           if ($ftype == TType::BOOL) {
             $xfer += $input->readBool($this->rely_cstr);
           } else {
@@ -1493,38 +1580,43 @@ class SQLNotNullConstraint {
   public function write($output) {
     $xfer = 0;
     $xfer += $output->writeStructBegin('SQLNotNullConstraint');
+    if ($this->catName !== null) {
+      $xfer += $output->writeFieldBegin('catName', TType::STRING, 1);
+      $xfer += $output->writeString($this->catName);
+      $xfer += $output->writeFieldEnd();
+    }
     if ($this->table_db !== null) {
-      $xfer += $output->writeFieldBegin('table_db', TType::STRING, 1);
+      $xfer += $output->writeFieldBegin('table_db', TType::STRING, 2);
       $xfer += $output->writeString($this->table_db);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->table_name !== null) {
-      $xfer += $output->writeFieldBegin('table_name', TType::STRING, 2);
+      $xfer += $output->writeFieldBegin('table_name', TType::STRING, 3);
       $xfer += $output->writeString($this->table_name);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->column_name !== null) {
-      $xfer += $output->writeFieldBegin('column_name', TType::STRING, 3);
+      $xfer += $output->writeFieldBegin('column_name', TType::STRING, 4);
       $xfer += $output->writeString($this->column_name);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->nn_name !== null) {
-      $xfer += $output->writeFieldBegin('nn_name', TType::STRING, 4);
+      $xfer += $output->writeFieldBegin('nn_name', TType::STRING, 5);
       $xfer += $output->writeString($this->nn_name);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->enable_cstr !== null) {
-      $xfer += $output->writeFieldBegin('enable_cstr', TType::BOOL, 5);
+      $xfer += $output->writeFieldBegin('enable_cstr', TType::BOOL, 6);
       $xfer += $output->writeBool($this->enable_cstr);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->validate_cstr !== null) {
-      $xfer += $output->writeFieldBegin('validate_cstr', TType::BOOL, 6);
+      $xfer += $output->writeFieldBegin('validate_cstr', TType::BOOL, 7);
       $xfer += $output->writeBool($this->validate_cstr);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->rely_cstr !== null) {
-      $xfer += $output->writeFieldBegin('rely_cstr', TType::BOOL, 7);
+      $xfer += $output->writeFieldBegin('rely_cstr', TType::BOOL, 8);
       $xfer += $output->writeBool($this->rely_cstr);
       $xfer += $output->writeFieldEnd();
     }
@@ -1541,6 +1633,10 @@ class SQLDefaultConstraint {
   /**
    * @var string
    */
+  public $catName = null;
+  /**
+   * @var string
+   */
   public $table_db = null;
   /**
    * @var string
@@ -1575,40 +1671,47 @@ class SQLDefaultConstraint {
     if (!isset(self::$_TSPEC)) {
       self::$_TSPEC = array(
         1 => array(
-          'var' => 'table_db',
+          'var' => 'catName',
           'type' => TType::STRING,
           ),
         2 => array(
-          'var' => 'table_name',
+          'var' => 'table_db',
           'type' => TType::STRING,
           ),
         3 => array(
-          'var' => 'column_name',
+          'var' => 'table_name',
           'type' => TType::STRING,
           ),
         4 => array(
-          'var' => 'default_value',
+          'var' => 'column_name',
           'type' => TType::STRING,
           ),
         5 => array(
-          'var' => 'dc_name',
+          'var' => 'default_value',
           'type' => TType::STRING,
           ),
         6 => array(
+          'var' => 'dc_name',
+          'type' => TType::STRING,
+          ),
+        7 => array(
           'var' => 'enable_cstr',
           'type' => TType::BOOL,
           ),
-        7 => array(
+        8 => array(
           'var' => 'validate_cstr',
           'type' => TType::BOOL,
           ),
-        8 => array(
+        9 => array(
           'var' => 'rely_cstr',
           'type' => TType::BOOL,
           ),
         );
     }
     if (is_array($vals)) {
+      if (isset($vals['catName'])) {
+        $this->catName = $vals['catName'];
+      }
       if (isset($vals['table_db'])) {
         $this->table_db = $vals['table_db'];
       }
@@ -1657,54 +1760,61 @@ class SQLDefaultConstraint {
       {
         case 1:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->table_db);
+            $xfer += $input->readString($this->catName);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 2:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->table_name);
+            $xfer += $input->readString($this->table_db);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 3:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->column_name);
+            $xfer += $input->readString($this->table_name);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 4:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->default_value);
+            $xfer += $input->readString($this->column_name);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 5:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->dc_name);
+            $xfer += $input->readString($this->default_value);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 6:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->dc_name);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 7:
           if ($ftype == TType::BOOL) {
             $xfer += $input->readBool($this->enable_cstr);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 7:
+        case 8:
           if ($ftype == TType::BOOL) {
             $xfer += $input->readBool($this->validate_cstr);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 8:
+        case 9:
           if ($ftype == TType::BOOL) {
             $xfer += $input->readBool($this->rely_cstr);
           } else {
@@ -1724,43 +1834,48 @@ class SQLDefaultConstraint {
   public function write($output) {
     $xfer = 0;
     $xfer += $output->writeStructBegin('SQLDefaultConstraint');
+    if ($this->catName !== null) {
+      $xfer += $output->writeFieldBegin('catName', TType::STRING, 1);
+      $xfer += $output->writeString($this->catName);
+      $xfer += $output->writeFieldEnd();
+    }
     if ($this->table_db !== null) {
-      $xfer += $output->writeFieldBegin('table_db', TType::STRING, 1);
+      $xfer += $output->writeFieldBegin('table_db', TType::STRING, 2);
       $xfer += $output->writeString($this->table_db);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->table_name !== null) {
-      $xfer += $output->writeFieldBegin('table_name', TType::STRING, 2);
+      $xfer += $output->writeFieldBegin('table_name', TType::STRING, 3);
       $xfer += $output->writeString($this->table_name);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->column_name !== null) {
-      $xfer += $output->writeFieldBegin('column_name', TType::STRING, 3);
+      $xfer += $output->writeFieldBegin('column_name', TType::STRING, 4);
       $xfer += $output->writeString($this->column_name);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->default_value !== null) {
-      $xfer += $output->writeFieldBegin('default_value', TType::STRING, 4);
+      $xfer += $output->writeFieldBegin('default_value', TType::STRING, 5);
       $xfer += $output->writeString($this->default_value);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->dc_name !== null) {
-      $xfer += $output->writeFieldBegin('dc_name', TType::STRING, 5);
+      $xfer += $output->writeFieldBegin('dc_name', TType::STRING, 6);
       $xfer += $output->writeString($this->dc_name);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->enable_cstr !== null) {
-      $xfer += $output->writeFieldBegin('enable_cstr', TType::BOOL, 6);
+      $xfer += $output->writeFieldBegin('enable_cstr', TType::BOOL, 7);
       $xfer += $output->writeBool($this->enable_cstr);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->validate_cstr !== null) {
-      $xfer += $output->writeFieldBegin('validate_cstr', TType::BOOL, 7);
+      $xfer += $output->writeFieldBegin('validate_cstr', TType::BOOL, 8);
       $xfer += $output->writeBool($this->validate_cstr);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->rely_cstr !== null) {
-      $xfer += $output->writeFieldBegin('rely_cstr', TType::BOOL, 8);
+      $xfer += $output->writeFieldBegin('rely_cstr', TType::BOOL, 9);
       $xfer += $output->writeBool($this->rely_cstr);
       $xfer += $output->writeFieldEnd();
     }
@@ -1777,6 +1892,10 @@ class SQLCheckConstraint {
   /**
    * @var string
    */
+  public $catName = null;
+  /**
+   * @var string
+   */
   public $table_db = null;
   /**
    * @var string
@@ -1811,40 +1930,47 @@ class SQLCheckConstraint {
     if (!isset(self::$_TSPEC)) {
       self::$_TSPEC = array(
         1 => array(
-          'var' => 'table_db',
+          'var' => 'catName',
           'type' => TType::STRING,
           ),
         2 => array(
-          'var' => 'table_name',
+          'var' => 'table_db',
           'type' => TType::STRING,
           ),
         3 => array(
-          'var' => 'column_name',
+          'var' => 'table_name',
           'type' => TType::STRING,
           ),
         4 => array(
-          'var' => 'check_expression',
+          'var' => 'column_name',
           'type' => TType::STRING,
           ),
         5 => array(
-          'var' => 'dc_name',
+          'var' => 'check_expression',
           'type' => TType::STRING,
           ),
         6 => array(
+          'var' => 'dc_name',
+          'type' => TType::STRING,
+          ),
+        7 => array(
           'var' => 'enable_cstr',
           'type' => TType::BOOL,
           ),
-        7 => array(
+        8 => array(
           'var' => 'validate_cstr',
           'type' => TType::BOOL,
           ),
-        8 => array(
+        9 => array(
           'var' => 'rely_cstr',
           'type' => TType::BOOL,
           ),
         );
     }
     if (is_array($vals)) {
+      if (isset($vals['catName'])) {
+        $this->catName = $vals['catName'];
+      }
       if (isset($vals['table_db'])) {
         $this->table_db = $vals['table_db'];
       }
@@ -1893,54 +2019,61 @@ class SQLCheckConstraint {
       {
         case 1:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->table_db);
+            $xfer += $input->readString($this->catName);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 2:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->table_name);
+            $xfer += $input->readString($this->table_db);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 3:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->column_name);
+            $xfer += $input->readString($this->table_name);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 4:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->check_expression);
+            $xfer += $input->readString($this->column_name);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 5:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->dc_name);
+            $xfer += $input->readString($this->check_expression);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 6:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->dc_name);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 7:
           if ($ftype == TType::BOOL) {
             $xfer += $input->readBool($this->enable_cstr);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 7:
+        case 8:
           if ($ftype == TType::BOOL) {
             $xfer += $input->readBool($this->validate_cstr);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 8:
+        case 9:
           if ($ftype == TType::BOOL) {
             $xfer += $input->readBool($this->rely_cstr);
           } else {
@@ -1960,43 +2093,48 @@ class SQLCheckConstraint {
   public function write($output) {
     $xfer = 0;
     $xfer += $output->writeStructBegin('SQLCheckConstraint');
+    if ($this->catName !== null) {
+      $xfer += $output->writeFieldBegin('catName', TType::STRING, 1);
+      $xfer += $output->writeString($this->catName);
+      $xfer += $output->writeFieldEnd();
+    }
     if ($this->table_db !== null) {
-      $xfer += $output->writeFieldBegin('table_db', TType::STRING, 1);
+      $xfer += $output->writeFieldBegin('table_db', TType::STRING, 2);
       $xfer += $output->writeString($this->table_db);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->table_name !== null) {
-      $xfer += $output->writeFieldBegin('table_name', TType::STRING, 2);
+      $xfer += $output->writeFieldBegin('table_name', TType::STRING, 3);
       $xfer += $output->writeString($this->table_name);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->column_name !== null) {
-      $xfer += $output->writeFieldBegin('column_name', TType::STRING, 3);
+      $xfer += $output->writeFieldBegin('column_name', TType::STRING, 4);
       $xfer += $output->writeString($this->column_name);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->check_expression !== null) {
-      $xfer += $output->writeFieldBegin('check_expression', TType::STRING, 4);
+      $xfer += $output->writeFieldBegin('check_expression', TType::STRING, 5);
       $xfer += $output->writeString($this->check_expression);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->dc_name !== null) {
-      $xfer += $output->writeFieldBegin('dc_name', TType::STRING, 5);
+      $xfer += $output->writeFieldBegin('dc_name', TType::STRING, 6);
       $xfer += $output->writeString($this->dc_name);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->enable_cstr !== null) {
-      $xfer += $output->writeFieldBegin('enable_cstr', TType::BOOL, 6);
+      $xfer += $output->writeFieldBegin('enable_cstr', TType::BOOL, 7);
       $xfer += $output->writeBool($this->enable_cstr);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->validate_cstr !== null) {
-      $xfer += $output->writeFieldBegin('validate_cstr', TType::BOOL, 7);
+      $xfer += $output->writeFieldBegin('validate_cstr', TType::BOOL, 8);
       $xfer += $output->writeBool($this->validate_cstr);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->rely_cstr !== null) {
-      $xfer += $output->writeFieldBegin('rely_cstr', TType::BOOL, 8);
+      $xfer += $output->writeFieldBegin('rely_cstr', TType::BOOL, 9);
       $xfer += $output->writeBool($this->rely_cstr);
       $xfer += $output->writeFieldEnd();
     }
@@ -2202,6 +2340,10 @@ class HiveObjectRef {
    * @var string
    */
   public $columnName = null;
+  /**
+   * @var string
+   */
+  public $catName = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -2230,6 +2372,10 @@ class HiveObjectRef {
           'var' => 'columnName',
           'type' => TType::STRING,
           ),
+        6 => array(
+          'var' => 'catName',
+          'type' => TType::STRING,
+          ),
         );
     }
     if (is_array($vals)) {
@@ -2248,6 +2394,9 @@ class HiveObjectRef {
       if (isset($vals['columnName'])) {
         $this->columnName = $vals['columnName'];
       }
+      if (isset($vals['catName'])) {
+        $this->catName = $vals['catName'];
+      }
     }
   }
 
@@ -2315,6 +2464,13 @@ class HiveObjectRef {
             $xfer += $input->skip($ftype);
           }
           break;
+        case 6:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->catName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
         default:
           $xfer += $input->skip($ftype);
           break;
@@ -2365,6 +2521,11 @@ class HiveObjectRef {
       $xfer += $output->writeString($this->columnName);
       $xfer += $output->writeFieldEnd();
     }
+    if ($this->catName !== null) {
+      $xfer += $output->writeFieldBegin('catName', TType::STRING, 6);
+      $xfer += $output->writeString($this->catName);
+      $xfer += $output->writeFieldEnd();
+    }
     $xfer += $output->writeFieldStop();
     $xfer += $output->writeStructEnd();
     return $xfer;
@@ -4296,7 +4457,7 @@ class GrantRevokeRoleResponse {
 
 }
 
-class Database {
+class Catalog {
   static $_TSPEC;
 
   /**
@@ -4311,22 +4472,6 @@ class Database {
    * @var string
    */
   public $locationUri = null;
-  /**
-   * @var array
-   */
-  public $parameters = null;
-  /**
-   * @var \metastore\PrincipalPrivilegeSet
-   */
-  public $privileges = null;
-  /**
-   * @var string
-   */
-  public $ownerName = null;
-  /**
-   * @var int
-   */
-  public $ownerType = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -4343,7 +4488,559 @@ class Database {
           'var' => 'locationUri',
           'type' => TType::STRING,
           ),
-        4 => array(
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['name'])) {
+        $this->name = $vals['name'];
+      }
+      if (isset($vals['description'])) {
+        $this->description = $vals['description'];
+      }
+      if (isset($vals['locationUri'])) {
+        $this->locationUri = $vals['locationUri'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'Catalog';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->name);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->description);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->locationUri);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('Catalog');
+    if ($this->name !== null) {
+      $xfer += $output->writeFieldBegin('name', TType::STRING, 1);
+      $xfer += $output->writeString($this->name);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->description !== null) {
+      $xfer += $output->writeFieldBegin('description', TType::STRING, 2);
+      $xfer += $output->writeString($this->description);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->locationUri !== null) {
+      $xfer += $output->writeFieldBegin('locationUri', TType::STRING, 3);
+      $xfer += $output->writeString($this->locationUri);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class CreateCatalogRequest {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\Catalog
+   */
+  public $catalog = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'catalog',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\Catalog',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['catalog'])) {
+        $this->catalog = $vals['catalog'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'CreateCatalogRequest';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->catalog = new \metastore\Catalog();
+            $xfer += $this->catalog->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('CreateCatalogRequest');
+    if ($this->catalog !== null) {
+      if (!is_object($this->catalog)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('catalog', TType::STRUCT, 1);
+      $xfer += $this->catalog->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class GetCatalogRequest {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $name = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'name',
+          'type' => TType::STRING,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['name'])) {
+        $this->name = $vals['name'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'GetCatalogRequest';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->name);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('GetCatalogRequest');
+    if ($this->name !== null) {
+      $xfer += $output->writeFieldBegin('name', TType::STRING, 1);
+      $xfer += $output->writeString($this->name);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class GetCatalogResponse {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\Catalog
+   */
+  public $catalog = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'catalog',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\Catalog',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['catalog'])) {
+        $this->catalog = $vals['catalog'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'GetCatalogResponse';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->catalog = new \metastore\Catalog();
+            $xfer += $this->catalog->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('GetCatalogResponse');
+    if ($this->catalog !== null) {
+      if (!is_object($this->catalog)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('catalog', TType::STRUCT, 1);
+      $xfer += $this->catalog->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class GetCatalogsResponse {
+  static $_TSPEC;
+
+  /**
+   * @var string[]
+   */
+  public $names = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'names',
+          'type' => TType::LST,
+          'etype' => TType::STRING,
+          'elem' => array(
+            'type' => TType::STRING,
+            ),
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['names'])) {
+        $this->names = $vals['names'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'GetCatalogsResponse';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::LST) {
+            $this->names = array();
+            $_size83 = 0;
+            $_etype86 = 0;
+            $xfer += $input->readListBegin($_etype86, $_size83);
+            for ($_i87 = 0; $_i87 < $_size83; ++$_i87)
+            {
+              $elem88 = null;
+              $xfer += $input->readString($elem88);
+              $this->names []= $elem88;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('GetCatalogsResponse');
+    if ($this->names !== null) {
+      if (!is_array($this->names)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('names', TType::LST, 1);
+      {
+        $output->writeListBegin(TType::STRING, count($this->names));
+        {
+          foreach ($this->names as $iter89)
+          {
+            $xfer += $output->writeString($iter89);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class DropCatalogRequest {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $name = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'name',
+          'type' => TType::STRING,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['name'])) {
+        $this->name = $vals['name'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'DropCatalogRequest';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->name);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('DropCatalogRequest');
+    if ($this->name !== null) {
+      $xfer += $output->writeFieldBegin('name', TType::STRING, 1);
+      $xfer += $output->writeString($this->name);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class Database {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $name = null;
+  /**
+   * @var string
+   */
+  public $description = null;
+  /**
+   * @var string
+   */
+  public $locationUri = null;
+  /**
+   * @var array
+   */
+  public $parameters = null;
+  /**
+   * @var \metastore\PrincipalPrivilegeSet
+   */
+  public $privileges = null;
+  /**
+   * @var string
+   */
+  public $ownerName = null;
+  /**
+   * @var int
+   */
+  public $ownerType = null;
+  /**
+   * @var string
+   */
+  public $catalogName = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'name',
+          'type' => TType::STRING,
+          ),
+        2 => array(
+          'var' => 'description',
+          'type' => TType::STRING,
+          ),
+        3 => array(
+          'var' => 'locationUri',
+          'type' => TType::STRING,
+          ),
+        4 => array(
           'var' => 'parameters',
           'type' => TType::MAP,
           'ktype' => TType::STRING,
@@ -4368,6 +5065,10 @@ class Database {
           'var' => 'ownerType',
           'type' => TType::I32,
           ),
+        8 => array(
+          'var' => 'catalogName',
+          'type' => TType::STRING,
+          ),
         );
     }
     if (is_array($vals)) {
@@ -4392,6 +5093,9 @@ class Database {
       if (isset($vals['ownerType'])) {
         $this->ownerType = $vals['ownerType'];
       }
+      if (isset($vals['catalogName'])) {
+        $this->catalogName = $vals['catalogName'];
+      }
     }
   }
 
@@ -4438,17 +5142,17 @@ class Database {
         case 4:
           if ($ftype == TType::MAP) {
             $this->parameters = array();
-            $_size83 = 0;
-            $_ktype84 = 0;
-            $_vtype85 = 0;
-            $xfer += $input->readMapBegin($_ktype84, $_vtype85, $_size83);
-            for ($_i87 = 0; $_i87 < $_size83; ++$_i87)
+            $_size90 = 0;
+            $_ktype91 = 0;
+            $_vtype92 = 0;
+            $xfer += $input->readMapBegin($_ktype91, $_vtype92, $_size90);
+            for ($_i94 = 0; $_i94 < $_size90; ++$_i94)
             {
-              $key88 = '';
-              $val89 = '';
-              $xfer += $input->readString($key88);
-              $xfer += $input->readString($val89);
-              $this->parameters[$key88] = $val89;
+              $key95 = '';
+              $val96 = '';
+              $xfer += $input->readString($key95);
+              $xfer += $input->readString($val96);
+              $this->parameters[$key95] = $val96;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -4477,6 +5181,13 @@ class Database {
             $xfer += $input->skip($ftype);
           }
           break;
+        case 8:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->catalogName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
         default:
           $xfer += $input->skip($ftype);
           break;
@@ -4513,10 +5224,10 @@ class Database {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->parameters));
         {
-          foreach ($this->parameters as $kiter90 => $viter91)
+          foreach ($this->parameters as $kiter97 => $viter98)
           {
-            $xfer += $output->writeString($kiter90);
-            $xfer += $output->writeString($viter91);
+            $xfer += $output->writeString($kiter97);
+            $xfer += $output->writeString($viter98);
           }
         }
         $output->writeMapEnd();
@@ -4541,6 +5252,11 @@ class Database {
       $xfer += $output->writeI32($this->ownerType);
       $xfer += $output->writeFieldEnd();
     }
+    if ($this->catalogName !== null) {
+      $xfer += $output->writeFieldBegin('catalogName', TType::STRING, 8);
+      $xfer += $output->writeString($this->catalogName);
+      $xfer += $output->writeFieldEnd();
+    }
     $xfer += $output->writeFieldStop();
     $xfer += $output->writeStructEnd();
     return $xfer;
@@ -4682,17 +5398,17 @@ class SerDeInfo {
         case 3:
           if ($ftype == TType::MAP) {
             $this->parameters = array();
-            $_size92 = 0;
-            $_ktype93 = 0;
-            $_vtype94 = 0;
-            $xfer += $input->readMapBegin($_ktype93, $_vtype94, $_size92);
-            for ($_i96 = 0; $_i96 < $_size92; ++$_i96)
+            $_size99 = 0;
+            $_ktype100 = 0;
+            $_vtype101 = 0;
+            $xfer += $input->readMapBegin($_ktype100, $_vtype101, $_size99);
+            for ($_i103 = 0; $_i103 < $_size99; ++$_i103)
             {
-              $key97 = '';
-              $val98 = '';
-              $xfer += $input->readString($key97);
-              $xfer += $input->readString($val98);
-              $this->parameters[$key97] = $val98;
+              $key104 = '';
+              $val105 = '';
+              $xfer += $input->readString($key104);
+              $xfer += $input->readString($val105);
+              $this->parameters[$key104] = $val105;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -4758,10 +5474,10 @@ class SerDeInfo {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->parameters));
         {
-          foreach ($this->parameters as $kiter99 => $viter100)
+          foreach ($this->parameters as $kiter106 => $viter107)
           {
-            $xfer += $output->writeString($kiter99);
-            $xfer += $output->writeString($viter100);
+            $xfer += $output->writeString($kiter106);
+            $xfer += $output->writeString($viter107);
           }
         }
         $output->writeMapEnd();
@@ -4985,14 +5701,14 @@ class SkewedInfo {
         case 1:
           if ($ftype == TType::LST) {
             $this->skewedColNames = array();
-            $_size101 = 0;
-            $_etype104 = 0;
-            $xfer += $input->readListBegin($_etype104, $_size101);
-            for ($_i105 = 0; $_i105 < $_size101; ++$_i105)
+            $_size108 = 0;
+            $_etype111 = 0;
+            $xfer += $input->readListBegin($_etype111, $_size108);
+            for ($_i112 = 0; $_i112 < $_size108; ++$_i112)
             {
-              $elem106 = null;
-              $xfer += $input->readString($elem106);
-              $this->skewedColNames []= $elem106;
+              $elem113 = null;
+              $xfer += $input->readString($elem113);
+              $this->skewedColNames []= $elem113;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -5002,24 +5718,24 @@ class SkewedInfo {
         case 2:
           if ($ftype == TType::LST) {
             $this->skewedColValues = array();
-            $_size107 = 0;
-            $_etype110 = 0;
-            $xfer += $input->readListBegin($_etype110, $_size107);
-            for ($_i111 = 0; $_i111 < $_size107; ++$_i111)
+            $_size114 = 0;
+            $_etype117 = 0;
+            $xfer += $input->readListBegin($_etype117, $_size114);
+            for ($_i118 = 0; $_i118 < $_size114; ++$_i118)
             {
-              $elem112 = null;
-              $elem112 = array();
-              $_size113 = 0;
-              $_etype116 = 0;
-              $xfer += $input->readListBegin($_etype116, $_size113);
-              for ($_i117 = 0; $_i117 < $_size113; ++$_i117)
+              $elem119 = null;
+              $elem119 = array();
+              $_size120 = 0;
+              $_etype123 = 0;
+              $xfer += $input->readListBegin($_etype123, $_size120);
+              for ($_i124 = 0; $_i124 < $_size120; ++$_i124)
               {
-                $elem118 = null;
-                $xfer += $input->readString($elem118);
-                $elem112 []= $elem118;
+                $elem125 = null;
+                $xfer += $input->readString($elem125);
+                $elem119 []= $elem125;
               }
               $xfer += $input->readListEnd();
-              $this->skewedColValues []= $elem112;
+              $this->skewedColValues []= $elem119;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -5029,27 +5745,27 @@ class SkewedInfo {
         case 3:
           if ($ftype == TType::MAP) {
             $this->skewedColValueLocationMaps = array();
-            $_size119 = 0;
-            $_ktype120 = 0;
-            $_vtype121 = 0;
-            $xfer += $input->readMapBegin($_ktype120, $_vtype121, $_size119);
-            for ($_i123 = 0; $_i123 < $_size119; ++$_i123)
+            $_size126 = 0;
+            $_ktype127 = 0;
+            $_vtype128 = 0;
+            $xfer += $input->readMapBegin($_ktype127, $_vtype128, $_size126);
+            for ($_i130 = 0; $_i130 < $_size126; ++$_i130)
             {
-              $key124 = array();
-              $val125 = '';
-              $key124 = array();
-              $_size126 = 0;
-              $_etype129 = 0;
-              $xfer += $input->readListBegin($_etype129, $_size126);
-              for ($_i130 = 0; $_i130 < $_size126; ++$_i130)
+              $key131 = array();
+              $val132 = '';
+              $key131 = array();
+              $_size133 = 0;
+              $_etype136 = 0;
+              $xfer += $input->readListBegin($_etype136, $_size133);
+              for ($_i137 = 0; $_i137 < $_size133; ++$_i137)
               {
-                $elem131 = null;
-                $xfer += $input->readString($elem131);
-                $key124 []= $elem131;
+                $elem138 = null;
+                $xfer += $input->readString($elem138);
+                $key131 []= $elem138;
               }
               $xfer += $input->readListEnd();
-              $xfer += $input->readString($val125);
-              $this->skewedColValueLocationMaps[$key124] = $val125;
+              $xfer += $input->readString($val132);
+              $this->skewedColValueLocationMaps[$key131] = $val132;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -5077,9 +5793,9 @@ class SkewedInfo {
       {
         $output->writeListBegin(TType::STRING, count($this->skewedColNames));
         {
-          foreach ($this->skewedColNames as $iter132)
+          foreach ($this->skewedColNames as $iter139)
           {
-            $xfer += $output->writeString($iter132);
+            $xfer += $output->writeString($iter139);
           }
         }
         $output->writeListEnd();
@@ -5094,14 +5810,14 @@ class SkewedInfo {
       {
         $output->writeListBegin(TType::LST, count($this->skewedColValues));
         {
-          foreach ($this->skewedColValues as $iter133)
+          foreach ($this->skewedColValues as $iter140)
           {
             {
-              $output->writeListBegin(TType::STRING, count($iter133));
+              $output->writeListBegin(TType::STRING, count($iter140));
               {
-                foreach ($iter133 as $iter134)
+                foreach ($iter140 as $iter141)
                 {
-                  $xfer += $output->writeString($iter134);
+                  $xfer += $output->writeString($iter141);
                 }
               }
               $output->writeListEnd();
@@ -5120,19 +5836,19 @@ class SkewedInfo {
       {
         $output->writeMapBegin(TType::LST, TType::STRING, count($this->skewedColValueLocationMaps));
         {
-          foreach ($this->skewedColValueLocationMaps as $kiter135 => $viter136)
+          foreach ($this->skewedColValueLocationMaps as $kiter142 => $viter143)
           {
             {
-              $output->writeListBegin(TType::STRING, count($kiter135));
+              $output->writeListBegin(TType::STRING, count($kiter142));
               {
-                foreach ($kiter135 as $iter137)
+                foreach ($kiter142 as $iter144)
                 {
-                  $xfer += $output->writeString($iter137);
+                  $xfer += $output->writeString($iter144);
                 }
               }
               $output->writeListEnd();
             }
-            $xfer += $output->writeString($viter136);
+            $xfer += $output->writeString($viter143);
           }
         }
         $output->writeMapEnd();
@@ -5337,15 +6053,15 @@ class StorageDescriptor {
         case 1:
           if ($ftype == TType::LST) {
             $this->cols = array();
-            $_size138 = 0;
-            $_etype141 = 0;
-            $xfer += $input->readListBegin($_etype141, $_size138);
-            for ($_i142 = 0; $_i142 < $_size138; ++$_i142)
+            $_size145 = 0;
+            $_etype148 = 0;
+            $xfer += $input->readListBegin($_etype148, $_size145);
+            for ($_i149 = 0; $_i149 < $_size145; ++$_i149)
             {
-              $elem143 = null;
-              $elem143 = new \metastore\FieldSchema();
-              $xfer += $elem143->read($input);
-              $this->cols []= $elem143;
+              $elem150 = null;
+              $elem150 = new \metastore\FieldSchema();
+              $xfer += $elem150->read($input);
+              $this->cols []= $elem150;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -5398,14 +6114,14 @@ class StorageDescriptor {
         case 8:
           if ($ftype == TType::LST) {
             $this->bucketCols = array();
-            $_size144 = 0;
-            $_etype147 = 0;
-            $xfer += $input->readListBegin($_etype147, $_size144);
-            for ($_i148 = 0; $_i148 < $_size144; ++$_i148)
+            $_size151 = 0;
+            $_etype154 = 0;
+            $xfer += $input->readListBegin($_etype154, $_size151);
+            for ($_i155 = 0; $_i155 < $_size151; ++$_i155)
             {
-              $elem149 = null;
-              $xfer += $input->readString($elem149);
-              $this->bucketCols []= $elem149;
+              $elem156 = null;
+              $xfer += $input->readString($elem156);
+              $this->bucketCols []= $elem156;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -5415,15 +6131,15 @@ class StorageDescriptor {
         case 9:
           if ($ftype == TType::LST) {
             $this->sortCols = array();
-            $_size150 = 0;
-            $_etype153 = 0;
-            $xfer += $input->readListBegin($_etype153, $_size150);
-            for ($_i154 = 0; $_i154 < $_size150; ++$_i154)
+            $_size157 = 0;
+            $_etype160 = 0;
+            $xfer += $input->readListBegin($_etype160, $_size157);
+            for ($_i161 = 0; $_i161 < $_size157; ++$_i161)
             {
-              $elem155 = null;
-              $elem155 = new \metastore\Order();
-              $xfer += $elem155->read($input);
-              $this->sortCols []= $elem155;
+              $elem162 = null;
+              $elem162 = new \metastore\Order();
+              $xfer += $elem162->read($input);
+              $this->sortCols []= $elem162;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -5433,17 +6149,17 @@ class StorageDescriptor {
         case 10:
           if ($ftype == TType::MAP) {
             $this->parameters = array();
-            $_size156 = 0;
-            $_ktype157 = 0;
-            $_vtype158 = 0;
-            $xfer += $input->readMapBegin($_ktype157, $_vtype158, $_size156);
-            for ($_i160 = 0; $_i160 < $_size156; ++$_i160)
+            $_size163 = 0;
+            $_ktype164 = 0;
+            $_vtype165 = 0;
+            $xfer += $input->readMapBegin($_ktype164, $_vtype165, $_size163);
+            for ($_i167 = 0; $_i167 < $_size163; ++$_i167)
             {
-              $key161 = '';
-              $val162 = '';
-              $xfer += $input->readString($key161);
-              $xfer += $input->readString($val162);
-              $this->parameters[$key161] = $val162;
+              $key168 = '';
+              $val169 = '';
+              $xfer += $input->readString($key168);
+              $xfer += $input->readString($val169);
+              $this->parameters[$key168] = $val169;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -5486,9 +6202,9 @@ class StorageDescriptor {
       {
         $output->writeListBegin(TType::STRUCT, count($this->cols));
         {
-          foreach ($this->cols as $iter163)
+          foreach ($this->cols as $iter170)
           {
-            $xfer += $iter163->write($output);
+            $xfer += $iter170->write($output);
           }
         }
         $output->writeListEnd();
@@ -5536,9 +6252,9 @@ class StorageDescriptor {
       {
         $output->writeListBegin(TType::STRING, count($this->bucketCols));
         {
-          foreach ($this->bucketCols as $iter164)
+          foreach ($this->bucketCols as $iter171)
           {
-            $xfer += $output->writeString($iter164);
+            $xfer += $output->writeString($iter171);
           }
         }
         $output->writeListEnd();
@@ -5553,9 +6269,9 @@ class StorageDescriptor {
       {
         $output->writeListBegin(TType::STRUCT, count($this->sortCols));
         {
-          foreach ($this->sortCols as $iter165)
+          foreach ($this->sortCols as $iter172)
           {
-            $xfer += $iter165->write($output);
+            $xfer += $iter172->write($output);
           }
         }
         $output->writeListEnd();
@@ -5570,10 +6286,10 @@ class StorageDescriptor {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->parameters));
         {
-          foreach ($this->parameters as $kiter166 => $viter167)
+          foreach ($this->parameters as $kiter173 => $viter174)
           {
-            $xfer += $output->writeString($kiter166);
-            $xfer += $output->writeString($viter167);
+            $xfer += $output->writeString($kiter173);
+            $xfer += $output->writeString($viter174);
           }
         }
         $output->writeMapEnd();
@@ -5667,6 +6383,10 @@ class Table {
    * @var \metastore\CreationMetadata
    */
   public $creationMetadata = null;
+  /**
+   * @var string
+   */
+  public $catName = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -5751,6 +6471,10 @@ class Table {
           'type' => TType::STRUCT,
           'class' => '\metastore\CreationMetadata',
           ),
+        17 => array(
+          'var' => 'catName',
+          'type' => TType::STRING,
+          ),
         );
     }
     if (is_array($vals)) {
@@ -5802,6 +6526,9 @@ class Table {
       if (isset($vals['creationMetadata'])) {
         $this->creationMetadata = $vals['creationMetadata'];
       }
+      if (isset($vals['catName'])) {
+        $this->catName = $vals['catName'];
+      }
     }
   }
 
@@ -5877,15 +6604,15 @@ class Table {
         case 8:
           if ($ftype == TType::LST) {
             $this->partitionKeys = array();
-            $_size168 = 0;
-            $_etype171 = 0;
-            $xfer += $input->readListBegin($_etype171, $_size168);
-            for ($_i172 = 0; $_i172 < $_size168; ++$_i172)
+            $_size175 = 0;
+            $_etype178 = 0;
+            $xfer += $input->readListBegin($_etype178, $_size175);
+            for ($_i179 = 0; $_i179 < $_size175; ++$_i179)
             {
-              $elem173 = null;
-              $elem173 = new \metastore\FieldSchema();
-              $xfer += $elem173->read($input);
-              $this->partitionKeys []= $elem173;
+              $elem180 = null;
+              $elem180 = new \metastore\FieldSchema();
+              $xfer += $elem180->read($input);
+              $this->partitionKeys []= $elem180;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -5895,17 +6622,17 @@ class Table {
         case 9:
           if ($ftype == TType::MAP) {
             $this->parameters = array();
-            $_size174 = 0;
-            $_ktype175 = 0;
-            $_vtype176 = 0;
-            $xfer += $input->readMapBegin($_ktype175, $_vtype176, $_size174);
-            for ($_i178 = 0; $_i178 < $_size174; ++$_i178)
+            $_size181 = 0;
+            $_ktype182 = 0;
+            $_vtype183 = 0;
+            $xfer += $input->readMapBegin($_ktype182, $_vtype183, $_size181);
+            for ($_i185 = 0; $_i185 < $_size181; ++$_i185)
             {
-              $key179 = '';
-              $val180 = '';
-              $xfer += $input->readString($key179);
-              $xfer += $input->readString($val180);
-              $this->parameters[$key179] = $val180;
+              $key186 = '';
+              $val187 = '';
+              $xfer += $input->readString($key186);
+              $xfer += $input->readString($val187);
+              $this->parameters[$key186] = $val187;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -5963,6 +6690,13 @@ class Table {
             $xfer += $input->skip($ftype);
           }
           break;
+        case 17:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->catName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
         default:
           $xfer += $input->skip($ftype);
           break;
@@ -6022,9 +6756,9 @@ class Table {
       {
         $output->writeListBegin(TType::STRUCT, count($this->partitionKeys));
         {
-          foreach ($this->partitionKeys as $iter181)
+          foreach ($this->partitionKeys as $iter188)
           {
-            $xfer += $iter181->write($output);
+            $xfer += $iter188->write($output);
           }
         }
         $output->writeListEnd();
@@ -6039,10 +6773,10 @@ class Table {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->parameters));
         {
-          foreach ($this->parameters as $kiter182 => $viter183)
+          foreach ($this->parameters as $kiter189 => $viter190)
           {
-            $xfer += $output->writeString($kiter182);
-            $xfer += $output->writeString($viter183);
+            $xfer += $output->writeString($kiter189);
+            $xfer += $output->writeString($viter190);
           }
         }
         $output->writeMapEnd();
@@ -6090,6 +6824,11 @@ class Table {
       $xfer += $this->creationMetadata->write($output);
       $xfer += $output->writeFieldEnd();
     }
+    if ($this->catName !== null) {
+      $xfer += $output->writeFieldBegin('catName', TType::STRING, 17);
+      $xfer += $output->writeString($this->catName);
+      $xfer += $output->writeFieldEnd();
+    }
     $xfer += $output->writeFieldStop();
     $xfer += $output->writeStructEnd();
     return $xfer;
@@ -6132,6 +6871,10 @@ class Partition {
    * @var \metastore\PrincipalPrivilegeSet
    */
   public $privileges = null;
+  /**
+   * @var string
+   */
+  public $catName = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -6182,6 +6925,10 @@ class Partition {
           'type' => TType::STRUCT,
           'class' => '\metastore\PrincipalPrivilegeSet',
           ),
+        9 => array(
+          'var' => 'catName',
+          'type' => TType::STRING,
+          ),
         );
     }
     if (is_array($vals)) {
@@ -6209,6 +6956,9 @@ class Partition {
       if (isset($vals['privileges'])) {
         $this->privileges = $vals['privileges'];
       }
+      if (isset($vals['catName'])) {
+        $this->catName = $vals['catName'];
+      }
     }
   }
 
@@ -6234,14 +6984,14 @@ class Partition {
         case 1:
           if ($ftype == TType::LST) {
             $this->values = array();
-            $_size184 = 0;
-            $_etype187 = 0;
-            $xfer += $input->readListBegin($_etype187, $_size184);
-            for ($_i188 = 0; $_i188 < $_size184; ++$_i188)
+            $_size191 = 0;
+            $_etype194 = 0;
+            $xfer += $input->readListBegin($_etype194, $_size191);
+            for ($_i195 = 0; $_i195 < $_size191; ++$_i195)
             {
-              $elem189 = null;
-              $xfer += $input->readString($elem189);
-              $this->values []= $elem189;
+              $elem196 = null;
+              $xfer += $input->readString($elem196);
+              $this->values []= $elem196;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -6287,17 +7037,17 @@ class Partition {
         case 7:
           if ($ftype == TType::MAP) {
             $this->parameters = array();
-            $_size190 = 0;
-            $_ktype191 = 0;
-            $_vtype192 = 0;
-            $xfer += $input->readMapBegin($_ktype191, $_vtype192, $_size190);
-            for ($_i194 = 0; $_i194 < $_size190; ++$_i194)
+            $_size197 = 0;
+            $_ktype198 = 0;
+            $_vtype199 = 0;
+            $xfer += $input->readMapBegin($_ktype198, $_vtype199, $_size197);
+            for ($_i201 = 0; $_i201 < $_size197; ++$_i201)
             {
-              $key195 = '';
-              $val196 = '';
-              $xfer += $input->readString($key195);
-              $xfer += $input->readString($val196);
-              $this->parameters[$key195] = $val196;
+              $key202 = '';
+              $val203 = '';
+              $xfer += $input->readString($key202);
+              $xfer += $input->readString($val203);
+              $this->parameters[$key202] = $val203;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -6312,6 +7062,13 @@ class Partition {
             $xfer += $input->skip($ftype);
           }
           break;
+        case 9:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->catName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
         default:
           $xfer += $input->skip($ftype);
           break;
@@ -6333,9 +7090,9 @@ class Partition {
       {
         $output->writeListBegin(TType::STRING, count($this->values));
         {
-          foreach ($this->values as $iter197)
+          foreach ($this->values as $iter204)
           {
-            $xfer += $output->writeString($iter197);
+            $xfer += $output->writeString($iter204);
           }
         }
         $output->writeListEnd();
@@ -6378,10 +7135,10 @@ class Partition {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->parameters));
         {
-          foreach ($this->parameters as $kiter198 => $viter199)
+          foreach ($this->parameters as $kiter205 => $viter206)
           {
-            $xfer += $output->writeString($kiter198);
-            $xfer += $output->writeString($viter199);
+            $xfer += $output->writeString($kiter205);
+            $xfer += $output->writeString($viter206);
           }
         }
         $output->writeMapEnd();
@@ -6396,6 +7153,11 @@ class Partition {
       $xfer += $this->privileges->write($output);
       $xfer += $output->writeFieldEnd();
     }
+    if ($this->catName !== null) {
+      $xfer += $output->writeFieldBegin('catName', TType::STRING, 9);
+      $xfer += $output->writeString($this->catName);
+      $xfer += $output->writeFieldEnd();
+    }
     $xfer += $output->writeFieldStop();
     $xfer += $output->writeStructEnd();
     return $xfer;
@@ -6517,14 +7279,14 @@ class PartitionWithoutSD {
         case 1:
           if ($ftype == TType::LST) {
             $this->values = array();
-            $_size200 = 0;
-            $_etype203 = 0;
-            $xfer += $input->readListBegin($_etype203, $_size200);
-            for ($_i204 = 0; $_i204 < $_size200; ++$_i204)
+            $_size207 = 0;
+            $_etype210 = 0;
+            $xfer += $input->readListBegin($_etype210, $_size207);
+            for ($_i211 = 0; $_i211 < $_size207; ++$_i211)
             {
-              $elem205 = null;
-              $xfer += $input->readString($elem205);
-              $this->values []= $elem205;
+              $elem212 = null;
+              $xfer += $input->readString($elem212);
+              $this->values []= $elem212;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -6555,17 +7317,17 @@ class PartitionWithoutSD {
         case 5:
           if ($ftype == TType::MAP) {
             $this->parameters = array();
-            $_size206 = 0;
-            $_ktype207 = 0;
-            $_vtype208 = 0;
-            $xfer += $input->readMapBegin($_ktype207, $_vtype208, $_size206);
-            for ($_i210 = 0; $_i210 < $_size206; ++$_i210)
+            $_size213 = 0;
+            $_ktype214 = 0;
+            $_vtype215 = 0;
+            $xfer += $input->readMapBegin($_ktype214, $_vtype215, $_size213);
+            for ($_i217 = 0; $_i217 < $_size213; ++$_i217)
             {
-              $key211 = '';
-              $val212 = '';
-              $xfer += $input->readString($key211);
-              $xfer += $input->readString($val212);
-              $this->parameters[$key211] = $val212;
+              $key218 = '';
+              $val219 = '';
+              $xfer += $input->readString($key218);
+              $xfer += $input->readString($val219);
+              $this->parameters[$key218] = $val219;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -6601,9 +7363,9 @@ class PartitionWithoutSD {
       {
         $output->writeListBegin(TType::STRING, count($this->values));
         {
-          foreach ($this->values as $iter213)
+          foreach ($this->values as $iter220)
           {
-            $xfer += $output->writeString($iter213);
+            $xfer += $output->writeString($iter220);
           }
         }
         $output->writeListEnd();
@@ -6633,10 +7395,10 @@ class PartitionWithoutSD {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->parameters));
         {
-          foreach ($this->parameters as $kiter214 => $viter215)
+          foreach ($this->parameters as $kiter221 => $viter222)
           {
-            $xfer += $output->writeString($kiter214);
-            $xfer += $output->writeString($viter215);
+            $xfer += $output->writeString($kiter221);
+            $xfer += $output->writeString($viter222);
           }
         }
         $output->writeMapEnd();
@@ -6721,15 +7483,15 @@ class PartitionSpecWithSharedSD {
         case 1:
           if ($ftype == TType::LST) {
             $this->partitions = array();
-            $_size216 = 0;
-            $_etype219 = 0;
-            $xfer += $input->readListBegin($_etype219, $_size216);
-            for ($_i220 = 0; $_i220 < $_size216; ++$_i220)
+            $_size223 = 0;
+            $_etype226 = 0;
+            $xfer += $input->readListBegin($_etype226, $_size223);
+            for ($_i227 = 0; $_i227 < $_size223; ++$_i227)
             {
-              $elem221 = null;
-              $elem221 = new \metastore\PartitionWithoutSD();
-              $xfer += $elem221->read($input);
-              $this->partitions []= $elem221;
+              $elem228 = null;
+              $elem228 = new \metastore\PartitionWithoutSD();
+              $xfer += $elem228->read($input);
+              $this->partitions []= $elem228;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -6765,9 +7527,9 @@ class PartitionSpecWithSharedSD {
       {
         $output->writeListBegin(TType::STRUCT, count($this->partitions));
         {
-          foreach ($this->partitions as $iter222)
+          foreach ($this->partitions as $iter229)
           {
-            $xfer += $iter222->write($output);
+            $xfer += $iter229->write($output);
           }
         }
         $output->writeListEnd();
@@ -6840,15 +7602,15 @@ class PartitionListComposingSpec {
         case 1:
           if ($ftype == TType::LST) {
             $this->partitions = array();
-            $_size223 = 0;
-            $_etype226 = 0;
-            $xfer += $input->readListBegin($_etype226, $_size223);
-            for ($_i227 = 0; $_i227 < $_size223; ++$_i227)
+            $_size230 = 0;
+            $_etype233 = 0;
+            $xfer += $input->readListBegin($_etype233, $_size230);
+            for ($_i234 = 0; $_i234 < $_size230; ++$_i234)
             {
-              $elem228 = null;
-              $elem228 = new \metastore\Partition();
-              $xfer += $elem228->read($input);
-              $this->partitions []= $elem228;
+              $elem235 = null;
+              $elem235 = new \metastore\Partition();
+              $xfer += $elem235->read($input);
+              $this->partitions []= $elem235;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -6876,9 +7638,9 @@ class PartitionListComposingSpec {
       {
         $output->writeListBegin(TType::STRUCT, count($this->partitions));
         {
-          foreach ($this->partitions as $iter229)
+          foreach ($this->partitions as $iter236)
           {
-            $xfer += $iter229->write($output);
+            $xfer += $iter236->write($output);
           }
         }
         $output->writeListEnd();
@@ -6915,6 +7677,10 @@ class PartitionSpec {
    * @var \metastore\PartitionListComposingSpec
    */
   public $partitionList = null;
+  /**
+   * @var string
+   */
+  public $catName = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -6941,6 +7707,10 @@ class PartitionSpec {
           'type' => TType::STRUCT,
           'class' => '\metastore\PartitionListComposingSpec',
           ),
+        6 => array(
+          'var' => 'catName',
+          'type' => TType::STRING,
+          ),
         );
     }
     if (is_array($vals)) {
@@ -6959,6 +7729,9 @@ class PartitionSpec {
       if (isset($vals['partitionList'])) {
         $this->partitionList = $vals['partitionList'];
       }
+      if (isset($vals['catName'])) {
+        $this->catName = $vals['catName'];
+      }
     }
   }
 
@@ -7018,6 +7791,13 @@ class PartitionSpec {
             $xfer += $input->skip($ftype);
           }
           break;
+        case 6:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->catName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
         default:
           $xfer += $input->skip($ftype);
           break;
@@ -7062,6 +7842,11 @@ class PartitionSpec {
       $xfer += $this->partitionList->write($output);
       $xfer += $output->writeFieldEnd();
     }
+    if ($this->catName !== null) {
+      $xfer += $output->writeFieldBegin('catName', TType::STRING, 6);
+      $xfer += $output->writeString($this->catName);
+      $xfer += $output->writeFieldEnd();
+    }
     $xfer += $output->writeFieldStop();
     $xfer += $output->writeStructEnd();
     return $xfer;
@@ -8782,6 +9567,10 @@ class ColumnStatisticsDesc {
    * @var int
    */
   public $lastAnalyzed = null;
+  /**
+   * @var string
+   */
+  public $catName = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -8806,6 +9595,10 @@ class ColumnStatisticsDesc {
           'var' => 'lastAnalyzed',
           'type' => TType::I64,
           ),
+        6 => array(
+          'var' => 'catName',
+          'type' => TType::STRING,
+          ),
         );
     }
     if (is_array($vals)) {
@@ -8824,6 +9617,9 @@ class ColumnStatisticsDesc {
       if (isset($vals['lastAnalyzed'])) {
         $this->lastAnalyzed = $vals['lastAnalyzed'];
       }
+      if (isset($vals['catName'])) {
+        $this->catName = $vals['catName'];
+      }
     }
   }
 
@@ -8881,6 +9677,13 @@ class ColumnStatisticsDesc {
             $xfer += $input->skip($ftype);
           }
           break;
+        case 6:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->catName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
         default:
           $xfer += $input->skip($ftype);
           break;
@@ -8919,6 +9722,11 @@ class ColumnStatisticsDesc {
       $xfer += $output->writeI64($this->lastAnalyzed);
       $xfer += $output->writeFieldEnd();
     }
+    if ($this->catName !== null) {
+      $xfer += $output->writeFieldBegin('catName', TType::STRING, 6);
+      $xfer += $output->writeString($this->catName);
+      $xfer += $output->writeFieldEnd();
+    }
     $xfer += $output->writeFieldStop();
     $xfer += $output->writeStructEnd();
     return $xfer;
@@ -8994,18 +9802,18 @@ class ColumnStatistics {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 2:
-          if ($ftype == TType::LST) {
-            $this->statsObj = array();
-            $_size230 = 0;
-            $_etype233 = 0;
-            $xfer += $input->readListBegin($_etype233, $_size230);
-            for ($_i234 = 0; $_i234 < $_size230; ++$_i234)
+        case 2:
+          if ($ftype == TType::LST) {
+            $this->statsObj = array();
+            $_size237 = 0;
+            $_etype240 = 0;
+            $xfer += $input->readListBegin($_etype240, $_size237);
+            for ($_i241 = 0; $_i241 < $_size237; ++$_i241)
             {
-              $elem235 = null;
-              $elem235 = new \metastore\ColumnStatisticsObj();
-              $xfer += $elem235->read($input);
-              $this->statsObj []= $elem235;
+              $elem242 = null;
+              $elem242 = new \metastore\ColumnStatisticsObj();
+              $xfer += $elem242->read($input);
+              $this->statsObj []= $elem242;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -9041,9 +9849,9 @@ class ColumnStatistics {
       {
         $output->writeListBegin(TType::STRUCT, count($this->statsObj));
         {
-          foreach ($this->statsObj as $iter236)
+          foreach ($this->statsObj as $iter243)
           {
-            $xfer += $iter236->write($output);
+            $xfer += $iter243->write($output);
           }
         }
         $output->writeListEnd();
@@ -9119,15 +9927,15 @@ class AggrStats {
         case 1:
           if ($ftype == TType::LST) {
             $this->colStats = array();
-            $_size237 = 0;
-            $_etype240 = 0;
-            $xfer += $input->readListBegin($_etype240, $_size237);
-            for ($_i241 = 0; $_i241 < $_size237; ++$_i241)
+            $_size244 = 0;
+            $_etype247 = 0;
+            $xfer += $input->readListBegin($_etype247, $_size244);
+            for ($_i248 = 0; $_i248 < $_size244; ++$_i248)
             {
-              $elem242 = null;
-              $elem242 = new \metastore\ColumnStatisticsObj();
-              $xfer += $elem242->read($input);
-              $this->colStats []= $elem242;
+              $elem249 = null;
+              $elem249 = new \metastore\ColumnStatisticsObj();
+              $xfer += $elem249->read($input);
+              $this->colStats []= $elem249;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -9162,9 +9970,9 @@ class AggrStats {
       {
         $output->writeListBegin(TType::STRUCT, count($this->colStats));
         {
-          foreach ($this->colStats as $iter243)
+          foreach ($this->colStats as $iter250)
           {
-            $xfer += $iter243->write($output);
+            $xfer += $iter250->write($output);
           }
         }
         $output->writeListEnd();
@@ -9245,15 +10053,15 @@ class SetPartitionsStatsRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->colStats = array();
-            $_size244 = 0;
-            $_etype247 = 0;
-            $xfer += $input->readListBegin($_etype247, $_size244);
-            for ($_i248 = 0; $_i248 < $_size244; ++$_i248)
+            $_size251 = 0;
+            $_etype254 = 0;
+            $xfer += $input->readListBegin($_etype254, $_size251);
+            for ($_i255 = 0; $_i255 < $_size251; ++$_i255)
             {
-              $elem249 = null;
-              $elem249 = new \metastore\ColumnStatistics();
-              $xfer += $elem249->read($input);
-              $this->colStats []= $elem249;
+              $elem256 = null;
+              $elem256 = new \metastore\ColumnStatistics();
+              $xfer += $elem256->read($input);
+              $this->colStats []= $elem256;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -9288,9 +10096,9 @@ class SetPartitionsStatsRequest {
       {
         $output->writeListBegin(TType::STRUCT, count($this->colStats));
         {
-          foreach ($this->colStats as $iter250)
+          foreach ($this->colStats as $iter257)
           {
-            $xfer += $iter250->write($output);
+            $xfer += $iter257->write($output);
           }
         }
         $output->writeListEnd();
@@ -9379,15 +10187,15 @@ class Schema {
         case 1:
           if ($ftype == TType::LST) {
             $this->fieldSchemas = array();
-            $_size251 = 0;
-            $_etype254 = 0;
-            $xfer += $input->readListBegin($_etype254, $_size251);
-            for ($_i255 = 0; $_i255 < $_size251; ++$_i255)
+            $_size258 = 0;
+            $_etype261 = 0;
+            $xfer += $input->readListBegin($_etype261, $_size258);
+            for ($_i262 = 0; $_i262 < $_size258; ++$_i262)
             {
-              $elem256 = null;
-              $elem256 = new \metastore\FieldSchema();
-              $xfer += $elem256->read($input);
-              $this->fieldSchemas []= $elem256;
+              $elem263 = null;
+              $elem263 = new \metastore\FieldSchema();
+              $xfer += $elem263->read($input);
+              $this->fieldSchemas []= $elem263;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -9397,17 +10205,17 @@ class Schema {
         case 2:
           if ($ftype == TType::MAP) {
             $this->properties = array();
-            $_size257 = 0;
-            $_ktype258 = 0;
-            $_vtype259 = 0;
-            $xfer += $input->readMapBegin($_ktype258, $_vtype259, $_size257);
-            for ($_i261 = 0; $_i261 < $_size257; ++$_i261)
+            $_size264 = 0;
+            $_ktype265 = 0;
+            $_vtype266 = 0;
+            $xfer += $input->readMapBegin($_ktype265, $_vtype266, $_size264);
+            for ($_i268 = 0; $_i268 < $_size264; ++$_i268)
             {
-              $key262 = '';
-              $val263 = '';
-              $xfer += $input->readString($key262);
-              $xfer += $input->readString($val263);
-              $this->properties[$key262] = $val263;
+              $key269 = '';
+              $val270 = '';
+              $xfer += $input->readString($key269);
+              $xfer += $input->readString($val270);
+              $this->properties[$key269] = $val270;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -9435,9 +10243,9 @@ class Schema {
       {
         $output->writeListBegin(TType::STRUCT, count($this->fieldSchemas));
         {
-          foreach ($this->fieldSchemas as $iter264)
+          foreach ($this->fieldSchemas as $iter271)
           {
-            $xfer += $iter264->write($output);
+            $xfer += $iter271->write($output);
           }
         }
         $output->writeListEnd();
@@ -9452,10 +10260,10 @@ class Schema {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->properties));
         {
-          foreach ($this->properties as $kiter265 => $viter266)
+          foreach ($this->properties as $kiter272 => $viter273)
           {
-            $xfer += $output->writeString($kiter265);
-            $xfer += $output->writeString($viter266);
+            $xfer += $output->writeString($kiter272);
+            $xfer += $output->writeString($viter273);
           }
         }
         $output->writeMapEnd();
@@ -9523,17 +10331,17 @@ class EnvironmentContext {
         case 1:
           if ($ftype == TType::MAP) {
             $this->properties = array();
-            $_size267 = 0;
-            $_ktype268 = 0;
-            $_vtype269 = 0;
-            $xfer += $input->readMapBegin($_ktype268, $_vtype269, $_size267);
-            for ($_i271 = 0; $_i271 < $_size267; ++$_i271)
+            $_size274 = 0;
+            $_ktype275 = 0;
+            $_vtype276 = 0;
+            $xfer += $input->readMapBegin($_ktype275, $_vtype276, $_size274);
+            for ($_i278 = 0; $_i278 < $_size274; ++$_i278)
             {
-              $key272 = '';
-              $val273 = '';
-              $xfer += $input->readString($key272);
-              $xfer += $input->readString($val273);
-              $this->properties[$key272] = $val273;
+              $key279 = '';
+              $val280 = '';
+              $xfer += $input->readString($key279);
+              $xfer += $input->readString($val280);
+              $this->properties[$key279] = $val280;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -9561,10 +10369,10 @@ class EnvironmentContext {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->properties));
         {
-          forea

<TRUNCATED>