You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jo...@apache.org on 2009/04/15 19:08:17 UTC

svn commit: r765271 - in /hadoop/hive/trunk: ./ cli/src/java/org/apache/hadoop/hive/cli/ conf/ metastore/if/ metastore/src/gen-javabean/org/apache/hadoop/hive/metastore/api/ metastore/src/gen-php/ metastore/src/gen-py/hive_metastore/ metastore/src/java...

Author: johan
Date: Wed Apr 15 17:08:16 2009
New Revision: 765271

URL: http://svn.apache.org/viewvc?rev=765271&view=rev
Log:
HIVE-143. Remove the old file based metastore, in favour of the RDBMS based metastore. (prasadc via johan)

Removed:
    hadoop/hive/trunk/cli/src/java/org/apache/hadoop/hive/cli/MetadataProcessor.java
    hadoop/hive/trunk/metastore/src/gen-javabean/org/apache/hadoop/hive/metastore/api/ExistingDependentsException.java
    hadoop/hive/trunk/metastore/src/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftMetaStore.java
    hadoop/hive/trunk/metastore/src/gen-php/ThriftMetaStore.php
    hadoop/hive/trunk/metastore/src/gen-py/hive_metastore/ThriftMetaStore-remote
    hadoop/hive/trunk/metastore/src/gen-py/hive_metastore/ThriftMetaStore.py
    hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/DB.java
    hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/FileStore.java
    hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStore.java
    hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreClient.java
    hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreServer.java
    hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/ROTable.java
    hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/RWTable.java
    hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/Table.java
    hadoop/hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/MetaStoreTestBase.java
    hadoop/hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestAlter.java
    hadoop/hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestCreateDB.java
    hadoop/hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestDBGetName.java
    hadoop/hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestDrop.java
    hadoop/hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestGetDBs.java
    hadoop/hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestGetSchema.java
    hadoop/hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestGetTable.java
    hadoop/hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestGetTables.java
    hadoop/hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestPartitions.java
    hadoop/hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestTableExists.java
    hadoop/hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestTablePath.java
    hadoop/hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestTruncate.java
Modified:
    hadoop/hive/trunk/CHANGES.txt
    hadoop/hive/trunk/conf/hive-default.xml
    hadoop/hive/trunk/metastore/if/hive_metastore.thrift
    hadoop/hive/trunk/metastore/src/gen-php/hive_metastore_types.php
    hadoop/hive/trunk/metastore/src/gen-py/hive_metastore/__init__.py
    hadoop/hive/trunk/metastore/src/gen-py/hive_metastore/ttypes.py
    hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
    hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
    hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java

Modified: hadoop/hive/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/CHANGES.txt?rev=765271&r1=765270&r2=765271&view=diff
==============================================================================
--- hadoop/hive/trunk/CHANGES.txt (original)
+++ hadoop/hive/trunk/CHANGES.txt Wed Apr 15 17:08:16 2009
@@ -3,6 +3,9 @@
 Trunk - unreleased changes
 
   INCOMPATIBLE CHANGES
+  
+    HIVE-143. Remove the old file based metastore, in favour of the
+    RDBMS based metastore. (prasadc via johan)
 
   NEW FEATURES
 

Modified: hadoop/hive/trunk/conf/hive-default.xml
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/conf/hive-default.xml?rev=765271&r1=765270&r2=765271&view=diff
==============================================================================
--- hadoop/hive/trunk/conf/hive-default.xml (original)
+++ hadoop/hive/trunk/conf/hive-default.xml Wed Apr 15 17:08:16 2009
@@ -35,18 +35,6 @@
 </property>
 
 <property>
-  <name>hive.metastore.metadb.dir</name>
-  <value>file:///var/metastore/metadb/</value>
-  <description>The location of filestore metadata base dir</description>
-</property>
-
-<property>
-  <name>hive.metastore.uris</name>
-  <value>file:///var/metastore/metadb/</value>
-  <description>Comma separated list of URIs of metastore servers. The first server that can be connected to will be used.</description>
-</property>
-
-<property>
   <name>hive.metastore.warehouse.dir</name>
   <value>/user/hive/warehouse</value>
   <description>location of default database for the warehouse</description>

