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

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

Repository: hive
Updated Branches:
  refs/heads/master 14e3f19d0 -> 93e33e119


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


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

Branch: refs/heads/master
Commit: 93e33e119e4cb4c41058ae23750835e3fe86cf56
Parents: 14e3f19
Author: Alan Gates <ga...@hortonworks.com>
Authored: Wed Jun 13 18:22:15 2018 -0700
Committer: Alan Gates <ga...@hortonworks.com>
Committed: Wed Jun 13 18:22:15 2018 -0700

----------------------------------------------------------------------
 .../hadoop/hive/metastore/HiveMetaStore.java    |  49 +++++
 .../hive/metastore/HiveMetaStoreClient.java     |   5 +
 .../hive/metastore/NonCatCallsWithCatalog.java  | 209 +++++++++++--------
 3 files changed, 174 insertions(+), 89 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/93e33e11/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 7dbdba8..278c58c 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
@@ -1865,6 +1865,25 @@ public class HiveMetaStore extends ThriftHiveMetastore {
             && checkConstraints == null) {
           ms.createTable(tbl);
         } else {
+          // Check that constraints have catalog name properly set first
+          if (primaryKeys != null && !primaryKeys.isEmpty() && !primaryKeys.get(0).isSetCatName()) {
+            for (SQLPrimaryKey pkcol : primaryKeys) pkcol.setCatName(tbl.getCatName());
+          }
+          if (foreignKeys != null && !foreignKeys.isEmpty() && !foreignKeys.get(0).isSetCatName()) {
+            for (SQLForeignKey fkcol : foreignKeys) fkcol.setCatName(tbl.getCatName());
+          }
+          if (uniqueConstraints != null && !uniqueConstraints.isEmpty() && !uniqueConstraints.get(0).isSetCatName()) {
+            for (SQLUniqueConstraint uccol : uniqueConstraints) uccol.setCatName(tbl.getCatName());
+          }
+          if (notNullConstraints != null && !notNullConstraints.isEmpty() && !notNullConstraints.get(0).isSetCatName()) {
+            for (SQLNotNullConstraint nncol : notNullConstraints) nncol.setCatName(tbl.getCatName());
+          }
+          if (defaultConstraints != null && !defaultConstraints.isEmpty() && !defaultConstraints.get(0).isSetCatName()) {
+            for (SQLDefaultConstraint dccol : defaultConstraints) dccol.setCatName(tbl.getCatName());
+          }
+          if (checkConstraints != null && !checkConstraints.isEmpty() && !checkConstraints.get(0).isSetCatName()) {
+            for (SQLCheckConstraint cccol : checkConstraints) cccol.setCatName(tbl.getCatName());
+          }
           // Set constraint name if null before sending to listener
           List<String> constraintNames = ms.createTableWithConstraints(tbl, primaryKeys, foreignKeys,
               uniqueConstraints, notNullConstraints, defaultConstraints, checkConstraints);
@@ -1875,6 +1894,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
               if (primaryKeys.get(i).getPk_name() == null) {
                 primaryKeys.get(i).setPk_name(constraintNames.get(i));
               }
+              if (!primaryKeys.get(i).isSetCatName()) primaryKeys.get(i).setCatName(tbl.getCatName());
             }
           }
           int foreignKeySize = 0;
@@ -1884,6 +1904,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
               if (foreignKeys.get(i).getFk_name() == null) {
                 foreignKeys.get(i).setFk_name(constraintNames.get(primaryKeySize + i));
               }
+              if (!foreignKeys.get(i).isSetCatName()) foreignKeys.get(i).setCatName(tbl.getCatName());
             }
           }
           int uniqueConstraintSize = 0;
@@ -1893,6 +1914,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
               if (uniqueConstraints.get(i).getUk_name() == null) {
                 uniqueConstraints.get(i).setUk_name(constraintNames.get(primaryKeySize + foreignKeySize + i));
               }
+              if (!uniqueConstraints.get(i).isSetCatName()) uniqueConstraints.get(i).setCatName(tbl.getCatName());
             }
           }
           int notNullConstraintSize =  0;
@@ -1901,6 +1923,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
               if (notNullConstraints.get(i).getNn_name() == null) {
                 notNullConstraints.get(i).setNn_name(constraintNames.get(primaryKeySize + foreignKeySize + uniqueConstraintSize + i));
               }
