You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by zs...@apache.org on 2008/12/22 21:59:24 UTC

svn commit: r728771 - in /hadoop/hive/trunk: ./ metastore/src/java/org/apache/hadoop/hive/metastore/ ql/src/java/org/apache/hadoop/hive/ql/metadata/ ql/src/test/org/apache/hadoop/hive/ql/metadata/

Author: zshao
Date: Mon Dec 22 12:59:23 2008
New Revision: 728771

URL: http://svn.apache.org/viewvc?rev=728771&view=rev
Log:
HIVE-192. Add database name parameter to getTable and dropTable. (Johan Oskarsson via zshao)

Added:
    hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestPartition.java
Modified:
    hadoop/hive/trunk/CHANGES.txt
    hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
    hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreClient.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Partition.java
    hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.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=728771&r1=728770&r2=728771&view=diff
==============================================================================
--- hadoop/hive/trunk/CHANGES.txt (original)
+++ hadoop/hive/trunk/CHANGES.txt Mon Dec 22 12:59:23 2008
@@ -26,6 +26,9 @@
 
   IMPROVEMENTS
 
+    HIVE-192. Add database name parameter to getTable and dropTable.
+    (Johan Oskarsson via zshao)
+
     HIVE-181. Restore UDFTestLength unit test for UDFs.
     (David Phillips via zshao)
 

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=728771&r1=728770&r2=728771&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 Mon Dec 22 12:59:23 2008
@@ -21,6 +21,7 @@
 import java.util.List;
 
 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;
@@ -42,14 +43,58 @@
   public List<String> getTables(String dbName, String tablePattern) throws MetaException, UnknownTableException, TException,
       UnknownDBException;
 
-  public void dropTable(String tableName, boolean deleteData) throws MetaException, UnknownTableException, TException, NoSuchObjectException;
+  /**
+   * Drop the table.
+   * @param tableName The table to drop
+   * @param deleteData Should we delete the underlying data
+   * @throws MetaException Could not drop table properly.
+   * @throws UnknownTableException The table wasn't found.
+   * @throws TException A thrift communication error occurred
+   * @throws NoSuchObjectException The table wasn't found.
+   */
+  public void dropTable(String tableName, boolean deleteData) 
+    throws MetaException, UnknownTableException, TException, NoSuchObjectException;
+
+  /**
+   * Drop the table.
+   * @param dbName The database for this table
+   * @param tableName The table to drop
+   * @throws MetaException Could not drop table properly.
+   * @throws NoSuchObjectException The table wasn't found.
+   * @throws TException A thrift communication error occurred
+   * @throws ExistingDependentsException
+   */
+  public void dropTable(String dbname, String tableName, boolean deleteData, 
+      boolean ignoreUknownTab) throws ExistingDependentsException, 
+      MetaException, TException, NoSuchObjectException;
 
   //public void createTable(String tableName, Properties schema) throws MetaException, UnknownTableException,
     //  TException;
 
   public boolean tableExists(String tableName) throws MetaException, TException, UnknownDBException;
 
-  public Table getTable(String tableName) throws MetaException, TException, NoSuchObjectException;
+  /**
+   * Get a table object. 
+   * @param tableName Name of the table to fetch.
+   * @return An object representing the table.
+   * @throws MetaException Could not fetch the table
+   * @throws TException A thrift communication error occurred 
+   * @throws NoSuchObjectException In case the table wasn't found.
+   */
+  public Table getTable(String tableName) throws MetaException, 
+    TException, NoSuchObjectException;
+  
+  /**
+   * Get a table object. 
+   * @param dbName The database the table is located in.
+   * @param tableName Name of the table to fetch.
+   * @return An object representing the table.
+   * @throws MetaException Could not fetch the table
+   * @throws TException A thrift communication error occurred 
+   * @throws NoSuchObjectException In case the table wasn't found.
+   */
+  public Table getTable(String dbName, String tableName) 
+    throws MetaException, TException, NoSuchObjectException;
 
   /**
    * @param table_name

Modified: hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreClient.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreClient.java?rev=728771&r1=728770&r2=728771&view=diff
==============================================================================
--- hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreClient.java (original)
+++ hadoop/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreClient.java Mon Dec 22 12:59:23 2008
@@ -36,6 +36,7 @@
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
 import org.apache.hadoop.hive.metastore.api.Constants;
+import org.apache.hadoop.hive.metastore.api.ExistingDependentsException;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
 import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
@@ -321,6 +322,17 @@
     Properties schema = this.getSchema(tableName);
     return MetaStoreUtils.getTable(conf, schema);
   }
+  
+  /**
+   * This operation is unsupported in this metastore.
+   * @see MetaStoreClient#getTable(String)
+   */
+  public Table getTable(String dbName, String tableName) throws MetaException,
+    TException, NoSuchObjectException {
+    throw new UnsupportedOperationException("getTable from a specific db " +
+    		"not supported by this metastore");
+  }
+  
 
   //These will disappear when the server is unified for both filestore and dbstore
   @Override