Modified: hadoop/hive/trunk/metastore/if/hive_metastore.thrift
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/metastore/if/hive_metastore.thrift?rev=765271&r1=765270&r2=765271&view=diff
==============================================================================
--- hadoop/hive/trunk/metastore/if/hive_metastore.thrift (original)
+++ hadoop/hive/trunk/metastore/if/hive_metastore.thrift Wed Apr 15 17:08:16 2009
@@ -112,10 +112,6 @@
   string message
 }
 
-exception ExistingDependentsException {
-  string message
-}
-
 exception NoSuchObjectException {
   string message
 }
@@ -193,49 +189,6 @@
   bool create_index(1:Index index_def) throws(1:IndexAlreadyExistsException o1, 2:MetaException o2)
 }
 
-
-/**
-* This interface is deprecated.
-*/
-service ThriftMetaStore extends fb303.FacebookService
-{
-  // retrieve a printable representation of the fields in a table (logfile, type) or table subtype
-  list<FieldSchema> get_fields(string db_name, string table_name) throws (MetaException o1, UnknownTableException o2, UnknownDBException o3),
-
-  // get all the tables (logfiles, types) in the metastore - no partitioning like different dbs yet
-  list<string> get_tables(string db_name, string pattern)  throws (MetaException o1, UnknownTableException o2, UnknownDBException o3),
-
-  // retrieve the opaque schema representation of this table (logfile, type) which contains enough
-  // information for the caller to instantiate some kind of object that will let it examine the type.
-  // That object might be a thrift, jute, or SerDe.
-  map<string,string> get_schema(string table_name) throws (MetaException o1, UnknownTableException o2, UnknownDBException o3),
-
-  // add some structure to the table or change its structure
-  void alter_table(string db_name, string table_name, map<string,string> schema) throws (MetaException o1, UnknownTableException o2, UnknownDBException o3),
-
-  // create_table == create_table4 (table_name, SIMPLE_META_SERDE, '\t', "",  dict [ META_COLUMNS => columns]
-  // bugbug do above transformation and deprecate this API
-  void create_table(string db_name, string table_name, map<string,string> schema) throws (MetaException o1, UnknownDBException o2),
-
-  // drop a table (i.e., remove it from the metastore) - for now allow metastore to do the delete (so python shell can do drops)
-  void drop_table(string db_name, string table_name) throws  (MetaException o1, UnknownTableException o2, UnknownDBException o3),
-
-  // truncate a table - i.e., delete its data, but keep the hdfs directory and the schema
-  void truncate_table(string db_name, string table_name, string partition)  throws (MetaException o1, UnknownTableException o2, UnknownDBException o3),
-
-  // generally does the table exist
-  bool table_exists(string db_name, string table_name) throws (MetaException o1, UnknownDBException o2),
-
-  // create a table with named columns
-  list<string> get_partitions(string db_name, string table_name) throws (MetaException o1, UnknownTableException o2, UnknownDBException o3),
-
-  // enumerate all the databases in this store
-  list<string> get_dbs() throws  (MetaException o),
-
-  // /bin/cat the table in human readable format
-  list<string> cat(string db_name, string table_name,string partition, i32 high) throws  (MetaException o1, UnknownDBException o2, UnknownTableException o3),
-}
-
 // these should be needed only for backward compatibility with filestore
 const string META_TABLE_COLUMNS   = "columns",
 const string META_TABLE_COLUMN_TYPES   = "columns.types",

Modified: hadoop/hive/trunk/metastore/src/gen-php/hive_metastore_types.php
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/metastore/src/gen-php/hive_metastore_types.php?rev=765271&r1=765270&r2=765271&view=diff
==============================================================================
--- hadoop/hive/trunk/metastore/src/gen-php/hive_metastore_types.php (original)
+++ hadoop/hive/trunk/metastore/src/gen-php/hive_metastore_types.php Wed Apr 15 17:08:16 2009
@@ -2127,78 +2127,6 @@
 
 }
 
