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/09/16 19:09:43 UTC

svn commit: r997851 [10/12] - in /hadoop/hive/branches/branch-0.6: ./ 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-php/hive_metastore/ ...

Modified: hadoop/hive/branches/branch-0.6/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hadoop/hive/branches/branch-0.6/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java?rev=997851&r1=997850&r2=997851&view=diff
==============================================================================
--- hadoop/hive/branches/branch-0.6/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java (original)
+++ hadoop/hive/branches/branch-0.6/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java Thu Sep 16 17:09:41 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.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Order;
 import org.apache.hadoop.hive.ql.exec.FetchTask;
@@ -45,16 +52,20 @@ import org.apache.hadoop.hive.ql.io.Igno
 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.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.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;
 import org.apache.hadoop.hive.ql.plan.ShowTablesDesc;
+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;
@@ -110,6 +121,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);
@@ -160,11 +174,65 @@ public class DDLSemanticAnalyzer extends
     } else if (ast.getToken().getType() == HiveParser.TOK_SHOWPARTITIONS) {
       ctx.setResFile(new Path(ctx.getLocalTmpFileURI()));
       analyzeShowPartitions(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());
@@ -383,6 +451,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) {
@@ -399,7 +479,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) {
@@ -569,7 +649,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));
@@ -591,7 +671,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));
@@ -618,14 +698,14 @@ 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);
       rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
           touchDesc), conf));
     } else {
       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));
@@ -654,7 +734,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));

Modified: hadoop/hive/branches/branch-0.6/ql/src/java/org/apache/hadoop/hive/ql/parse/Hive.g
URL: http://svn.apache.org/viewvc/hadoop/hive/branches/branch-0.6/ql/src/java/org/apache/hadoop/hive/ql/parse/Hive.g?rev=997851&r1=997850&r2=997851&view=diff
==============================================================================
--- hadoop/hive/branches/branch-0.6/ql/src/java/org/apache/hadoop/hive/ql/parse/Hive.g (original)
+++ hadoop/hive/branches/branch-0.6/ql/src/java/org/apache/hadoop/hive/ql/parse/Hive.g Thu Sep 16 17:09:41 2010
@@ -85,6 +85,7 @@ TOK_STRING;
 TOK_LIST;
 TOK_STRUCT;
 TOK_MAP;
+TOK_CREATEDATABASE;
 TOK_CREATETABLE;
 TOK_LIKETABLE;
 TOK_DESCTABLE;
@@ -104,10 +105,14 @@ TOK_ALTERTABLE_FILEFORMAT;
 TOK_ALTERTABLE_PROPERTIES;
 TOK_ALTERTABLE_CHANGECOL_AFTER_POSITION;
 TOK_MSCK;
+TOK_SHOWDATABASES;
 TOK_SHOWTABLES;
 TOK_SHOWFUNCTIONS;
 TOK_SHOWPARTITIONS;
 TOK_SHOW_TABLESTATUS;
+TOK_SWITCHDATABASE;
+TOK_DROPDATABASE;
+TOK_DATABASECOMMENT;
 TOK_DROPTABLE;
 TOK_TABCOLLIST;
 TOK_TABCOL;
@@ -146,6 +151,7 @@ TOK_TABTYPE;
 TOK_LIMIT;
 TOK_TABLEPROPERTY;
 TOK_IFNOTEXISTS;
+TOK_IFEXISTS;
 TOK_HINTLIST;
 TOK_HINT;
 TOK_MAPJOIN;
@@ -209,7 +215,10 @@ loadStatement
 ddlStatement
 @init { msgs.push("ddl statement"); }
 @after { msgs.pop(); }
-    : createTableStatement
+    : createDatabaseStatement
+    | switchDatabaseStatement
+    | dropDatabaseStatement
+    | createTableStatement
     | dropTableStatement
     | alterStatement
     | descStatement
@@ -221,6 +230,13 @@ ddlStatement
     | dropFunctionStatement
     ;
 
+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(); }
@@ -228,6 +244,37 @@ 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(); }
@@ -438,7 +485,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?
@@ -1617,6 +1665,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';
@@ -1649,6 +1698,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/branches/branch-0.6/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hadoop/hive/branches/branch-0.6/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java?rev=997851&r1=997850&r2=997851&view=diff
==============================================================================
--- hadoop/hive/branches/branch-0.6/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java (original)
+++ hadoop/hive/branches/branch-0.6/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java Thu Sep 16 17:09:41 2010
@@ -42,7 +42,6 @@ import org.apache.hadoop.fs.Path;
 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.exec.AbstractMapJoinOperator;
