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/07/10 17:44:40 UTC

hive git commit: HIVE-19983 Backport HIVE-19769 (dedicated DatabaseName and TableName objects) to branch-3 (Alan Gates, original patch reviewed by Zoltan Haindrich)

Repository: hive
Updated Branches:
  refs/heads/branch-3 086266963 -> 416d7eb60


HIVE-19983 Backport HIVE-19769 (dedicated DatabaseName and TableName objects) to branch-3 (Alan Gates, original patch reviewed by Zoltan Haindrich)


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

Branch: refs/heads/branch-3
Commit: 416d7eb60d25e8cc737c8a6c145df2116475116d
Parents: 0862669
Author: Alan Gates <ga...@hortonworks.com>
Authored: Tue Jul 10 10:42:54 2018 -0700
Committer: Alan Gates <ga...@hortonworks.com>
Committed: Tue Jul 10 10:42:54 2018 -0700

----------------------------------------------------------------------
 .../listener/DummyRawStoreFailEvent.java        |   6 +-
 .../hadoop/hive/ql/session/SessionState.java    |  14 +++
 .../hive/ql/stats/StatsUpdaterThread.java       |  32 ++---
 .../hadoop/hive/metastore/HiveAlterHandler.java |  16 +--
 .../hadoop/hive/metastore/HiveMetaStore.java    |  51 ++++----
 .../hive/metastore/MetaStoreDirectSql.java      |  13 +-
 .../hadoop/hive/metastore/ObjectStore.java      |  61 +++++----
 .../apache/hadoop/hive/metastore/RawStore.java  |   6 +-
 .../apache/hadoop/hive/metastore/Warehouse.java |  24 +---
 .../hive/metastore/cache/CachedStore.java       |  15 +--
 .../hive/metastore/utils/MetaStoreUtils.java    |  33 +----
 .../DummyRawStoreControlledCommit.java          |   6 +-
 .../DummyRawStoreForJdoConnection.java          |   6 +-
 .../apache/hadoop/hive/common/DatabaseName.java |  97 ++++++++++++++
 .../apache/hadoop/hive/common/TableName.java    | 125 +++++++++++++++++++
 .../hadoop/hive/common/TestDatabaseName.java    |  44 +++++++
 .../hadoop/hive/common/TestTableName.java       |  51 ++++++++
 17 files changed, 445 insertions(+), 155 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/416d7eb6/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 7b8c4a7..7271c3d 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
@@ -18,6 +18,7 @@
 
 package org.apache.hive.hcatalog.listener;
 
+import org.apache.hadoop.hive.common.TableName;
 import org.apache.hadoop.hive.metastore.api.ISchemaName;
 import org.apache.hadoop.hive.metastore.api.SchemaVersionDescriptor;
 import org.apache.hadoop.hive.metastore.api.Catalog;
@@ -90,7 +91,6 @@ import org.apache.hadoop.hive.metastore.api.WMPool;
 import org.apache.hadoop.hive.metastore.api.WMNullablePool;
 import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.ColStatsObjWithSourceInfo;
-import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.FullTableName;
 import org.apache.thrift.TException;
 
 /**
@@ -1226,13 +1226,13 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
 
 
   @Override
-  public List<FullTableName> getTableNamesWithStats() throws MetaException,
+  public List<TableName> getTableNamesWithStats() throws MetaException,
       NoSuchObjectException {
     return null;
   }
 
   @Override
-  public List<FullTableName> getAllTableNamesForStats() throws MetaException,
+  public List<TableName> getAllTableNamesForStats() throws MetaException,
       NoSuchObjectException {
     return null;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/416d7eb6/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
index ad2ec76..6f39b03 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
@@ -64,6 +64,7 @@ import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.cache.CachedStore;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.MapRedStats;
 import org.apache.hadoop.hive.ql.exec.FunctionInfo;
 import org.apache.hadoop.hive.ql.exec.Registry;
@@ -232,6 +233,8 @@ public class SessionState {
 
   private String currentDatabase;
 
+  private String currentCatalog;
+
   private final String CONFIG_AUTHZ_SETTINGS_APPLIED_MARKER =
       "hive.internal.ss.authz.settings.applied.marker";
 
@@ -1730,6 +1733,17 @@ public class SessionState {
     this.currentDatabase = currentDatabase;
   }
 
+  public String getCurrentCatalog() {
+    if (currentCatalog == null) {
+      currentCatalog = MetaStoreUtils.getDefaultCatalog(getConf());
+    }
+    return currentCatalog;
+  }
+
+  public void setCurrentCatalog(String currentCatalog) {
+    this.currentCatalog = currentCatalog;
+  }
+
   public void close() throws IOException {
     for (Closeable cleanupItem : cleanupItems) {
       try {

http://git-wip-us.apache.org/repos/asf/hive/blob/416d7eb6/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUpdaterThread.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUpdaterThread.java b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUpdaterThread.java
index 285db31..ddca704 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUpdaterThread.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUpdaterThread.java
@@ -32,6 +32,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.hadoop.hive.common.TableName;
 import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.MetaStoreThread;
@@ -52,7 +53,6 @@ import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf.StatsUpdateMode;
 import org.apache.hadoop.hive.metastore.txn.TxnStore;
 import org.apache.hadoop.hive.metastore.txn.TxnUtils;
-import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.FullTableName;
 import org.apache.hadoop.hive.ql.DriverUtils;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.session.SessionState;
@@ -75,7 +75,7 @@ public class StatsUpdaterThread extends Thread implements MetaStoreThread {
   private RawStore rs;
   private TxnStore txnHandler;
   /** Full tables, and partitions that currently have analyze commands queued or in progress. */
-  private ConcurrentHashMap<FullTableName, Boolean> tablesInProgress = new ConcurrentHashMap<>();
+  private ConcurrentHashMap<TableName, Boolean> tablesInProgress = new ConcurrentHashMap<>();
   private ConcurrentHashMap<String, Boolean> partsInProgress = new ConcurrentHashMap<>();
   private AtomicInteger itemsInProgress = new AtomicInteger(0);
 