+              if (!notNullConstraints.get(i).isSetCatName()) notNullConstraints.get(i).setCatName(tbl.getCatName());
             }
           }
           int defaultConstraintSize =  0;
@@ -1910,6 +1933,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
                 defaultConstraints.get(i).setDc_name(constraintNames.get(primaryKeySize + foreignKeySize
                     + uniqueConstraintSize + notNullConstraintSize + i));
               }
+              if (!defaultConstraints.get(i).isSetCatName()) defaultConstraints.get(i).setCatName(tbl.getCatName());
             }
           }
           if (checkConstraints!= null) {
@@ -1920,6 +1944,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
                                                                              + defaultConstraintSize
                                                                            + notNullConstraintSize + i));
               }
+              if (!checkConstraints.get(i).isSetCatName()) checkConstraints.get(i).setCatName(tbl.getCatName());
             }
           }
         }
@@ -2091,6 +2116,10 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       startFunction("add_primary_key", ": " + constraintName);
       boolean success = false;
       Exception ex = null;
+      if (!primaryKeyCols.isEmpty() && !primaryKeyCols.get(0).isSetCatName()) {
+        String defaultCat = getDefaultCatalog(conf);
+        primaryKeyCols.forEach(pk -> pk.setCatName(defaultCat));
+      }
       RawStore ms = getMS();
       try {
         ms.openTransaction();
@@ -2140,6 +2169,10 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       startFunction("add_foreign_key", ": " + constraintName);
       boolean success = false;
       Exception ex = null;
+      if (!foreignKeyCols.isEmpty() && !foreignKeyCols.get(0).isSetCatName()) {
+        String defaultCat = getDefaultCatalog(conf);
+        foreignKeyCols.forEach(pk -> pk.setCatName(defaultCat));
+      }
       RawStore ms = getMS();
       try {
         ms.openTransaction();
@@ -2189,6 +2222,10 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       startFunction("add_unique_constraint", ": " + constraintName);
       boolean success = false;
       Exception ex = null;
+      if (!uniqueConstraintCols.isEmpty() && !uniqueConstraintCols.get(0).isSetCatName()) {
+        String defaultCat = getDefaultCatalog(conf);
+        uniqueConstraintCols.forEach(pk -> pk.setCatName(defaultCat));
+      }
       RawStore ms = getMS();
       try {
         ms.openTransaction();
@@ -2238,6 +2275,10 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       startFunction("add_not_null_constraint", ": " + constraintName);
       boolean success = false;
       Exception ex = null;
+      if (!notNullConstraintCols.isEmpty() && !notNullConstraintCols.get(0).isSetCatName()) {
+        String defaultCat = getDefaultCatalog(conf);
+        notNullConstraintCols.forEach(pk -> pk.setCatName(defaultCat));
+      }
       RawStore ms = getMS();
       try {
         ms.openTransaction();
@@ -2287,6 +2328,10 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       startFunction("add_default_constraint", ": " + constraintName);
       boolean success = false;
       Exception ex = null;
+      if (!defaultConstraintCols.isEmpty() && !defaultConstraintCols.get(0).isSetCatName()) {
+        String defaultCat = getDefaultCatalog(conf);
+        defaultConstraintCols.forEach(pk -> pk.setCatName(defaultCat));
+      }
       RawStore ms = getMS();
       try {
         ms.openTransaction();
@@ -2337,6 +2382,10 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       startFunction("add_check_constraint", ": " + constraintName);
       boolean success = false;
       Exception ex = null;
+      if (!checkConstraintCols.isEmpty() && !checkConstraintCols.get(0).isSetCatName()) {
+        String defaultCat = getDefaultCatalog(conf);
+        checkConstraintCols.forEach(pk -> pk.setCatName(defaultCat));
+      }
       RawStore ms = getMS();
       try {
         ms.openTransaction();

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

http://git-wip-us.apache.org/repos/asf/hive/blob/93e33e11/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/NonCatCallsWithCatalog.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/NonCatCallsWithCatalog.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/NonCatCallsWithCatalog.java
index 0194178..f750ca2 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/NonCatCallsWithCatalog.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/NonCatCallsWithCatalog.java
@@ -44,8 +44,6 @@ 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;
@@ -66,7 +64,6 @@ 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 java.io.File;
@@ -75,7 +72,6 @@ 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;
@@ -91,8 +87,8 @@ public abstract class NonCatCallsWithCatalog {
       "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;
@@ -116,13 +112,18 @@ public abstract class NonCatCallsWithCatalog {
       client.dropTable(DEFAULT_DATABASE_NAME, tableName, true, true, true);
     }
 
+    Database db = new DatabaseBuilder().setName(OTHER_DATABASE).build(conf);
+    db.unsetCatalogName();
+    client.createDatabase(db);
+
+
     testTables[0] =
         new TableBuilder()
             .setTableName("test_table")
             .addCol("test_col1", "int")
             .addCol("test_col2", "int")
             .addCol("test_col3", "int")
-            .create(client, conf);
+            .build(conf);
 
     testTables[1] =
         new TableBuilder()
@@ -131,7 +132,7 @@ public abstract class NonCatCallsWithCatalog {
             .addCol("test_col2", "int")
             .addCol("test_col3", "int")
             .setType("VIRTUAL_VIEW")
-            .create(client, conf);
+            .build(conf);
 
     testTables[2] =
         new TableBuilder()
@@ -139,7 +140,7 @@ public abstract class NonCatCallsWithCatalog {
             .addCol("test_col1", "int")
             .addCol("test_col2", "int")
             .addCol("test_col3", "int")
-            .create(client, conf);
+            .build(conf);
 
     testTables[3] =
         new TableBuilder()
@@ -147,7 +148,7 @@ public abstract class NonCatCallsWithCatalog {
             .addCol("test_col1", "int")
             .addCol("test_col2", "int")
             .addPartCol("test_part_col", "int")
-            .create(client, conf);
+            .build(conf);
 
     testTables[4] =
         new TableBuilder()
@@ -156,24 +157,28 @@ public abstract class NonCatCallsWithCatalog {
             .setLocation(MetaStoreTestUtils.getTestWarehouseDir("/external/table_dir"))
             .addTableParam("EXTERNAL", "TRUE")
             .setType("EXTERNAL_TABLE")
-            .create(client, conf);
-
-
-    new DatabaseBuilder().setName(OTHER_DATABASE).create(client, conf);
+            .build(conf);
 
     testTables[5] =
         new TableBuilder()
             .setDbName(OTHER_DATABASE)
             .setTableName("test_table")
             .addCol("test_col", "int")
-            .create(client, conf);
+            .build(conf);
+
+    for (Table t : testTables) {
+      t.unsetCatName();
+      client.createTable(t);
+    }
 
     // Create partitions for the partitioned table
     for(int i=0; i < 3; i++) {
-      new PartitionBuilder()
+      Partition p = new PartitionBuilder()
           .inTable(testTables[3])
           .addValue("a" + i)
-          .addToTable(client, conf);
+          .build(conf);
+      p.unsetCatName();
+      client.add_partition(p);
     }
 
   }
@@ -196,14 +201,19 @@ public abstract class NonCatCallsWithCatalog {
     // 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);
+        .build(conf);
 
     // For the second one, explicitly set a location to make sure it ends up in the specified place.
     String db1Location = MetaStoreTestUtils.getTestWarehouseDir(dbNames[1]);
     dbs[1] = new DatabaseBuilder()
         .setName(dbNames[1])
         .setLocation(db1Location)
-        .create(client, conf);
+        .build(conf);
+
+    for (Database db : dbs) {
+      db.unsetCatalogName();
+      client.createDatabase(db);
+    }
 
     Database fetched = client.getDatabase(dbNames[0]);
     String expectedLocation = new File(expectedBaseDir(), dbNames[0] + ".db").toURI().toString();
@@ -245,7 +255,9 @@ public abstract class NonCatCallsWithCatalog {
     // 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);
+        .build(conf);
+    db.unsetCatalogName();
+    client.createDatabase(db);
 
     String[] tableNames = new String[4];
     for (int i = 0; i < tableNames.length; i++) {
@@ -268,7 +280,9 @@ public abstract class NonCatCallsWithCatalog {
             .addMaterializedViewReferencedTable(dbName + "." + tableNames[0]);
       }
       */
-      client.createTable(builder.build(conf));
+      Table t = builder.build(conf);
+      t.unsetCatName();
+      client.createTable(t);
     }
 
     // Add partitions for the partitioned table
@@ -276,10 +290,12 @@ public abstract class NonCatCallsWithCatalog {
     Table partitionedTable = client.getTable(dbName, tableNames[2]);
     for (int i = 0; i < partVals.length; i++) {
       partVals[i] = "part" + i;
-      new PartitionBuilder()
+      Partition p = new PartitionBuilder()
           .inTable(partitionedTable)
           .addValue(partVals[i])
-          .addToTable(client, conf);
+          .build(conf);
+      p.unsetCatName();
+      client.add_partition(p);
     }
 
     // Get tables, make sure the locations are correct
@@ -386,17 +402,21 @@ public abstract class NonCatCallsWithCatalog {
     // 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);
+        .build(conf);
+    db.unsetCatalogName();
+    client.createDatabase(db);
 
     String[] tableNames = new String[4];
     for (int i = 0; i < tableNames.length; i++) {
       tableNames[i] = "table_in_other_catalog_" + i;
-      new TableBuilder()
+      Table table = new TableBuilder()
           .inDb(db)
           .setTableName(tableNames[i])
           .addCol("col1_" + i, ColumnType.STRING_TYPE_NAME)
           .addCol("col2_" + i, ColumnType.INT_TYPE_NAME)
-          .create(client, conf);
+          .build(conf);
+      table.unsetCatName();
+      client.createTable(table);
     }
 
     Set<String> tables = new HashSet<>(client.getTables(dbName, "*e_in_other_*"));
@@ -417,7 +437,9 @@ public abstract class NonCatCallsWithCatalog {
     // 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);
+        .build(conf);
+    db.unsetCatalogName();
+    client.createDatabase(db);
 
     String[] tableNames = new String[4];
     for (int i = 0; i < tableNames.length; i++) {
@@ -428,7 +450,9 @@ public abstract class NonCatCallsWithCatalog {
           .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);
+      Table table = builder.build(conf);
+      table.unsetCatName();
+      client.createTable(table);
     }
 
     String filter = hive_metastoreConstants.HIVE_FILTER_FIELD_PARAMS + "the_key=\"the_value\"";
@@ -443,17 +467,21 @@ public abstract class NonCatCallsWithCatalog {
     // 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);
+        .build(conf);
+    db.unsetCatalogName();
+    client.createDatabase(db);
 
     String[] tableNames = {"table_in_other_catalog_1", "table_in_other_catalog_2", "random_name"};
     List<TableMeta> expected = new ArrayList<>(tableNames.length);
     for (int i = 0; i < tableNames.length; i++) {
-      client.createTable(new TableBuilder()
+      Table table = new TableBuilder()
           .inDb(db)
           .setTableName(tableNames[i])
           .addCol("id", "int")
           .addCol("name", "string")
-          .build(conf));
+          .build(conf);
+      table.unsetCatName();
+      client.createTable(table);
       expected.add(new TableMeta(dbName, tableNames[i], TableType.MANAGED_TABLE.name()));
     }
 
@@ -471,7 +499,9 @@ public abstract class NonCatCallsWithCatalog {
     String dbName = "add_partition_database_in_other_catalog";
     Database db = new DatabaseBuilder()
         .setName(dbName)
-        .create(client, conf);
+        .build(conf);
+    db.unsetCatalogName();
+    client.createDatabase(db);
 
     String tableName = "table_in_other_catalog";
     Table table = new TableBuilder()
@@ -480,7 +510,9 @@ public abstract class NonCatCallsWithCatalog {
         .addCol("id", "int")
         .addCol("name", "string")
         .addPartCol("partcol", "string")
-        .create(client, conf);
+        .build(conf);
+    table.unsetCatName();
+    client.createTable(table);
 
     Partition[] parts = new Partition[5];
     for (int i = 0; i < parts.length; i++) {
@@ -488,10 +520,11 @@ public abstract class NonCatCallsWithCatalog {
           .inTable(table)
           .addValue("a" + i)
           .build(conf);
+      parts[i].unsetCatName();
     }
     client.add_partition(parts[0]);
     Assert.assertEquals(2, client.add_partitions(Arrays.asList(parts[1], parts[2])));
-    client.add_partitions(Arrays.asList(parts), true, false);
+    client.add_partitions(Arrays.asList(parts[3], parts[4]), true, false);
 
     for (int i = 0; i < parts.length; i++) {
       Partition fetched = client.getPartition(dbName, tableName,
@@ -509,7 +542,9 @@ public abstract class NonCatCallsWithCatalog {
     String dbName = "get_partition_database_in_other_catalog";
     Database db = new DatabaseBuilder()
         .setName(dbName)
-        .create(client, conf);
+        .build(conf);
+    db.unsetCatalogName();
+    client.createDatabase(db);
 
     String tableName = "table_in_other_catalog";
     Table table = new TableBuilder()
@@ -519,7 +554,9 @@ public abstract class NonCatCallsWithCatalog {
         .addCol("name", "string")
         .addPartCol("partcol", "string")
         .addTableParam("PARTITION_LEVEL_PRIVILEGE", "true")
-        .create(client, conf);
+        .build(conf);
+    table.unsetCatName();
+    client.createTable(table);
 
     Partition[] parts = new Partition[5];
     for (int i = 0; i < parts.length; i++) {
@@ -527,6 +564,7 @@ public abstract class NonCatCallsWithCatalog {
           .inTable(table)
           .addValue("a" + i)
           .build(conf);
+      parts[i].unsetCatName();
     }
     client.add_partitions(Arrays.asList(parts));
 
@@ -554,7 +592,9 @@ public abstract class NonCatCallsWithCatalog {
     String dbName = "list_partition_database_in_other_catalog";
     Database db = new DatabaseBuilder()
         .setName(dbName)
-        .create(client, conf);
+        .build(conf);
+    db.unsetCatalogName();
+    client.createDatabase(db);
 
     String tableName = "table_in_other_catalog";
     Table table = new TableBuilder()
@@ -563,7 +603,9 @@ public abstract class NonCatCallsWithCatalog {
         .addCol("id", "int")
         .addCol("name", "string")
         .addPartCol("partcol", "string")
-        .create(client, conf);
+        .build(conf);
+    table.unsetCatName();
+    client.createTable(table);
 
     Partition[] parts = new Partition[5];
     for (int i = 0; i < parts.length; i++) {
@@ -571,6 +613,7 @@ public abstract class NonCatCallsWithCatalog {
           .inTable(table)
           .addValue("a" + i)
           .build(conf);
+      parts[i].unsetCatName();
     }
     client.add_partitions(Arrays.asList(parts));
 
@@ -616,7 +659,9 @@ public abstract class NonCatCallsWithCatalog {
     String dbName = "alter_partition_database_in_other_catalog";
     Database db = new DatabaseBuilder()
         .setName(dbName)
-        .create(client, conf);
+        .build(conf);
+    db.unsetCatalogName();
+    client.createDatabase(db);
 
     String tableName = "table_in_other_catalog";
     Table table = new TableBuilder()
@@ -625,7 +670,9 @@ public abstract class NonCatCallsWithCatalog {
         .addCol("id", "int")
         .addCol("name", "string")
         .addPartCol("partcol", "string")
-        .create(client, conf);
+        .build(conf);
+    table.unsetCatName();
+    client.createTable(table);
 
     Partition[] parts = new Partition[5];
     for (int i = 0; i < 5; i++) {
@@ -634,6 +681,7 @@ public abstract class NonCatCallsWithCatalog {
           .addValue("a" + i)
           .setLocation(MetaStoreTestUtils.getTestWarehouseDir("b" + i))
           .build(conf);
+      parts[i].unsetCatName();
     }
     client.add_partitions(Arrays.asList(parts));
 
@@ -679,7 +727,9 @@ public abstract class NonCatCallsWithCatalog {
     String dbName = "drop_partition_database_in_other_catalog";
     Database db = new DatabaseBuilder()
         .setName(dbName)
-        .create(client, conf);
+        .build(conf);
+    db.unsetCatalogName();
+    client.createDatabase(db);
 
     String tableName = "table_in_other_catalog";
     Table table = new TableBuilder()
@@ -688,7 +738,9 @@ public abstract class NonCatCallsWithCatalog {
         .addCol("id", "int")
         .addCol("name", "string")
         .addPartCol("partcol", "string")
-        .create(client, conf);
+        .build(conf);
+    table.unsetCatName();
+    client.createTable(table);
 
     Partition[] parts = new Partition[2];
     for (int i = 0; i < parts.length; i++) {
@@ -696,6 +748,7 @@ public abstract class NonCatCallsWithCatalog {
           .inTable(table)
           .addValue("a" + i)
           .build(conf);
+      parts[i].unsetCatName();
     }
     client.add_partitions(Arrays.asList(parts));
     List<Partition> fetched = client.listPartitions(dbName, tableName, (short)-1);
@@ -897,45 +950,53 @@ public abstract class NonCatCallsWithCatalog {
         .addCol("col5", "int")
         .addCol("col6", "int")
         .build(conf);
+    table.unsetCatName();
 
     List<SQLPrimaryKey> parentPk = new SQLPrimaryKeyBuilder()
         .onTable(parentTable)
         .addColumn("test_col1")
         .build(conf);
+    for (SQLPrimaryKey pkcol : parentPk) pkcol.unsetCatName();
     client.addPrimaryKey(parentPk);
 
     List<SQLPrimaryKey> pk = new SQLPrimaryKeyBuilder()
         .onTable(table)
         .addColumn("col2")
         .build(conf);
+    for (SQLPrimaryKey pkcol : pk) pkcol.unsetCatName();
 
     List<SQLForeignKey> fk = new SQLForeignKeyBuilder()
         .fromPrimaryKey(parentPk)
         .onTable(table)
         .addColumn("col1")
         .build(conf);
+    for (SQLForeignKey fkcol : fk) fkcol.unsetCatName();
 
     List<SQLDefaultConstraint> dv = new SQLDefaultConstraintBuilder()
         .onTable(table)
         .addColumn("col3")
         .setDefaultVal(0)
         .build(conf);
+    for (SQLDefaultConstraint dccol : dv) dccol.unsetCatName();
 
     List<SQLNotNullConstraint> nn = new SQLNotNullConstraintBuilder()
         .onTable(table)
         .addColumn("col4")
         .build(conf);
+    for (SQLNotNullConstraint nncol : nn) nncol.unsetCatName();
 
     List<SQLUniqueConstraint> uc = new SQLUniqueConstraintBuilder()
         .onTable(table)
         .addColumn("col5")
         .build(conf);
+    for (SQLUniqueConstraint uccol : uc) uccol.unsetCatName();
 
     List<SQLCheckConstraint> cc = new SQLCheckConstraintBuilder()
         .onTable(table)
         .addColumn("col6")
         .setCheckExpression("> 0")
         .build(conf);
+    for (SQLCheckConstraint cccol : cc) cccol.unsetCatName();
 
     client.createTableWithConstraints(table, pk, fk, uc, nn, dv, cc);
 
@@ -1036,7 +1097,9 @@ public abstract class NonCatCallsWithCatalog {
     String dbName = "functions_other_catalog_db";
     Database db = new DatabaseBuilder()
         .setName(dbName)
-        .create(client, conf);
+        .build(conf);
+    db.unsetCatalogName();
+    client.createDatabase(db);
 
     String functionName = "test_function";
     Function function =
@@ -1047,23 +1110,32 @@ public abstract class NonCatCallsWithCatalog {
             .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);
+            .build(conf);
+    function.unsetCatName();
+    client.createFunction(function);
 
     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);
+    // Creation time will be set by server and not us.
+    Assert.assertEquals(function.getFunctionName(), createdFunction.getFunctionName());
+    Assert.assertEquals(function.getDbName(), createdFunction.getDbName());
+    Assert.assertEquals(expectedCatalog(), createdFunction.getCatName());
+    Assert.assertEquals(function.getClassName(), createdFunction.getClassName());
+    Assert.assertEquals(function.getOwnerName(), createdFunction.getOwnerName());
+    Assert.assertEquals(function.getOwnerType(), createdFunction.getOwnerType());
+    Assert.assertEquals(function.getFunctionType(), createdFunction.getFunctionType());
+    Assert.assertEquals(function.getResourceUris(), createdFunction.getResourceUris());
 
     String f2Name = "testy_function2";
     Function f2 = new FunctionBuilder()
         .inDb(db)
         .setName(f2Name)
         .setClass(TEST_FUNCTION_CLASS)
-        .create(client, conf);
+        .build(conf);
+    f2.unsetCatName();
+    client.createFunction(f2);
 
     Set<String> functions = new HashSet<>(client.getFunctions(dbName, "test*"));
     Assert.assertEquals(2, functions.size());
@@ -1083,45 +1155,4 @@ public abstract class NonCatCallsWithCatalog {
       // 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);
-  }
 }