@@ -443,5 +455,15 @@
       throws NoSuchObjectException, MetaException, TException {
     return false;
   }
+  
+  /**
+   * This operation is unsupported in this metastore.
+   * @see MetaStoreClient#dropTable(String, String) 
+   */
+  public void dropTable(String dbname, String name, boolean deleteData, boolean ignoreUknownTab) throws 
+    ExistingDependentsException, MetaException, TException, NoSuchObjectException {
+    
+    throw new UnsupportedOperationException("Operation not supported in this metastore");
+  }  
 
 }

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=728771&r1=728770&r2=728771&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 Mon Dec 22 12:59:23 2008
@@ -42,7 +42,6 @@
 import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
-import org.apache.hadoop.hive.metastore.api.UnknownTableException;
 import org.apache.hadoop.hive.ql.parse.ParseDriver;
 import org.apache.hadoop.hive.serde2.Deserializer;
 import org.apache.hadoop.hive.serde2.SerDeException;
@@ -214,29 +213,57 @@
    * Drops table along with the data in it. If the table doesn't exist then it is a no-op
    * @param tableName
    * @throws HiveException
+   * @deprecated Use {@link #dropTable(String, String)} instead
    */
   public void dropTable(String tableName) throws HiveException {
     dropTable(tableName, true, true);
   }
 
+  
+  /**
+   * Drops table along with the data in it. If the table doesn't exist
+   * then it is a no-op
+   * @param dbName database where the table lives
+   * @param tableName table to drop
+   * @throws HiveException thrown if the drop fails
+   */
+  public void dropTable(String dbName, String tableName) throws HiveException {
+    dropTable(dbName, tableName, true, true);
+  }  
+  
+  /**
+   * Drops the table. 
+   * @param tableName
+   * @param deleteData deletes the underlying data along with metadata
+   * @param ignoreUnknownTab an exception if thrown if this is falser
+   * and table doesn't exist
+   * @throws HiveException
+   * @deprecated Use {@link #dropTable(String, String, boolean, boolean)} instead
+   */
+  public void dropTable(String tableName, boolean deleteData, 
+      boolean ignoreUnknownTab) throws HiveException {
+    
+    dropTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName,
+        deleteData, ignoreUnknownTab);
+  }
+
   /**
    * Drops the table. 
    * @param tableName
    * @param deleteData deletes the underlying data along with metadata
-   * @param ignoreUnknownTab an exception if thrown if this is falser and table doesn't exist
+   * @param ignoreUnknownTab an exception if thrown if this is falser and
+   * table doesn't exist
    * @throws HiveException
    */