@@ -738,7 +737,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)));
@@ -6605,9 +6604,8 @@ public class SemanticAnalyzer extends Ba
     // check for existence of table
     if (ifNotExists) {
       try {
-        List<String> tables = db.getTablesByPattern(tableName);
-        if (tables != null && tables.size() > 0) { // table exists
-          return null;
+        if (null != db.getTable(db.getCurrentDatabase(), tableName, false)) {
+            return null;
         }
       } catch (HiveException e) {
         e.printStackTrace();
@@ -6641,8 +6639,8 @@ public class SemanticAnalyzer extends Ba
 
       // check for existence of table. Throw an exception if it exists.
       try {
-        Table tab = db.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME,
-            tableName, false); // do not throw exception if table does not exist
+        Table tab = db.getTable(db.getCurrentDatabase(), tableName, false);
+        // do not throw exception if table does not exist
 
         if (tab != null) {
           throw new SemanticException(ErrorMsg.TABLE_ALREADY_EXISTS

Modified: hadoop/hive/branches/branch-0.6/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
URL: http://svn.apache.org/viewvc/hadoop/hive/branches/branch-0.6/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java?rev=997851&r1=997850&r2=997851&view=diff
==============================================================================
--- hadoop/hive/branches/branch-0.6/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java (original)
+++ hadoop/hive/branches/branch-0.6/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java Thu Sep 16 17:09:41 2010
@@ -34,6 +34,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");
@@ -51,6 +54,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");
@@ -78,6 +82,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:
@@ -93,6 +100,7 @@ public final class SemanticAnalyzerFacto
       case HiveParser.TOK_ALTERTABLE_SERIALIZER:
       case HiveParser.TOK_ALTERTABLE_SERDEPROPERTIES:
       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/branches/branch-0.6/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateDatabaseDesc.java
URL: http://svn.apache.org/viewvc/hadoop/hive/branches/branch-0.6/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateDatabaseDesc.java?rev=997851&view=auto
==============================================================================
--- hadoop/hive/branches/branch-0.6/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateDatabaseDesc.java (added)
+++ hadoop/hive/branches/branch-0.6/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateDatabaseDesc.java Thu Sep 16 17:09:41 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/branches/branch-0.6/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java
URL: http://svn.apache.org/viewvc/hadoop/hive/branches/branch-0.6/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java?rev=997851&r1=997850&r2=997851&view=diff
==============================================================================
--- hadoop/hive/branches/branch-0.6/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java (original)
+++ hadoop/hive/branches/branch-0.6/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java Thu Sep 16 17:09:41 2010
@@ -31,11 +31,15 @@ import org.apache.hadoop.hive.ql.hooks.W
 public class DDLWork implements Serializable {
   private static final long serialVersionUID = 1L;
 
+  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 ShowFunctionsDesc showFuncsDesc;
   private DescFunctionDesc descFunctionDesc;
@@ -64,6 +68,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
    */
@@ -128,6 +162,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,
@@ -208,6 +252,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")
@@ -288,6 +377,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/branches/branch-0.6/ql/src/java/org/apache/hadoop/hive/ql/plan/DropDatabaseDesc.java
URL: http://svn.apache.org/viewvc/hadoop/hive/branches/branch-0.6/ql/src/java/org/apache/hadoop/hive/ql/plan/DropDatabaseDesc.java?rev=997851&view=auto
==============================================================================
--- hadoop/hive/branches/branch-0.6/ql/src/java/org/apache/hadoop/hive/ql/plan/DropDatabaseDesc.java (added)
+++ hadoop/hive/branches/branch-0.6/ql/src/java/org/apache/hadoop/hive/ql/plan/DropDatabaseDesc.java Thu Sep 16 17:09:41 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/branches/branch-0.6/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowDatabasesDesc.java
URL: http://svn.apache.org/viewvc/hadoop/hive/branches/branch-0.6/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowDatabasesDesc.java?rev=997851&view=auto
==============================================================================
--- hadoop/hive/branches/branch-0.6/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowDatabasesDesc.java (added)
+++ hadoop/hive/branches/branch-0.6/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowDatabasesDesc.java Thu Sep 16 17:09:41 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/branches/branch-0.6/ql/src/java/org/apache/hadoop/hive/ql/plan/SwitchDatabaseDesc.java
URL: http://svn.apache.org/viewvc/hadoop/hive/branches/branch-0.6/ql/src/java/org/apache/hadoop/hive/ql/plan/SwitchDatabaseDesc.java?rev=997851&view=auto
==============================================================================
--- hadoop/hive/branches/branch-0.6/ql/src/java/org/apache/hadoop/hive/ql/plan/SwitchDatabaseDesc.java (added)
+++ hadoop/hive/branches/branch-0.6/ql/src/java/org/apache/hadoop/hive/ql/plan/SwitchDatabaseDesc.java Thu Sep 16 17:09:41 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/branches/branch-0.6/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java
URL: http://svn.apache.org/viewvc/hadoop/hive/branches/branch-0.6/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java?rev=997851&r1=997850&r2=997851&view=diff
==============================================================================
--- hadoop/hive/branches/branch-0.6/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java (original)
+++ hadoop/hive/branches/branch-0.6/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java Thu Sep 16 17:09:41 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;
@@ -26,8 +28,8 @@ import java.io.FileOutputStream;
 import java.io.FileWriter;
 import java.io.PrintStream;
 import java.io.Serializable;
-import java.net.URI;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Deque;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -84,11 +86,15 @@ public class QTestUtil {
   private final String logDir;
   private final TreeMap<String, String> qMap;
   private final Set<String> qSkipSet;
-  private final LinkedList<String> srcTables;
+  public static final HashSet<String> srcTables = new HashSet<String>
+    (Arrays.asList(new String [] {
+        "src", "src1", "srcbucket", "srcbucket2", "src_json", "src_thrift",
+        "src_sequencefile", "srcpart"
+      }));
 
   private ParseDriver pd;
   private Hive db;
-  protected final HiveConf conf;
+  protected HiveConf conf;
   private Driver drv;
   private SemanticAnalyzer sem;
   private FileSystem fs;
@@ -184,6 +190,18 @@ public class QTestUtil {
     return null;
   }
 
+  public void initConf() {
+    if (miniMr) {
+      String fsName = conf.get("fs.default.name");
+      assert fsName != null;
+      // hive.metastore.warehouse.dir needs to be set relative to the jobtracker
+      conf.set("hive.metastore.warehouse.dir", fsName
+               .concat("/build/ql/test/data/warehouse/"));
+      conf.set("mapred.job.tracker", "localhost:" + mr.getJobTrackerPort());
+    }
+
+  }
+
   public QTestUtil(String outDir, String logDir, boolean miniMr, String hadoopVer) throws Exception {
     this.outDir = outDir;
     this.logDir = logDir;
@@ -197,17 +215,10 @@ public class QTestUtil {
       dfs = ShimLoader.getHadoopShims().getMiniDfs(conf, 4, true, null);
       FileSystem fs = dfs.getFileSystem();
       mr = new MiniMRCluster(4, fs.getUri().toString(), 1);
-
-      // hive.metastore.warehouse.dir needs to be set relative to the jobtracker
-      String fsName = conf.get("fs.default.name");
-      assert fsName != null;
-      conf.set("hive.metastore.warehouse.dir", fsName
-          .concat("/build/ql/test/data/warehouse/"));
-
-      conf.set("mapred.job.tracker", "localhost:" + mr.getJobTrackerPort());
     }
 
-    // System.out.println(conf.toString());
+    initConf();
+
     testFiles = conf.get("test.data.files").replace('\\', '/')
         .replace("c:", "");
 
@@ -217,8 +228,6 @@ public class QTestUtil {
       overWrite = true;
     }
 
-    srcTables = new LinkedList<String>();
-
     init();
   }
 
@@ -290,18 +299,50 @@ public class QTestUtil {
     dis.close();
   }
 
+  /**
+   * Clear out any side effects of running tests
+   */
+  public void clearTestSideEffects () throws Exception {
+    // 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);
+    initConf();
+  }
+
+
   public void cleanUp() throws Exception {
-    String warehousePath = ((new URI(testWarehouse)).getPath());
     // Drop any tables that remain due to unsuccessful runs
-    for (String s : new String[] {"src", "src1", "src_json", "src_thrift",
-        "src_sequencefile", "srcpart", "srcbucket", "srcbucket2", "dest1",
-        "dest2", "dest3", "dest4", "dest4_sequencefile", "dest_j1", "dest_j2",
-        "dest_g1", "dest_g2", "fetchtask_ioexception"}) {
-      db.dropTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, s);
+    for (String dbName : db.getAllDatabases()) {
+      for (String tableName : db.getAllTables(dbName)) {
+        db.dropTable(dbName, tableName, true, true);
+      }
+      if (!DEFAULT_DATABASE_NAME.equalsIgnoreCase(dbName)) {
+        db.dropDatabase(dbName, true, true);
+      }
     }
-    for (String s : new String[] {"dest4.out", "union.out"}) {
-      deleteDirectory(new File(warehousePath, s));
+
+   // delete any contents in the warehouse dir
+    Path p = new Path(testWarehouse);
+    FileSystem fs = p.getFileSystem(conf);
+    FileStatus [] ls = fs.listStatus(p);
+    for (int i=0; (ls != null) && (i<ls.length); i++) {
+      fs.delete(ls[i].getPath(), true);
     }
+
     FunctionRegistry.unregisterTemporaryUDF("test_udaf");
     FunctionRegistry.unregisterTemporaryUDF("test_error");
   }
@@ -329,6 +370,7 @@ public class QTestUtil {
   }
 
   public void createSources() throws Exception {
+    db.setCurrentDatabase(DEFAULT_DATABASE_NAME);
     // Next create the three tables src, dest1 and dest2 each with two columns
     // key and value
     LinkedList<String> cols = new LinkedList<String>();
@@ -340,7 +382,6 @@ public class QTestUtil {
     part_cols.add("hr");
     db.createTable("srcpart", cols, part_cols, TextInputFormat.class,
         IgnoreKeyTextOutputFormat.class);
-    srcTables.add("srcpart");
 
     Path fpath;
     Path newfpath;
@@ -367,7 +408,6 @@ public class QTestUtil {
     runCreateTableCmd("CREATE TABLE srcbucket(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE");
     // db.createTable("srcbucket", cols, null, TextInputFormat.class,
     // IgnoreKeyTextOutputFormat.class, 2, bucketCols);
-    srcTables.add("srcbucket");
     for (String fname : new String[] {"srcbucket0.txt", "srcbucket1.txt"}) {
       fpath = new Path(testFiles, fname);
       newfpath = new Path(tmppath, fname);
@@ -380,7 +420,6 @@ public class QTestUtil {
         + "CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE");
     // db.createTable("srcbucket", cols, null, TextInputFormat.class,
     // IgnoreKeyTextOutputFormat.class, 2, bucketCols);
-    srcTables.add("srcbucket2");
     for (String fname : new String[] {"srcbucket20.txt", "srcbucket21.txt",
         "srcbucket22.txt", "srcbucket23.txt"}) {
       fpath = new Path(testFiles, fname);
@@ -393,13 +432,11 @@ public class QTestUtil {
     for (String tname : new String[] {"src", "src1"}) {
       db.createTable(tname, cols, null, TextInputFormat.class,
           IgnoreKeyTextOutputFormat.class);
-      srcTables.add(tname);
     }
     db.createTable("src_sequencefile", cols, null,
         SequenceFileInputFormat.class, SequenceFileOutputFormat.class);
-    srcTables.add("src_sequencefile");
 
-    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());
@@ -408,13 +445,11 @@ public class QTestUtil {
     srcThrift.setSerdeParam(Constants.SERIALIZATION_FORMAT,
         TBinaryProtocol.class.getName());
     db.createTable(srcThrift);
-    srcTables.add("src_thrift");
 
     LinkedList<String> json_cols = new LinkedList<String>();
     json_cols.add("json");
     db.createTable("src_json", json_cols, null, TextInputFormat.class,
         IgnoreKeyTextOutputFormat.class);
-    srcTables.add("src_json");
 
     // load the input data into the src table
     fpath = new Path(testFiles, "kv1.txt");
@@ -489,7 +524,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");
@@ -512,7 +547,7 @@ public class QTestUtil {
       createSources();
     }
 
-    CliSessionState ss = new CliSessionState(conf);
+    CliSessionState ss = new CliSessionState(new HiveConf(Driver.class));
     assert ss != null;
     ss.in = System.in;
 
@@ -973,6 +1008,7 @@ public class QTestUtil {
       for (int i = 0; i < qfiles.length; i++) {
         qt[i] = new QTestUtil(resDirs[i], logDirs[i]);
         qt[i].addFile(qfiles[i]);
+        qt[i].clearTestSideEffects();
       }
 
       if (mt) {
@@ -980,6 +1016,7 @@ public class QTestUtil {
 
         qt[0].cleanUp();
         qt[0].createSources();
+        qt[0].clearTestSideEffects();
 
         QTRunner[] qtRunners = new QTestUtil.QTRunner[qfiles.length];
         Thread[] qtThread = new Thread[qfiles.length];

Modified: hadoop/hive/branches/branch-0.6/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java
URL: http://svn.apache.org/viewvc/hadoop/hive/branches/branch-0.6/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java?rev=997851&r1=997850&r2=997851&view=diff
==============================================================================
--- hadoop/hive/branches/branch-0.6/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java (original)
+++ hadoop/hive/branches/branch-0.6/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java Thu Sep 16 17:09:41 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;
@@ -90,7 +93,7 @@ public class TestHive extends TestCase {
         e1.printStackTrace();
         assertTrue("Unable to drop table", false);
       }
-      Table tbl = new Table(tableName);
+      Table tbl = new Table(DEFAULT_DATABASE_NAME, tableName);
       List<FieldSchema> fields = tbl.getCols();
 
       fields.add(new FieldSchema("col1", Constants.INT_TYPE_NAME,
@@ -158,6 +161,7 @@ public class TestHive extends TestCase {
             .getOwner(), ft.getOwner());
         assertEquals("Table retention didn't match for table: " + tableName,
             tbl.getRetention(), ft.getRetention());
+        String dbPath = wh.getDefaultDatabasePath(DEFAULT_DATABASE_NAME).toString();
         assertEquals("Data location is not set correctly", wh
             .getDefaultTablePath(MetaStoreUtils.DEFAULT_DATABASE_NAME,
             tableName).toString(), ft.getDataLocation().toString());
@@ -205,7 +209,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(DEFAULT_DATABASE_NAME, tableName);
       tbl.setInputFormatClass(SequenceFileInputFormat.class.getName());
       tbl.setOutputFormatClass(SequenceFileOutputFormat.class.getName());
       tbl.setSerializationLib(ThriftDeserializer.class.getName());
@@ -256,8 +260,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());
@@ -283,8 +286,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);
@@ -313,6 +319,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/branches/branch-0.6/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveMetaStoreChecker.java
URL: http://svn.apache.org/viewvc/hadoop/hive/branches/branch-0.6/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveMetaStoreChecker.java?rev=997851&r1=997850&r2=997851&view=diff
==============================================================================
--- hadoop/hive/branches/branch-0.6/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveMetaStoreChecker.java (original)
+++ hadoop/hive/branches/branch-0.6/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveMetaStoreChecker.java Thu Sep 16 17:09:41 2010
@@ -11,8 +11,10 @@ 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;
@@ -60,7 +62,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 +95,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,10 +167,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.setDbName(dbName);
+    Table table = new Table(dbName, tableName);
     table.setInputFormatClass(TextInputFormat.class);
     table.setOutputFormatClass(HiveIgnoreKeyTextOutputFormat.class);
     table.setPartCols(partCols);

Modified: hadoop/hive/branches/branch-0.6/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestPartition.java
URL: http://svn.apache.org/viewvc/hadoop/hive/branches/branch-0.6/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestPartition.java?rev=997851&r1=997850&r2=997851&view=diff
==============================================================================
--- hadoop/hive/branches/branch-0.6/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestPartition.java (original)
+++ hadoop/hive/branches/branch-0.6/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestPartition.java Thu Sep 16 17:09:41 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/branches/branch-0.6/ql/src/test/queries/clientnegative/database_create_already_exists.q
URL: http://svn.apache.org/viewvc/hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientnegative/database_create_already_exists.q?rev=997851&view=auto
==============================================================================
--- hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientnegative/database_create_already_exists.q (added)
+++ hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientnegative/database_create_already_exists.q Thu Sep 16 17:09:41 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/branches/branch-0.6/ql/src/test/queries/clientnegative/database_create_invalid_name.q
URL: http://svn.apache.org/viewvc/hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientnegative/database_create_invalid_name.q?rev=997851&view=auto
==============================================================================
--- hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientnegative/database_create_invalid_name.q (added)
+++ hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientnegative/database_create_invalid_name.q Thu Sep 16 17:09:41 2010
@@ -0,0 +1,4 @@
+SHOW DATABASES;
+
+-- Try to create a database with an invalid name
+CREATE DATABASE `test.db`;

Added: hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientnegative/database_drop_does_not_exist.q
URL: http://svn.apache.org/viewvc/hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientnegative/database_drop_does_not_exist.q?rev=997851&view=auto
==============================================================================
--- hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientnegative/database_drop_does_not_exist.q (added)
+++ hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientnegative/database_drop_does_not_exist.q Thu Sep 16 17:09:41 2010
@@ -0,0 +1,4 @@
+SHOW DATABASES;
+
+-- Try to drop a database that does not exist
+DROP DATABASE does_not_exist;

Added: hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientnegative/database_drop_not_empty.q
URL: http://svn.apache.org/viewvc/hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientnegative/database_drop_not_empty.q?rev=997851&view=auto
==============================================================================
--- hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientnegative/database_drop_not_empty.q (added)
+++ hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientnegative/database_drop_not_empty.q Thu Sep 16 17:09:41 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/branches/branch-0.6/ql/src/test/queries/clientnegative/database_switch_does_not_exist.q
URL: http://svn.apache.org/viewvc/hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientnegative/database_switch_does_not_exist.q?rev=997851&view=auto
==============================================================================
--- hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientnegative/database_switch_does_not_exist.q (added)
+++ hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientnegative/database_switch_does_not_exist.q Thu Sep 16 17:09:41 2010
@@ -0,0 +1,4 @@
+SHOW DATABASES;
+
+-- Try to switch to a database that does not exist
+USE does_not_exist;

Modified: hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientpositive/add_part_exist.q
URL: http://svn.apache.org/viewvc/hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientpositive/add_part_exist.q?rev=997851&r1=997850&r2=997851&view=diff
==============================================================================
--- hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientpositive/add_part_exist.q (original)
+++ hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientpositive/add_part_exist.q Thu Sep 16 17:09:41 2010
@@ -14,3 +14,26 @@ ALTER TABLE add_part_test ADD IF NOT EXI
 SHOW PARTITIONS add_part_test;
 
 DROP TABLE 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/branches/branch-0.6/ql/src/test/queries/clientpositive/alter1.q
URL: http://svn.apache.org/viewvc/hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientpositive/alter1.q?rev=997851&r1=997850&r2=997851&view=diff
==============================================================================
--- hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientpositive/alter1.q (original)
+++ hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientpositive/alter1.q Thu Sep 16 17:09:41 2010
@@ -27,3 +27,48 @@ alter table alter1 replace columns (a in
 describe alter1;
 
 drop table 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;

Modified: hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientpositive/alter2.q
URL: http://svn.apache.org/viewvc/hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientpositive/alter2.q?rev=997851&r1=997850&r2=997851&view=diff
==============================================================================
--- hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientpositive/alter2.q (original)
+++ hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientpositive/alter2.q Thu Sep 16 17:09:41 2010
@@ -19,4 +19,39 @@ show partitions alter2;
 alter table alter2 add partition (insertdate='2008-01-02') location '2008/01/02';
 describe extended alter2;
 show partitions alter2;
-drop table alter2;
\ No newline at end of file
+drop table alter2;
+
+-- Cleanup
+DROP TABLE alter2;
+SHOW TABLES;
+
+-- Using non-default Database
+
+CREATE DATABASE alter2_db;
+USE alter2_db;
+SHOW TABLES;
+
+CREATE TABLE alter2(a int, b int) PARTITIONED BY (insertdate string);
+DESCRIBE EXTENDED alter2;
+SHOW PARTITIONS alter2;
+ALTER TABLE alter2 ADD PARTITION (insertdate='2008-01-01') LOCATION '2008/01/01';
+DESCRIBE EXTENDED alter2;
+SHOW PARTITIONS alter2;
+ALTER TABLE alter2 ADD PARTITION (insertdate='2008-01-02') LOCATION '2008/01/02';
+DESCRIBE EXTENDED alter2;
+SHOW PARTITIONS alter2;
+DROP TABLE alter2;
+
+CREATE EXTERNAL TABLE alter2(a int, b int) PARTITIONED BY (insertdate string);
+DESCRIBE EXTENDED alter2;
+SHOW PARTITIONS alter2;
+ALTER TABLE alter2 ADD PARTITION (insertdate='2008-01-01') LOCATION '2008/01/01';
+DESCRIBE EXTENDED alter2;
+SHOW PARTITIONS alter2;
+ALTER TABLE alter2 ADD PARTITION (insertdate='2008-01-02') LOCATION '2008/01/02';
+DESCRIBE EXTENDED alter2;
+SHOW PARTITIONS alter2;
+
+DROP TABLE alter2;
+USE default;
+DROP DATABASE alter2_db;

Modified: hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientpositive/alter3.q
URL: http://svn.apache.org/viewvc/hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientpositive/alter3.q?rev=997851&r1=997850&r2=997851&view=diff
==============================================================================
--- hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientpositive/alter3.q (original)
+++ hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientpositive/alter3.q Thu Sep 16 17:09:41 2010
@@ -29,3 +29,37 @@ drop table alter3_src;
 drop table alter3;
 drop table alter3_renamed;
 drop table alter3_like_renamed;
+
+-- Cleanup
+DROP TABLE alter3_src;
+DROP TABLE alter3_renamed;
+DROP TABLE alter3_like_renamed;
+SHOW TABLES;
+
+-- With non-default Database
+
+CREATE DATABASE alter3_db;
+USE alter3_db;
+SHOW TABLES;
+
+CREATE TABLE alter3_src (col1 STRING) STORED AS TEXTFILE ;
+LOAD DATA LOCAL INPATH '../data/files/test.dat' OVERWRITE INTO TABLE alter3_src ;
+
+CREATE TABLE alter3 (col1 STRING) PARTITIONED BY (pcol1 STRING, pcol2 STRING) STORED AS SEQUENCEFILE;
+
+CREATE TABLE alter3_like LIKE alter3;
+
+INSERT OVERWRITE TABLE alter3 PARTITION (pCol1='test_part', pcol2='test_part') SELECT col1 FROM alter3_src ;
+SELECT * FROM alter3 WHERE pcol1='test_part' AND pcol2='test_part';
+
+ALTER TABLE alter3 RENAME TO alter3_renamed;
+DESCRIBE EXTENDED alter3_renamed;
+DESCRIBE EXTENDED alter3_renamed PARTITION (pCol1='test_part', pcol2='test_part');
+SELECT * FROM alter3_renamed WHERE pcol1='test_part' AND pcol2='test_part';
+
+INSERT OVERWRITE TABLE alter3_like
+PARTITION (pCol1='test_part', pcol2='test_part')
+SELECT col1 FROM alter3_src;
+ALTER TABLE alter3_like RENAME TO alter3_like_renamed;
+
+DESCRIBE EXTENDED alter3_like_renamed;

Added: hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientpositive/database.q
URL: http://svn.apache.org/viewvc/hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientpositive/database.q?rev=997851&view=auto
==============================================================================
--- hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientpositive/database.q (added)
+++ hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientpositive/database.q Thu Sep 16 17:09:41 2010
@@ -0,0 +1,91 @@
+SHOW DATABASES;
+
+-- CREATE with comment
+CREATE DATABASE test_db COMMENT 'Hive test database';
+SHOW DATABASES;
+
+-- CREATE INE already exists
+CREATE DATABASE IF NOT EXISTS test_db;
+SHOW DATABASES;
+
+-- SHOW DATABASES synonym
+SHOW SCHEMAS;
+
+-- DROP
+DROP DATABASE test_db;
+SHOW DATABASES;
+
+-- CREATE INE doesn't exist
+CREATE DATABASE IF NOT EXISTS test_db COMMENT 'Hive test database';
+SHOW DATABASES;
+
+-- DROP IE exists
+DROP DATABASE IF EXISTS test_db;
+SHOW DATABASES;
+
+-- DROP IE doesn't exist
+DROP DATABASE IF EXISTS test_db;
+
+-- SHOW
+CREATE DATABASE test_db;
+SHOW DATABASES;
+
+-- SHOW pattern
+SHOW DATABASES LIKE 'test*';
+
+-- SHOW pattern
+SHOW DATABASES LIKE '*ef*';
+
+
+USE test_db;
+SHOW DATABASES;
+
+-- CREATE table in non-default DB
+CREATE TABLE test_table (col1 STRING) STORED AS TEXTFILE;
+SHOW TABLES;
+
+-- DESCRIBE table in non-default DB
+DESCRIBE test_table;
+
+-- DESCRIBE EXTENDED in non-default DB
+DESCRIBE EXTENDED test_table;
+
+-- CREATE LIKE in non-default DB
+CREATE TABLE test_table_like LIKE test_table;
+SHOW TABLES;
+DESCRIBE EXTENDED test_table_like;
+
+-- LOAD and SELECT
+LOAD DATA LOCAL INPATH '../data/files/test.dat' OVERWRITE INTO TABLE test_table ;
+SELECT * FROM test_table;
+
+-- DROP and CREATE w/o LOAD
+DROP TABLE test_table;
+SHOW TABLES;
+
+CREATE TABLE test_table (col1 STRING) STORED AS TEXTFILE;
+SHOW TABLES;
+
+SELECT * FROM test_table;
+
+-- CREATE table that already exists in DEFAULT
+USE test_db;
+CREATE TABLE src (col1 STRING) STORED AS TEXTFILE;
+SHOW TABLES;
+
+SELECT * FROM src LIMIT 10;
+
+USE default;
+SELECT * FROM src LIMIT 10;
+
+-- DROP DATABASE
+USE test_db;
+
+DROP TABLE src;
+DROP TABLE test_table;
+DROP TABLE test_table_like;
+SHOW TABLES;
+
+USE default;
+DROP DATABASE test_db;
+SHOW DATABASES;

Modified: hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientpositive/rename_column.q
URL: http://svn.apache.org/viewvc/hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientpositive/rename_column.q?rev=997851&r1=997850&r2=997851&view=diff
==============================================================================
--- hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientpositive/rename_column.q (original)
+++ hadoop/hive/branches/branch-0.6/ql/src/test/queries/clientpositive/rename_column.q Thu Sep 16 17:09:41 2010
@@ -24,3 +24,36 @@ DESCRIBE kv_rename_test;
 
 DROP TABLE kv_rename_test;
 
+DROP TABLE kv_rename_test;
+SHOW TABLES;
+
+-- Using non-default Database
+CREATE DATABASE kv_rename_test_db;
+USE kv_rename_test_db;
+
+CREATE TABLE kv_rename_test(a int, b int, c int);
+DESCRIBE kv_rename_test;
+
+ALTER TABLE kv_rename_test CHANGE a a STRING;
+DESCRIBE kv_rename_test;
+
+ALTER TABLE kv_rename_test CHANGE a a1 INT;
+DESCRIBE kv_rename_test;
+
+ALTER TABLE kv_rename_test CHANGE a1 a2 INT FIRST;
+DESCRIBE kv_rename_test;
+
+ALTER TABLE kv_rename_test CHANGE a2 a INT AFTER b;
+DESCRIBE kv_rename_test;
+
+ALTER TABLE kv_rename_test CHANGE a a1 INT COMMENT 'test comment1';
+DESCRIBE kv_rename_test;
+
+ALTER TABLE kv_rename_test CHANGE a1 a2 INT COMMENT 'test comment2' FIRST;
+DESCRIBE kv_rename_test;
+
+ALTER TABLE kv_rename_test CHANGE COLUMN a2 a INT AFTER b;
+DESCRIBE kv_rename_test;
+
+DROP TABLE kv_rename_test;
+SHOW TABLES;

Added: hadoop/hive/branches/branch-0.6/ql/src/test/results/clientnegative/database_create_already_exists.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/branches/branch-0.6/ql/src/test/results/clientnegative/database_create_already_exists.q.out?rev=997851&view=auto
==============================================================================
--- hadoop/hive/branches/branch-0.6/ql/src/test/results/clientnegative/database_create_already_exists.q.out (added)
+++ hadoop/hive/branches/branch-0.6/ql/src/test/results/clientnegative/database_create_already_exists.q.out Thu Sep 16 17:09:41 2010
@@ -0,0 +1,15 @@
+PREHOOK: query: SHOW DATABASES
+PREHOOK: type: SHOWDATABASES
+POSTHOOK: query: SHOW DATABASES
+POSTHOOK: type: SHOWDATABASES
+default
+PREHOOK: query: -- Try to create a database that already exists
+CREATE DATABASE test_db
+PREHOOK: type: CREATEDATABASE
+POSTHOOK: query: -- Try to create a database that already exists
+CREATE DATABASE test_db
+POSTHOOK: type: CREATEDATABASE
+PREHOOK: query: CREATE DATABASE test_db
+PREHOOK: type: CREATEDATABASE
+Failed with exception Database test_db already exists
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask

Added: hadoop/hive/branches/branch-0.6/ql/src/test/results/clientnegative/database_create_invalid_name.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/branches/branch-0.6/ql/src/test/results/clientnegative/database_create_invalid_name.q.out?rev=997851&view=auto
==============================================================================
--- hadoop/hive/branches/branch-0.6/ql/src/test/results/clientnegative/database_create_invalid_name.q.out (added)
+++ hadoop/hive/branches/branch-0.6/ql/src/test/results/clientnegative/database_create_invalid_name.q.out Thu Sep 16 17:09:41 2010
@@ -0,0 +1,10 @@
+PREHOOK: query: SHOW DATABASES
+PREHOOK: type: SHOWDATABASES
+POSTHOOK: query: SHOW DATABASES
+POSTHOOK: type: SHOWDATABASES
+default
+PREHOOK: query: -- Try to create a database with an invalid name
+CREATE DATABASE `test.db`
+PREHOOK: type: CREATEDATABASE
+FAILED: Error in metadata: InvalidObjectException(message:test.db is not a valid database name)
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask

Added: hadoop/hive/branches/branch-0.6/ql/src/test/results/clientnegative/database_drop_does_not_exist.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/branches/branch-0.6/ql/src/test/results/clientnegative/database_drop_does_not_exist.q.out?rev=997851&view=auto
==============================================================================
--- hadoop/hive/branches/branch-0.6/ql/src/test/results/clientnegative/database_drop_does_not_exist.q.out (added)
+++ hadoop/hive/branches/branch-0.6/ql/src/test/results/clientnegative/database_drop_does_not_exist.q.out Thu Sep 16 17:09:41 2010
@@ -0,0 +1,10 @@
+PREHOOK: query: SHOW DATABASES
+PREHOOK: type: SHOWDATABASES
+POSTHOOK: query: SHOW DATABASES
+POSTHOOK: type: SHOWDATABASES
+default
+PREHOOK: query: -- Try to drop a database that does not exist
+DROP DATABASE does_not_exist
+PREHOOK: type: DROPDATABASE
+Failed with exception There is no database named does_not_exist
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask

Added: hadoop/hive/branches/branch-0.6/ql/src/test/results/clientnegative/database_drop_not_empty.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/branches/branch-0.6/ql/src/test/results/clientnegative/database_drop_not_empty.q.out?rev=997851&view=auto
==============================================================================
--- hadoop/hive/branches/branch-0.6/ql/src/test/results/clientnegative/database_drop_not_empty.q.out (added)
+++ hadoop/hive/branches/branch-0.6/ql/src/test/results/clientnegative/database_drop_not_empty.q.out Thu Sep 16 17:09:41 2010
@@ -0,0 +1,28 @@
+PREHOOK: query: SHOW DATABASES
+PREHOOK: type: SHOWDATABASES
+POSTHOOK: query: SHOW DATABASES
+POSTHOOK: type: SHOWDATABASES
+default
+PREHOOK: query: -- Try to drop a non-empty database
+CREATE DATABASE test_db
+PREHOOK: type: CREATEDATABASE
+POSTHOOK: query: -- Try to drop a non-empty database
+CREATE DATABASE test_db
+POSTHOOK: type: CREATEDATABASE
+PREHOOK: query: USE test_db
+PREHOOK: type: SWITCHDATABASE
+POSTHOOK: query: USE test_db
+POSTHOOK: type: SWITCHDATABASE
+PREHOOK: query: CREATE TABLE t(a INT)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: CREATE TABLE t(a INT)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: test_db@t
+PREHOOK: query: USE default
+PREHOOK: type: SWITCHDATABASE
+POSTHOOK: query: USE default
+POSTHOOK: type: SWITCHDATABASE
+PREHOOK: query: DROP DATABASE test_db
+PREHOOK: type: DROPDATABASE
+FAILED: Error in metadata: InvalidOperationException(message:Database test_db is not empty)
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask

Added: hadoop/hive/branches/branch-0.6/ql/src/test/results/clientnegative/database_switch_does_not_exist.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/branches/branch-0.6/ql/src/test/results/clientnegative/database_switch_does_not_exist.q.out?rev=997851&view=auto
==============================================================================
--- hadoop/hive/branches/branch-0.6/ql/src/test/results/clientnegative/database_switch_does_not_exist.q.out (added)
+++ hadoop/hive/branches/branch-0.6/ql/src/test/results/clientnegative/database_switch_does_not_exist.q.out Thu Sep 16 17:09:41 2010
@@ -0,0 +1,10 @@
+PREHOOK: query: SHOW DATABASES
+PREHOOK: type: SHOWDATABASES
+POSTHOOK: query: SHOW DATABASES
+POSTHOOK: type: SHOWDATABASES
+default
+PREHOOK: query: -- Try to switch to a database that does not exist
+USE does_not_exist
+PREHOOK: type: SWITCHDATABASE
+FAILED: Error in metadata: ERROR: The database does_not_exist does not exist.
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask

Modified: hadoop/hive/branches/branch-0.6/ql/src/test/results/clientnegative/invalid_t_alter2.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/branches/branch-0.6/ql/src/test/results/clientnegative/invalid_t_alter2.q.out?rev=997851&r1=997850&r2=997851&view=diff
==============================================================================
--- hadoop/hive/branches/branch-0.6/ql/src/test/results/clientnegative/invalid_t_alter2.q.out (original)
+++ hadoop/hive/branches/branch-0.6/ql/src/test/results/clientnegative/invalid_t_alter2.q.out Thu Sep 16 17:09:41 2010
@@ -1,4 +1,6 @@
 PREHOOK: query: CREATE TABLE alter_test (d STRING)
 PREHOOK: type: CREATETABLE
-FAILED: Error in metadata: AlreadyExistsException(message:Table alter_test already exists)
-FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask
+POSTHOOK: query: CREATE TABLE alter_test (d STRING)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@alter_test
+FAILED: Error in semantic analysis: DATE, DATETIME, and TIMESTAMP types aren't supported yet. Please use STRING instead.

Modified: hadoop/hive/branches/branch-0.6/ql/src/test/results/clientnegative/load_wrong_fileformat_txt_seq.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/branches/branch-0.6/ql/src/test/results/clientnegative/load_wrong_fileformat_txt_seq.q.out?rev=997851&r1=997850&r2=997851&view=diff
==============================================================================
--- hadoop/hive/branches/branch-0.6/ql/src/test/results/clientnegative/load_wrong_fileformat_txt_seq.q.out (original)
+++ hadoop/hive/branches/branch-0.6/ql/src/test/results/clientnegative/load_wrong_fileformat_txt_seq.q.out Thu Sep 16 17:09:41 2010
@@ -8,7 +8,6 @@ POSTHOOK: query: -- test for loading int
 
 DROP TABLE T1
 POSTHOOK: type: DROPTABLE
-POSTHOOK: Output: default@t1
 PREHOOK: query: CREATE TABLE T1(name STRING) STORED AS TEXTFILE
 PREHOOK: type: CREATETABLE
 POSTHOOK: query: CREATE TABLE T1(name STRING) STORED AS TEXTFILE

Modified: hadoop/hive/branches/branch-0.6/ql/src/test/results/clientpositive/add_part_exist.q.out
URL: http://svn.apache.org/viewvc/hadoop/hive/branches/branch-0.6/ql/src/test/results/clientpositive/add_part_exist.q.out?rev=997851&r1=997850&r2=997851&view=diff
==============================================================================
--- hadoop/hive/branches/branch-0.6/ql/src/test/results/clientpositive/add_part_exist.q.out (original)
+++ hadoop/hive/branches/branch-0.6/ql/src/test/results/clientpositive/add_part_exist.q.out Thu Sep 16 17:09:41 2010
@@ -54,3 +54,84 @@ PREHOOK: type: DROPTABLE
 POSTHOOK: query: DROP TABLE add_part_test
 POSTHOOK: type: DROPTABLE
 POSTHOOK: Output: default@add_part_test
+PREHOOK: query: DROP TABLE add_part_test
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE add_part_test
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: SHOW TABLES
+PREHOOK: type: SHOWTABLES
+POSTHOOK: query: SHOW TABLES
+POSTHOOK: type: SHOWTABLES
+src
+src1
+src_json
+src_sequencefile
+src_thrift
+srcbucket
+srcbucket2
+srcpart
+PREHOOK: query: -- Test ALTER TABLE ADD PARTITION in non-default Database
+CREATE DATABASE add_part_test_db
+PREHOOK: type: CREATEDATABASE
+POSTHOOK: query: -- Test ALTER TABLE ADD PARTITION in non-default Database
+CREATE DATABASE add_part_test_db
+POSTHOOK: type: CREATEDATABASE
+PREHOOK: query: USE add_part_test_db
+PREHOOK: type: SWITCHDATABASE
+POSTHOOK: query: USE add_part_test_db
+POSTHOOK: type: SWITCHDATABASE
+PREHOOK: query: SHOW TABLES
+PREHOOK: type: SHOWTABLES
+POSTHOOK: query: SHOW TABLES
+POSTHOOK: type: SHOWTABLES
+PREHOOK: query: CREATE TABLE add_part_test (key STRING, value STRING) PARTITIONED BY (ds STRING)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: CREATE TABLE add_part_test (key STRING, value STRING) PARTITIONED BY (ds STRING)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: add_part_test_db@add_part_test
+PREHOOK: query: SHOW PARTITIONS add_part_test
+PREHOOK: type: SHOWPARTITIONS
+POSTHOOK: query: SHOW PARTITIONS add_part_test
+POSTHOOK: type: SHOWPARTITIONS
+PREHOOK: query: ALTER TABLE add_part_test ADD PARTITION (ds='2010-01-01')
+PREHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: query: ALTER TABLE add_part_test ADD PARTITION (ds='2010-01-01')
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Output: add_part_test_db@add_part_test@ds=2010-01-01
+PREHOOK: query: SHOW PARTITIONS add_part_test
+PREHOOK: type: SHOWPARTITIONS
+POSTHOOK: query: SHOW PARTITIONS add_part_test
+POSTHOOK: type: SHOWPARTITIONS
+ds=2010-01-01
+PREHOOK: query: ALTER TABLE add_part_test ADD IF NOT EXISTS PARTITION (ds='2010-01-01')
+PREHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: query: ALTER TABLE add_part_test ADD IF NOT EXISTS PARTITION (ds='2010-01-01')
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: query: SHOW PARTITIONS add_part_test
+PREHOOK: type: SHOWPARTITIONS
+POSTHOOK: query: SHOW PARTITIONS add_part_test
+POSTHOOK: type: SHOWPARTITIONS
+ds=2010-01-01
+PREHOOK: query: ALTER TABLE add_part_test ADD IF NOT EXISTS PARTITION (ds='2010-01-02')
+PREHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: query: ALTER TABLE add_part_test ADD IF NOT EXISTS PARTITION (ds='2010-01-02')
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Output: add_part_test_db@add_part_test@ds=2010-01-02
+PREHOOK: query: SHOW PARTITIONS add_part_test
+PREHOOK: type: SHOWPARTITIONS
+POSTHOOK: query: SHOW PARTITIONS add_part_test
+POSTHOOK: type: SHOWPARTITIONS
+ds=2010-01-01
+ds=2010-01-02
+PREHOOK: query: ALTER TABLE add_part_test ADD IF NOT EXISTS PARTITION (ds='2010-01-01') PARTITION (ds='2010-01-02') PARTITION (ds='2010-01-03')
+PREHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: query: ALTER TABLE add_part_test ADD IF NOT EXISTS PARTITION (ds='2010-01-01') PARTITION (ds='2010-01-02') PARTITION (ds='2010-01-03')
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Output: add_part_test_db@add_part_test@ds=2010-01-03
+PREHOOK: query: SHOW PARTITIONS add_part_test
+PREHOOK: type: SHOWPARTITIONS
+POSTHOOK: query: SHOW PARTITIONS add_part_test
+POSTHOOK: type: SHOWPARTITIONS
+ds=2010-01-01
+ds=2010-01-02
+ds=2010-01-03