You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by na...@apache.org on 2010/08/27 07:41:45 UTC

svn commit: r990026 [8/10] - in /hadoop/hive/trunk: ./ eclipse-templates/ metastore/if/ metastore/src/gen-cpp/ metastore/src/gen-javabean/org/apache/hadoop/hive/metastore/api/ metastore/src/gen-php/ metastore/src/gen-py/hive_metastore/ metastore/src/ja...

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java?rev=990026&r1=990025&r2=990026&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java Fri Aug 27 05:41:43 2010
@@ -32,10 +32,10 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.DriverContext;
-import org.apache.hadoop.hive.ql.hooks.LineageInfo.DataContainer;
 import org.apache.hadoop.hive.ql.hooks.WriteEntity;
+import org.apache.hadoop.hive.ql.hooks.LineageInfo.DataContainer;
 import org.apache.hadoop.hive.ql.io.HiveFileFormatUtils;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.Partition;
@@ -46,7 +46,6 @@ import org.apache.hadoop.hive.ql.plan.Lo
 import org.apache.hadoop.hive.ql.plan.MoveWork;
 import org.apache.hadoop.hive.ql.plan.api.StageType;
 import org.apache.hadoop.hive.ql.session.SessionState;
-import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.util.StringUtils;
 
 /**
@@ -132,8 +131,7 @@ public class MoveTask extends Task<MoveW
         }
         String mesg_detail = " from " + tbd.getSourceDir();
         console.printInfo(mesg.toString(), mesg_detail);
-        Table table = db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tbd
-            .getTable().getTableName());
+        Table table = db.getTable(tbd.getTable().getTableName());
 
         if (work.getCheckFileFormat()) {
           // Get all files from the src directory

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=990026&r1=990025&r2=990026&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 Fri Aug 27 05:41:43 2010
@@ -18,6 +18,16 @@
 
 package org.apache.hadoop.hive.ql.metadata;
 
+import static org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME;
+import static org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_STORAGE;
+import static org.apache.hadoop.hive.serde.Constants.COLLECTION_DELIM;
+import static org.apache.hadoop.hive.serde.Constants.ESCAPE_CHAR;
+import static org.apache.hadoop.hive.serde.Constants.FIELD_DELIM;
+import static org.apache.hadoop.hive.serde.Constants.LINE_DELIM;
+import static org.apache.hadoop.hive.serde.Constants.MAPKEY_DELIM;
+import static org.apache.hadoop.hive.serde.Constants.SERIALIZATION_FORMAT;
+import static org.apache.hadoop.hive.serde.Constants.STRING_TYPE_NAME;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -42,6 +52,7 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
+import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Index;
 import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
@@ -51,7 +62,6 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.index.HiveIndexHandler;
-import org.apache.hadoop.hive.serde.Constants;
 import org.apache.hadoop.hive.serde2.Deserializer;
 import org.apache.hadoop.hive.serde2.SerDeException;
 import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
@@ -72,6 +82,7 @@ public class Hive {
 
   private HiveConf conf = null;
   private IMetaStoreClient metaStoreClient;
+  private String currentDatabase;
 
   private static ThreadLocal<Hive> hiveDB = new ThreadLocal() {
     @Override
@@ -172,6 +183,69 @@ public class Hive {
   }
 
   /**
+   * Create a database
+   * @param db
+   * @param ifNotExist if true, will ignore AlreadyExistsException exception
+   * @throws AlreadyExistsException
+   * @throws HiveException
+   */
+  public void createDatabase(Database db, boolean ifNotExist)
+      throws AlreadyExistsException, HiveException {
+    try {
+      getMSC().createDatabase(db);
+    } catch (AlreadyExistsException e) {
+      if (!ifNotExist) {
+        throw e;
+      }
+    } catch (Exception e) {
+      throw new HiveException(e);
+    }
+  }
+
+  /**
+   * Create a Database. Raise an error if a database with the same name already exists.
+   * @param db
+   * @throws AlreadyExistsException
+   * @throws HiveException
+   */
+  public void createDatabase(Database db) throws AlreadyExistsException, HiveException {
+    createDatabase(db, false);
+  }
+
+  /**
+   * Drop a database.
+   * @param name
+   * @throws NoSuchObjectException
+   * @throws HiveException
+   * @see org.apache.hadoop.hive.metastore.HiveMetaStoreClient#dropDatabase(java.lang.String)
+   */
+  public void dropDatabase(String name) throws HiveException, NoSuchObjectException {
+    dropDatabase(name, true, false);
+  }
+
+
+  /**
+   * Drop a database
+   * @param name
+   * @param deleteData
+   * @param ignoreUnknownDb if true, will ignore NoSuchObjectException
+   * @return
+   * @throws HiveException
+   * @throws NoSuchObjectException
+   */
+  public void dropDatabase(String name, boolean deleteData, boolean ignoreUnknownDb)
+      throws HiveException, NoSuchObjectException {
+    try {
+      getMSC().dropDatabase(name, deleteData, ignoreUnknownDb);
+    } catch (NoSuchObjectException e) {
+      throw e;
+    } catch (Exception e) {
+      throw new HiveException(e);
+    }
+  }
+
+
+  /**
    * Creates a table metdata and the directory for the table data
    *
    * @param tableName
@@ -223,13 +297,12 @@ public class Hive {
       throw new HiveException("columns not specified for table " + tableName);
     }
 
-    Table tbl = new Table(tableName);
+    Table tbl = new Table(getCurrentDatabase(), tableName);
     tbl.setInputFormatClass(fileInputFormat.getName());
     tbl.setOutputFormatClass(fileOutputFormat.getName());
 
     for (String col : columns) {
-      FieldSchema field = new FieldSchema(col,
-          org.apache.hadoop.hive.serde.Constants.STRING_TYPE_NAME, "default");
+      FieldSchema field = new FieldSchema(col, STRING_TYPE_NAME, "default");
       tbl.getCols().add(field);
     }
 
@@ -237,9 +310,7 @@ public class Hive {
       for (String partCol : partCols) {
         FieldSchema part = new FieldSchema();
         part.setName(partCol);
-        part.setType(org.apache.hadoop.hive.serde.Constants.STRING_TYPE_NAME); // default
-                                                                               // partition
-                                                                               // key
+        part.setType(STRING_TYPE_NAME); // default partition key
         tbl.getPartCols().add(part);
       }
     }
@@ -263,8 +334,7 @@ public class Hive {
   public void alterTable(String tblName, Table newTbl)
       throws InvalidOperationException, HiveException {
     try {
-      getMSC().alter_table(MetaStoreUtils.DEFAULT_DATABASE_NAME, tblName,
-          newTbl.getTTable());
+      getMSC().alter_table(getCurrentDatabase(), tblName, newTbl.getTTable());
     } catch (MetaException e) {
       throw new HiveException("Unable to alter table.", e);
     } catch (TException e) {
@@ -286,7 +356,7 @@ public class Hive {
   public void alterPartition(String tblName, Partition newPart)
       throws InvalidOperationException, HiveException {
     try {
-      getMSC().alter_partition(MetaStoreUtils.DEFAULT_DATABASE_NAME, tblName,
+      getMSC().alter_partition(getCurrentDatabase(), tblName,
           newPart.getTPartition());
 
     } catch (MetaException e) {
@@ -318,6 +388,9 @@ public class Hive {
    */
   public void createTable(Table tbl, boolean ifNotExists) throws HiveException {
     try {
+      if (tbl.getDbName() == null || "".equals(tbl.getDbName().trim())) {
+        tbl.setDbName(getCurrentDatabase());
+      }
       if (tbl.getCols().size() == 0) {
         tbl.setFields(MetaStoreUtils.getFieldsFromDeserializer(tbl.getTableName(),
             tbl.getDeserializer()));
@@ -377,7 +450,7 @@ public class Hive {
       throws HiveException {
 
     try {
-      String dbName = MetaStoreUtils.DEFAULT_DATABASE_NAME;
+      String dbName = getCurrentDatabase();
       Index old_index = null;
       try {
         old_index = getIndex(dbName, tableName, indexName);
@@ -420,22 +493,20 @@ public class Hive {
       }
 
       if (fieldDelim != null) {
-        serdeInfo.getParameters().put(Constants.FIELD_DELIM, fieldDelim);
-        serdeInfo.getParameters().put(Constants.SERIALIZATION_FORMAT,
-            fieldDelim);
+        serdeInfo.getParameters().put(FIELD_DELIM, fieldDelim);
+        serdeInfo.getParameters().put(SERIALIZATION_FORMAT, fieldDelim);
       }
       if (fieldEscape != null) {
-        serdeInfo.getParameters().put(Constants.ESCAPE_CHAR, fieldEscape);
+        serdeInfo.getParameters().put(ESCAPE_CHAR, fieldEscape);
       }
       if (collItemDelim != null) {
-        serdeInfo.getParameters()
-            .put(Constants.COLLECTION_DELIM, collItemDelim);
+        serdeInfo.getParameters().put(COLLECTION_DELIM, collItemDelim);
       }
       if (mapKeyDelim != null) {
-        serdeInfo.getParameters().put(Constants.MAPKEY_DELIM, mapKeyDelim);
+        serdeInfo.getParameters().put(MAPKEY_DELIM, mapKeyDelim);
       }
       if (lineDelim != null) {
-        serdeInfo.getParameters().put(Constants.LINE_DELIM, lineDelim);
+        serdeInfo.getParameters().put(LINE_DELIM, lineDelim);
       }
 
       if (serdeProps != null) {
@@ -481,7 +552,7 @@ public class Hive {
       HiveIndexHandler indexHandler = HiveUtils.getIndexHandler(this.getConf(), indexHandlerClass);
 
       if (indexHandler.usesIndexTable()) {
-        tt = new org.apache.hadoop.hive.ql.metadata.Table(indexTblName).getTTable();
+        tt = new org.apache.hadoop.hive.ql.metadata.Table(dbName, indexTblName).getTTable();
         List<FieldSchema> partKeys = baseTbl.getPartitionKeys();
         tt.setPartitionKeys(partKeys);
         tt.setTableType(TableType.INDEX_TABLE.toString());
@@ -537,6 +608,26 @@ public class Hive {
    * @throws HiveException
    *           thrown if the drop fails
    */
+  public void dropTable(String tableName) throws HiveException {
+    dropTable(getCurrentDatabase(), 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
+   * 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);
   }
@@ -570,7 +661,18 @@ public class Hive {
   }
 
   /**
-   * Returns metadata of the table.
+   * Returns metadata for the table named tableName in the current database.
+   * @param tableName the name of the table
+   * @return
+   * @throws HiveException if there's an internal error or if the
+   * table doesn't exist
+   */
+  public Table getTable(final String tableName) throws HiveException {
+    return this.getTable(getCurrentDatabase(), tableName, true);
+  }
+
+  /**
+   * Returns metadata of the table
    *
    * @param dbName
    *          the name of the database
@@ -580,9 +682,7 @@ public class Hive {
    * @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 {
-
+  public Table getTable(final String dbName, final String tableName) throws HiveException {
     return this.getTable(dbName, tableName, true);
   }
 
@@ -623,12 +723,11 @@ public class Hive {
     if (!TableType.VIRTUAL_VIEW.toString().equals(tTable.getTableType())) {
       // Fix the non-printable chars
       Map<String, String> parameters = tTable.getSd().getParameters();
-      String sf = parameters.get(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_FORMAT);
+      String sf = parameters.get(SERIALIZATION_FORMAT);
       if (sf != null) {
         char[] b = sf.toCharArray();
         if ((b.length == 1) && (b[0] < 10)) { // ^A, ^B, ^C, ^D, \t
-          parameters.put(org.apache.hadoop.hive.serde.Constants.SERIALIZATION_FORMAT,
-              Integer.toString(b[0]));
+          parameters.put(SERIALIZATION_FORMAT, Integer.toString(b[0]));
         }
       }
 
@@ -654,12 +753,27 @@ public class Hive {
     return table;
   }
 
+  /**
+   * Get all table names for the current database.
+   * @return List of table names
+   * @throws HiveException
+   */
   public List<String> getAllTables() throws HiveException {
-    return getTablesByPattern(".*");
+    return getAllTables(getCurrentDatabase());
+  }
+
+  /**
+   * Get all table names for the specified database.
+   * @param dbName
+   * @return List of table names
+   * @throws HiveException
+   */
+  public List<String> getAllTables(String dbName) throws HiveException {
+    return getTablesByPattern(dbName, ".*");
   }
 
   /**
-   * returns all existing tables from default database which match the given
+   * Returns all existing tables from default database which match the given
    * pattern. The matching occurs as per Java regular expressions
    *
    * @param tablePattern
@@ -667,13 +781,28 @@ public class Hive {
    * @return list of table names
    * @throws HiveException
    */
-  public List<String> getTablesByPattern(String tablePattern)
-      throws HiveException {
-    return getTablesForDb(MetaStoreUtils.DEFAULT_DATABASE_NAME, tablePattern);
+  public List<String> getTablesByPattern(String tablePattern) throws HiveException {
+    return getTablesByPattern(getCurrentDatabase(), tablePattern);
+  }
+
+  /**
+   * Returns all existing tables from the specified database which match the given
+   * pattern. The matching occurs as per Java regular expressions.
+   * @param dbName
+   * @param tablePattern
+   * @return list of table names
+   * @throws HiveException
+   */
+  public List<String> getTablesByPattern(String dbName, String tablePattern) throws HiveException {
+    try {
+      return getMSC().getTables(dbName, tablePattern);
+    } catch (Exception e) {
+      throw new HiveException(e);
+    }
   }
 
   /**
-   * returns all existing tables from the given database which match the given
+   * Returns all existing tables from the given database which match the given
    * pattern. The matching occurs as per Java regular expressions
    *
    * @param database
@@ -693,31 +822,58 @@ public class Hive {
   }
 
   /**
-   * @param name
-   * @param locationUri
-   * @return true or false
-   * @throws AlreadyExistsException
-   * @throws MetaException
-   * @throws TException
-   * @see org.apache.hadoop.hive.metastore.HiveMetaStoreClient#createDatabase(java.lang.String,
-   *      java.lang.String)
+   * Get all existing database names.
+   *
+   * @return List of database names.
+   * @throws HiveException
    */
-  protected boolean createDatabase(String name, String locationUri)
-      throws AlreadyExistsException, MetaException, TException {
-    return getMSC().createDatabase(name, locationUri);
+  public List<String> getAllDatabases() throws HiveException {
+    try {
+      return getMSC().getAllDatabases();
+    } catch (Exception e) {
+      throw new HiveException(e);
+    }
   }
 
   /**
-   * @param name
-   * @return true or false
-   * @throws MetaException
-   * @throws TException
-   * @see org.apache.hadoop.hive.metastore.HiveMetaStoreClient#dropDatabase(java.lang.String)
+   * Get all existing databases that match the given
+   * pattern. The matching occurs as per Java regular expressions
+   *
+   * @param databasePattern
+   *          java re pattern
+   * @return list of database names
+   * @throws HiveException
+   */
+  public List<String> getDatabasesByPattern(String databasePattern) throws HiveException {
+    try {
+      return getMSC().getDatabases(databasePattern);
+    } catch (Exception e) {
+      throw new HiveException(e);
+    }
+  }
+
+  /**
+   * Query metadata to see if a database with the given name already exists.
+   *
+   * @param dbName
+   * @return true if a database with the given name already exists, false if
+   *         does not exist.
+   * @throws HiveException
    */
-  protected boolean dropDatabase(String name) throws MetaException, TException {
-    return getMSC().dropDatabase(name);
+  public boolean databaseExists(String dbName) throws HiveException {
+    try {
+      if (null != getMSC().getDatabase(dbName)) {
+        return true;
+      }
+      return false;
+    } catch (NoSuchObjectException e) {
+      return false;
+    } catch (Exception e) {
+      throw new HiveException(e);
+    }
   }
 
+
   /**
    * Load a directory into a Hive Table Partition - Alters existing content of
    * the partition with the contents of loadPath. - If he partition does not
@@ -740,7 +896,7 @@ public class Hive {
       Map<String, String> partSpec, boolean replace, Path tmpDirPath,
       boolean holdDDLTime)
       throws HiveException {
-    Table tbl = getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName);
+    Table tbl = getTable(tableName);
     try {
       /**
        * Move files before creating the partition since down stream processes
@@ -865,7 +1021,7 @@ public class Hive {
    */
   public void loadTable(Path loadPath, String tableName, boolean replace,
       Path tmpDirPath, boolean holdDDLTime) throws HiveException {
-    Table tbl = getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName);
+    Table tbl = getTable(tableName);
 
     if (replace) {
       tbl.replaceFiles(loadPath, tmpDirPath);
@@ -1122,6 +1278,25 @@ public class Hive {
     return qlPartitions;
   }
 
+  /**
+   * Get the name of the current database
+   * @return
+   */
+  public String getCurrentDatabase() {
+    if (null == currentDatabase) {
+      currentDatabase = DEFAULT_DATABASE_NAME;
+    }
+    return currentDatabase;
+  }
+
+  /**
+   * Set the name of the current database
+   * @param currentDatabase
+   */
+  public void setCurrentDatabase(String currentDatabase) {
+    this.currentDatabase = currentDatabase;
+  }
+
   static private void checkPaths(FileSystem fs, FileStatus[] srcs, Path destf,
       boolean replace) throws HiveException {
     try {
@@ -1284,10 +1459,8 @@ public class Hive {
               return null;
             }
             HiveStorageHandler storageHandler =
-              HiveUtils.getStorageHandler(
-                conf,
-                tbl.getParameters().get(
-                    org.apache.hadoop.hive.metastore.api.Constants.META_TABLE_STORAGE));
+              HiveUtils.getStorageHandler(conf,
+                tbl.getParameters().get(META_TABLE_STORAGE));
             if (storageHandler == null) {
               return null;
             }

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=990026&r1=990025&r2=990026&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 Fri Aug 27 05:41:43 2010
@@ -97,8 +97,8 @@ public class Table implements Serializab
     }
   }
 
-  public Table(String name) {
-    this(getEmptyTable(name));
+  public Table(String databaseName, String tableName) {
+    this(getEmptyTable(databaseName, tableName));
   }
 
   /**
@@ -120,7 +120,8 @@ public class Table implements Serializab
   /**
    * Initialize an emtpy table.
    */
-  static org.apache.hadoop.hive.metastore.api.Table getEmptyTable(String name) {
+  static org.apache.hadoop.hive.metastore.api.Table
+  getEmptyTable(String databaseName, String tableName) {
     StorageDescriptor sd = new StorageDescriptor();
     {
       sd.setSerdeInfo(new SerDeInfo());
@@ -144,8 +145,8 @@ public class Table implements Serializab
       t.setPartitionKeys(new ArrayList<FieldSchema>());
       t.setParameters(new HashMap<String, String>());
       t.setTableType(TableType.MANAGED_TABLE.toString());
-      t.setTableName(name);
-      t.setDbName(MetaStoreUtils.DEFAULT_DATABASE_NAME);
+      t.setDbName(databaseName);
+      t.setTableName(tableName);
     }
     return t;
   }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java?rev=990026&r1=990025&r2=990026&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java Fri Aug 27 05:41:43 2010
@@ -33,7 +33,6 @@ import org.antlr.runtime.tree.Tree;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Order;
 import org.apache.hadoop.hive.ql.Context;
@@ -571,8 +570,7 @@ public abstract class BaseSemanticAnalyz
               + tableName;
         }
 
-        tableHandle = db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME,
-            tableName);
+        tableHandle = db.getTable(tableName);
       } catch (InvalidTableException ite) {
         throw new SemanticException(ErrorMsg.INVALID_TABLE.getMsg(ast
             .getChild(0)), ite);

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java?rev=990026&r1=990025&r2=990026&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java Fri Aug 27 05:41:43 2010
@@ -18,6 +18,14 @@
 
 package org.apache.hadoop.hive.ql.parse;
 
+import static org.apache.hadoop.hive.ql.parse.HiveParser.TOK_CREATEDATABASE;
+import static org.apache.hadoop.hive.ql.parse.HiveParser.TOK_DATABASECOMMENT;
+import static org.apache.hadoop.hive.ql.parse.HiveParser.TOK_DROPDATABASE;
+import static org.apache.hadoop.hive.ql.parse.HiveParser.TOK_IFEXISTS;
+import static org.apache.hadoop.hive.ql.parse.HiveParser.TOK_IFNOTEXISTS;
+import static org.apache.hadoop.hive.ql.parse.HiveParser.TOK_SHOWDATABASES;
+import static org.apache.hadoop.hive.ql.parse.HiveParser.TOK_SWITCHDATABASE;
+
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -36,7 +44,6 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-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.metastore.api.Index;
@@ -45,6 +52,8 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.ql.exec.FetchTask;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.hooks.ReadEntity;
+import org.apache.hadoop.hive.ql.hooks.WriteEntity;
 import org.apache.hadoop.hive.ql.index.HiveIndex;
 import org.apache.hadoop.hive.ql.index.HiveIndexHandler;
 import org.apache.hadoop.hive.ql.index.HiveIndex.IndexType;
@@ -57,14 +66,17 @@ import org.apache.hadoop.hive.ql.metadat
 import org.apache.hadoop.hive.ql.plan.AddPartitionDesc;
 import org.apache.hadoop.hive.ql.plan.AlterTableDesc;
 import org.apache.hadoop.hive.ql.plan.AlterTableSimpleDesc;
+import org.apache.hadoop.hive.ql.plan.CreateDatabaseDesc;
 import org.apache.hadoop.hive.ql.plan.CreateIndexDesc;
 import org.apache.hadoop.hive.ql.plan.DDLWork;
 import org.apache.hadoop.hive.ql.plan.DescFunctionDesc;
 import org.apache.hadoop.hive.ql.plan.DescTableDesc;
+import org.apache.hadoop.hive.ql.plan.DropDatabaseDesc;
 import org.apache.hadoop.hive.ql.plan.DropIndexDesc;
 import org.apache.hadoop.hive.ql.plan.DropTableDesc;
 import org.apache.hadoop.hive.ql.plan.FetchWork;
 import org.apache.hadoop.hive.ql.plan.MsckDesc;
+import org.apache.hadoop.hive.ql.plan.ShowDatabasesDesc;
 import org.apache.hadoop.hive.ql.plan.ShowFunctionsDesc;
 import org.apache.hadoop.hive.ql.plan.ShowPartitionsDesc;
 import org.apache.hadoop.hive.ql.plan.ShowTableStatusDesc;
@@ -72,13 +84,12 @@ import org.apache.hadoop.hive.ql.plan.Sh
 import org.apache.hadoop.hive.ql.plan.ShowLocksDesc;
 import org.apache.hadoop.hive.ql.plan.LockTableDesc;
 import org.apache.hadoop.hive.ql.plan.UnlockTableDesc;
+import org.apache.hadoop.hive.ql.plan.SwitchDatabaseDesc;
 import org.apache.hadoop.hive.ql.plan.TableDesc;
 import org.apache.hadoop.hive.ql.plan.AlterTableDesc.AlterTableTypes;
 import org.apache.hadoop.hive.serde.Constants;
 import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
 import org.apache.hadoop.mapred.TextInputFormat;
-import org.apache.hadoop.hive.ql.hooks.ReadEntity;
-import org.apache.hadoop.hive.ql.hooks.WriteEntity;
 
 /**
  * DDLSemanticAnalyzer.
@@ -165,6 +176,9 @@ public class DDLSemanticAnalyzer extends
     } else if (ast.getToken().getType() == HiveParser.TOK_DESCTABLE) {
       ctx.setResFile(new Path(ctx.getLocalTmpFileURI()));
       analyzeDescribeTable(ast);
+    } else if (ast.getToken().getType() == TOK_SHOWDATABASES) {
+      ctx.setResFile(new Path(ctx.getLocalTmpFileURI()));
+      analyzeShowDatabases(ast);
     } else if (ast.getToken().getType() == HiveParser.TOK_SHOWTABLES) {
       ctx.setResFile(new Path(ctx.getLocalTmpFileURI()));
       analyzeShowTables(ast);
@@ -222,16 +236,72 @@ public class DDLSemanticAnalyzer extends
       analyzeLockTable(ast);
     } else if (ast.getToken().getType() == HiveParser.TOK_UNLOCKTABLE) {
       analyzeUnlockTable(ast);
+    } else if (ast.getToken().getType() == TOK_CREATEDATABASE) {
+      analyzeCreateDatabase(ast);
+    } else if (ast.getToken().getType() == TOK_DROPDATABASE) {
+      analyzeDropDatabase(ast);
+    } else if (ast.getToken().getType() == TOK_SWITCHDATABASE) {
+      analyzeSwitchDatabase(ast);
     } else {
       throw new SemanticException("Unsupported command.");
     }
   }
 
+  private void analyzeCreateDatabase(ASTNode ast) throws SemanticException {
+    String dbName = unescapeIdentifier(ast.getChild(0).getText());
+    boolean ifNotExists = false;
+    String dbComment = null;
+
+    for (int i = 1; i < ast.getChildCount(); i++) {
+      ASTNode childNode = (ASTNode) ast.getChild(i);
+      switch (childNode.getToken().getType()) {
+      case TOK_IFNOTEXISTS:
+        ifNotExists = true;
+        break;
+      case TOK_DATABASECOMMENT:
+        dbComment = unescapeSQLString(childNode.getChild(0).getText());
+        break;
+      default:
+        throw new SemanticException("Unrecognized token in CREATE DATABASE statement");
+      }
+    }
+
+    CreateDatabaseDesc createDatabaseDesc = new CreateDatabaseDesc();
+    createDatabaseDesc.setName(dbName);
+    createDatabaseDesc.setComment(dbComment);
+    createDatabaseDesc.setIfNotExists(ifNotExists);
+    createDatabaseDesc.setLocationUri(null);
+
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
+        createDatabaseDesc), conf));
+  }
+
+  private void analyzeDropDatabase(ASTNode ast) throws SemanticException {
+    String dbName = unescapeIdentifier(ast.getChild(0).getText());
+    boolean ifExists = false;
+
+    if (null != ast.getFirstChildWithType(TOK_IFEXISTS)) {
+      ifExists = true;
+    }
+
+    DropDatabaseDesc dropDatabaseDesc = new DropDatabaseDesc(dbName, ifExists);
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), dropDatabaseDesc), conf));
+  }
+
+  private void analyzeSwitchDatabase(ASTNode ast) {
+    String dbName = unescapeIdentifier(ast.getChild(0).getText());
+    SwitchDatabaseDesc switchDatabaseDesc = new SwitchDatabaseDesc(dbName);
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
+        switchDatabaseDesc), conf));
+  }
+
+
+
   private void analyzeDropTable(ASTNode ast, boolean expectView)
       throws SemanticException {
     String tableName = unescapeIdentifier(ast.getChild(0).getText());
     try {
-      Table tab = db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName, false);
+      Table tab = db.getTable(db.getCurrentDatabase(), tableName, false);
       // Ignore if table does not exist
       if (tab != null) {
         inputs.add(new ReadEntity(tab));
@@ -340,11 +410,11 @@ public class DDLSemanticAnalyzer extends
   private List<Task<?>> getIndexBuilderMapRed(String baseTableName, String indexName,
       HashMap<String, String> partSpec) throws SemanticException {
     try {
-      Index index = db.getIndex(MetaStoreUtils.DEFAULT_DATABASE_NAME, baseTableName, indexName);
-      Table indexTbl = db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME,index.getIndexTableName());
+      String dbName = db.getCurrentDatabase();
+      Index index = db.getIndex(dbName, baseTableName, indexName);
+      Table indexTbl = db.getTable(dbName, index.getIndexTableName());
       String baseTblName = index.getOrigTableName();
-      Table baseTbl = db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME,
-          baseTblName);
+      Table baseTbl = db.getTable(dbName, baseTblName);
 
       String handlerCls = index.getIndexHandlerClass();
       HiveIndexHandler handler = HiveUtils.getIndexHandler(conf, handlerCls);
@@ -415,7 +485,7 @@ public class DDLSemanticAnalyzer extends
     alterTblDesc.setOldName(tableName);
 
     try {
-      Table tab = db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName, false);
+      Table tab = db.getTable(db.getCurrentDatabase(), tableName, false);
       if (tab != null) {
         inputs.add(new ReadEntity(tab));
         outputs.add(new WriteEntity(tab));
@@ -439,7 +509,7 @@ public class DDLSemanticAnalyzer extends
     alterTblDesc.setOldName(tableName);
 
     try {
-      Table tab = db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName, false);
+      Table tab = db.getTable(db.getCurrentDatabase(), tableName, false);
       if (tab != null) {
         inputs.add(new ReadEntity(tab));
         outputs.add(new WriteEntity(tab));
@@ -465,7 +535,7 @@ public class DDLSemanticAnalyzer extends
     alterTblDesc.setSerdeName(serdeName);
 
     try {
-      Table tab = db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName, false);
+      Table tab = db.getTable(db.getCurrentDatabase(), tableName, false);
       if (tab != null) {
         inputs.add(new ReadEntity(tab));
         outputs.add(new WriteEntity(tab));
@@ -529,7 +599,7 @@ public class DDLSemanticAnalyzer extends
         outputFormat, serde, storageHandler, partSpec);
 
     try {
-      Table tab = db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName, false);
+      Table tab = db.getTable(db.getCurrentDatabase(), tableName, false);
       if (tab != null) {
         inputs.add(new ReadEntity(tab));
         outputs.add(new WriteEntity(tab));
@@ -550,7 +620,7 @@ public class DDLSemanticAnalyzer extends
     AlterTableDesc alterTblDesc = new AlterTableDesc (tableName, newLocation, partSpec);
 
     try {
-      Table tab = db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName, false);
+      Table tab = db.getTable(db.getCurrentDatabase(), tableName, false);
       if (tab != null) {
         inputs.add(new ReadEntity(tab));
         outputs.add(new WriteEntity(tab));
@@ -604,7 +674,7 @@ public class DDLSemanticAnalyzer extends
     }
 
     try {
-      Table tab = db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName, false);
+      Table tab = db.getTable(db.getCurrentDatabase(), tableName, false);
       if (tab != null) {
         inputs.add(new ReadEntity(tab));
         outputs.add(new WriteEntity(tab));
@@ -622,7 +692,7 @@ public class DDLSemanticAnalyzer extends
     String tableName = unescapeIdentifier(ast.getChild(0).getText());
 
     try {
-      Table tab = db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName, false);
+      Table tab = db.getTable(db.getCurrentDatabase(), tableName, false);
       if (tab != null) {
         inputs.add(new ReadEntity(tab));
         outputs.add(new WriteEntity(tab));
@@ -753,6 +823,18 @@ public class DDLSemanticAnalyzer extends
     setFetchTask(createFetchTask(showPartsDesc.getSchema()));
   }
 
+  private void analyzeShowDatabases(ASTNode ast) throws SemanticException {
+    ShowDatabasesDesc showDatabasesDesc;
+    if (ast.getChildCount() == 1) {
+      String databasePattern = unescapeSQLString(ast.getChild(0).getText());
+      showDatabasesDesc = new ShowDatabasesDesc(ctx.getResFile(), databasePattern);
+    } else {
+      showDatabasesDesc = new ShowDatabasesDesc(ctx.getResFile());
+    }
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), showDatabasesDesc), conf));
+    setFetchTask(createFetchTask(showDatabasesDesc.getSchema()));
+  }
+
   private void analyzeShowTables(ASTNode ast) throws SemanticException {
     ShowTablesDesc showTblsDesc;
     if (ast.getChildCount() == 1) {
@@ -769,7 +851,7 @@ public class DDLSemanticAnalyzer extends
   private void analyzeShowTableStatus(ASTNode ast) throws SemanticException {
     ShowTableStatusDesc showTblStatusDesc;
     String tableNames = unescapeIdentifier(ast.getChild(0).getText());
-    String dbName = MetaStoreUtils.DEFAULT_DATABASE_NAME;
+    String dbName = db.getCurrentDatabase();
     int children = ast.getChildCount();
     HashMap<String, String> partSpec = null;
     if (children >= 2) {
@@ -929,7 +1011,7 @@ public class DDLSemanticAnalyzer extends
     AlterTableDesc alterTblDesc = new AlterTableDesc(tblName,
       unescapeIdentifier(ast.getChild(1).getText()));
     try {
-      Table tab = db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tblName, false);
+      Table tab = db.getTable(db.getCurrentDatabase(), tblName, false);
       if (tab != null) {
         inputs.add(new ReadEntity(tab));
         outputs.add(new WriteEntity(tab));
@@ -974,7 +1056,7 @@ public class DDLSemanticAnalyzer extends
         .getChild(2).getText()), newType, newComment, first, flagCol);
 
     try {
-      Table tab = db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tblName, false);
+      Table tab = db.getTable(db.getCurrentDatabase(), tblName, false);
       if (tab != null) {
         inputs.add(new ReadEntity(tab));
         outputs.add(new WriteEntity(tab));
@@ -995,7 +1077,7 @@ public class DDLSemanticAnalyzer extends
         alterType);
 
     try {
-      Table tab = db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tblName, false);
+      Table tab = db.getTable(db.getCurrentDatabase(), tblName, false);
       if (tab != null) {
         inputs.add(new ReadEntity(tab));
         outputs.add(new WriteEntity(tab));
@@ -1015,7 +1097,7 @@ public class DDLSemanticAnalyzer extends
     DropTableDesc dropTblDesc = new DropTableDesc(tblName, partSpecs);
 
     try {
-      Table tab = db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tblName, false);
+      Table tab = db.getTable(db.getCurrentDatabase(), tblName, false);
       if (tab != null) {
         inputs.add(new ReadEntity(tab));
       }
@@ -1045,7 +1127,7 @@ public class DDLSemanticAnalyzer extends
 
     String tblName = unescapeIdentifier(ast.getChild(0).getText());
     try {
-      Table tab = db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tblName, false);
+      Table tab = db.getTable(db.getCurrentDatabase(), tblName, false);
       if (tab != null) {
         inputs.add(new ReadEntity(tab));
       }
@@ -1074,7 +1156,7 @@ public class DDLSemanticAnalyzer extends
         if (currentPart != null) {
           validatePartitionValues(currentPart);
           AddPartitionDesc addPartitionDesc = new AddPartitionDesc(
-              MetaStoreUtils.DEFAULT_DATABASE_NAME, tblName, currentPart,
+              db.getCurrentDatabase(), tblName, currentPart,
               currentLocation, ifNotExists);
           rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
               addPartitionDesc), conf));
@@ -1096,7 +1178,7 @@ public class DDLSemanticAnalyzer extends
     if (currentPart != null) {
       validatePartitionValues(currentPart);
       AddPartitionDesc addPartitionDesc = new AddPartitionDesc(
-          MetaStoreUtils.DEFAULT_DATABASE_NAME, tblName, currentPart,
+          db.getCurrentDatabase(), tblName, currentPart,
           currentLocation, ifNotExists);
       rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
           addPartitionDesc), conf));
@@ -1120,7 +1202,7 @@ public class DDLSemanticAnalyzer extends
     Table tab;
 
     try {
-      tab = db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tblName, false);
+      tab = db.getTable(db.getCurrentDatabase(), tblName, false);
       if (tab != null) {
         inputs.add(new ReadEntity(tab));
       }
@@ -1133,7 +1215,7 @@ public class DDLSemanticAnalyzer extends
 
     if (partSpecs.size() == 0) {
       AlterTableSimpleDesc touchDesc = new AlterTableSimpleDesc(
-          MetaStoreUtils.DEFAULT_DATABASE_NAME, tblName, null,
+          db.getCurrentDatabase(), tblName, null,
           AlterTableDesc.AlterTableTypes.TOUCH);
       outputs.add(new WriteEntity(tab));
       rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
@@ -1142,7 +1224,7 @@ public class DDLSemanticAnalyzer extends
       addTablePartsOutputs(tblName, partSpecs);
       for (Map<String, String> partSpec : partSpecs) {
         AlterTableSimpleDesc touchDesc = new AlterTableSimpleDesc(
-            MetaStoreUtils.DEFAULT_DATABASE_NAME, tblName, partSpec,
+            db.getCurrentDatabase(), tblName, partSpec,
             AlterTableDesc.AlterTableTypes.TOUCH);
         rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
                                                   touchDesc), conf));
@@ -1162,7 +1244,7 @@ public class DDLSemanticAnalyzer extends
     List<Map<String, String>> partSpecs = getPartitionSpecs(ast);
 
     try {
-      Table tab = db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tblName, false);
+      Table tab = db.getTable(db.getCurrentDatabase(), tblName, false);
       if (tab != null) {
         inputs.add(new ReadEntity(tab));
       }
@@ -1182,7 +1264,7 @@ public class DDLSemanticAnalyzer extends
 
     Map<String,String> partSpec = partSpecs.get(0);
       AlterTableSimpleDesc archiveDesc = new AlterTableSimpleDesc(
-          MetaStoreUtils.DEFAULT_DATABASE_NAME, tblName, partSpec,
+          db.getCurrentDatabase(), tblName, partSpec,
           (isUnArchive ? AlterTableTypes.UNARCHIVE : AlterTableTypes.ARCHIVE));
       rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
           archiveDesc), conf));
@@ -1273,7 +1355,7 @@ public class DDLSemanticAnalyzer extends
     throws SemanticException {
     Table tab;
     try {
-      tab = db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tblName);
+      tab = db.getTable(tblName);
     } catch (HiveException e) {
       throw new SemanticException(ErrorMsg.INVALID_TABLE.getMsg(tblName));
     }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/Hive.g
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/Hive.g?rev=990026&r1=990025&r2=990026&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/Hive.g (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/Hive.g Fri Aug 27 05:41:43 2010
@@ -85,6 +85,7 @@ TOK_STRING;
 TOK_LIST;
 TOK_STRUCT;
 TOK_MAP;
+TOK_CREATEDATABASE;
 TOK_CREATETABLE;
 TOK_CREATEINDEX;
 TOK_CREATEINDEX_INDEXTBLNAME;
@@ -113,6 +114,7 @@ TOK_ALTERTABLE_PROPERTIES;
 TOK_ALTERTABLE_CHANGECOL_AFTER_POSITION;
 TOK_ALTERINDEX_REBUILD;
 TOK_MSCK;
+TOK_SHOWDATABASES;
 TOK_SHOWTABLES;
 TOK_SHOWFUNCTIONS;
 TOK_SHOWPARTITIONS;
@@ -120,7 +122,10 @@ TOK_SHOW_TABLESTATUS;
 TOK_SHOWLOCKS;
 TOK_LOCKTABLE;
 TOK_UNLOCKTABLE;
+TOK_SWITCHDATABASE;
+TOK_DROPDATABASE;
 TOK_DROPTABLE;
+TOK_DATABASECOMMENT;
 TOK_TABCOLLIST;
 TOK_TABCOL;
 TOK_TABLECOMMENT;
@@ -162,6 +167,7 @@ TOK_TABLEPROPLIST;
 TOK_TABTYPE;
 TOK_LIMIT;
 TOK_TABLEPROPERTY;
+TOK_IFEXISTS;
 TOK_IFNOTEXISTS;
 TOK_HINTLIST;
 TOK_HINT;
@@ -227,7 +233,10 @@ loadStatement
 ddlStatement
 @init { msgs.push("ddl statement"); }
 @after { msgs.pop(); }
-    : createTableStatement
+    : createDatabaseStatement
+    | switchDatabaseStatement
+    | dropDatabaseStatement
+    | createTableStatement
     | dropTableStatement
     | alterStatement
     | descStatement
@@ -244,6 +253,13 @@ ddlStatement
     | unlockStatement
     ;
 
+ifExists
+@init { msgs.push("if exists clause"); }
+@after { msgs.pop(); }
+    : KW_IF KW_EXISTS
+    -> ^(TOK_IFEXISTS)
+    ;
+
 ifNotExists
 @init { msgs.push("if not exists clause"); }
 @after { msgs.pop(); }
@@ -251,6 +267,38 @@ ifNotExists
     -> ^(TOK_IFNOTEXISTS)
     ;
 
+
+createDatabaseStatement
+@init { msgs.push("create database statement"); }
+@after { msgs.pop(); }
+    : KW_CREATE (KW_DATABASE|KW_SCHEMA)
+        ifNotExists?
+        name=Identifier
+        databaseComment?
+    -> ^(TOK_CREATEDATABASE $name ifNotExists? databaseComment?)
+    ;
+
+switchDatabaseStatement
+@init { msgs.push("switch database statement"); }
+@after { msgs.pop(); }
+    : KW_USE Identifier
+    -> ^(TOK_SWITCHDATABASE Identifier)
+    ;
+
+dropDatabaseStatement
+@init { msgs.push("drop database statement"); }
+@after { msgs.pop(); }
+    : KW_DROP (KW_DATABASE|KW_SCHEMA) ifExists? Identifier
+    -> ^(TOK_DROPDATABASE Identifier ifExists?)
+    ;
+
+databaseComment
+@init { msgs.push("database's comment"); }
+@after { msgs.pop(); }
+    : KW_COMMENT comment=StringLiteral
+    -> ^(TOK_DATABASECOMMENT $comment)
+    ;
+
 createTableStatement
 @init { msgs.push("create table statement"); }
 @after { msgs.pop(); }
@@ -577,7 +625,8 @@ descStatement
 showStatement
 @init { msgs.push("show statement"); }
 @after { msgs.pop(); }
-    : KW_SHOW KW_TABLES showStmtIdentifier?  -> ^(TOK_SHOWTABLES showStmtIdentifier?)
+    : KW_SHOW (KW_DATABASES|KW_SCHEMAS) (KW_LIKE showStmtIdentifier)? -> ^(TOK_SHOWDATABASES showStmtIdentifier?)
+    | KW_SHOW KW_TABLES showStmtIdentifier?  -> ^(TOK_SHOWTABLES showStmtIdentifier?)
     | KW_SHOW KW_FUNCTIONS showStmtIdentifier?  -> ^(TOK_SHOWFUNCTIONS showStmtIdentifier?)
     | KW_SHOW KW_PARTITIONS Identifier partitionSpec? -> ^(TOK_SHOWPARTITIONS Identifier partitionSpec?)
     | KW_SHOW KW_TABLE KW_EXTENDED ((KW_FROM|KW_IN) db_name=Identifier)? KW_LIKE showStmtIdentifier partitionSpec?
@@ -1786,6 +1835,7 @@ KW_INTERSECT: 'INTERSECT';
 KW_VIEW: 'VIEW';
 KW_IN: 'IN';
 KW_DATABASE: 'DATABASE';
+KW_DATABASES: 'DATABASES';
 KW_MATERIALIZED: 'MATERIALIZED';
 KW_SCHEMA: 'SCHEMA';
 KW_SCHEMAS: 'SCHEMAS';
@@ -1821,6 +1871,7 @@ KW_LATERAL: 'LATERAL';
 KW_TOUCH: 'TOUCH';
 KW_ARCHIVE: 'ARCHIVE';
 KW_UNARCHIVE: 'UNARCHIVE';
+KW_USE: 'USE';
 
 // Operators
 // NOTE: if you add a new function/operator, add it to sysFuncNames so that describe function _FUNC_ will work.

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java?rev=990026&r1=990025&r2=990026&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java Fri Aug 27 05:41:43 2010
@@ -32,9 +32,9 @@ import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Map.Entry;
 import java.util.Set;
 import java.util.TreeSet;
+import java.util.Map.Entry;
 import java.util.regex.Pattern;
 import java.util.regex.PatternSyntaxException;
 
@@ -45,7 +45,6 @@ import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.common.JavaUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Order;
 import org.apache.hadoop.hive.ql.Context;
@@ -94,7 +93,6 @@ import org.apache.hadoop.hive.ql.metadat
 import org.apache.hadoop.hive.ql.optimizer.GenMRFileSink1;
 import org.apache.hadoop.hive.ql.optimizer.GenMROperator;
 import org.apache.hadoop.hive.ql.optimizer.GenMRProcContext;
-import org.apache.hadoop.hive.ql.optimizer.GenMRProcContext.GenMapRedCtx;
 import org.apache.hadoop.hive.ql.optimizer.GenMRRedSink1;
 import org.apache.hadoop.hive.ql.optimizer.GenMRRedSink2;
 import org.apache.hadoop.hive.ql.optimizer.GenMRRedSink3;
@@ -104,6 +102,7 @@ import org.apache.hadoop.hive.ql.optimiz
 import org.apache.hadoop.hive.ql.optimizer.GenMapRedUtils;
 import org.apache.hadoop.hive.ql.optimizer.MapJoinFactory;
 import org.apache.hadoop.hive.ql.optimizer.Optimizer;
+import org.apache.hadoop.hive.ql.optimizer.GenMRProcContext.GenMapRedCtx;
 import org.apache.hadoop.hive.ql.optimizer.physical.PhysicalContext;
 import org.apache.hadoop.hive.ql.optimizer.physical.PhysicalOptimizer;
 import org.apache.hadoop.hive.ql.optimizer.ppr.PartitionPruner;
@@ -123,7 +122,6 @@ import org.apache.hadoop.hive.ql.plan.Ex
 import org.apache.hadoop.hive.ql.plan.FetchWork;
 import org.apache.hadoop.hive.ql.plan.FileSinkDesc;
 import org.apache.hadoop.hive.ql.plan.FilterDesc;
-import org.apache.hadoop.hive.ql.plan.FilterDesc.sampleDesc;
 import org.apache.hadoop.hive.ql.plan.ForwardDesc;
 import org.apache.hadoop.hive.ql.plan.GroupByDesc;
 import org.apache.hadoop.hive.ql.plan.JoinCondDesc;
@@ -145,11 +143,12 @@ import org.apache.hadoop.hive.ql.plan.Ta
 import org.apache.hadoop.hive.ql.plan.TableScanDesc;
 import org.apache.hadoop.hive.ql.plan.UDTFDesc;
 import org.apache.hadoop.hive.ql.plan.UnionDesc;
+import org.apache.hadoop.hive.ql.plan.FilterDesc.sampleDesc;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.Mode;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFHash;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDTF;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.Mode;
 import org.apache.hadoop.hive.serde.Constants;
 import org.apache.hadoop.hive.serde2.Deserializer;
 import org.apache.hadoop.hive.serde2.MetadataTypedColumnsetSerDe;
@@ -157,9 +156,9 @@ import org.apache.hadoop.hive.serde2.Ser
 import org.apache.hadoop.hive.serde2.SerDeUtils;
 import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
 import org.apache.hadoop.hive.serde2.objectinspector.StructField;
 import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
@@ -744,7 +743,7 @@ public class SemanticAnalyzer extends Ba
         String tab_name = qb.getTabNameForAlias(alias);
         Table tab = null;
         try {
-          tab = db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tab_name);
+          tab = db.getTable(tab_name);
         } catch (InvalidTableException ite) {
           throw new SemanticException(ErrorMsg.INVALID_TABLE.getMsg(qb
               .getParseInfo().getSrcForAlias(alias)));
@@ -6768,16 +6767,12 @@ public class SemanticAnalyzer extends Ba
 
     case CTAS: // create table as select
 
-      // check for existence of table. Throw an exception if it exists.
+      // Verify that the table does not already exist
       try {
-        Table tab = db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME,
-            tableName, false); // do not throw exception if table does not exist
-
-        if (tab != null) {
-          throw new SemanticException(ErrorMsg.TABLE_ALREADY_EXISTS
-              .getMsg(tableName));
+        if (null != db.getTable(db.getCurrentDatabase(), tableName, false)) {
+          throw new SemanticException(ErrorMsg.TABLE_ALREADY_EXISTS.getMsg(tableName));
         }
-      } catch (HiveException e) { // may be unable to get meta data
+      } catch (HiveException e) {
         throw new SemanticException(e);
       }
 
@@ -6790,7 +6785,7 @@ public class SemanticAnalyzer extends Ba
 
       return selectStmt;
     default:
-      assert false; // should never be unknown command type
+      throw new SemanticException("Unrecognized command.");
     }
     return null;
   }

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java?rev=990026&r1=990025&r2=990026&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java Fri Aug 27 05:41:43 2010
@@ -35,6 +35,9 @@ public final class SemanticAnalyzerFacto
   static {
     commandType.put(HiveParser.TOK_EXPLAIN, "EXPLAIN");
     commandType.put(HiveParser.TOK_LOAD, "LOAD");
+    commandType.put(HiveParser.TOK_CREATEDATABASE, "CREATEDATABASE");
+    commandType.put(HiveParser.TOK_DROPDATABASE, "DROPDATABASE");
+    commandType.put(HiveParser.TOK_SWITCHDATABASE, "SWITCHDATABASE");
     commandType.put(HiveParser.TOK_CREATETABLE, "CREATETABLE");
     commandType.put(HiveParser.TOK_DROPTABLE, "DROPTABLE");
     commandType.put(HiveParser.TOK_DESCTABLE, "DESCTABLE");
@@ -52,6 +55,7 @@ public final class SemanticAnalyzerFacto
     commandType.put(HiveParser.TOK_ALTERTABLE_PROPERTIES, "ALTERTABLE_PROPERTIES");
     commandType.put(HiveParser.TOK_ALTERTABLE_SERIALIZER, "ALTERTABLE_SERIALIZER");
     commandType.put(HiveParser.TOK_ALTERTABLE_SERDEPROPERTIES, "ALTERTABLE_SERDEPROPERTIES");
+    commandType.put(HiveParser.TOK_SHOWDATABASES, "SHOWDATABASES");
     commandType.put(HiveParser.TOK_SHOWTABLES, "SHOWTABLES");
     commandType.put(HiveParser.TOK_SHOW_TABLESTATUS, "SHOW_TABLESTATUS");
     commandType.put(HiveParser.TOK_SHOWFUNCTIONS, "SHOWFUNCTIONS");
@@ -92,6 +96,9 @@ public final class SemanticAnalyzerFacto
         return new ExplainSemanticAnalyzer(conf);
       case HiveParser.TOK_LOAD:
         return new LoadSemanticAnalyzer(conf);
+      case HiveParser.TOK_CREATEDATABASE:
+      case HiveParser.TOK_DROPDATABASE:
+      case HiveParser.TOK_SWITCHDATABASE:
       case HiveParser.TOK_DROPTABLE:
       case HiveParser.TOK_DROPVIEW:
       case HiveParser.TOK_DESCTABLE:
@@ -108,6 +115,7 @@ public final class SemanticAnalyzerFacto
       case HiveParser.TOK_ALTERTABLE_SERDEPROPERTIES:
       case HiveParser.TOK_ALTERINDEX_REBUILD:
       case HiveParser.TOK_ALTERVIEW_PROPERTIES:
+      case HiveParser.TOK_SHOWDATABASES:
       case HiveParser.TOK_SHOWTABLES:
       case HiveParser.TOK_SHOW_TABLESTATUS:
       case HiveParser.TOK_SHOWFUNCTIONS:

Added: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateDatabaseDesc.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateDatabaseDesc.java?rev=990026&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateDatabaseDesc.java (added)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateDatabaseDesc.java Fri Aug 27 05:41:43 2010
@@ -0,0 +1,93 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.plan;
+
+import java.io.Serializable;
+
+/**
+ * CreateDatabaseDesc.
+ *
+ */
+@Explain(displayName = "Create Database")
+public class CreateDatabaseDesc extends DDLDesc implements Serializable {
+
+  private static final long serialVersionUID = 1L;
+
+  String databaseName;
+  String locationUri;
+  String comment;
+  boolean ifNotExists;
+
+  /**
+   * For serialization only.
+   */
+  public CreateDatabaseDesc() {
+  }
+
+  public CreateDatabaseDesc(String databaseName, String comment,
+      String locationUri, boolean ifNotExists) {
+    super();
+    this.databaseName = databaseName;
+    this.comment = comment;
+    this.locationUri = locationUri;
+    this.ifNotExists = ifNotExists;
+  }
+
+  public CreateDatabaseDesc(String databaseName, boolean ifNotExists) {
+    this(databaseName, null, null, ifNotExists);
+  }
+
+
+
+  @Explain(displayName="if not exists")
+  public boolean getIfNotExists() {
+    return ifNotExists;
+  }
+
+  public void setIfNotExists(boolean ifNotExists) {
+    this.ifNotExists = ifNotExists;
+  }
+
+  @Explain(displayName="name")
+  public String getName() {
+    return databaseName;
+  }
+
+  public void setName(String databaseName) {
+    this.databaseName = databaseName;
+  }
+
+  @Explain(displayName="comment")
+  public String getComment() {
+    return comment;
+  }
+
+  public void setComment(String comment) {
+    this.comment = comment;
+  }
+
+  @Explain(displayName="locationUri")
+  public String getLocationUri() {
+    return locationUri;
+  }
+
+  public void setLocationUri(String locationUri) {
+    this.locationUri = locationUri;
+  }
+}

Modified: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java?rev=990026&r1=990025&r2=990026&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java (original)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java Fri Aug 27 05:41:43 2010
@@ -30,13 +30,17 @@ import org.apache.hadoop.hive.ql.hooks.W
  */
 public class DDLWork implements Serializable {
   private static final long serialVersionUID = 1L;
-  private CreateIndexDesc      createIndexDesc;
+  private CreateIndexDesc createIndexDesc;
   private DropIndexDesc dropIdxDesc;
+  private CreateDatabaseDesc createDatabaseDesc;
+  private SwitchDatabaseDesc switchDatabaseDesc;
+  private DropDatabaseDesc dropDatabaseDesc;
   private CreateTableDesc createTblDesc;
   private CreateTableLikeDesc createTblLikeDesc;
   private CreateViewDesc createVwDesc;
   private DropTableDesc dropTblDesc;
   private AlterTableDesc alterTblDesc;
+  private ShowDatabasesDesc showDatabasesDesc;
   private ShowTablesDesc showTblsDesc;
   private LockTableDesc lockTblDesc;
   private UnlockTableDesc unlockTblDesc;
@@ -72,6 +76,36 @@ public class DDLWork implements Serializ
   }
   
   /**
+   * @param createDatabaseDesc
+   *          Create Database descriptor
+   */
+  public DDLWork(HashSet<ReadEntity> inputs, HashSet<WriteEntity> outputs,
+      CreateDatabaseDesc createDatabaseDesc) {
+    this(inputs, outputs);
+    this.createDatabaseDesc = createDatabaseDesc;
+  }
+
+  /**
+   * @param dropDatabaseDesc
+   *          Drop Database descriptor
+   */
+  public DDLWork(HashSet<ReadEntity> inputs, HashSet<WriteEntity> outputs,
+      DropDatabaseDesc dropDatabaseDesc) {
+    this(inputs, outputs);
+    this.dropDatabaseDesc = dropDatabaseDesc;
+  }
+
+  /**
+   * @param switchDatabaseDesc
+   *          Switch Database descriptor
+   */
+  public DDLWork(HashSet<ReadEntity> inputs, HashSet<WriteEntity> outputs,
+      SwitchDatabaseDesc switchDatabaseDesc) {
+    this(inputs, outputs);
+    this.switchDatabaseDesc = switchDatabaseDesc;
+  }
+
+  /**
    * @param alterTblDesc
    *          alter table descriptor
    */
@@ -136,6 +170,16 @@ public class DDLWork implements Serializ
   }
 
   /**
+   * @param showDatabasesDesc
+   */
+  public DDLWork(HashSet<ReadEntity> inputs, HashSet<WriteEntity> outputs,
+      ShowDatabasesDesc showDatabasesDesc) {
+    this(inputs, outputs);
+
+    this.showDatabasesDesc = showDatabasesDesc;
+  }
+
+  /**
    * @param showTblsDesc
    */
   public DDLWork(HashSet<ReadEntity> inputs, HashSet<WriteEntity> outputs,
@@ -252,6 +296,51 @@ public class DDLWork implements Serializ
   }
 
   /**
+   * @return Create Database descriptor
+   */
+  public CreateDatabaseDesc getCreateDatabaseDesc() {
+    return createDatabaseDesc;
+  }
+
+  /**
+   * Set Create Database descriptor
+   * @param createDatabaseDesc
+   */
+  public void setCreateDatabaseDesc(CreateDatabaseDesc createDatabaseDesc) {
+    this.createDatabaseDesc = createDatabaseDesc;
+  }
+
+  /**
+   * @return Drop Database descriptor
+   */
+  public DropDatabaseDesc getDropDatabaseDesc() {
+    return dropDatabaseDesc;
+  }
+
+  /**
+   * Set Drop Database descriptor
+   * @param dropDatabaseDesc
+   */
+  public void setDropDatabaseDesc(DropDatabaseDesc dropDatabaseDesc) {
+    this.dropDatabaseDesc = dropDatabaseDesc;
+  }
+
+  /**
+   * @return Switch Database descriptor
+   */
+  public SwitchDatabaseDesc getSwitchDatabaseDesc() {
+    return switchDatabaseDesc;
+  }
+
+  /**
+   * Set Switch Database descriptor
+   * @param switchDatabaseDesc
+   */
+  public void setSwitchDatabaseDesc(SwitchDatabaseDesc switchDatabaseDesc) {
+    this.switchDatabaseDesc = switchDatabaseDesc;
+  }
+
+  /**
    * @return the createTblDesc
    */
   @Explain(displayName = "Create Table Operator")
@@ -340,6 +429,22 @@ public class DDLWork implements Serializ
   }
 
   /**
+   * @return the showDatabasesDesc
+   */
+  @Explain(displayName = "Show Databases Operator")
+  public ShowDatabasesDesc getShowDatabasesDesc() {
+    return showDatabasesDesc;
+  }
+
+  /**
+   * @param showDatabasesDesc
+   *          the showDatabasesDesc to set
+   */
+  public void setShowDatabasesDesc(ShowDatabasesDesc showDatabasesDesc) {
+    this.showDatabasesDesc = showDatabasesDesc;
+  }
+
+  /**
    * @return the showTblsDesc
    */
   @Explain(displayName = "Show Table Operator")

Added: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/DropDatabaseDesc.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/DropDatabaseDesc.java?rev=990026&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/DropDatabaseDesc.java (added)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/DropDatabaseDesc.java Fri Aug 27 05:41:43 2010
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.plan;
+
+import java.io.Serializable;
+
+/**
+ * DropDatabaseDesc.
+ *
+ */
+@Explain(displayName = "Drop Database")
+public class DropDatabaseDesc extends DDLDesc implements Serializable {
+  private static final long serialVersionUID = 1L;
+
+  String databaseName;
+  boolean ifExists;
+
+  public DropDatabaseDesc(String databaseName, boolean ifExists) {
+    super();
+    this.databaseName = databaseName;
+    this.ifExists = ifExists;
+  }
+
+  @Explain(displayName = "database")
+  public String getDatabaseName() {
+    return databaseName;
+  }
+
+  public void setDatabaseName(String databaseName) {
+    this.databaseName = databaseName;
+  }
+
+  @Explain(displayName = "if exists")
+  public boolean getIfExists() {
+    return ifExists;
+  }
+
+  public void setIfExists(boolean ifExists) {
+    this.ifExists = ifExists;
+  }
+}

Added: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowDatabasesDesc.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowDatabasesDesc.java?rev=990026&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowDatabasesDesc.java (added)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowDatabasesDesc.java Fri Aug 27 05:41:43 2010
@@ -0,0 +1,104 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.plan;
+
+import java.io.Serializable;
+
+import org.apache.hadoop.fs.Path;
+
+/**
+ * ShowDatabasesDesc.
+ *
+ */
+@Explain(displayName = "Show Databases")
+public class ShowDatabasesDesc extends DDLDesc implements Serializable {
+  private static final long serialVersionUID = 1L;
+  String pattern;
+  String resFile;
+
+  /**
+   * table name for the result of show databases.
+   */
+  private static final String table = "show_databases";
+
+  /**
+   * thrift ddl for the result of show databases.
+   */
+  private static final String schema = "database_name#string";
+
+  public String getTable() {
+    return table;
+  }
+
+  public String getSchema() {
+    return schema;
+  }
+
+  public ShowDatabasesDesc() {
+  }
+
+  /**
+   * @param resFile
+   */
+  public ShowDatabasesDesc(Path resFile) {
+    this.resFile = resFile.toString();
+    pattern = null;
+  }
+
+  /**
+   * @param pattern
+   *          names of databases to show
+   */
+  public ShowDatabasesDesc(Path resFile, String pattern) {
+    this.resFile = resFile.toString();
+    this.pattern = pattern;
+  }
+
+  /**
+   * @return the pattern
+   */
+  @Explain(displayName = "pattern")
+  public String getPattern() {
+    return pattern;
+  }
+
+  /**
+   * @param pattern
+   *          the pattern to set
+   */
+  public void setPattern(String pattern) {
+    this.pattern = pattern;
+  }
+
+  /**
+   * @return the resFile
+   */
+  @Explain(displayName = "result file", normalExplain = false)
+  public String getResFile() {
+    return resFile;
+  }
+
+  /**
+   * @param resFile
+   *          the resFile to set
+   */
+  public void setResFile(String resFile) {
+    this.resFile = resFile;
+  }
+}

Added: hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/SwitchDatabaseDesc.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/SwitchDatabaseDesc.java?rev=990026&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/SwitchDatabaseDesc.java (added)
+++ hadoop/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/SwitchDatabaseDesc.java Fri Aug 27 05:41:43 2010
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.plan;
+
+import java.io.Serializable;
+
+/**
+ * SwitchDatabaseDesc.
+ *
+ */
+@Explain(displayName = "Switch Database")
+public class SwitchDatabaseDesc extends DDLDesc implements Serializable {
+
+  private static final long serialVersionUID = 1L;
+
+  String databaseName;
+
+  public SwitchDatabaseDesc() {
+  }
+
+  public SwitchDatabaseDesc(String databaseName) {
+    super();
+    this.databaseName = databaseName;
+  }
+
+  @Explain(displayName = "name")
+  public String getDatabaseName() {
+    return databaseName;
+  }
+
+  public void setDatabaseName(String databaseName) {
+    this.databaseName = databaseName;
+  }
+}

Modified: hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java?rev=990026&r1=990025&r2=990026&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java (original)
+++ hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java Fri Aug 27 05:41:43 2010
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.hive.ql;
 
+import static org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME;
+
 import java.io.BufferedInputStream;
 import java.io.DataInputStream;
 import java.io.File;
@@ -319,12 +321,21 @@ public class QTestUtil {
    * Clear out any side effects of running tests
    */
   public void clearTestSideEffects () throws Exception {
-    // delete any tables other than the source tables
-    for (String s: db.getAllTables()) {
-      if (!srcTables.contains(s)) {
-        db.dropTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, s);
+    // Delete any tables other than the source tables
+    // and any databases other than the default database.
+    for (String dbName : db.getAllDatabases()) {
+      db.setCurrentDatabase(dbName);
+      for (String tblName : db.getAllTables()) {
+        if (!DEFAULT_DATABASE_NAME.equals(dbName) || !srcTables.contains(tblName)) {
+          db.dropTable(dbName, tblName);
+        }
+      }
+      if (!DEFAULT_DATABASE_NAME.equals(dbName)) {
+        db.dropDatabase(dbName);
       }
     }
+    db.setCurrentDatabase(DEFAULT_DATABASE_NAME);
+
     // allocate and initialize a new conf since a test can
     // modify conf by using 'set' commands
     conf = new HiveConf (Driver.class);
@@ -433,7 +444,7 @@ public class QTestUtil {
     db.createTable("src_sequencefile", cols, null,
         SequenceFileInputFormat.class, SequenceFileOutputFormat.class);
 
-    Table srcThrift = new Table("src_thrift");
+    Table srcThrift = new Table(db.getCurrentDatabase(), "src_thrift");
     srcThrift.setInputFormatClass(SequenceFileInputFormat.class.getName());
     srcThrift.setOutputFormatClass(SequenceFileOutputFormat.class.getName());
     srcThrift.setSerializationLib(ThriftDeserializer.class.getName());
@@ -506,7 +517,7 @@ public class QTestUtil {
 
     db.createTable("dest3", cols, part_cols, TextInputFormat.class,
         IgnoreKeyTextOutputFormat.class);
-    Table dest3 = db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, "dest3");
+    Table dest3 = db.getTable("dest3");
 
     HashMap<String, String> part_spec = new HashMap<String, String>();
     part_spec.put("ds", "2008-04-08");

Modified: hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/hooks/EnforceReadOnlyTables.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/hooks/EnforceReadOnlyTables.java?rev=990026&r1=990025&r2=990026&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/hooks/EnforceReadOnlyTables.java (original)
+++ hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/hooks/EnforceReadOnlyTables.java Fri Aug 27 05:41:43 2010
@@ -18,12 +18,14 @@
 
 package org.apache.hadoop.hive.ql.hooks;
 
+import static org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME;
+
 import java.util.Set;
 
-import org.apache.hadoop.hive.ql.session.SessionState;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.hive.ql.QTestUtil;
 import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.security.UserGroupInformation;
 
 /**
  * Implementation of a pre execute hook that prevents modifications
@@ -40,8 +42,10 @@ public class EnforceReadOnlyTables imple
       if ((w.getTyp() == WriteEntity.Type.TABLE) ||
           (w.getTyp() == WriteEntity.Type.PARTITION)) {
         Table t = w.getTable();
-        if (QTestUtil.srcTables.contains(t.getTableName()))
+        if (DEFAULT_DATABASE_NAME.equalsIgnoreCase(t.getDbName())
+            && QTestUtil.srcTables.contains(t.getTableName())) {
           throw new RuntimeException ("Cannot overwrite read-only table: " + t.getTableName());
+        }
       }
     }
   }

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=990026&r1=990025&r2=990026&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 Fri Aug 27 05:41:43 2010
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.hive.ql.metadata;
 
+import static org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME;
+
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.LinkedList;
@@ -29,6 +31,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.Warehouse;
+import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat;
 import org.apache.hadoop.hive.serde.Constants;
@@ -88,22 +91,18 @@ public class TestHive extends TestCase {
         e1.printStackTrace();
         assertTrue("Unable to drop table", false);
       }
-      Table tbl = new Table(tableName);
+      Table tbl = new Table(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName);
       List<FieldSchema> fields = tbl.getCols();
 
-      fields.add(new FieldSchema("col1", Constants.INT_TYPE_NAME,
-          "int -- first column"));
-      fields.add(new FieldSchema("col2", Constants.STRING_TYPE_NAME,
-          "string -- second column"));
-      fields.add(new FieldSchema("col3", Constants.DOUBLE_TYPE_NAME,
-          "double -- thrift column"));
+      fields.add(new FieldSchema("col1", Constants.INT_TYPE_NAME, "int -- first column"));
+      fields.add(new FieldSchema("col2", Constants.STRING_TYPE_NAME, "string -- second column"));
+      fields.add(new FieldSchema("col3", Constants.DOUBLE_TYPE_NAME, "double -- thrift column"));
       tbl.setFields(fields);
 
       tbl.setOutputFormatClass(HiveIgnoreKeyTextOutputFormat.class);
       tbl.setInputFormatClass(SequenceFileInputFormat.class);
 
-      tbl.setProperty("comment",
-          "this is a test table created as part junit tests");
+      tbl.setProperty("comment", "this is a test table created as part junit tests");
 
       List<String> bucketCols = tbl.getBucketCols();
       bucketCols.add("col1");
@@ -156,9 +155,10 @@ public class TestHive extends TestCase {
             .getOwner(), ft.getOwner());
         assertEquals("Table retention didn't match for table: " + tableName,
             tbl.getRetention(), ft.getRetention());
-        assertEquals("Data location is not set correctly", wh
-            .getDefaultTablePath(MetaStoreUtils.DEFAULT_DATABASE_NAME,
-            tableName).toString(), ft.getDataLocation().toString());
+        String dbPath = wh.getDefaultDatabasePath(DEFAULT_DATABASE_NAME).toString();
+        assertEquals("Data location is not set correctly",
+            wh.getDefaultTablePath(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());
@@ -191,7 +191,7 @@ public class TestHive extends TestCase {
 
   /**
    * Tests create and fetch of a thrift based table.
-   * 
+   *
    * @throws Throwable
    */
   public void testThriftTable() throws Throwable {
@@ -203,7 +203,7 @@ public class TestHive extends TestCase {
         System.err.println(StringUtils.stringifyException(e1));
         assertTrue("Unable to drop table", false);
       }
-      Table tbl = new Table(tableName);
+      Table tbl = new Table(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName);
       tbl.setInputFormatClass(SequenceFileInputFormat.class.getName());
       tbl.setOutputFormatClass(SequenceFileOutputFormat.class.getName());
       tbl.setSerializationLib(ThriftDeserializer.class.getName());
@@ -229,9 +229,10 @@ public class TestHive extends TestCase {
             .getOwner(), ft.getOwner());
         assertEquals("Table retention didn't match for table: " + tableName,
             tbl.getRetention(), ft.getRetention());
-        assertEquals("Data location is not set correctly", wh
-            .getDefaultTablePath(MetaStoreUtils.DEFAULT_DATABASE_NAME,
-            tableName).toString(), ft.getDataLocation().toString());
+        String dbPath = wh.getDefaultDatabasePath(DEFAULT_DATABASE_NAME).toString();
+        assertEquals("Data location is not set correctly",
+            wh.getDefaultTablePath(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());
@@ -245,7 +246,7 @@ public class TestHive extends TestCase {
         System.err.println(StringUtils.stringifyException(e));
         assertTrue("Unable to fetch table correctly: " + tableName, false);
       }
-      hm.dropTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName);
+      hm.dropTable(DEFAULT_DATABASE_NAME, tableName);
     } catch (Throwable e) {
       System.err.println(StringUtils.stringifyException(e));
       System.err.println("testThriftTable() failed");
@@ -254,8 +255,7 @@ public class TestHive extends TestCase {
   }
 
   private static Table createTestTable(String dbName, String tableName) throws HiveException {
-    Table tbl = new Table(tableName);
-    tbl.setDbName(dbName);
+    Table tbl = new Table(dbName, tableName);
     tbl.setInputFormatClass(SequenceFileInputFormat.class.getName());
     tbl.setOutputFormatClass(SequenceFileOutputFormat.class.getName());
     tbl.setSerializationLib(ThriftDeserializer.class.getName());
@@ -281,8 +281,11 @@ public class TestHive extends TestCase {
     try {
       String dbName = "db_for_testgettables";
       String table1Name = "table1";
-      hm.dropDatabase(dbName);
-      hm.createDatabase(dbName, "");
+      hm.dropDatabase(dbName, true, true);
+
+      Database db = new Database();
+      db.setName(dbName);
+      hm.createDatabase(db);
 
       List<String> ts = new ArrayList<String>(2);
       ts.add(table1Name);
@@ -312,6 +315,10 @@ public class TestHive extends TestCase {
       hm.dropTable(dbName, table1Name);
       assertFalse(fs.exists(table1.getPath()));
 
+      // Drop all tables
+      for (String tableName : hm.getAllTables(dbName)) {
+        hm.dropTable(dbName, tableName);
+      }
       hm.dropDatabase(dbName);
     } catch (Throwable e) {
       System.err.println(StringUtils.stringifyException(e));

Modified: hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveMetaStoreChecker.java
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveMetaStoreChecker.java?rev=990026&r1=990025&r2=990026&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveMetaStoreChecker.java (original)
+++ hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveMetaStoreChecker.java Fri Aug 27 05:41:43 2010
@@ -11,10 +11,11 @@ import junit.framework.TestCase;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
+import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat;
-import org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat;
 import org.apache.hadoop.hive.serde.Constants;
 import org.apache.hadoop.mapred.TextInputFormat;
 import org.apache.thrift.TException;
@@ -60,7 +61,11 @@ public class TestHiveMetaStoreChecker ex
 
     // cleanup
     hive.dropTable(dbName, tableName, true, true);
-    hive.dropDatabase(dbName);
+    try {
+      hive.dropDatabase(dbName);
+    } catch (NoSuchObjectException e) {
+      // ignore
+    }
   }
 
   @Override
@@ -89,9 +94,11 @@ public class TestHiveMetaStoreChecker ex
     assertTrue(result.getPartitionsNotOnFs().isEmpty());
     assertTrue(result.getPartitionsNotInMs().isEmpty());
 
-    hive.createDatabase(dbName, "");
+    Database db = new Database();
+    db.setName(dbName);
+    hive.createDatabase(db);
 
-    Table table = new Table(tableName);
+    Table table = new Table(dbName, tableName);
     table.setDbName(dbName);
     table.setInputFormatClass(TextInputFormat.class);
     table.setOutputFormatClass(HiveIgnoreKeyTextOutputFormat.class);
@@ -159,9 +166,11 @@ public class TestHiveMetaStoreChecker ex
   public void testPartitionsCheck() throws HiveException, MetaException,
       IOException, TException, AlreadyExistsException {
 
-    hive.createDatabase(dbName, "");
+    Database db = new Database();
+    db.setName(dbName);
+    hive.createDatabase(db);
 
-    Table table = new Table(tableName);
+    Table table = new Table(dbName, tableName);
     table.setDbName(dbName);
     table.setInputFormatClass(TextInputFormat.class);
     table.setOutputFormatClass(HiveIgnoreKeyTextOutputFormat.class);

Modified: 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=990026&r1=990025&r2=990026&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestPartition.java (original)
+++ hadoop/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestPartition.java Fri Aug 27 05:41:43 2010
@@ -39,7 +39,7 @@ public class TestPartition extends TestC
     List<FieldSchema> partCols = new ArrayList<FieldSchema>();
     partCols.add(new FieldSchema(PARTITION_COL, "string", ""));
 
-    Table tbl = new Table(TABLENAME);
+    Table tbl = new Table("default", TABLENAME);
     tbl.setDataLocation(new URI("tmplocation"));
     tbl.setPartCols(partCols);
 

Added: hadoop/hive/trunk/ql/src/test/queries/clientnegative/database_create_already_exists.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientnegative/database_create_already_exists.q?rev=990026&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientnegative/database_create_already_exists.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientnegative/database_create_already_exists.q Fri Aug 27 05:41:43 2010
@@ -0,0 +1,5 @@
+SHOW DATABASES;
+
+-- Try to create a database that already exists
+CREATE DATABASE test_db;
+CREATE DATABASE test_db;

Added: hadoop/hive/trunk/ql/src/test/queries/clientnegative/database_create_invalid_name.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientnegative/database_create_invalid_name.q?rev=990026&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientnegative/database_create_invalid_name.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientnegative/database_create_invalid_name.q Fri Aug 27 05:41:43 2010
@@ -0,0 +1,4 @@
+SHOW DATABASES;
+
+-- Try to create a database with an invalid name
+CREATE DATABASE `test.db`;

Added: hadoop/hive/trunk/ql/src/test/queries/clientnegative/database_drop_does_not_exist.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientnegative/database_drop_does_not_exist.q?rev=990026&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientnegative/database_drop_does_not_exist.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientnegative/database_drop_does_not_exist.q Fri Aug 27 05:41:43 2010
@@ -0,0 +1,4 @@
+SHOW DATABASES;
+
+-- Try to drop a database that does not exist
+DROP DATABASE does_not_exist;

Added: hadoop/hive/trunk/ql/src/test/queries/clientnegative/database_drop_not_empty.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientnegative/database_drop_not_empty.q?rev=990026&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientnegative/database_drop_not_empty.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientnegative/database_drop_not_empty.q Fri Aug 27 05:41:43 2010
@@ -0,0 +1,8 @@
+SHOW DATABASES;
+
+-- Try to drop a non-empty database
+CREATE DATABASE test_db;
+USE test_db;
+CREATE TABLE t(a INT);
+USE default;
+DROP DATABASE test_db;

Added: hadoop/hive/trunk/ql/src/test/queries/clientnegative/database_switch_does_not_exist.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientnegative/database_switch_does_not_exist.q?rev=990026&view=auto
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientnegative/database_switch_does_not_exist.q (added)
+++ hadoop/hive/trunk/ql/src/test/queries/clientnegative/database_switch_does_not_exist.q Fri Aug 27 05:41:43 2010
@@ -0,0 +1,4 @@
+SHOW DATABASES;
+
+-- Try to switch to a database that does not exist
+USE does_not_exist;

Modified: hadoop/hive/trunk/ql/src/test/queries/clientpositive/add_part_exist.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientpositive/add_part_exist.q?rev=990026&r1=990025&r2=990026&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientpositive/add_part_exist.q (original)
+++ hadoop/hive/trunk/ql/src/test/queries/clientpositive/add_part_exist.q Fri Aug 27 05:41:43 2010
@@ -12,3 +12,26 @@ SHOW PARTITIONS add_part_test;
 
 ALTER TABLE add_part_test ADD IF NOT EXISTS PARTITION (ds='2010-01-01') PARTITION (ds='2010-01-02') PARTITION (ds='2010-01-03');
 SHOW PARTITIONS add_part_test;
+
+DROP TABLE add_part_test;
+SHOW TABLES;
+
+-- Test ALTER TABLE ADD PARTITION in non-default Database
+CREATE DATABASE add_part_test_db;
+USE add_part_test_db;
+SHOW TABLES;
+
+CREATE TABLE add_part_test (key STRING, value STRING) PARTITIONED BY (ds STRING);
+SHOW PARTITIONS add_part_test;
+
+ALTER TABLE add_part_test ADD PARTITION (ds='2010-01-01');
+SHOW PARTITIONS add_part_test;
+
+ALTER TABLE add_part_test ADD IF NOT EXISTS PARTITION (ds='2010-01-01');
+SHOW PARTITIONS add_part_test;
+
+ALTER TABLE add_part_test ADD IF NOT EXISTS PARTITION (ds='2010-01-02');
+SHOW PARTITIONS add_part_test;
+
+ALTER TABLE add_part_test ADD IF NOT EXISTS PARTITION (ds='2010-01-01') PARTITION (ds='2010-01-02') PARTITION (ds='2010-01-03');
+SHOW PARTITIONS add_part_test;

Modified: hadoop/hive/trunk/ql/src/test/queries/clientpositive/alter1.q
URL: http://svn.apache.org/viewvc/hadoop/hive/trunk/ql/src/test/queries/clientpositive/alter1.q?rev=990026&r1=990025&r2=990026&view=diff
==============================================================================
--- hadoop/hive/trunk/ql/src/test/queries/clientpositive/alter1.q (original)
+++ hadoop/hive/trunk/ql/src/test/queries/clientpositive/alter1.q Fri Aug 27 05:41:43 2010
@@ -24,3 +24,48 @@ describe extended alter1;
 
 alter table alter1 replace columns (a int, b int, c string);
 describe alter1;
+
+-- Cleanup
+DROP TABLE alter1;
+SHOW TABLES;
+
+-- With non-default Database
+
+CREATE DATABASE alter1_db;
+USE alter1_db;
+SHOW TABLES;
+
+CREATE TABLE alter1(a INT, b INT);
+DESCRIBE EXTENDED alter1;
+
+ALTER TABLE alter1 SET TBLPROPERTIES ('a'='1', 'c'='3');
+DESCRIBE EXTENDED alter1;
+
+ALTER TABLE alter1 SET TBLPROPERTIES ('a'='1', 'c'='4', 'd'='3');
+DESCRIBE EXTENDED alter1;
+
+ALTER TABLE alter1 SET TBLPROPERTIES ('EXTERNAL'='TRUE');
+DESCRIBE EXTENDED alter1;
+
+ALTER TABLE alter1 SET TBLPROPERTIES ('EXTERNAL'='FALSE');
+DESCRIBE EXTENDED alter1;
+
+ALTER TABLE alter1 SET SERDEPROPERTIES('s1'='9');
+DESCRIBE EXTENDED alter1;
+
+ALTER TABLE alter1 SET SERDEPROPERTIES('s1'='10', 's2' ='20');
+DESCRIBE EXTENDED alter1;
+
+add jar ../data/files/TestSerDe.jar;
+ALTER TABLE alter1 SET SERDE 'org.apache.hadoop.hive.serde2.TestSerDe' WITH SERDEPROPERTIES ('s1'='9');
+DESCRIBE EXTENDED alter1;
+
+ALTER TABLE alter1 SET SERDE 'org.apache.hadoop.hive.serde2.MetadataTypedColumnsetSerDe';
+DESCRIBE EXTENDED alter1;
+
+ALTER TABLE alter1 REPLACE COLUMNS (a int, b int, c string);
+DESCRIBE alter1;
+
+DROP TABLE alter1;
+USE default;
+DROP DATABASE alter1_db;