-  public void dropTable(String tableName, boolean deleteData, boolean ignoreUnknownTab) throws HiveException {
+  public void dropTable(String dbName, String tableName, boolean deleteData,
+      boolean ignoreUnknownTab) throws HiveException {
+    
     try {
-      msc.dropTable(tableName, deleteData);
+      msc.dropTable(dbName, tableName, deleteData, ignoreUnknownTab);
     } catch (NoSuchObjectException e) {
       if (!ignoreUnknownTab) {
         throw new HiveException(e);
       }
-    } catch (UnknownTableException e) {
-      if (!ignoreUnknownTab) {
-        throw new HiveException(e);
-      }
     } catch (Exception e) {
       throw new HiveException(e);
     } 
@@ -250,20 +277,55 @@
    * Returns metadata of the table. 
    * @param tableName the name of the table
    * @return the table
-   * @exception HiveException if there's an internal error or if the table doesn't exist 
+   * @exception HiveException if there's an internal error or if the 
+   * table doesn't exist 
+   * @deprecated Use {@link #getTable(String, String)} instead
    */
   public Table getTable(final String tableName) throws HiveException {
     return this.getTable(tableName, true);
   }
   
   /**
+   * Returns metadata of the table. 
+   * @param dbName the name of the database
+   * @param tableName the name of the table
+   * @return the table
+   * @exception HiveException if there's an internal error or if the 
+   * table doesn't exist 
+   */
+  public Table getTable(final String dbName, final String tableName) 
+    throws HiveException {
+    
+    return this.getTable(dbName, tableName, true);
+  }  
+  
+  /**
    * Returns metadata of the table
-   * @param tableName
-   * @param throwException controls whether an exception is thrown or a returns a null
-   * @return
+   * @param tableName the name of the table
+   * @param throwException controls whether an exception is 
+   * thrown or a null returned
+   * @return the table or if something false and 
+   * throwException is false a null value.
    * @throws HiveException
+   * @deprecated Use {@link #getTable(String, String, boolean)} instead
    */
-  public Table getTable(final String tableName, boolean throwException) throws HiveException {
+  public Table getTable(final String tableName, boolean throwException) 
+    throws HiveException {
+    return getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName, 
+        throwException);
+  }
+  
+  /**
+   * Returns metadata of the table
+   * @param dbName the name of the database
+   * @param tableName the name of the table
+   * @param throwException controls whether an exception is thrown 
+   * or a returns a null
+   * @return the table or if throwException is false a null value.
+   * @throws HiveException
+   */
+  public Table getTable(final String dbName, final String tableName, 
+      boolean throwException) throws HiveException {
 
     if(tableName == null || tableName.equals("")) {
       throw new HiveException("empty table creation??");
@@ -271,7 +333,7 @@
     Table table = new Table();
     org.apache.hadoop.hive.metastore.api.Table tTable = null;
     try {
-      tTable = msc.getTable(tableName);
+      tTable = msc.getTable(dbName, tableName);
     } catch (NoSuchObjectException e) {
       if(throwException) {
         LOG.error(StringUtils.stringifyException(e));
@@ -510,16 +572,6 @@
       for (org.apache.hadoop.hive.metastore.api.Partition tpart : tParts) {
         parts.add(new Partition(tbl, tpart));
       }
-      // get the partitions on the HDFS location
-      List<Partition> hdfsPartitions = tbl.getPartitionsFromHDFS();
-      if(hdfsPartitions.size() != parts.size()) {
-        // HACK: either we are connecting to old metastore or metadata is out of sync with data
-        // TODO: for the former case, move this logic into OLD metastore and compare 
-        // the two lists here for any conflict between metadata and data
-        LOG.error("Metadata for partitions doesn't match the data in HDFS. Table name: " + tbl.getName());
-        // let's trust hdfs partitions for now
-        return hdfsPartitions;
-      }
       return parts;
     } else {
       // create an empty partition. 

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Partition.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Partition.java?rev=728771&r1=728770&r2=728771&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Partition.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Partition.java Mon Dec 22 12:59:23 2008
@@ -22,7 +22,6 @@
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.regex.Matcher;
@@ -36,7 +35,6 @@
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 
 /**
  * A Hive Table Partition: is a fundamental storage unit within a Table
@@ -84,7 +82,7 @@
         // is same as the table partition. 
         this.partPath = table.getPath();
       }
-      spec = makeSpecFromPath();
+      spec = createSpec(tbl, tp);
       
       URI tmpURI = table.getDataLocation();
       try {
@@ -95,83 +93,24 @@
       }
     }
     
-    // This is used when a Partition object is created solely from the hdfs partition directories
-    Partition(Table tbl, Path path) throws HiveException {
-      this.table = tbl;
-      // initialize the tPartition(thrift object) with the data from path and  table
-      this.tPartition = new org.apache.hadoop.hive.metastore.api.Partition();
-      this.tPartition.setDbName(tbl.getDbName());
-      this.tPartition.setTableName(tbl.getName());
-      StorageDescriptor sd = tbl.getTTable().getSd();
-      StorageDescriptor psd = new StorageDescriptor(
-          sd.getCols(), sd.getLocation(), sd.getInputFormat(), sd.getOutputFormat(),
-          sd.isCompressed(), sd.getNumBuckets(), sd.getSerdeInfo(), sd.getBucketCols(),
-          sd.getSortCols(), new HashMap<String, String>());
-      this.tPartition.setSd(psd);
-      // change the partition location
-      if(table.isPartitioned()) {
-        this.partPath = path;
-      } else {
-        // We are in the HACK territory. SemanticAnalyzer expects a single partition whose schema
-        // is same as the table partition. 
-        this.partPath = table.getPath();
-      }
-      spec = makeSpecFromPath();
-      psd.setLocation(this.partPath.toString());
-      List<String> partVals = new ArrayList<String> ();
-      tPartition.setValues(partVals);
-      for (FieldSchema field : tbl.getPartCols()) {
-        partVals.add(spec.get(field.getName()));
-      }
-      try {
-        this.partName = Warehouse.makePartName(tbl.getPartCols(), partVals);
-      } catch (MetaException e) {
-        throw new HiveException("Invalid partition key values", e);
-      }
-    }
-    
-    static final Pattern pat = Pattern.compile("([^/]+)=([^/]+)");
-    private LinkedHashMap<String, String> makeSpecFromPath() throws HiveException {
-      // Keep going up the path till it equals the parent
-      Path currPath = this.partPath;
-      LinkedHashMap<String, String> partSpec = new LinkedHashMap<String, String>();
-      List<FieldSchema> pcols = this.table.getPartCols();
-      for(int i = 0; i < pcols.size(); i++) {
-        FieldSchema col =  pcols.get(pcols.size() - i - 1);
-        if (currPath == null) {
-          throw new HiveException("Out of path components while expecting key: " + col.getName());
-        }
-        String component = currPath.getName();
-        // Check if the component is either of the form k=v
-        // or is the first component
-        // if neither is true then this is an invalid path
-        Matcher m = pat.matcher(component);
-        if (m.matches()) {
-          String k = m.group(1);
-          String v = m.group(2);
-
-          if (!k.equals(col.getName())) {
-            throw new HiveException("Key mismatch expected: " + col.getName() + " and got: " + k);
-          }
-          if (partSpec.containsKey(k)) {
-            throw new HiveException("Key " + k + " defined at two levels");
-          }
-
-          partSpec.put(k, v);
-        }
-        else {
-          throw new HiveException("Path " + currPath.toString() + " not a valid path");
-        }
-        currPath = currPath.getParent();
-      }
-      // reverse the list since we checked the part from leaf dir to table's base dir
-      LinkedHashMap<String, String> newSpec = new LinkedHashMap<String, String>();
-      for(int i = 0; i < table.getPartCols().size(); i++) {
-        FieldSchema  field = table.getPartCols().get(i);
-        String val = partSpec.get(field.getName());
-        newSpec.put(field.getName(), val);
+    /**
+     * Creates a partition name -> value spec map object
+     * @param tbl Use the information from this table.
+     * @param tp Use the information from this partition.
+     * @return Partition name to value mapping.
+     */
+    private LinkedHashMap<String, String> createSpec(Table tbl, 
+        org.apache.hadoop.hive.metastore.api.Partition tp) {
+      
+      List<FieldSchema> fsl = tbl.getPartCols();
+      List<String> tpl = tp.getValues();
+      LinkedHashMap<String, String> spec = new LinkedHashMap<String, String>();
+      for (int i = 0; i < tbl.getPartCols().size(); i++) {
+        FieldSchema fs = fsl.get(i);
+        String value = tpl.get(i);
+        spec.put(fs.getName(), value);
       }
-      return newSpec;
+      return spec;
     }
 
     public URI makePartURI(LinkedHashMap<String, String> spec) throws HiveException {

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java?rev=728771&r1=728770&r2=728771&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java Mon Dec 22 12:59:23 2008
@@ -28,11 +28,9 @@
 import java.util.Map;
 import java.util.Properties;
 import java.util.Vector;
-import java.util.regex.Pattern;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
@@ -52,7 +50,6 @@
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapred.InputFormat;
 import org.apache.hadoop.mapred.OutputFormat;
-import org.apache.hadoop.util.StringUtils;
 
 
 /**
@@ -566,55 +563,5 @@
   public List<Order> getSortCols() {
     return getTTable().getSd().getSortCols();
   }
-
-  private static void getPartPaths(FileSystem fs, Path p, Vector<String> partPaths) throws IOException {
-    // Base case for recursion
-    if (fs.isFile(p)) {
-      if (!partPaths.contains(p.getParent().toString())) {
-        partPaths.add(p.getParent().toString());
-      }
-    }
-    else {
-      FileStatus [] dirs = fs.listStatus(p);
-
-      if (dirs.length != 0 ) {
-        for(int i=0; i < dirs.length; ++i) {
-          getPartPaths(fs, dirs[i].getPath(), partPaths);
-        }
-      }
-      else {
-        // This is an empty partition
-        if (!partPaths.contains(p.toString())) {
-          partPaths.add(p.toString());
-        }
-      }
-    }
-
-    return;
-  }
-
-  static final Pattern pat = Pattern.compile("([^/]+)=([^/]+)");
-  public List<Partition> getPartitionsFromHDFS() throws HiveException {
-    ArrayList<Partition> ret = new ArrayList<Partition> ();
-    FileSystem fs = null;
-    Vector<String> partPaths = new Vector<String>();
-
-    try {
-      fs = FileSystem.get(getDataLocation(), Hive.get().getConf());
-      getPartPaths(fs, new Path(getDataLocation().getPath()), partPaths);
-      for(String partPath: partPaths) {
-        Path tmpPath = new Path(partPath);
-        if(!fs.getFileStatus(tmpPath).isDir()) {
-          throw new HiveException("Data in hdfs is messed up. Table " + getName() + " has a partition " + partPath + " that is not a directory");
-        }
-        ret.add(new Partition(this, tmpPath));
-      }
-    } catch (IOException e) {
-      LOG.error(StringUtils.stringifyException(e));
-      throw new HiveException("DB Error: Table " + getDataLocation() + " message: " + e.getMessage());
-    }
-
-    return ret;
-  }
   
 };

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=728771&r1=728770&r2=728771&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 Mon Dec 22 12:59:23 2008
@@ -25,6 +25,7 @@
 
 import junit.framework.TestCase;
 
+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.api.FieldSchema;
@@ -43,10 +44,12 @@
 public class TestHive extends TestCase {
   private Hive hm;
   private HiveConf hiveConf;
+  private FileSystem fs;
 
   protected void setUp() throws Exception {
     super.setUp();
     hiveConf = new HiveConf(this.getClass());
+    fs = FileSystem.get(hiveConf);
     try {
       this.hm = Hive.get(hiveConf);
     } catch (Exception e) {
@@ -221,14 +224,15 @@
     return tbl;
   }
 
-  public void testGetTables() throws Throwable {
+  public void testGetAndDropTables() throws Throwable {
     try {
       String dbName = "db_for_testgettables";
+      String table1Name = "table1";
       hm.dropDatabase(dbName);
       hm.createDatabase(dbName, "");
 
       List<String> ts = new ArrayList<String>(2);
-      ts.add("table1");
+      ts.add(table1Name);
       ts.add("table2");
       Table tbl1 = createTestTable(dbName, ts.get(0));
       hm.createTable(tbl1);
@@ -243,6 +247,17 @@
       fts = hm.getTablesForDb(dbName, ".*1");
       assertEquals(1, fts.size());
       assertEquals(ts.get(0), fts.get(0));
+      
+      //also test getting a table from a specific db
+      Table table1 = hm.getTable(dbName, table1Name);
+      assertNotNull(table1);
+      assertEquals(table1Name, table1.getName());
+      
+      assertTrue(fs.exists(table1.getPath()));
+      //and test dropping this specific table
+      hm.dropTable(dbName, table1Name);
+      assertFalse(fs.exists(table1.getPath()));
+      
       hm.dropDatabase(dbName);
     } catch (Throwable e) {
       System.err.println(StringUtils.stringifyException(e));

Added: hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestPartition.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestPartition.java?rev=728771&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestPartition.java (added)
+++ hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestPartition.java Mon Dec 22 12:59:23 2008
@@ -0,0 +1,46 @@
+package org.apache.hadoop.hive.ql.metadata;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import junit.framework.TestCase;
+
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.Partition;
+
+/**
+ * Test the partition class.
+ */
+public class TestPartition extends TestCase {
+
+  private static final String PARTITION_COL = "partcol";
+  private static final String PARTITION_VALUE = "value";
+  private static final String TABLENAME = "tablename";
+
+  /**
+   * Test that the Partition spec is created properly.
+   */
+  public void testPartition() throws HiveException, URISyntaxException {
+    Partition tp = new Partition();
+    tp.setTableName(TABLENAME);
+    
+    List<String> values = new ArrayList<String>();
+    values.add(PARTITION_VALUE);
+    tp.setValues(values);
+    
+    List<FieldSchema> partCols = new ArrayList<FieldSchema>();
+    partCols.add(new FieldSchema(PARTITION_COL, "string", ""));
+    
+    Table tbl = new Table(TABLENAME);
+    tbl.setDataLocation(new URI("tmplocation"));
+    tbl.setPartCols(partCols);
+    
+    Map<String, String> spec = new org.apache.hadoop.hive.ql.metadata.Partition(tbl, tp).getSpec();
+    assertFalse(spec.isEmpty());
+    assertEquals(spec.get(PARTITION_COL), PARTITION_VALUE);
+  }
+  
+}