@@ -170,7 +170,7 @@ public class StatsUpdaterThread extends Thread implements MetaStoreThread {
 
   @VisibleForTesting
   boolean runOneIteration() {
-    List<FullTableName> fullTableNames;
+    List<TableName> fullTableNames;
     try {
       fullTableNames = getTablesToCheck();
     } catch (Throwable t) {
@@ -180,7 +180,7 @@ public class StatsUpdaterThread extends Thread implements MetaStoreThread {
     }
     LOG.debug("Processing {}", fullTableNames);
     boolean hadUpdates = false;
-    for (FullTableName fullTableName : fullTableNames) {
+    for (TableName fullTableName : fullTableNames) {
       try {
         List<AnalyzeWork> commands = processOneTable(fullTableName);
         hadUpdates = hadUpdates || commands != null;
@@ -203,10 +203,10 @@ public class StatsUpdaterThread extends Thread implements MetaStoreThread {
     }
   }
 
-  private List<AnalyzeWork> processOneTable(FullTableName fullTableName)
+  private List<AnalyzeWork> processOneTable(TableName fullTableName)
       throws MetaException, NoSuchTxnException, NoSuchObjectException {
     if (isAnalyzeTableInProgress(fullTableName)) return null;
-    String cat = fullTableName.catalog, db = fullTableName.db, tbl = fullTableName.table;
+    String cat = fullTableName.getCat(), db = fullTableName.getDb(), tbl = fullTableName.getTable();
     Table table = rs.getTable(cat, db, tbl);
     LOG.debug("Processing table {}", table);
 
@@ -262,7 +262,7 @@ public class StatsUpdaterThread extends Thread implements MetaStoreThread {
     }
   }
 
-  private List<String> findPartitionsToAnalyze(FullTableName fullTableName, String cat, String db,
+  private List<String> findPartitionsToAnalyze(TableName fullTableName, String cat, String db,
       String tbl, List<String> allCols, Map<String, List<String>> partsToAnalyze)
           throws MetaException, NoSuchObjectException {
     // TODO: ideally when col-stats-accurate stuff is stored in some sane structure, this should
@@ -404,7 +404,7 @@ public class StatsUpdaterThread extends Thread implements MetaStoreThread {
     return partColStr;
   }
 
-  private List<String> getExistingNonPartTableStatsToUpdate(FullTableName fullTableName,
+  private List<String> getExistingNonPartTableStatsToUpdate(TableName fullTableName,
       String cat, String db, String tbl, Map<String, String> params,
       List<String> allCols) throws MetaException {
     ColumnStatistics existingStats = null;
@@ -445,7 +445,7 @@ public class StatsUpdaterThread extends Thread implements MetaStoreThread {
     return colsToUpdate;
   }
 
-  private List<FullTableName> getTablesToCheck() throws MetaException, NoSuchObjectException {
+  private List<TableName> getTablesToCheck() throws MetaException, NoSuchObjectException {
     if (isExistingOnly) {
       try {
         return rs.getTableNamesWithStats();
@@ -457,7 +457,7 @@ public class StatsUpdaterThread extends Thread implements MetaStoreThread {
   }
 
   private ValidReaderWriteIdList getWriteIds(
-      FullTableName fullTableName) throws NoSuchTxnException, MetaException {
+      TableName fullTableName) throws NoSuchTxnException, MetaException {
     GetValidWriteIdsRequest req = new GetValidWriteIdsRequest();
     req.setFullTableNames(Lists.newArrayList(fullTableName.toString()));
     return TxnUtils.createValidReaderWriteIdList(
@@ -504,24 +504,24 @@ public class StatsUpdaterThread extends Thread implements MetaStoreThread {
     }
   }
 
-  private boolean isAnalyzeTableInProgress(FullTableName fullTableName) {
+  private boolean isAnalyzeTableInProgress(TableName fullTableName) {
     return tablesInProgress.containsKey(fullTableName);
   }
 
-  private boolean isAnalyzePartInProgress(FullTableName tableName, String partName) {
+  private boolean isAnalyzePartInProgress(TableName tableName, String partName) {
     return partsInProgress.containsKey(makeFullPartName(tableName, partName));
   }
 
-  private static String makeFullPartName(FullTableName tableName, String partName) {
+  private static String makeFullPartName(TableName tableName, String partName) {
     return tableName + "/" + partName;
   }
 
   private final static class AnalyzeWork {
-    FullTableName tableName;
+    TableName tableName;
     String partName, allParts;
     List<String> cols;
 
-    public AnalyzeWork(FullTableName tableName, String partName, String allParts, List<String> cols) {
+    public AnalyzeWork(TableName tableName, String partName, String allParts, List<String> cols) {
       this.tableName = tableName;
       this.partName = partName;
       this.allParts = allParts;
@@ -534,7 +534,7 @@ public class StatsUpdaterThread extends Thread implements MetaStoreThread {
 
     public String buildCommand() {
       // Catalogs cannot be parsed as part of the query. Seems to be a bug.
-      String cmd = "analyze table " + tableName.db + "." + tableName.table;
+      String cmd = "analyze table " + tableName.getDb() + "." + tableName.getTable();
       assert partName == null || allParts == null;
       if (partName != null) {
         cmd += " partition(" + partName + ")";

http://git-wip-us.apache.org/repos/asf/hive/blob/416d7eb6/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 f328ad1..f7444a1 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
@@ -21,6 +21,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
 
 import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.hive.common.TableName;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.events.AlterPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.AlterTableEvent;
@@ -151,7 +152,7 @@ public class HiveAlterHandler implements AlterHandler {
       oldt = msdb.getTable(catName, dbname, name);
       if (oldt == null) {
         throw new InvalidOperationException("table " +
-            Warehouse.getCatalogQualifiedTableName(catName, dbname, name) + " doesn't exist");
+            TableName.getQualified(catName, dbname, name) + " doesn't exist");
       }
 
       if (oldt.getPartitionKeysSize() != 0) {
@@ -226,7 +227,7 @@ public class HiveAlterHandler implements AlterHandler {
           try {
             if (destFs.exists(destPath)) {
               throw new InvalidOperationException("New location for this table " +
-                  Warehouse.getCatalogQualifiedTableName(catName, newDbName, newTblName) +
+                  TableName.getQualified(catName, newDbName, newTblName) +
                       " already exists : " + destPath);
             }
             // check that src exists and also checks permissions necessary, rename src to dest
@@ -242,12 +243,12 @@ public class HiveAlterHandler implements AlterHandler {
           }
 
           if (!HiveMetaStore.isRenameAllowed(olddb, db)) {
-            LOG.error("Alter Table operation for " + Warehouse.getCatalogQualifiedTableName(catName, dbname, name) +
-                    "to new table = " + Warehouse.getCatalogQualifiedTableName(catName, newDbName, newTblName) +
+            LOG.error("Alter Table operation for " + TableName.getQualified(catName, dbname, name) +
+                    "to new table = " + TableName.getQualified(catName, newDbName, newTblName) +
                     " failed ");
             throw new MetaException("Alter table not allowed for table " +
-                    Warehouse.getCatalogQualifiedTableName(catName, dbname, name) +
-                    "to new table = " + Warehouse.getCatalogQualifiedTableName(catName, newDbName, newTblName));
+                    TableName.getQualified(catName, dbname, name) +
+                    "to new table = " + TableName.getQualified(catName, newDbName, newTblName));
           }
         }
 
@@ -369,8 +370,7 @@ public class HiveAlterHandler implements AlterHandler {
               + " Check metastore logs for detailed stack." + e.getMessage());
     } finally {
       if (!success) {
-        LOG.error("Failed to alter table " +
-            Warehouse.getCatalogQualifiedTableName(catName, dbname, name));
+        LOG.error("Failed to alter table " + TableName.getQualified(catName, dbname, name));
         msdb.rollbackTransaction();
         if (dataWasMoved) {
           try {

http://git-wip-us.apache.org/repos/asf/hive/blob/416d7eb6/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 cd68cbe..2ccaaa4 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
@@ -79,6 +79,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.hadoop.hive.common.TableName;
 import org.apache.hadoop.hive.metastore.api.*;
 import org.apache.hadoop.hive.metastore.events.AddForeignKeyEvent;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
@@ -919,7 +920,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
 
     private void startTableFunction(String function, String catName, String db, String tbl) {
       startFunction(function, " : tbl=" +
-          getCatalogQualifiedTableName(catName, db, tbl));
+          TableName.getQualified(catName, db, tbl));
     }
 
     private void startMultiTableFunction(String function, String db, List<String> tbls) {
@@ -930,13 +931,13 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     private void startPartitionFunction(String function, String cat, String db, String tbl,
                                         List<String> partVals) {
       startFunction(function, " : tbl=" +
-          getCatalogQualifiedTableName(cat, db, tbl) + "[" + join(partVals, ",") + "]");
+          TableName.getQualified(cat, db, tbl) + "[" + join(partVals, ",") + "]");
     }
 
     private void startPartitionFunction(String function, String catName, String db, String tbl,
                                         Map<String, String> partName) {
       startFunction(function, " : tbl=" +
-          getCatalogQualifiedTableName(catName, db, tbl) + "partition=" + partName);
+          TableName.getQualified(catName, db, tbl) + "partition=" + partName);
     }
 
     private void endFunction(String function, boolean successful, Exception e) {
@@ -2524,7 +2525,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         ms.dropConstraint(catName, dbname, name, null, true);
 
         if (!ms.dropTable(catName, dbname, name)) {
-          String tableName = getCatalogQualifiedTableName(catName, dbname, name);
+          String tableName = TableName.getQualified(catName, dbname, name);
           throw new MetaException(indexName == null ? "Unable to drop table " + tableName:
               "Unable to drop index table " + tableName + " for index " + indexName);
         } else {
@@ -2945,7 +2946,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       try {
         t = getMS().getTable(catName, dbname, name);
         if (t == null) {
-          throw new NoSuchObjectException(getCatalogQualifiedTableName(catName, dbname, name) +
+          throw new NoSuchObjectException(TableName.getQualified(catName, dbname, name) +
             " table not found");
         }
       } catch (Exception e) {
@@ -3376,7 +3377,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         tbl = ms.getTable(catName, dbName, tblName);
         if (tbl == null) {
           throw new InvalidObjectException("Unable to add partitions because "
-              + getCatalogQualifiedTableName(catName, dbName, tblName) +
+              + TableName.getQualified(catName, dbName, tblName) +
               " does not exist");
         }
 
@@ -3391,7 +3392,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         for (final Partition part : parts) {
           if (!part.getTableName().equals(tblName) || !part.getDbName().equals(dbName)) {
             throw new MetaException("Partition does not belong to target table " +
-                getCatalogQualifiedTableName(catName, dbName, tblName) + ": " +
+                TableName.getQualified(catName, dbName, tblName) + ": " +
                     part);
           }
 
@@ -3944,14 +3945,14 @@ public class HiveMetaStore extends ThriftHiveMetastore {
           ms.getTable(parsedDestDbName[CAT_NAME], parsedDestDbName[DB_NAME], destTableName);
       if (destinationTable == null) {
         throw new MetaException( "The destination table " +
-            getCatalogQualifiedTableName(parsedDestDbName[CAT_NAME],
+            TableName.getQualified(parsedDestDbName[CAT_NAME],
                 parsedDestDbName[DB_NAME], destTableName) + " not found");
       }
       Table sourceTable =
           ms.getTable(parsedSourceDbName[CAT_NAME], parsedSourceDbName[DB_NAME], sourceTableName);
       if (sourceTable == null) {
         throw new MetaException("The source table " +
-            getCatalogQualifiedTableName(parsedSourceDbName[CAT_NAME],
+            TableName.getQualified(parsedSourceDbName[CAT_NAME],
                 parsedSourceDbName[DB_NAME], sourceTableName) + " not found");
       }
 
@@ -4011,7 +4012,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       Database destDb = ms.getDatabase(parsedDestDbName[CAT_NAME], parsedDestDbName[DB_NAME]);
       if (!isRenameAllowed(srcDb, destDb)) {
         throw new MetaException("Exchange partition not allowed for " +
-                getCatalogQualifiedTableName(parsedSourceDbName[CAT_NAME],
+                TableName.getQualified(parsedSourceDbName[CAT_NAME],
                 parsedSourceDbName[DB_NAME], sourceTableName) + " Dest db : " + destDbName);
       }
       try {
@@ -4469,7 +4470,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         // metastore api call)
         Table t = getMS().getTable(catName, dbName, tblName);
         if (t == null) {
-          throw new NoSuchObjectException(getCatalogQualifiedTableName(catName, dbName, tblName)
+          throw new NoSuchObjectException(TableName.getQualified(catName, dbName, tblName)
               + " table not found");
         }
         firePreEvent(new PreReadTableEvent(t, this));
@@ -4998,7 +4999,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     private void alter_table_core(final String catName, final String dbname, final String name,
                                   final Table newTable, final EnvironmentContext envContext)
         throws InvalidOperationException, MetaException {
-      startFunction("alter_table", ": " + getCatalogQualifiedTableName(catName, dbname, name)
+      startFunction("alter_table", ": " + TableName.getQualified(catName, dbname, name)
           + " newtbl=" + newTable.getTableName());
 
       // Update the time if it hasn't been specified.
@@ -5389,7 +5390,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       Partition p = ms.getPartition(catName, db_name, tbl_name, partVals);
 
       if (p == null) {
-        throw new NoSuchObjectException(getCatalogQualifiedTableName(catName, db_name, tbl_name)
+        throw new NoSuchObjectException(TableName.getQualified(catName, db_name, tbl_name)
             + " partition (" + part_name + ") not found");
       }
       return p;
@@ -5401,7 +5402,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
 
       String[] parsedDbName = parseDbName(db_name, conf);
       startFunction("get_partition_by_name", ": tbl=" +
-          getCatalogQualifiedTableName(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name)
+          TableName.getQualified(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name)
           + " part=" + part_name);
       Partition ret = null;
       Exception ex = null;
@@ -5428,7 +5429,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         throws TException {
       String[] parsedDbName = parseDbName(db_name, conf);
       startFunction("append_partition_by_name", ": tbl="
-          + getCatalogQualifiedTableName(parsedDbName[CAT_NAME], parsedDbName[DB_NAME],
+          + TableName.getQualified(parsedDbName[CAT_NAME], parsedDbName[DB_NAME],
           tbl_name) + " part=" + part_name);
 
       Partition ret = null;
@@ -5483,7 +5484,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         final EnvironmentContext envContext) throws TException {
       String[] parsedDbName = parseDbName(db_name, conf);
       startFunction("drop_partition_by_name", ": tbl=" +
-          getCatalogQualifiedTableName(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name)
+          TableName.getQualified(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tbl_name)
           + " part=" + part_name);
 
       boolean ret = false;
@@ -5623,7 +5624,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       tableName = tableName.toLowerCase();
       colName = colName.toLowerCase();
       startFunction("get_column_statistics_by_table", ": table=" +
-          getCatalogQualifiedTableName(parsedDbName[CAT_NAME], parsedDbName[DB_NAME],
+          TableName.getQualified(parsedDbName[CAT_NAME], parsedDbName[DB_NAME],
               tableName) + " column=" + colName);
       ColumnStatistics statsObj = null;
       try {
@@ -5645,7 +5646,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       String dbName = request.getDbName().toLowerCase();
       String tblName = request.getTblName().toLowerCase();
       startFunction("get_table_statistics_req", ": table=" +
-          getCatalogQualifiedTableName(catName, dbName, tblName));
+          TableName.getQualified(catName, dbName, tblName));
       TableStatsResult result = null;
       List<String> lowerCaseColNames = new ArrayList<>(request.getColNames().size());
       for (String colName : request.getColNames()) {
@@ -5670,7 +5671,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       colName = colName.toLowerCase();
       String convertedPartName = lowerCaseConvertPartName(partName);
       startFunction("get_column_statistics_by_partition", ": table=" +
-          getCatalogQualifiedTableName(parsedDbName[CAT_NAME], parsedDbName[DB_NAME],
+          TableName.getQualified(parsedDbName[CAT_NAME], parsedDbName[DB_NAME],
           tableName) + " partition=" + convertedPartName + " column=" + colName);
       ColumnStatistics statsObj = null;
 
@@ -5698,7 +5699,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       String dbName = request.getDbName().toLowerCase();
       String tblName = request.getTblName().toLowerCase();
       startFunction("get_partitions_statistics_req", ": table=" +
-          getCatalogQualifiedTableName(catName, dbName, tblName));
+          TableName.getQualified(catName, dbName, tblName));
 
       PartitionsStatsResult result = null;
       List<String> lowerCaseColNames = new ArrayList<>(request.getColNames().size());
@@ -5743,7 +5744,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       List<ColumnStatisticsObj> statsObjs =  colStats.getStatsObj();
 
       startFunction("write_column_statistics", ":  table=" +
-          Warehouse.getCatalogQualifiedTableName(catName, dbName, tableName));
+          TableName.getQualified(catName, dbName, tableName));
       for (ColumnStatisticsObj statsObj:statsObjs) {
         colName = statsObj.getColName().toLowerCase();
         statsObj.setColName(colName);
@@ -5829,7 +5830,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       }
       String convertedPartName = lowerCaseConvertPartName(partName);
       startFunction("delete_column_statistics_by_partition",": table=" +
-          getCatalogQualifiedTableName(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tableName) +
+          TableName.getQualified(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tableName) +
           " partition=" + convertedPartName + " column=" + colName);
       boolean ret = false;
 
@@ -5855,7 +5856,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         colName = colName.toLowerCase();
       }
       startFunction("delete_column_statistics_by_table", ": table=" +
-          getCatalogQualifiedTableName(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tableName) + " column=" +
+          TableName.getQualified(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tableName) + " column=" +
           colName);
 
       boolean ret = false;
@@ -7318,7 +7319,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       String dbName = request.getDbName().toLowerCase();
       String tblName = request.getTblName().toLowerCase();
       startFunction("get_aggr_stats_for", ": table=" +
-          getCatalogQualifiedTableName(catName, dbName, tblName));
+          TableName.getQualified(catName, dbName, tblName));
 
       List<String> lowerCaseColNames = new ArrayList<>(request.getColNames().size());
       for (String colName : request.getColNames()) {
@@ -7450,7 +7451,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         throws MetaException, InvalidObjectException {
       Table t = getMS().getTable(catName, dbName, tableName);
       if (t == null) {
-        throw new InvalidObjectException(getCatalogQualifiedTableName(catName, dbName, tableName)
+        throw new InvalidObjectException(TableName.getQualified(catName, dbName, tableName)
             + " table not found");
       }
       return t;

http://git-wip-us.apache.org/repos/asf/hive/blob/416d7eb6/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 4df43d6..c7d2856 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
@@ -88,7 +88,6 @@ import org.apache.hadoop.hive.metastore.parser.ExpressionTree.TreeNode;
 import org.apache.hadoop.hive.metastore.parser.ExpressionTree.TreeVisitor;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.ColStatsObjWithSourceInfo;
-import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.FullTableName;
 import org.apache.hive.common.util.BloomFilter;
 import org.datanucleus.store.rdbms.query.ForwardQueryResult;
 import org.slf4j.Logger;
@@ -2469,7 +2468,7 @@ class MetaStoreDirectSql {
     TableType.MANAGED_TABLE.toString(), TableType.MATERIALIZED_VIEW.toString()
   };
 
-  public List<FullTableName> getTableNamesWithStats() throws MetaException {
+  public List<org.apache.hadoop.hive.common.TableName> getTableNamesWithStats() throws MetaException {
     // Could we also join with ACID tables to only get tables with outdated stats?
     String queryText0 = "SELECT DISTINCT " + TBLS + ".\"TBL_NAME\", " + DBS + ".\"NAME\", "
         + DBS + ".\"CTLG_NAME\" FROM " + TBLS + " INNER JOIN " + DBS + " ON "
@@ -2477,7 +2476,7 @@ class MetaStoreDirectSql {
     String queryText1 = " WHERE " + TBLS + ".\"TBL_TYPE\" IN ("
         + makeParams(STATS_TABLE_TYPES.length) + ")";
 
-    List<FullTableName> result = new ArrayList<>();
+    List<org.apache.hadoop.hive.common.TableName> result = new ArrayList<>();
 
     String queryText = queryText0 + " INNER JOIN " + TAB_COL_STATS
         + " ON " + TBLS + ".\"TBL_ID\" = " + TAB_COL_STATS + ".\"TBL_ID\"" + queryText1;
@@ -2531,24 +2530,24 @@ class MetaStoreDirectSql {
     }
   }
 
-  public List<FullTableName> getAllTableNamesForStats() throws MetaException {
+  public List<org.apache.hadoop.hive.common.TableName> getAllTableNamesForStats() throws MetaException {
     String queryText = "SELECT " + TBLS + ".\"TBL_NAME\", " + DBS + ".\"NAME\", "
         + DBS + ".\"CTLG_NAME\" FROM " + TBLS + " INNER JOIN " + DBS + " ON " + TBLS
         + ".\"DB_ID\" = " + DBS + ".\"DB_ID\""
         + " WHERE " + TBLS + ".\"TBL_TYPE\" IN (" + makeParams(STATS_TABLE_TYPES.length) + ")";
-    List<FullTableName> result = new ArrayList<>();
+    List<org.apache.hadoop.hive.common.TableName> result = new ArrayList<>();
     getStatsTableListResult(queryText, result);
     return result;
   }
 
   private void getStatsTableListResult(
-      String queryText, List<FullTableName> result) throws MetaException {
+      String queryText, List<org.apache.hadoop.hive.common.TableName> result) throws MetaException {
     LOG.debug("Running {}", queryText);
     Query<?> query = pm.newQuery("javax.jdo.query.SQL", queryText);
     try {
       List<Object[]> sqlResult = ensureList(executeWithArray(query, STATS_TABLE_TYPES, queryText));
       for (Object[] line : sqlResult) {
-        result.add(new FullTableName(
+        result.add(new org.apache.hadoop.hive.common.TableName(
             extractSqlString(line[2]), extractSqlString(line[1]), extractSqlString(line[0])));
       }
     } finally {

http://git-wip-us.apache.org/repos/asf/hive/blob/416d7eb6/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 5cdbbfc..b8eb173 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,8 +19,6 @@
 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;
 
@@ -82,7 +80,9 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.common.DatabaseName;
 import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.hadoop.hive.common.TableName;
 import org.apache.hadoop.hive.metastore.MetaStoreDirectSql.SqlFilterForPushdown;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
@@ -213,7 +213,6 @@ import org.apache.hadoop.hive.metastore.tools.SQLGenerator;
 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.MetaStoreUtils.FullTableName;
 import org.apache.hadoop.hive.metastore.utils.ObjectPair;
 import org.apache.thrift.TException;
 import org.datanucleus.AbstractNucleusContext;
@@ -1401,7 +1400,7 @@ public class ObjectStore implements RawStore, Configurable {
           deleteTableColumnStatistics(catName, dbName, tableName, null);
         } catch (NoSuchObjectException e) {
           LOG.info("Found no table level column statistics associated with {} to delete",
-              getCatalogQualifiedTableName(catName, dbName, tableName));
+              TableName.getQualified(catName, dbName, tableName));
         }
 
         List<MConstraint> tabConstraints = listAllTableConstraintsWithOptionalConstraintName(
@@ -1539,17 +1538,17 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @Override
-  public List<FullTableName> getTableNamesWithStats() throws MetaException, NoSuchObjectException {
-    return new GetListHelper<FullTableName>(null, null, null, true, false) {
+  public List<TableName> getTableNamesWithStats() throws MetaException, NoSuchObjectException {
+    return new GetListHelper<TableName>(null, null, null, true, false) {
       @Override
-      protected List<FullTableName> getSqlResult(
-          GetHelper<List<FullTableName>> ctx) throws MetaException {
+      protected List<TableName> getSqlResult(
+          GetHelper<List<TableName>> ctx) throws MetaException {
         return directSql.getTableNamesWithStats();
       }
 
       @Override
-      protected List<FullTableName> getJdoResult(
-          GetHelper<List<FullTableName>> ctx) throws MetaException {
+      protected List<TableName> getJdoResult(
+          GetHelper<List<TableName>> ctx) throws MetaException {
         throw new UnsupportedOperationException("UnsupportedOperationException"); // TODO: implement?
       }
     }.run(false);
@@ -1584,20 +1583,20 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @Override
-  public List<FullTableName> getAllTableNamesForStats() throws MetaException, NoSuchObjectException {
-    return new GetListHelper<FullTableName>(null, null, null, true, false) {
+  public List<TableName> getAllTableNamesForStats() throws MetaException, NoSuchObjectException {
+    return new GetListHelper<TableName>(null, null, null, true, false) {
       @Override
-      protected List<FullTableName> getSqlResult(
-          GetHelper<List<FullTableName>> ctx) throws MetaException {
+      protected List<TableName> getSqlResult(
+          GetHelper<List<TableName>> ctx) throws MetaException {
         return directSql.getAllTableNamesForStats();
       }
 
       @Override
-      protected List<FullTableName> getJdoResult(
-          GetHelper<List<FullTableName>> ctx) throws MetaException {
+      protected List<TableName> getJdoResult(
+          GetHelper<List<TableName>> ctx) throws MetaException {
         boolean commited = false;
         Query query = null;
-        List<FullTableName> result = new ArrayList<>();
+        List<TableName> result = new ArrayList<>();
         openTransaction();
         try {
           String paramStr = "", whereStr = "";
@@ -1616,7 +1615,7 @@ public class ObjectStore implements RawStore, Configurable {
               query, MetaStoreDirectSql.STATS_TABLE_TYPES);
           pm.retrieveAll(tbls);
           for (MTable tbl : tbls) {
-            result.add(new FullTableName(
+            result.add(new TableName(
                 tbl.getDatabase().getCatalogName(), tbl.getDatabase().getName(), tbl.getTableName()));
           }
           commited = commitTransaction();
@@ -1859,7 +1858,7 @@ public class ObjectStore implements RawStore, Configurable {
           "java.lang.String table, java.lang.String db, java.lang.String catname");
       query.setUnique(true);
       LOG.debug("Executing getMTable for " +
-          getCatalogQualifiedTableName(catName, db, table));
+          TableName.getQualified(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.
@@ -1930,7 +1929,7 @@ public class ObjectStore implements RawStore, Configurable {
         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));
+              DatabaseName.getQualified(catName, db));
         }
       } else {
         for (Iterator iter = mtables.iterator(); iter.hasNext(); ) {
@@ -2011,7 +2010,7 @@ public class ObjectStore implements RawStore, Configurable {
     } catch (NoSuchObjectException e) {
       LOG.error("Could not convert to MTable", e);
       throw new InvalidObjectException("Database " +
-          getCatalogQualifiedDbName(catName, tbl.getDbName()) + " doesn't exist.");
+          DatabaseName.getQualified(catName, tbl.getDbName()) + " doesn't exist.");
     }
 
     // If the table has property EXTERNAL set, update table type
@@ -2934,7 +2933,7 @@ public class ObjectStore implements RawStore, Configurable {
       throws MetaException, NoSuchObjectException {
 
     LOG.info("Table: {} filter: \"{}\" cols: {}",
-        getCatalogQualifiedTableName(catName, dbName, tableName), filter, cols);
+        TableName.getQualified(catName, dbName, tableName), filter, cols);
     List<String> partitionNames = null;
     List<Partition> partitions = null;
     Table tbl = getTable(catName, dbName, tableName);
@@ -2962,7 +2961,7 @@ public class ObjectStore implements RawStore, Configurable {
 
     if (partitionNames == null && partitions == null) {
       throw new MetaException("Cannot obtain list of partitions by filter:\"" + filter +
-          "\" for " + getCatalogQualifiedTableName(catName, dbName, tableName));
+          "\" for " + TableName.getQualified(catName, dbName, tableName));
     }
 
     if (!ascending) {
@@ -3154,7 +3153,7 @@ public class ObjectStore implements RawStore, Configurable {
     tableName = normalizeIdentifier(tableName);
     Table table = getTable(catName, dbName, tableName);
     if (table == null) {
-      throw new NoSuchObjectException(getCatalogQualifiedTableName(catName, dbName, tableName)
+      throw new NoSuchObjectException(TableName.getQualified(catName, dbName, tableName)
           + " table not found");
     }
     String partNameMatcher = MetaStoreUtils.makePartNameMatcher(table, part_vals);
@@ -3872,7 +3871,7 @@ public class ObjectStore implements RawStore, Configurable {
     MTable mtable = getMTable(catName, dbName, tblName);
     if (mtable == null) {
       throw new NoSuchObjectException("Specified catalog.database.table does not exist : "
-          + getCatalogQualifiedTableName(catName, dbName, tblName));
+          + TableName.getQualified(catName, dbName, tblName));
     }
     return mtable;
   }
@@ -4677,7 +4676,7 @@ public class ObjectStore implements RawStore, Configurable {
       if (getPrimaryKeyConstraintName(parentTable.getDatabase().getCatalogName(),
           parentTable.getDatabase().getName(), parentTable.getTableName()) != null) {
         throw new MetaException(" Primary key already exists for: " +
-            getCatalogQualifiedTableName(catName, tableDB, tableName));
+            TableName.getQualified(catName, tableDB, tableName));
       }
       if (pks.get(i).getPk_name() == null) {
         if (pks.get(i).getKey_seq() == 1) {
@@ -8138,7 +8137,7 @@ public class ObjectStore implements RawStore, Configurable {
 
     try {
       LOG.info("Updating table level column statistics for table={}" +
-        " colName={}", getCatalogQualifiedTableName(table), colName);
+        " colName={}", Warehouse.getCatalogQualifiedTableName(table), colName);
       validateTableCols(table, Lists.newArrayList(colName));
 
       if (oldStats != null) {
@@ -8166,7 +8165,7 @@ public class ObjectStore implements RawStore, Configurable {
     String colName = mStatsObj.getColName();
 
     LOG.info("Updating partition level column statistics for table=" +
-        getCatalogQualifiedTableName(catName, dbName, tableName) +
+        TableName.getQualified(catName, dbName, tableName) +
         " partName=" + partName + " colName=" + colName);
 
     boolean foundCol = false;
@@ -8696,7 +8695,7 @@ public class ObjectStore implements RawStore, Configurable {
           pm.deletePersistent(mStatsObj);
         } else {
           throw new NoSuchObjectException("Column stats doesn't exist for table="
-              + getCatalogQualifiedTableName(catName, dbName, tableName) +
+              + TableName.getQualified(catName, dbName, tableName) +
               " partition=" + partName + " col=" + colName);
         }
       } else {
@@ -8710,7 +8709,7 @@ public class ObjectStore implements RawStore, Configurable {
           pm.deletePersistentAll(mStatsObjColl);
         } else {
           throw new NoSuchObjectException("Column stats don't exist for table="
-              + getCatalogQualifiedTableName(catName, dbName, tableName) + " partition" + partName);
+              + TableName.getQualified(catName, dbName, tableName) + " partition" + partName);
         }
       }
       ret = commitTransaction();
@@ -8741,7 +8740,7 @@ public class ObjectStore implements RawStore, Configurable {
       List<MTableColumnStatistics> mStatsObjColl;
       if (mTable == null) {
         throw new NoSuchObjectException("Table " +
-            getCatalogQualifiedTableName(catName, dbName, tableName)
+            TableName.getQualified(catName, dbName, tableName)
             + "  for which stats deletion is requested doesn't exist");
       }
       query = pm.newQuery(MTableColumnStatistics.class);

http://git-wip-us.apache.org/repos/asf/hive/blob/416d7eb6/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 a78d51b..c429048 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
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hive.metastore;
 
+import org.apache.hadoop.hive.common.TableName;
 import org.apache.hadoop.hive.metastore.api.CreationMetadata;
 import org.apache.hadoop.hive.metastore.api.ISchemaName;
 import org.apache.hadoop.hive.metastore.api.SchemaVersionDescriptor;
@@ -88,7 +89,6 @@ 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.hadoop.hive.metastore.utils.MetaStoreUtils.FullTableName;
 import org.apache.thrift.TException;
 
 public interface RawStore extends Configurable {
@@ -1641,9 +1641,9 @@ public interface RawStore extends Configurable {
   /** Removes outdated statistics. */
   int deleteRuntimeStats(int maxRetainSecs) throws MetaException;
 
-  List<FullTableName> getTableNamesWithStats() throws MetaException, NoSuchObjectException;
+  List<TableName> getTableNamesWithStats() throws MetaException, NoSuchObjectException;
 
-  List<FullTableName> getAllTableNamesForStats() throws MetaException, NoSuchObjectException;
+  List<TableName> getAllTableNamesForStats() throws MetaException, NoSuchObjectException;
 
   Map<String, List<String>> getPartitionColsWithStats(String catName, String dbName,
       String tableName) throws MetaException, NoSuchObjectException;

http://git-wip-us.apache.org/repos/asf/hive/blob/416d7eb6/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 5a6a5fa..da5a71c 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.common.TableName;
 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;
@@ -283,12 +284,14 @@ public class Warehouse {
     return getDefaultTablePath(db, table.getTableName(), MetaStoreUtils.isExternalTable(table));
   }
 
+  @Deprecated // Use TableName
   public static String getQualifiedName(Table table) {
-    return getQualifiedName(table.getDbName(), table.getTableName());
+    return TableName.getDbTable(table.getDbName(), table.getTableName());
   }
 
+  @Deprecated // Use TableName
   public static String getQualifiedName(String dbName, String tableName) {
-    return dbName + CAT_DB_TABLE_SEPARATOR + tableName;
+    return TableName.getDbTable(dbName, tableName);
   }
 
   public static String getQualifiedName(Partition partition) {
@@ -301,22 +304,7 @@ public class Warehouse {
    * @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;
+    return TableName.getQualified(table.getCatName(), table.getDbName(), table.getTableName());
   }
 
   public boolean mkdirs(Path f) throws MetaException {

http://git-wip-us.apache.org/repos/asf/hive/blob/416d7eb6/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 5a8b564..f6286ea 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
@@ -38,7 +38,9 @@ import java.util.regex.Pattern;
 
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.common.DatabaseName;
 import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.hadoop.hive.common.TableName;
 import org.apache.hadoop.hive.metastore.Deadline;
 import org.apache.hadoop.hive.metastore.FileMetadataHandler;
 import org.apache.hadoop.hive.metastore.ObjectStore;
@@ -116,7 +118,6 @@ 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.MetaStoreUtils.ColStatsObjWithSourceInfo;
-import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.FullTableName;
 import org.apache.hadoop.hive.metastore.utils.StringUtils;
 import org.apache.thrift.TException;
 import org.slf4j.Logger;
@@ -244,7 +245,7 @@ public class CachedStore implements RawStore, Configurable {
             } catch (NoSuchObjectException e) {
               // Continue with next database
               LOG.warn("Failed to cache database "
-                  + Warehouse.getCatalogQualifiedDbName(catName, dbName) + ", moving on", e);
+                  + DatabaseName.getQualified(catName, dbName) + ", moving on", e);
             }
           }
         } catch (MetaException e) {
@@ -263,7 +264,7 @@ public class CachedStore implements RawStore, Configurable {
           tblNames = rawStore.getAllTables(catName, dbName);
         } catch (MetaException e) {
           LOG.warn("Failed to cache tables for database "
-              + Warehouse.getCatalogQualifiedDbName(catName, dbName) + ", moving on");
+              + DatabaseName.getQualified(catName, dbName) + ", moving on");
           // Continue with next database
           continue;
         }
@@ -2407,7 +2408,7 @@ public class CachedStore implements RawStore, Configurable {
   }
 
   static boolean isNotInBlackList(String catName, String dbName, String tblName) {
-    String str = Warehouse.getCatalogQualifiedTableName(catName, dbName, tblName);
+    String str = TableName.getQualified(catName, dbName, tblName);
     for (Pattern pattern : blacklistPatterns) {
       LOG.debug("Trying to match: {} against blacklist pattern: {}", str, pattern);
       Matcher matcher = pattern.matcher(str);
@@ -2421,7 +2422,7 @@ public class CachedStore implements RawStore, Configurable {
   }
 
   private static boolean isInWhitelist(String catName, String dbName, String tblName) {
-    String str = Warehouse.getCatalogQualifiedTableName(catName, dbName, tblName);
+    String str = TableName.getQualified(catName, dbName, tblName);
     for (Pattern pattern : whitelistPatterns) {
       LOG.debug("Trying to match: {} against whitelist pattern: {}", str, pattern);
       Matcher matcher = pattern.matcher(str);
@@ -2495,12 +2496,12 @@ public class CachedStore implements RawStore, Configurable {
   }
 
   @Override
-  public List<FullTableName> getTableNamesWithStats() throws MetaException, NoSuchObjectException {
+  public List<TableName> getTableNamesWithStats() throws MetaException, NoSuchObjectException {
     return rawStore.getTableNamesWithStats();
   }
 
   @Override
-  public List<FullTableName> getAllTableNamesForStats() throws MetaException, NoSuchObjectException {
+  public List<TableName> getAllTableNamesForStats() throws MetaException, NoSuchObjectException {
     return rawStore.getAllTableNamesForStats();
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/416d7eb6/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 83952eb..32850fd 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
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hive.metastore.utils;
 
+import org.apache.hadoop.hive.common.TableName;
 import org.apache.hadoop.hive.metastore.api.WMPoolSchedulingPolicy;
 
 import com.google.common.base.Joiner;
@@ -270,7 +271,7 @@ public class MetaStoreUtils {
     }
     if (colStatsMap.size() < 1) {
       LOG.debug("No stats data found for: tblName= {}, partNames= {}, colNames= {}",
-          Warehouse.getCatalogQualifiedTableName(catName, dbName, tableName), partNames, colNames);
+          TableName.getQualified(catName, dbName, tableName), partNames, colNames);
       return new ArrayList<ColumnStatisticsObj>();
     }
     return aggrPartitionStats(colStatsMap, partNames, areAllPartsFound,
@@ -1806,34 +1807,4 @@ public class MetaStoreUtils {
     return catName;
   }
 
-
-  public static class FullTableName {
-    public final String catalog, db, table;
-
-    public FullTableName(String catalog, String db, String table) {
-      assert catalog != null && db != null && table != null : catalog + ", " + db + ", " + table;
-      this.catalog = catalog;
-      this.db = db;
-      this.table = table;
-    }
-
-    @Override
-    public String toString() {
-      return catalog + MetaStoreUtils.CATALOG_DB_SEPARATOR + db + "." + table;
-    }
-
-    @Override
-    public int hashCode() {
-      final int prime = 31;
-      return prime * (prime * (prime + catalog.hashCode()) + db.hashCode()) + table.hashCode();
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-      if (this == obj) return true;
-      if (obj == null || getClass() != obj.getClass()) return false;
-      FullTableName other = (FullTableName) obj;
-      return catalog.equals(other.catalog) && db.equals(other.db) && table.equals(other.table);
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/416d7eb6/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 9d50b25..bf00d2e 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
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hive.metastore;
 
+import org.apache.hadoop.hive.common.TableName;
 import org.apache.hadoop.hive.metastore.api.CreationMetadata;
 import org.apache.hadoop.hive.metastore.api.ISchemaName;
 import org.apache.hadoop.hive.metastore.api.SchemaVersionDescriptor;
@@ -86,7 +87,6 @@ import org.apache.hadoop.hive.metastore.api.WMMapping;
 import org.apache.hadoop.hive.metastore.api.WMPool;
 import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.ColStatsObjWithSourceInfo;
-import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.FullTableName;
 import org.apache.thrift.TException;
 
 /**
@@ -1188,13 +1188,13 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
   }
 
   @Override
-  public List<FullTableName> getTableNamesWithStats() throws MetaException,
+  public List<TableName> getTableNamesWithStats() throws MetaException,
       NoSuchObjectException {
     return null;
   }
 
   @Override
-  public List<FullTableName> getAllTableNamesForStats() throws MetaException,
+  public List<TableName> getAllTableNamesForStats() throws MetaException,
       NoSuchObjectException {
     return null;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/416d7eb6/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 7e15292..7ea09a0 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
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hive.metastore;
 
+import org.apache.hadoop.hive.common.TableName;
 import org.apache.hadoop.hive.metastore.api.CreationMetadata;
 import org.apache.hadoop.hive.metastore.api.ISchemaName;
 import org.apache.hadoop.hive.metastore.api.SchemaVersionDescriptor;
@@ -85,7 +86,6 @@ import org.apache.hadoop.hive.metastore.api.WMPool;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
-import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.FullTableName;
 import org.apache.thrift.TException;
 import org.junit.Assert;
 
@@ -1175,13 +1175,13 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
-  public List<FullTableName> getTableNamesWithStats() throws MetaException,
+  public List<TableName> getTableNamesWithStats() throws MetaException,
       NoSuchObjectException {
     return null;
   }
 
   @Override
-  public List<FullTableName> getAllTableNamesForStats() throws MetaException,
+  public List<TableName> getAllTableNamesForStats() throws MetaException,
       NoSuchObjectException {
     return null;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/416d7eb6/storage-api/src/java/org/apache/hadoop/hive/common/DatabaseName.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/common/DatabaseName.java b/storage-api/src/java/org/apache/hadoop/hive/common/DatabaseName.java
new file mode 100644
index 0000000..6a030bb
--- /dev/null
+++ b/storage-api/src/java/org/apache/hadoop/hive/common/DatabaseName.java
@@ -0,0 +1,97 @@
+/*
+ * 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.common;
+
+/**
+ * A container for fully qualified database name, i.e. catalogname.databasename.  Also contains
+ * utilities for string parsing.
+ */
+public class DatabaseName {
+  static final String CAT_DB_TABLE_SEPARATOR = ".";
+  private final String cat;
+  private final String db;
+
+  /**
+   *
+   * @param cat catalog name.  This cannot be null.  If you don't know the value, then likely the
+   *            right answer is to fetch it from SessionState.getCurrentCatalog() if you want to
+   *            get the catalog being used in the current session or
+   *            MetaStoreUtils.getDefaultCatalog() if you want to get the default catalog for
+   *            this Hive instance.
+   * @param db database name.  This cannot be null.
+   */
+  public DatabaseName(String cat, String db) {
+    this.cat = cat;
+    this.db = db;
+  }
+
+  /**
+   * Build a DatabaseName from a string of the form [catalog.]database.
+   * @param name name, can be "db" or "cat.db"
+   * @param defaultCatalog default catalog to use if catalog name is not in the name.  This can
+   *                       be null if you are absolutely certain that the catalog name is
+   *                       embedded in name.  If you want the default catalog to be determined by
+   *                       the session, use SessionState.getCurrentCatalog().  If you want it to
+   *                       be determined by the default for the Hive instance or you are not in a
+   *                       session, use MetaStoreUtils.getDefaultCatalog().
+   * @return new DatabaseName object.
+   */
+  public static DatabaseName fromString(String name, String defaultCatalog) {
+    if (name.contains(CAT_DB_TABLE_SEPARATOR)) {
+      String[] names = name.split("\\.");
+      if (names.length != 2) {
+        throw new RuntimeException("Database name must be either <dbname> or <catname>.<dbname>");
+      }
+      return new DatabaseName(names[0], names[1]);
+    } else {
+      assert defaultCatalog != null;
+      return new DatabaseName(defaultCatalog, name);
+    }
+  }
+
+  public String getCat() {
+    return cat;
+  }
+
+  public String getDb() {
+    return db;
+  }
+
+  public static String getQualified(String catName, String dbName) {
+    return catName + CAT_DB_TABLE_SEPARATOR + dbName;
+  }
+
+  @Override
+  public int hashCode() {
+    return cat.hashCode() * 31 + db.hashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj != null && obj instanceof DatabaseName) {
+      DatabaseName that = (DatabaseName)obj;
+      return db.equals(that.db) && cat.equals(that.cat);
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return cat + CAT_DB_TABLE_SEPARATOR + db;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/416d7eb6/storage-api/src/java/org/apache/hadoop/hive/common/TableName.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/common/TableName.java b/storage-api/src/java/org/apache/hadoop/hive/common/TableName.java
new file mode 100644
index 0000000..f5cb192
--- /dev/null
+++ b/storage-api/src/java/org/apache/hadoop/hive/common/TableName.java
@@ -0,0 +1,125 @@
+/*
+ * 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.common;
+
+/**
+ * A container for a fully qualified table name, i.e. catalogname.databasename.tablename.  Also
+ * includes utilities for string parsing.
+ */
+public class TableName {
+  private final String cat;
+  private final String db;
+  private final String table;
+
+  /**
+   *
+   * @param cat catalog name.  Cannot be null.  If you do not know it you can get it from
+   *            SessionState.getCurrentCatalog() if you want to use the catalog from the current
+   *            session, or from MetaStoreUtils.getDefaultCatalog() if you do not have a session
+   *            or want to use the default catalog for the Hive instance.
+   * @param db database name.  Cannot be null.  If you do not now it you can get it from
+   *           SessionState.getCurrentDatabase() or use Warehouse.DEFAULT_DATABASE_NAME.
+   * @param table  table name, cannot be null
+   */
+  public TableName(String cat, String db, String table) {
+    this.cat = cat;
+    this.db = db;
+    this.table = table;
+  }
+
+  /**
+   * Build a TableName from a string of the form [[catalog.]database.]table.
+   * @param name name in string form
+   * @param defaultCatalog default catalog to use if catalog is not in the name.  If you do not
+   *                       know it you can get it from SessionState.getCurrentCatalog() if you
+   *                       want to use the catalog from the current session, or from
+   *                       MetaStoreUtils.getDefaultCatalog() if you do not have a session or
+   *                       want to use the default catalog for the Hive instance.
+   * @param defaultDatabase default database to use if database is not in the name.  If you do
+   *                        not now it you can get it from SessionState.getCurrentDatabase() or
+   *                        use Warehouse.DEFAULT_DATABASE_NAME.
+   * @return TableName
+   */
+  public static TableName fromString(String name, String defaultCatalog, String defaultDatabase) {
+    if (name.contains(DatabaseName.CAT_DB_TABLE_SEPARATOR)) {
+      String names[] = name.split("\\.");
+      if (names.length == 2) {
+        return new TableName(defaultCatalog, names[0], names[1]);
+      } else if (names.length == 3) {
+        return new TableName(names[0], names[1], names[2]);
+      } else {
+        throw new RuntimeException("Table name must be either <tablename>, <dbname>.<tablename> " +
+            "or <catname>.<dbname>.<tablename>");
+      }
+
+    } else {
+      return new TableName(defaultCatalog, defaultDatabase, name);
+    }
+  }
+
+  public String getCat() {
+    return cat;
+  }
+
+  public String getDb() {
+    return db;
+  }
+
+  public String getTable() {
+    return table;
+  }
+
+  /**
+   * Get the name in db.table format, for use with stuff not yet converted to use the catalog.
+   */
+  public String getDbTable() {
+    return db + DatabaseName.CAT_DB_TABLE_SEPARATOR + table;
+
+  }
+
+  /**
+   * Get the name in db.table format, for use with stuff not yet converted to use the catalog.
+   */
+  public static String getDbTable(String dbName, String tableName) {
+    return dbName + DatabaseName.CAT_DB_TABLE_SEPARATOR + tableName;
+
+  }
+
+  public static String getQualified(String catName, String dbName, String tableName) {
+    return catName + DatabaseName.CAT_DB_TABLE_SEPARATOR + dbName + DatabaseName.CAT_DB_TABLE_SEPARATOR + tableName;
+  }
+
+  @Override
+  public int hashCode() {
+    return (cat.hashCode() * 31 + db.hashCode()) * 31 + table.hashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj != null && obj instanceof TableName) {
+      TableName that = (TableName)obj;
+      return table.equals(that.table) && db.equals(that.db) && cat.equals(that.cat);
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return cat + DatabaseName.CAT_DB_TABLE_SEPARATOR + db + DatabaseName.CAT_DB_TABLE_SEPARATOR + table;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/416d7eb6/storage-api/src/test/org/apache/hadoop/hive/common/TestDatabaseName.java
----------------------------------------------------------------------
diff --git a/storage-api/src/test/org/apache/hadoop/hive/common/TestDatabaseName.java b/storage-api/src/test/org/apache/hadoop/hive/common/TestDatabaseName.java
new file mode 100644
index 0000000..76c3a70
--- /dev/null
+++ b/storage-api/src/test/org/apache/hadoop/hive/common/TestDatabaseName.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.common;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestDatabaseName {
+
+  @Test
+  public void differentFromConf() {
+    String cat = "cat";
+    String db = "db";
+    DatabaseName dbName = new DatabaseName(cat, db);
+    Assert.assertEquals(cat, dbName.getCat());
+    Assert.assertEquals(db, dbName.getDb());
+    Assert.assertEquals("cat.db", dbName.toString());
+  }
+
+  @Test
+  public void fromString() {
+    DatabaseName dbName = DatabaseName.fromString("cat.db", null);
+    Assert.assertEquals("cat", dbName.getCat());
+    Assert.assertEquals("db", dbName.getDb());
+    dbName = DatabaseName.fromString("db", "cat");
+    Assert.assertEquals("cat", dbName.getCat());
+    Assert.assertEquals("db", dbName.getDb());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/416d7eb6/storage-api/src/test/org/apache/hadoop/hive/common/TestTableName.java
----------------------------------------------------------------------
diff --git a/storage-api/src/test/org/apache/hadoop/hive/common/TestTableName.java b/storage-api/src/test/org/apache/hadoop/hive/common/TestTableName.java
new file mode 100644
index 0000000..0a8cb2a
--- /dev/null
+++ b/storage-api/src/test/org/apache/hadoop/hive/common/TestTableName.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.common;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestTableName {
+  @Test
+  public void fullname() {
+    TableName name = new TableName("cat", "db", "t");
+    Assert.assertEquals("cat", name.getCat());
+    Assert.assertEquals("db", name.getDb());
+    Assert.assertEquals("t", name.getTable());
+    Assert.assertEquals("cat.db.t", name.toString());
+    Assert.assertEquals("db.t", name.getDbTable());
+  }
+
+  @Test
+  public void fromString() {
+    TableName name = TableName.fromString("cat.db.tab", null, null);
+    Assert.assertEquals("cat", name.getCat());
+    Assert.assertEquals("db", name.getDb());
+    Assert.assertEquals("tab", name.getTable());
+
+    name = TableName.fromString("db.tab", "cat", null);
+    Assert.assertEquals("cat", name.getCat());
+    Assert.assertEquals("db", name.getDb());
+    Assert.assertEquals("tab", name.getTable());
+
+    name = TableName.fromString("tab", "cat", "db");
+    Assert.assertEquals("cat", name.getCat());
+    Assert.assertEquals("db", name.getDb());
+    Assert.assertEquals("tab", name.getTable());
+  }
+}