-class metastore_ExistingDependentsException extends TException {
-  static $_TSPEC;
-
-  public $message = null;
-
-  public function __construct($vals=null) {
-    if (!isset(self::$_TSPEC)) {
-      self::$_TSPEC = array(
-        -1 => array(
-          'var' => 'message',
-          'type' => TType::STRING,
-          ),
-        );
-    }
-    if (is_array($vals)) {
-      if (isset($vals['message'])) {
-        $this->message = $vals['message'];
-      }
-    }
-  }
-
-  public function getName() {
-    return 'ExistingDependentsException';
-  }
-
-  public function read($input)
-  {
-    $xfer = 0;
-    $fname = null;
-    $ftype = 0;
-    $fid = 0;
-    $xfer += $input->readStructBegin($fname);
-    while (true)
-    {
-      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
-      if ($ftype == TType::STOP) {
-        break;
-      }
-      switch ($fid)
-      {
-        case -1:
-          if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->message);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        default:
-          $xfer += $input->skip($ftype);
-          break;
-      }
-      $xfer += $input->readFieldEnd();
-    }
-    $xfer += $input->readStructEnd();
-    return $xfer;
-  }
-
-  public function write($output) {
-    $xfer = 0;
-    $xfer += $output->writeStructBegin('ExistingDependentsException');
-    if ($this->message !== null) {
-      $xfer += $output->writeFieldBegin('message', TType::STRING, -1);
-      $xfer += $output->writeString($this->message);
-      $xfer += $output->writeFieldEnd();
-    }
-    $xfer += $output->writeFieldStop();
-    $xfer += $output->writeStructEnd();
-    return $xfer;
-  }
-
-}
-
 class metastore_NoSuchObjectException extends TException {
   static $_TSPEC;
 

Modified: hadoop/hive/trunk/metastore/src/gen-py/hive_metastore/__init__.py
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/metastore/src/gen-py/hive_metastore/__init__.py?rev=765271&r1=765270&r2=765271&view=diff
==============================================================================
--- hadoop/hive/trunk/metastore/src/gen-py/hive_metastore/__init__.py (original)
+++ hadoop/hive/trunk/metastore/src/gen-py/hive_metastore/__init__.py Wed Apr 15 17:08:16 2009
@@ -1 +1 @@
-__all__ = ['ttypes', 'constants', 'ThriftHiveMetastore', 'ThriftMetaStore']
+__all__ = ['ttypes', 'constants', 'ThriftHiveMetastore']

Modified: hadoop/hive/trunk/metastore/src/gen-py/hive_metastore/ttypes.py
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/metastore/src/gen-py/hive_metastore/ttypes.py?rev=765271&r1=765270&r2=765271&view=diff
==============================================================================
--- hadoop/hive/trunk/metastore/src/gen-py/hive_metastore/ttypes.py (original)
+++ hadoop/hive/trunk/metastore/src/gen-py/hive_metastore/ttypes.py Wed Apr 15 17:08:16 2009
@@ -1420,58 +1420,6 @@
   def __ne__(self, other):
     return not (self == other)
 
-class ExistingDependentsException(Exception):
-
-  thrift_spec = None
-  def __init__(self, d=None):
-    self.message = None
-    if isinstance(d, dict):
-      if 'message' in d:
-        self.message = d['message']
-
-  def read(self, iprot):
-    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
-      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
-      return
-    iprot.readStructBegin()
-    while True:
-      (fname, ftype, fid) = iprot.readFieldBegin()
-      if ftype == TType.STOP:
-        break
-      if fid == -1:
-        if ftype == TType.STRING:
-          self.message = iprot.readString();
-        else:
-          iprot.skip(ftype)
-      else:
-        iprot.skip(ftype)
-      iprot.readFieldEnd()
-    iprot.readStructEnd()
-
-  def write(self, oprot):
-    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
-      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
-      return
-    oprot.writeStructBegin('ExistingDependentsException')
-    if self.message != None:
-      oprot.writeFieldBegin('message', TType.STRING, -1)
-      oprot.writeString(self.message)
-      oprot.writeFieldEnd()
-    oprot.writeFieldStop()
-    oprot.writeStructEnd()
-
-  def __str__(self): 
-    return str(self.__dict__)
-
-  def __repr__(self): 
-    return repr(self.__dict__)
-
-  def __eq__(self, other):
-    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
-
-  def __ne__(self, other):
-    return not (self == other)
-
 class NoSuchObjectException(Exception):
 
   thrift_spec = None

Modified: hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java?rev=765271&r1=765270&r2=765271&view=diff
==============================================================================
--- hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java (original)
+++ hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java Wed Apr 15 17:08:16 2009
@@ -28,7 +28,6 @@
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
 import org.apache.hadoop.hive.metastore.api.Database;
-import org.apache.hadoop.hive.metastore.api.ExistingDependentsException;
 import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
 import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
 import org.apache.hadoop.hive.metastore.api.MetaException;
@@ -318,7 +317,7 @@
    * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#drop_table(java.lang.String, java.lang.String, boolean)
    */
   public void dropTable(String dbname, String name) throws NoSuchObjectException,
-      ExistingDependentsException, MetaException, TException {
+      MetaException, TException {
         dropTable(dbname, name, true, true);
       }
 
@@ -333,7 +332,7 @@
    * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#drop_table(java.lang.String, java.lang.String, boolean)
    */
   public void dropTable(String dbname, String name, boolean deleteData, boolean ignoreUknownTab) throws 
-      ExistingDependentsException, MetaException, TException, NoSuchObjectException {
+      MetaException, TException, NoSuchObjectException {
     try {
       client.drop_table(dbname, name, deleteData);
     } catch (NoSuchObjectException e) {

Modified: hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java?rev=765271&r1=765270&r2=765271&view=diff
==============================================================================
--- hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java (original)
+++ hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java Wed Apr 15 17:08:16 2009
@@ -20,9 +20,7 @@
 
 import java.util.List;
 
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
-import org.apache.hadoop.hive.metastore.api.ExistingDependentsException;
 import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
 import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
 import org.apache.hadoop.hive.metastore.api.MetaException;
@@ -66,7 +64,7 @@
    * @throws ExistingDependentsException
    */
   public void dropTable(String dbname, String tableName, boolean deleteData, 
-      boolean ignoreUknownTab) throws ExistingDependentsException, 
+      boolean ignoreUknownTab) throws  
       MetaException, TException, NoSuchObjectException;
 
   //public void createTable(String tableName, Properties schema) throws MetaException, UnknownTableException,

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java?rev=765271&r1=765270&r2=765271&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java Wed Apr 15 17:08:16 2009
@@ -36,7 +36,6 @@
 import org.apache.hadoop.hive.metastore.HiveMetaException;
 import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
-import org.apache.hadoop.hive.metastore.MetaStoreClient;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
@@ -778,15 +777,13 @@
   }
 
   /**
-   * @return
-   * @throws HiveMetaException 
+   * Creates a metastore client. Currently it creates only JDBC based client as File based store
+   * support is removed
+   * @returns a Meta Store Client
+   * @throws HiveMetaException  if a working client can't be created
    */
   private IMetaStoreClient createMetaStoreClient() throws MetaException {
-    boolean useFileStore = conf.getBoolean("hive.metastore.usefilestore", false);
-    if(!useFileStore) {
       return new HiveMetaStoreClient(this.conf);
-    }
-    return new MetaStoreClient(this.conf);
   }
   
   /**

Modified: hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java?rev=765271&r1=765270&r2=765271&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java (original)
+++ hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java Wed Apr 15 17:08:16 2009
@@ -27,12 +27,12 @@
 
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.DB;
+import org.apache.hadoop.hive.metastore.MetaStoreUtils;
+import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat;
+import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat;
 import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
 import org.apache.hadoop.hive.serde2.thrift.test.Complex;
-import org.apache.hadoop.hive.serde2.MetadataTypedColumnsetSerDe;
 import org.apache.hadoop.hive.serde2.ThriftDeserializer;
 import org.apache.hadoop.hive.serde.Constants;
 import org.apache.hadoop.mapred.SequenceFileInputFormat;
@@ -76,7 +76,7 @@
       // create a simple table and test create, drop, get
       String tableName = "table_for_testtable";
       try {
-        this.hm.dropTable(tableName);
+        this.hm.dropTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName);
       } catch (HiveException e1) {
         e1.printStackTrace();
         assertTrue("Unable to drop table", false);
@@ -89,7 +89,7 @@
       fields.add(new FieldSchema("col3", Constants.DOUBLE_TYPE_NAME, "double -- thrift column"));
       tbl.setFields(fields);
 
-      tbl.setOutputFormatClass(IgnoreKeyTextOutputFormat.class);
+      tbl.setOutputFormatClass(HiveIgnoreKeyTextOutputFormat.class);
       tbl.setInputFormatClass(SequenceFileInputFormat.class);
 
       tbl.setProperty("comment", "this is a test table created as part junit tests");
@@ -131,13 +131,14 @@
 
       // get table
       Table ft = null;
+      Warehouse wh = new Warehouse(hiveConf);
       try {
-        ft = hm.getTable(tableName);
+        ft = hm.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName);
         ft.checkValidity();
         assertEquals("Table names didn't match for table: " + tableName, tbl.getName(), ft.getName());
         assertEquals("Table owners didn't match for table: " + tableName, tbl.getOwner(), ft.getOwner());
         assertEquals("Table retention didn't match for table: " + tableName, tbl.getRetention(), ft.getRetention());
-        assertEquals("Data location is not set correctly", DB.getDefaultTablePath(tableName, this.hiveConf).toString(), ft.getDataLocation().toString());
+        assertEquals("Data location is not set correctly", wh.getDefaultTablePath(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName).toString(), ft.getDataLocation().toString());
         // now that URI is set correctly, set the original table's uri and then compare the two tables
         tbl.setDataLocation(ft.getDataLocation());
         assertTrue("Tables doesn't match: " + tableName, ft.getTTable().equals(tbl.getTTable()));
@@ -149,8 +150,8 @@
       }
 
       try {
-        hm.dropTable(tableName, true, false);
-        Table ft2 = hm.getTable(tableName, false);
+        hm.dropTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName, true, false);
+        Table ft2 = hm.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName, false);
         assertNull("Unable to drop table ", ft2);
       } catch (HiveException e) {
         assertTrue("Unable to drop table: " + tableName, false);
@@ -170,7 +171,7 @@
     String tableName = "table_for_test_thrifttable";
     try {
       try {
-        this.hm.dropTable(tableName);
+        this.hm.dropTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName);
       } catch (HiveException e1) {
         System.err.println(StringUtils.stringifyException(e1));
         assertTrue("Unable to drop table", false);
@@ -188,15 +189,16 @@
         assertTrue("Unable to create table: " + tableName, false);
       }
       // get table
+      Warehouse wh = new Warehouse(hiveConf);
       Table ft = null;
       try {
-        ft = hm.getTable(tableName);
+        ft = hm.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName);
         assertNotNull("Unable to fetch table", ft);
         ft.checkValidity();
         assertEquals("Table names didn't match for table: " + tableName, tbl.getName(), ft.getName());
         assertEquals("Table owners didn't match for table: " + tableName, tbl.getOwner(), ft.getOwner());
         assertEquals("Table retention didn't match for table: " + tableName, tbl.getRetention(), ft.getRetention());
-        assertEquals("Data location is not set correctly", DB.getDefaultTablePath(tableName, this.hiveConf).toString(), ft.getDataLocation().toString());
+        assertEquals("Data location is not set correctly", wh.getDefaultTablePath(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName).toString(), ft.getDataLocation().toString());
         // now that URI is set correctly, set the original table's uri and then compare the two tables
         tbl.setDataLocation(ft.getDataLocation());
         assertTrue("Tables  doesn't match: " + tableName, ft.getTTable().equals(tbl.getTTable()));
@@ -206,7 +208,7 @@
         System.err.println(StringUtils.stringifyException(e));
         assertTrue("Unable to fetch table correctly: " + tableName, false);
       }
-      hm.dropTable(tableName);
+      hm.dropTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName);
     } catch (Throwable e) {
       System.err.println(StringUtils.stringifyException(e));
       System.err.println("testThriftTable() failed");
@@ -271,7 +273,7 @@
     try {
       String tableName = "table_for_testpartition";
       try {
-        hm.dropTable(tableName);
+        hm.dropTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName);
       } catch (HiveException e) {
         System.err.println(StringUtils.stringifyException(e));
         assertTrue("Unable to drop table: " + tableName, false);
@@ -284,14 +286,14 @@
       part_cols.add("ds");
       part_cols.add("hr");
       try {
-        hm.createTable(tableName, cols, part_cols, TextInputFormat.class, IgnoreKeyTextOutputFormat.class);
+        hm.createTable(tableName, cols, part_cols, TextInputFormat.class, HiveIgnoreKeyTextOutputFormat.class);
       } catch (HiveException e) {
         System.err.println(StringUtils.stringifyException(e));
         assertTrue("Unable to create table: " + tableName, false);
       }
       Table tbl = null;
       try {
-        tbl = hm.getTable(tableName);
+        tbl = hm.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName);
       } catch (HiveException e) {
         System.err.println(StringUtils.stringifyException(e));
         assertTrue("Unable to fetch table: " + tableName, false);
@@ -306,7 +308,7 @@
         System.err.println(StringUtils.stringifyException(e));
         assertTrue("Unable to create parition for table: " + tableName, false);
       }
-      hm.dropTable(tableName);
+      hm.dropTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName);
     } catch (Throwable e) {
       System.err.println(StringUtils.stringifyException(e));
       System.err.println("testPartition() failed");