You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by br...@apache.org on 2014/08/30 08:44:53 UTC

svn commit: r1621416 [4/11] - in /hive/branches/spark: ./ common/src/java/org/apache/hadoop/hive/common/ common/src/java/org/apache/hadoop/hive/conf/ contrib/src/test/results/clientnegative/ contrib/src/test/results/clientpositive/ hbase-handler/src/te...

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java Sat Aug 30 06:44:46 2014
@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.ql.parse;
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Set;
@@ -227,11 +228,19 @@ public abstract class TaskCompiler {
 
       crtTblDesc.validate(conf);
 
-      // Clear the output for CTAS since we don't need the output from the
-      // mapredWork, the
+      // clear the mapredWork output file from outputs for CTAS
       // DDLWork at the tail of the chain will have the output
-      outputs.clear();
-
+      Iterator<WriteEntity> outIter = outputs.iterator();
+      while (outIter.hasNext()) {
+        switch (outIter.next().getType()) {
+        case DFS_DIR:
+        case LOCAL_DIR:
+          outIter.remove();
+          break;
+        default:
+          break;
+        }
+      }
       Task<? extends Serializable> crtTblTask = TaskFactory.get(new DDLWork(
           inputs, outputs, crtTblDesc), conf);
 

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/HiveAuthorizationTaskFactoryImpl.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/HiveAuthorizationTaskFactoryImpl.java?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/HiveAuthorizationTaskFactoryImpl.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/HiveAuthorizationTaskFactoryImpl.java Sat Aug 30 06:44:46 2014
@@ -206,7 +206,7 @@ public class HiveAuthorizationTaskFactor
 
     List<String> roles = new ArrayList<String>();
     for (int i = rolesStartPos; i < ast.getChildCount(); i++) {
-      roles.add(BaseSemanticAnalyzer.unescapeIdentifier(ast.getChild(i).getText()).toLowerCase());
+      roles.add(BaseSemanticAnalyzer.unescapeIdentifier(ast.getChild(i).getText()));
     }
 
     String roleOwnerName = SessionState.getUserFromAuthenticator();

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java Sat Aug 30 06:44:46 2014
@@ -44,11 +44,19 @@ public class AlterTableDesc extends DDLD
    *
    */
   public static enum AlterTableTypes {
-    RENAME, ADDCOLS, REPLACECOLS, ADDPROPS, DROPPROPS, ADDSERDE, ADDSERDEPROPS,
-    ADDFILEFORMAT, ADDCLUSTERSORTCOLUMN, RENAMECOLUMN, ADDPARTITION,
-    TOUCH, ARCHIVE, UNARCHIVE, ALTERPROTECTMODE, ALTERPARTITIONPROTECTMODE,
-    ALTERLOCATION, DROPPARTITION, RENAMEPARTITION, ADDSKEWEDBY, ALTERSKEWEDLOCATION,
-    ALTERBUCKETNUM, ALTERPARTITION, COMPACT
+    RENAME("rename"), ADDCOLS("add columns"), REPLACECOLS("replace columns"),
+    ADDPROPS("add props"), DROPPROPS("drop props"), ADDSERDE("add serde"), ADDSERDEPROPS("add serde props"),
+    ADDFILEFORMAT("add fileformat"), ADDCLUSTERSORTCOLUMN("add cluster sort column"),
+    RENAMECOLUMN("rename column"), ADDPARTITION("add partition"), TOUCH("touch"), ARCHIVE("archieve"),
+    UNARCHIVE("unarchieve"), ALTERPROTECTMODE("alter protect mode"),
+    ALTERPARTITIONPROTECTMODE("alter partition protect mode"), ALTERLOCATION("alter location"),
+    DROPPARTITION("drop partition"), RENAMEPARTITION("rename partition"), ADDSKEWEDBY("add skew column"),
+    ALTERSKEWEDLOCATION("alter skew location"), ALTERBUCKETNUM("alter bucket number"),
+    ALTERPARTITION("alter partition"), COMPACT("compact");
+
+    private final String name;
+    private AlterTableTypes(String name) { this.name = name; }
+    public String getName() { return name; }
   }
 
   public static enum ProtectModeType {
@@ -236,16 +244,7 @@ public class AlterTableDesc extends DDLD
 
   @Explain(displayName = "type")
   public String getAlterTableTypeString() {
-    switch (op) {
-    case RENAME:
-      return "rename";
-    case ADDCOLS:
-      return "add columns";
-    case REPLACECOLS:
-      return "replace columns";
-    }
-
-    return "unknown";
+    return op.getName();
   }
 
   /**

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java Sat Aug 30 06:44:46 2014
@@ -114,8 +114,8 @@ public enum HiveOperation {
       new Privilege[] {Privilege.ALTER_DATA}, null),
   ALTERTABLE_PARTCOLTYPE("ALTERTABLE_PARTCOLTYPE", new Privilege[] { Privilege.SELECT }, new Privilege[] { Privilege.ALTER_DATA }),
   ALTERVIEW_RENAME("ALTERVIEW_RENAME", new Privilege[] {Privilege.ALTER_METADATA}, null),
-  ALTERTABLE_COMPACT("ALTERTABLE_COMPACT", new Privilege[]{Privilege.SELECT},
-      new Privilege[]{Privilege.ALTER_DATA}),
+  ALTERVIEW_AS("ALTERVIEW_AS", new Privilege[] {Privilege.ALTER_METADATA}, null),
+  ALTERTABLE_COMPACT("ALTERTABLE_COMPACT", new Privilege[]{Privilege.SELECT}, new Privilege[]{Privilege.ALTER_DATA}),
   SHOW_COMPACTIONS("SHOW COMPACTIONS", null, null),
   SHOW_TRANSACTIONS("SHOW TRANSACTIONS", null, null);
   ;

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java Sat Aug 30 06:44:46 2014
@@ -347,7 +347,7 @@ public final class PlanUtils {
 
       if (crtTblDesc.getTableName() != null && crtTblDesc.getDatabaseName() != null) {
         properties.setProperty(org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_NAME,
-            crtTblDesc.getDatabaseName() + "." + crtTblDesc.getTableName());
+            crtTblDesc.getTableName());
       }
 
       if (crtTblDesc.getTblProps() != null) {

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/plan/RoleDDLDesc.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/plan/RoleDDLDesc.java?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/plan/RoleDDLDesc.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/plan/RoleDDLDesc.java Sat Aug 30 06:44:46 2014
@@ -102,8 +102,7 @@ public class RoleDDLDesc extends DDLDesc
 
   public RoleDDLDesc(String principalName, PrincipalType principalType,
       RoleOperation operation, String roleOwnerName) {
-    this.name = (principalName != null  && principalType == PrincipalType.ROLE) ?
-      principalName.toLowerCase() : principalName;
+    this.name = principalName;
     this.principalType = principalType;
     this.operation = operation;
     this.roleOwnerName = roleOwnerName;

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationUtils.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationUtils.java?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationUtils.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationUtils.java Sat Aug 30 06:44:46 2014
@@ -67,9 +67,6 @@ public class AuthorizationUtils {
     case ROLE:
       return HivePrincipalType.ROLE;
     case GROUP:
-      if (SessionState.get().getAuthorizationMode() == SessionState.AuthorizationMode.V2) {
-        throw new HiveException(ErrorMsg.UNSUPPORTED_AUTHORIZATION_PRINCIPAL_TYPE_GROUP);
-      }
       return HivePrincipalType.GROUP;
     default:
       //should not happen as we take care of all existing types

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/StorageBasedAuthorizationProvider.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/StorageBasedAuthorizationProvider.java?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/StorageBasedAuthorizationProvider.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/StorageBasedAuthorizationProvider.java Sat Aug 30 06:44:46 2014
@@ -21,7 +21,7 @@ package org.apache.hadoop.hive.ql.securi
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.security.AccessControlException;
-import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.List;
 
@@ -34,12 +34,9 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsAction;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.hive.common.FileUtils;
-import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
+import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.MetaException;
@@ -48,7 +45,6 @@ import org.apache.hadoop.hive.ql.metadat
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.metadata.Table;
-import org.apache.hadoop.hive.shims.ShimLoader;
 
 /**
  * StorageBasedAuthorizationProvider is an implementation of
@@ -141,28 +137,77 @@ public class StorageBasedAuthorizationPr
   public void authorize(Database db, Privilege[] readRequiredPriv, Privilege[] writeRequiredPriv)
       throws HiveException, AuthorizationException {
     Path path = getDbLocation(db);
+
+    // extract drop privileges
+    DropPrivilegeExtractor privExtractor = new DropPrivilegeExtractor(readRequiredPriv,
+        writeRequiredPriv);
+    readRequiredPriv = privExtractor.getReadReqPriv();
+    writeRequiredPriv = privExtractor.getWriteReqPriv();
+
+    // authorize drops if there was a drop privilege requirement
+    if(privExtractor.hasDropPrivilege()) {
+      checkDeletePermission(path, getConf(), authenticator.getUserName());
+    }
+
     authorize(path, readRequiredPriv, writeRequiredPriv);
   }
 
   @Override
   public void authorize(Table table, Privilege[] readRequiredPriv, Privilege[] writeRequiredPriv)
       throws HiveException, AuthorizationException {
-
-    // To create/drop/alter a table, the owner should have WRITE permission on the database directory
-    authorize(hive_db.getDatabase(table.getDbName()), readRequiredPriv, writeRequiredPriv);
-
-    // If the user has specified a location - external or not, check if the user has the
     try {
       initWh();
-      String location = table.getTTable().getSd().getLocation();
-      if (location != null && !location.isEmpty()) {
-        authorize(new Path(location), readRequiredPriv, writeRequiredPriv);
-      }
     } catch (MetaException ex) {
       throw hiveException(ex);
     }
+
+    // extract any drop privileges out of required privileges
+    DropPrivilegeExtractor privExtractor = new DropPrivilegeExtractor(readRequiredPriv,
+        writeRequiredPriv);
+    readRequiredPriv = privExtractor.getReadReqPriv();
+    writeRequiredPriv = privExtractor.getWriteReqPriv();
+
+    // if CREATE or DROP priv requirement is there, the owner should have WRITE permission on
+    // the database directory
+    if (privExtractor.hasDropPrivilege || requireCreatePrivilege(readRequiredPriv)
+        || requireCreatePrivilege(writeRequiredPriv)) {
+      authorize(hive_db.getDatabase(table.getDbName()), new Privilege[] {},
+          new Privilege[] { Privilege.ALTER_DATA });
+    }
+
+    Path path = table.getDataLocation();
+    // authorize drops if there was a drop privilege requirement, and
+    // table is not external (external table data is not dropped)
+    if (privExtractor.hasDropPrivilege() && table.getTableType() != TableType.EXTERNAL_TABLE) {
+      checkDeletePermission(path, getConf(), authenticator.getUserName());
+    }
+
+    // If the user has specified a location - external or not, check if the user
+    // has the permissions on the table dir
+    if (path != null) {
+      authorize(path, readRequiredPriv, writeRequiredPriv);
+    }
   }
 
+
+  /**
+   *
+   * @param privs
+   * @return true, if set of given privileges privs contain CREATE privilege
+   */
+  private boolean requireCreatePrivilege(Privilege[] privs) {
+    if(privs == null) {
+      return false;
+    }
+    for (Privilege priv : privs) {
+      if (priv.equals(Privilege.CREATE)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+
   @Override
   public void authorize(Partition part, Privilege[] readRequiredPriv, Privilege[] writeRequiredPriv)
       throws HiveException, AuthorizationException {
@@ -173,17 +218,39 @@ public class StorageBasedAuthorizationPr
       Privilege[] writeRequiredPriv)
       throws HiveException, AuthorizationException {
 
+    // extract drop privileges
+    DropPrivilegeExtractor privExtractor = new DropPrivilegeExtractor(readRequiredPriv,
+        writeRequiredPriv);
+    readRequiredPriv = privExtractor.getReadReqPriv();
+    writeRequiredPriv = privExtractor.getWriteReqPriv();
+
+    // authorize drops if there was a drop privilege requirement
+    if(privExtractor.hasDropPrivilege()) {
+      checkDeletePermission(part.getDataLocation(), getConf(), authenticator.getUserName());
+    }
+
     // Partition path can be null in the case of a new create partition - in this case,
     // we try to default to checking the permissions of the parent table.
     // Partition itself can also be null, in cases where this gets called as a generic
     // catch-all call in cases like those with CTAS onto an unpartitioned table (see HIVE-1887)
     if ((part == null) || (part.getLocation() == null)) {
-      authorize(table, readRequiredPriv, writeRequiredPriv);
+      // this should be the case only if this is a create partition.
+      // The privilege needed on the table should be ALTER_DATA, and not CREATE
+      authorize(table, new Privilege[]{}, new Privilege[]{Privilege.ALTER_DATA});
     } else {
       authorize(part.getDataLocation(), readRequiredPriv, writeRequiredPriv);
     }
   }
 
+  private void checkDeletePermission(Path dataLocation, Configuration conf, String userName)
+      throws HiveException {
+    try {
+      FileUtils.checkDeletePermission(dataLocation, conf, userName);
+    } catch (Exception e) {
+      throw new HiveException(e);
+    }
+  }
+
   @Override
   public void authorize(Table table, Partition part, List<String> columns,
       Privilege[] readRequiredPriv, Privilege[] writeRequiredPriv) throws HiveException,
@@ -191,11 +258,7 @@ public class StorageBasedAuthorizationPr
     // In a simple storage-based auth, we have no information about columns
     // living in different files, so we do simple partition-auth and ignore
     // the columns parameter.
-    if ((part != null) && (part.getTable() != null)) {
-      authorize(part.getTable(), part, readRequiredPriv, writeRequiredPriv);
-    } else {
-      authorize(table, part, readRequiredPriv, writeRequiredPriv);
-    }
+    authorize(table, part, readRequiredPriv, writeRequiredPriv);
   }
 
   @Override
@@ -373,4 +436,48 @@ public class StorageBasedAuthorizationPr
     // no-op - SBA does not attempt to authorize auth api call. Allow it
   }
 
+  public class DropPrivilegeExtractor {
+
+    private boolean hasDropPrivilege = false;
+    private final Privilege[] readReqPriv;
+    private final Privilege[] writeReqPriv;
+
+    public DropPrivilegeExtractor(Privilege[] readRequiredPriv, Privilege[] writeRequiredPriv) {
+      this.readReqPriv = extractDropPriv(readRequiredPriv);
+      this.writeReqPriv = extractDropPriv(writeRequiredPriv);
+    }
+
+    private Privilege[] extractDropPriv(Privilege[] requiredPrivs) {
+      if (requiredPrivs == null) {
+        return null;
+      }
+      List<Privilege> privList = new ArrayList<Privilege>();
+      for (Privilege priv : requiredPrivs) {
+        if (priv.equals(Privilege.DROP)) {
+          hasDropPrivilege = true;
+        } else {
+          privList.add(priv);
+        }
+      }
+      return privList.toArray(new Privilege[0]);
+    }
+
+    public boolean hasDropPrivilege() {
+      return hasDropPrivilege;
+    }
+
+    public void setHasDropPrivilege(boolean hasDropPrivilege) {
+      this.hasDropPrivilege = hasDropPrivilege;
+    }
+
+    public Privilege[] getReadReqPriv() {
+      return readReqPriv;
+    }
+
+    public Privilege[] getWriteReqPriv() {
+      return writeReqPriv;
+    }
+
+  }
+
 }

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java Sat Aug 30 06:44:46 2014
@@ -114,6 +114,7 @@ public enum HiveOperationType {
   ALTERTABLE_SKEWED,
   ALTERTBLPART_SKEWED_LOCATION,
   ALTERVIEW_RENAME,
+  ALTERVIEW_AS,
   ALTERTABLE_COMPACT,
   SHOW_COMPACTIONS,
   SHOW_TRANSACTIONS,

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HivePrincipal.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HivePrincipal.java?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HivePrincipal.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HivePrincipal.java Sat Aug 30 06:44:46 2014
@@ -50,16 +50,9 @@ public class HivePrincipal implements Co
 
   public HivePrincipal(String name, HivePrincipalType type){
     this.type = type;
-    if (type == HivePrincipalType.ROLE) {
-      // lower case role to make operations on it case insensitive
-      // when the old default authorization gets deprecated, this can move
-      // to ObjectStore code base
-      this.name = name.toLowerCase();
-    } else {
-      this.name = name;
-    }
-
+    this.name = name;
   }
+
   public String getName() {
     return name;
   }

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java Sat Aug 30 06:44:46 2014
@@ -246,6 +246,8 @@ public class Operation2Privilege {
 (OWNER_PRIV_AR, OWNER_PRIV_AR));
     op2Priv.put(HiveOperationType.ALTERVIEW_RENAME, PrivRequirement.newIOPrivRequirement
 (OWNER_PRIV_AR, OWNER_PRIV_AR));
+    op2Priv.put(HiveOperationType.ALTERVIEW_AS, PrivRequirement.newIOPrivRequirement
+(OWNER_PRIV_AR, OWNER_PRIV_AR));
     op2Priv.put(HiveOperationType.DROPVIEW, PrivRequirement.newIOPrivRequirement
 (OWNER_PRIV_AR, OWNER_PRIV_AR));
 
@@ -276,8 +278,9 @@ public class Operation2Privilege {
 (SEL_NOGRANT_AR, null));
     op2Priv.put(HiveOperationType.SHOW_TBLPROPERTIES, PrivRequirement.newIOPrivRequirement
 (SEL_NOGRANT_AR, null));
-    op2Priv.put(HiveOperationType.CREATETABLE_AS_SELECT, PrivRequirement.newIOPrivRequirement
-(SEL_NOGRANT_AR, null));
+    op2Priv.put(HiveOperationType.CREATETABLE_AS_SELECT, PrivRequirement.newPrivRequirementList(
+        new PrivRequirement(SEL_NOGRANT_AR, IOType.INPUT),
+        new PrivRequirement(OWNER_PRIV_AR, HivePrivilegeObjectType.DATABASE)));
 
     // QUERY,LOAD op can contain an insert & overwrite,
     // require delete privilege if this is an insert-overwrite
@@ -300,8 +303,9 @@ public class Operation2Privilege {
 
     // for now allow only create-view with 'select with grant'
     // the owner will also have select with grant privileges on new view
-    op2Priv.put(HiveOperationType.CREATEVIEW, PrivRequirement.newIOPrivRequirement
-(SEL_GRANT_AR, null));
+    op2Priv.put(HiveOperationType.CREATEVIEW, PrivRequirement.newPrivRequirementList(
+        new PrivRequirement(SEL_GRANT_AR, IOType.INPUT),
+        new PrivRequirement(OWNER_PRIV_AR, HivePrivilegeObjectType.DATABASE)));
 
     op2Priv.put(HiveOperationType.SHOWFUNCTIONS, PrivRequirement.newIOPrivRequirement
 (null, null));

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLAuthorizationUtils.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLAuthorizationUtils.java?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLAuthorizationUtils.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLAuthorizationUtils.java Sat Aug 30 06:44:46 2014
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd;
 
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -25,6 +24,7 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
+import java.util.ListIterator;
 import java.util.Locale;
 import java.util.Map;
 import java.util.Set;
@@ -416,4 +416,43 @@ public class SQLAuthorizationUtils {
     return new HiveAuthzPluginException(prefix + ": " + e.getMessage(), e);
   }
 
+  /**
+   * Validate the principal type, and convert role name to lower case
+   * @param hPrincipal
+   * @return validated principal
+   * @throws HiveAuthzPluginException
+   */
+  public static HivePrincipal getValidatedPrincipal(HivePrincipal hPrincipal)
+      throws HiveAuthzPluginException {
+    if (hPrincipal == null || hPrincipal.getType() == null) {
+      // null principal
+      return hPrincipal;
+    }
+    switch (hPrincipal.getType()) {
+    case USER:
+      return hPrincipal;
+    case ROLE:
+      // lower case role names, for case insensitive behavior
+      return new HivePrincipal(hPrincipal.getName().toLowerCase(), hPrincipal.getType());
+    default:
+      throw new HiveAuthzPluginException("Invalid principal type in principal " + hPrincipal);
+    }
+  }
+
+  /**
+   * Calls getValidatedPrincipal on each principal in list and updates the list
+   * @param hivePrincipals
+   * @return
+   * @return
+   * @throws HiveAuthzPluginException
+   */
+  public static List<HivePrincipal> getValidatedPrincipals(List<HivePrincipal> hivePrincipals)
+      throws HiveAuthzPluginException {
+    ListIterator<HivePrincipal> it = hivePrincipals.listIterator();
+    while(it.hasNext()){
+      it.set(getValidatedPrincipal(it.next()));
+    }
+    return hivePrincipals;
+  }
+
 }

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizationValidator.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizationValidator.java?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizationValidator.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizationValidator.java Sat Aug 30 06:44:46 2014
@@ -43,17 +43,17 @@ public class SQLStdHiveAuthorizationVali
   private final HiveMetastoreClientFactory metastoreClientFactory;
   private final HiveConf conf;
   private final HiveAuthenticationProvider authenticator;
-  private final SQLStdHiveAccessController privController;
+  private final SQLStdHiveAccessControllerWrapper privController;
   public static final Log LOG = LogFactory.getLog(SQLStdHiveAuthorizationValidator.class);
 
   public SQLStdHiveAuthorizationValidator(HiveMetastoreClientFactory metastoreClientFactory,
       HiveConf conf, HiveAuthenticationProvider authenticator,
-      SQLStdHiveAccessController privController) {
+      SQLStdHiveAccessControllerWrapper privilegeManager) {
 
     this.metastoreClientFactory = metastoreClientFactory;
     this.conf = conf;
     this.authenticator = authenticator;
-    this.privController = privController;
+    this.privController = privilegeManager;
   }
 
   @Override

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizerFactory.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizerFactory.java?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizerFactory.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizerFactory.java Sat Aug 30 06:44:46 2014
@@ -32,8 +32,8 @@ public class SQLStdHiveAuthorizerFactory
   @Override
   public HiveAuthorizer createHiveAuthorizer(HiveMetastoreClientFactory metastoreClientFactory,
       HiveConf conf, HiveAuthenticationProvider authenticator, HiveAuthzSessionContext ctx) throws HiveAuthzPluginException {
-    SQLStdHiveAccessController privilegeManager =
-        new SQLStdHiveAccessController(metastoreClientFactory, conf, authenticator, ctx);
+    SQLStdHiveAccessControllerWrapper privilegeManager =
+        new SQLStdHiveAccessControllerWrapper(metastoreClientFactory, conf, authenticator, ctx);
     return new HiveAuthorizerImpl(
         privilegeManager,
         new SQLStdHiveAuthorizationValidator(metastoreClientFactory, conf, authenticator,

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java Sat Aug 30 06:44:46 2014
@@ -45,6 +45,8 @@ import org.apache.hadoop.fs.permission.F
 import org.apache.hadoop.hive.common.JavaUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
 import org.apache.hadoop.hive.ql.MapRedStats;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.exec.spark.session.SparkSession;
@@ -88,6 +90,8 @@ public class SessionState {
   private static final String HDFS_SESSION_PATH_KEY = "_hive.hdfs.session.path";
   private static final String TMP_TABLE_SPACE_KEY = "_hive.tmp_table_space";
   private final Map<String, Map<String, Table>> tempTables = new HashMap<String, Map<String, Table>>();
+  private final Map<String, Map<String, ColumnStatisticsObj>> tempTableColStats =
+      new HashMap<String, Map<String, ColumnStatisticsObj>>();
 
   protected ClassLoader parentLoader;
 
@@ -1159,6 +1163,10 @@ public class SessionState {
     return tempTables;
   }
 
+  public Map<String, Map<String, ColumnStatisticsObj>> getTempTableColStats() {
+    return tempTableColStats;
+  }
+
   /**
    * @return ip address for user running the query
    */

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java Sat Aug 30 06:44:46 2014
@@ -1166,6 +1166,10 @@ public class StatsUtils {
     return getFullyQualifiedName(dbName, tabName, partName, colName);
   }
 
+  public static String getFullyQualifiedTableName(String dbName, String tabName) {
+    return getFullyQualifiedName(dbName, tabName);
+  }
+
   private static String getFullyQualifiedName(String... names) {
     List<String> nonNullAndEmptyNames = Lists.newArrayList();
     for (String name : names) {

Modified: hive/branches/spark/ql/src/protobuf/org/apache/hadoop/hive/ql/io/orc/orc_proto.proto
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/protobuf/org/apache/hadoop/hive/ql/io/orc/orc_proto.proto?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/protobuf/org/apache/hadoop/hive/ql/io/orc/orc_proto.proto (original)
+++ hive/branches/spark/ql/src/protobuf/org/apache/hadoop/hive/ql/io/orc/orc_proto.proto Sat Aug 30 06:44:46 2014
@@ -53,6 +53,12 @@ message DateStatistics {
   optional sint32 maximum = 2;
 }
 
+message TimestampStatistics {
+  // min,max values saved as milliseconds since epoch
+  optional sint64 minimum = 1;
+  optional sint64 maximum = 2;
+}
+
 message BinaryStatistics {
   // sum will store the total binary blob length in a stripe
   optional sint64 sum = 1;
@@ -67,6 +73,7 @@ message ColumnStatistics {
   optional DecimalStatistics decimalStatistics = 6;
   optional DateStatistics dateStatistics = 7;
   optional BinaryStatistics binaryStatistics = 8;
+  optional TimestampStatistics timestampStatistics = 9;
 }
 
 message RowIndexEntry {

Modified: hive/branches/spark/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestFileDump.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestFileDump.java?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestFileDump.java (original)
+++ hive/branches/spark/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestFileDump.java Sat Aug 30 06:44:46 2014
@@ -92,7 +92,7 @@ public class TestFileDump {
     }
     conf.set(HiveConf.ConfVars.HIVE_ORC_ENCODING_STRATEGY.varname, "COMPRESSION");
     Writer writer = OrcFile.createWriter(fs, testFilePath, conf, inspector,
-        100000, CompressionKind.ZLIB, 10000, 10000);
+        100000, CompressionKind.ZLIB, 10000, 1000);
     Random r1 = new Random(1);
     String[] words = new String[]{"It", "was", "the", "best", "of", "times,",
         "it", "was", "the", "worst", "of", "times,", "it", "was", "the", "age",
@@ -116,7 +116,7 @@ public class TestFileDump {
 
     // replace stdout and run command
     System.setOut(new PrintStream(myOut));
-    FileDump.main(new String[]{testFilePath.toString()});
+    FileDump.main(new String[]{testFilePath.toString(), "--rowindex=1,2,3"});
     System.out.flush();
     System.setOut(origOut);
 
@@ -138,7 +138,7 @@ public class TestFileDump {
     conf.set(HiveConf.ConfVars.HIVE_ORC_ENCODING_STRATEGY.varname, "COMPRESSION");
     conf.setFloat(HiveConf.ConfVars.HIVE_ORC_DICTIONARY_KEY_SIZE_THRESHOLD.varname, 0.49f);
     Writer writer = OrcFile.createWriter(fs, testFilePath, conf, inspector,
-        100000, CompressionKind.ZLIB, 10000, 10000);
+        100000, CompressionKind.ZLIB, 10000, 1000);
     Random r1 = new Random(1);
     String[] words = new String[]{"It", "was", "the", "best", "of", "times,",
         "it", "was", "the", "worst", "of", "times,", "it", "was", "the", "age",
@@ -171,7 +171,7 @@ public class TestFileDump {
 
     // replace stdout and run command
     System.setOut(new PrintStream(myOut));
-    FileDump.main(new String[]{testFilePath.toString()});
+    FileDump.main(new String[]{testFilePath.toString(), "--rowindex=1,2,3"});
     System.out.flush();
     System.setOut(origOut);
 

Modified: hive/branches/spark/ql/src/test/queries/clientnegative/authorization_public_create.q
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/test/queries/clientnegative/authorization_public_create.q?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/test/queries/clientnegative/authorization_public_create.q (original)
+++ hive/branches/spark/ql/src/test/queries/clientnegative/authorization_public_create.q Sat Aug 30 06:44:46 2014
@@ -1 +1 @@
-create role PUBLIC;
+create role public;

Modified: hive/branches/spark/ql/src/test/queries/clientnegative/authorization_public_drop.q
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/test/queries/clientnegative/authorization_public_drop.q?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/test/queries/clientnegative/authorization_public_drop.q (original)
+++ hive/branches/spark/ql/src/test/queries/clientnegative/authorization_public_drop.q Sat Aug 30 06:44:46 2014
@@ -1 +1 @@
-drop role PUBLIC;
+drop role public;

Modified: hive/branches/spark/ql/src/test/queries/clientnegative/authorize_grant_public.q
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/test/queries/clientnegative/authorize_grant_public.q?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/test/queries/clientnegative/authorize_grant_public.q (original)
+++ hive/branches/spark/ql/src/test/queries/clientnegative/authorize_grant_public.q Sat Aug 30 06:44:46 2014
@@ -1 +1 @@
-grant role PUBLIC to user hive_test_user;
+grant role public to user hive_test_user;

Modified: hive/branches/spark/ql/src/test/queries/clientnegative/authorize_revoke_public.q
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/test/queries/clientnegative/authorize_revoke_public.q?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/test/queries/clientnegative/authorize_revoke_public.q (original)
+++ hive/branches/spark/ql/src/test/queries/clientnegative/authorize_revoke_public.q Sat Aug 30 06:44:46 2014
@@ -1 +1 @@
-revoke role PUBLIC from user hive_test_user;
+revoke role public from user hive_test_user;

Modified: hive/branches/spark/ql/src/test/queries/clientpositive/add_part_exist.q
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/test/queries/clientpositive/add_part_exist.q?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/test/queries/clientpositive/add_part_exist.q (original)
+++ hive/branches/spark/ql/src/test/queries/clientpositive/add_part_exist.q Sat Aug 30 06:44:46 2014
@@ -18,20 +18,21 @@ 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;
+CREATE TABLE add_part_test_db.add_part_test (key STRING, value STRING) PARTITIONED BY (ds STRING);
+SHOW PARTITIONS add_part_test_db.add_part_test;
 
-ALTER TABLE add_part_test ADD PARTITION (ds='2010-01-01');
-SHOW PARTITIONS add_part_test;
+ALTER TABLE add_part_test_db.add_part_test ADD PARTITION (ds='2010-01-01');
+SHOW PARTITIONS add_part_test_db.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_db.add_part_test ADD IF NOT EXISTS PARTITION (ds='2010-01-01');
+SHOW PARTITIONS add_part_test_db.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_db.add_part_test ADD IF NOT EXISTS PARTITION (ds='2010-01-02');
+SHOW PARTITIONS add_part_test_db.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;
+ALTER TABLE add_part_test_db.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_db.add_part_test;
+
+DROP TABLE add_part_test_db.add_part_test;
+DROP DATABASE add_part_test_db;

Modified: hive/branches/spark/ql/src/test/queries/clientpositive/alter1.q
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/test/queries/clientpositive/alter1.q?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/test/queries/clientpositive/alter1.q (original)
+++ hive/branches/spark/ql/src/test/queries/clientpositive/alter1.q Sat Aug 30 06:44:46 2014
@@ -32,40 +32,38 @@ SHOW TABLES;
 -- With non-default Database
 
 CREATE DATABASE alter1_db;
-USE alter1_db;
-SHOW TABLES;
+SHOW TABLES alter1_db;
 
-CREATE TABLE alter1(a INT, b INT);
-DESCRIBE EXTENDED alter1;
+CREATE TABLE alter1_db.alter1(a INT, b INT);
+DESCRIBE EXTENDED alter1_db.alter1;
 
-ALTER TABLE alter1 SET TBLPROPERTIES ('a'='1', 'c'='3');
-DESCRIBE EXTENDED alter1;
+ALTER TABLE alter1_db.alter1 SET TBLPROPERTIES ('a'='1', 'c'='3');
+DESCRIBE EXTENDED alter1_db.alter1;
 
-ALTER TABLE alter1 SET TBLPROPERTIES ('a'='1', 'c'='4', 'd'='3');
-DESCRIBE EXTENDED alter1;
+ALTER TABLE alter1_db.alter1 SET TBLPROPERTIES ('a'='1', 'c'='4', 'd'='3');
+DESCRIBE EXTENDED alter1_db.alter1;
 
-ALTER TABLE alter1 SET TBLPROPERTIES ('EXTERNAL'='TRUE');
-DESCRIBE EXTENDED alter1;
+ALTER TABLE alter1_db.alter1 SET TBLPROPERTIES ('EXTERNAL'='TRUE');
+DESCRIBE EXTENDED alter1_db.alter1;
 
-ALTER TABLE alter1 SET TBLPROPERTIES ('EXTERNAL'='FALSE');
-DESCRIBE EXTENDED alter1;
+ALTER TABLE alter1_db.alter1 SET TBLPROPERTIES ('EXTERNAL'='FALSE');
+DESCRIBE EXTENDED alter1_db.alter1;
 
-ALTER TABLE alter1 SET SERDEPROPERTIES('s1'='9');
-DESCRIBE EXTENDED alter1;
+ALTER TABLE alter1_db.alter1 SET SERDEPROPERTIES('s1'='9');
+DESCRIBE EXTENDED alter1_db.alter1;
 
-ALTER TABLE alter1 SET SERDEPROPERTIES('s1'='10', 's2' ='20');
-DESCRIBE EXTENDED alter1;
+ALTER TABLE alter1_db.alter1 SET SERDEPROPERTIES('s1'='10', 's2' ='20');
+DESCRIBE EXTENDED alter1_db.alter1;
 
 add jar ${system:maven.local.repository}/org/apache/hive/hive-it-test-serde/${system:hive.version}/hive-it-test-serde-${system:hive.version}.jar;
-ALTER TABLE alter1 SET SERDE 'org.apache.hadoop.hive.serde2.TestSerDe' WITH SERDEPROPERTIES ('s1'='9');
-DESCRIBE EXTENDED alter1;
+ALTER TABLE alter1_db.alter1 SET SERDE 'org.apache.hadoop.hive.serde2.TestSerDe' WITH SERDEPROPERTIES ('s1'='9');
+DESCRIBE EXTENDED alter1_db.alter1;
 
-ALTER TABLE alter1 SET SERDE 'org.apache.hadoop.hive.serde2.MetadataTypedColumnsetSerDe';
-DESCRIBE EXTENDED alter1;
+ALTER TABLE alter1_db.alter1 SET SERDE 'org.apache.hadoop.hive.serde2.MetadataTypedColumnsetSerDe';
+DESCRIBE EXTENDED alter1_db.alter1;
 
-ALTER TABLE alter1 REPLACE COLUMNS (a int, b int, c string);
-DESCRIBE alter1;
+ALTER TABLE alter1_db.alter1 REPLACE COLUMNS (a int, b int, c string);
+DESCRIBE alter1_db.alter1;
 
-DROP TABLE alter1;
-USE default;
+DROP TABLE alter1_db.alter1;
 DROP DATABASE alter1_db;

Modified: hive/branches/spark/ql/src/test/queries/clientpositive/alter_char1.q
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/test/queries/clientpositive/alter_char1.q?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/test/queries/clientpositive/alter_char1.q (original)
+++ hive/branches/spark/ql/src/test/queries/clientpositive/alter_char1.q Sat Aug 30 06:44:46 2014
@@ -1,34 +1,35 @@
 -- SORT_QUERY_RESULTS
 
-drop table alter_char_1;
+create database ac;
 
-create table alter_char_1 (key string, value string);
-insert overwrite table alter_char_1
+create table ac.alter_char_1 (key string, value string);
+insert overwrite table ac.alter_char_1
   select key, value from src order by key limit 5;
 
-select * from alter_char_1;
+select * from ac.alter_char_1;
 
 -- change column to char
-alter table alter_char_1 change column value value char(20);
+alter table ac.alter_char_1 change column value value char(20);
 -- contents should still look the same
-select * from alter_char_1;
+select * from ac.alter_char_1;
 
 -- change column to smaller char
-alter table alter_char_1 change column value value char(3);
+alter table ac.alter_char_1 change column value value char(3);
 -- value column should be truncated now
-select * from alter_char_1;
+select * from ac.alter_char_1;
 
 -- change back to bigger char
-alter table alter_char_1 change column value value char(20);
+alter table ac.alter_char_1 change column value value char(20);
 -- column values should be full size again
-select * from alter_char_1;
+select * from ac.alter_char_1;
 
 -- add char column
-alter table alter_char_1 add columns (key2 int, value2 char(10));
-select * from alter_char_1;
+alter table ac.alter_char_1 add columns (key2 int, value2 char(10));
+select * from ac.alter_char_1;
 
-insert overwrite table alter_char_1
+insert overwrite table ac.alter_char_1
   select key, value, key, value from src order by key limit 5;
-select * from alter_char_1;
+select * from ac.alter_char_1;
 
-drop table alter_char_1;
+drop table ac.alter_char_1;
+drop database ac;

Modified: hive/branches/spark/ql/src/test/queries/clientpositive/alter_index.q
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/test/queries/clientpositive/alter_index.q?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/test/queries/clientpositive/alter_index.q (original)
+++ hive/branches/spark/ql/src/test/queries/clientpositive/alter_index.q Sat Aug 30 06:44:46 2014
@@ -1,11 +1,11 @@
 drop index src_index_8 on src;
 
-create index src_index_8 on table src(key) as 'compact' WITH DEFERRED REBUILD IDXPROPERTIES ("prop1"="val1", "prop2"="val2"); 
+create index src_index_8 on table default.src(key) as 'compact' WITH DEFERRED REBUILD IDXPROPERTIES ("prop1"="val1", "prop2"="val2");
 desc extended default__src_src_index_8__;
 
-alter index src_index_8 on src set IDXPROPERTIES ("prop1"="val1_new", "prop3"="val3"); 
+alter index src_index_8 on default.src set IDXPROPERTIES ("prop1"="val1_new", "prop3"="val3");
 desc extended default__src_src_index_8__;
 
-drop index src_index_8 on src;
+drop index src_index_8 on default.src;
 
 show tables;

Modified: hive/branches/spark/ql/src/test/queries/clientpositive/alter_partition_coltype.q
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/test/queries/clientpositive/alter_partition_coltype.q?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/test/queries/clientpositive/alter_partition_coltype.q (original)
+++ hive/branches/spark/ql/src/test/queries/clientpositive/alter_partition_coltype.q Sat Aug 30 06:44:46 2014
@@ -51,18 +51,23 @@ desc alter_coltype partition (dt='100', 
 
 drop table alter_coltype;
 
-create table alterdynamic_part_table(intcol string) partitioned by (partcol1 string, partcol2 string);
+create database pt;
+
+create table pt.alterdynamic_part_table(intcol string) partitioned by (partcol1 string, partcol2 string);
 
 set hive.exec.dynamic.partition.mode=nonstrict;
 
-insert into table alterdynamic_part_table partition(partcol1, partcol2) select '1', '1', '1' from src where key=150 limit 5;
+insert into table pt.alterdynamic_part_table partition(partcol1, partcol2) select '1', '1', '1' from src where key=150 limit 5;
+
+insert into table pt.alterdynamic_part_table partition(partcol1, partcol2) select '1', '2', '1' from src where key=150 limit 5;
+insert into table pt.alterdynamic_part_table partition(partcol1, partcol2) select NULL, '1', '1' from src where key=150 limit 5;
 
-insert into table alterdynamic_part_table partition(partcol1, partcol2) select '1', '2', '1' from src where key=150 limit 5;
-insert into table alterdynamic_part_table partition(partcol1, partcol2) select NULL, '1', '1' from src where key=150 limit 5;
+alter table pt.alterdynamic_part_table partition column (partcol1 int);
 
-alter table alterdynamic_part_table partition column (partcol1 int);
+explain extended select intcol from pt.alterdynamic_part_table where partcol1='1' and partcol2='1';
 
-explain extended select intcol from alterdynamic_part_table where partcol1='1' and partcol2='1';
+explain extended select intcol from pt.alterdynamic_part_table where (partcol1='2' and partcol2='1')or (partcol1='1' and partcol2='__HIVE_DEFAULT_PARTITION__');
+select intcol from pt.alterdynamic_part_table where (partcol1='2' and partcol2='1')or (partcol1='1' and partcol2='__HIVE_DEFAULT_PARTITION__');
 
-explain extended select intcol from alterdynamic_part_table where (partcol1='2' and partcol2='1')or (partcol1='1' and partcol2='__HIVE_DEFAULT_PARTITION__');
-select intcol from alterdynamic_part_table where (partcol1='2' and partcol2='1')or (partcol1='1' and partcol2='__HIVE_DEFAULT_PARTITION__');
\ No newline at end of file
+drop table pt.alterdynamic_part_table;
+drop database pt;

Modified: hive/branches/spark/ql/src/test/queries/clientpositive/alter_skewed_table.q
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/test/queries/clientpositive/alter_skewed_table.q?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/test/queries/clientpositive/alter_skewed_table.q (original)
+++ hive/branches/spark/ql/src/test/queries/clientpositive/alter_skewed_table.q Sat Aug 30 06:44:46 2014
@@ -1,6 +1,6 @@
 set hive.mapred.supports.subdirectories=true;
 
-create table original (key STRING, value STRING); 
+create table original (key STRING, value STRING);
 
 describe formatted original;
 
@@ -10,23 +10,27 @@ describe formatted original;
 
 drop table original;
 
-create table original2 (key STRING, value STRING) ; 
+create database skew_test;
 
-describe formatted original2;
+create table skew_test.original2 (key STRING, value STRING) ;
 
-alter table original2 SKEWED BY (key, value) ON ((1,1),(5,6));
+describe formatted skew_test.original2;
 
-describe formatted original2;
+alter table skew_test.original2 SKEWED BY (key, value) ON ((1,1),(5,6));
 
-drop table original2;
+describe formatted skew_test.original2;
 
-create table original3 (key STRING, value STRING) SKEWED BY (key, value) ON ((1,1),(5,6)); 
+drop table skew_test.original2;
 
-describe formatted original3;
+create table skew_test.original3 (key STRING, value STRING) SKEWED BY (key, value) ON ((1,1),(5,6));
 
-alter table original3 not skewed;
+describe formatted skew_test.original3;
 
-describe formatted original3;
+alter table skew_test.original3 not skewed;
 
-drop table original3;
+describe formatted skew_test.original3;
+
+drop table skew_test.original3;
+
+drop database skew_test;
 

Modified: hive/branches/spark/ql/src/test/queries/clientpositive/alter_varchar1.q
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/test/queries/clientpositive/alter_varchar1.q?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/test/queries/clientpositive/alter_varchar1.q (original)
+++ hive/branches/spark/ql/src/test/queries/clientpositive/alter_varchar1.q Sat Aug 30 06:44:46 2014
@@ -1,34 +1,35 @@
 -- SORT_QUERY_RESULTS
 
-drop table alter_varchar_1;
+create database avc;
 
-create table alter_varchar_1 (key string, value string);
-insert overwrite table alter_varchar_1
+create table avc.alter_varchar_1 (key string, value string);
+insert overwrite table avc.alter_varchar_1
   select key, value from src order by key limit 5;
 
-select * from alter_varchar_1;
+select * from avc.alter_varchar_1;
 
 -- change column to varchar
-alter table alter_varchar_1 change column value value varchar(20);
+alter table avc.alter_varchar_1 change column value value varchar(20);
 -- contents should still look the same
-select * from alter_varchar_1;
+select * from avc.alter_varchar_1;
 
 -- change column to smaller varchar
-alter table alter_varchar_1 change column value value varchar(3);
+alter table avc.alter_varchar_1 change column value value varchar(3);
 -- value column should be truncated now
-select * from alter_varchar_1;
+select * from avc.alter_varchar_1;
 
 -- change back to bigger varchar
-alter table alter_varchar_1 change column value value varchar(20);
+alter table avc.alter_varchar_1 change column value value varchar(20);
 -- column values should be full size again
-select * from alter_varchar_1;
+select * from avc.alter_varchar_1;
 
 -- add varchar column
-alter table alter_varchar_1 add columns (key2 int, value2 varchar(10));
-select * from alter_varchar_1;
+alter table avc.alter_varchar_1 add columns (key2 int, value2 varchar(10));
+select * from avc.alter_varchar_1;
 
-insert overwrite table alter_varchar_1
+insert overwrite table avc.alter_varchar_1
   select key, value, key, value from src order by key limit 5;
-select * from alter_varchar_1;
+select * from avc.alter_varchar_1;
 
-drop table alter_varchar_1;
+drop table avc.alter_varchar_1;
+drop database avc;

Modified: hive/branches/spark/ql/src/test/queries/clientpositive/alter_view_as_select.q
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/test/queries/clientpositive/alter_view_as_select.q?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/test/queries/clientpositive/alter_view_as_select.q (original)
+++ hive/branches/spark/ql/src/test/queries/clientpositive/alter_view_as_select.q Sat Aug 30 06:44:46 2014
@@ -1,13 +1,16 @@
-DROP VIEW testView;
-CREATE VIEW testView as SELECT * FROM srcpart;
-DESCRIBE FORMATTED testView;
+CREATE DATABASE tv;
+CREATE VIEW tv.testView as SELECT * FROM srcpart;
+DESCRIBE FORMATTED tv.testView;
 
-ALTER VIEW testView AS SELECT value FROM src WHERE key=86;
-DESCRIBE FORMATTED testView;
+ALTER VIEW tv.testView AS SELECT value FROM src WHERE key=86;
+DESCRIBE FORMATTED tv.testView;
 
-ALTER VIEW testView AS
+ALTER VIEW tv.testView AS
 SELECT * FROM src
 WHERE key > 80 AND key < 100
 ORDER BY key, value
 LIMIT 10;
-DESCRIBE FORMATTED testView;
+DESCRIBE FORMATTED tv.testView;
+
+DROP VIEW tv.testView;
+DROP DATABASE tv;
\ No newline at end of file

Modified: hive/branches/spark/ql/src/test/queries/clientpositive/alter_view_rename.q
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/test/queries/clientpositive/alter_view_rename.q?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/test/queries/clientpositive/alter_view_rename.q (original)
+++ hive/branches/spark/ql/src/test/queries/clientpositive/alter_view_rename.q Sat Aug 30 06:44:46 2014
@@ -1,10 +1,16 @@
+CREATE DATABASE tv1;
+CREATE DATABASE tv2;
+
 CREATE TABLE invites (foo INT, bar STRING) PARTITIONED BY (ds STRING);
-CREATE VIEW view1 as SELECT * FROM invites;
-DESCRIBE EXTENDED view1;
+CREATE VIEW tv1.view1 as SELECT * FROM invites;
+DESCRIBE EXTENDED tv1.view1;
 
-ALTER VIEW view1 RENAME TO view2;
-DESCRIBE EXTENDED view2;
-SELECT * FROM view2;
+ALTER VIEW tv1.view1 RENAME TO tv2.view2;
+DESCRIBE EXTENDED tv2.view2;
+SELECT * FROM tv2.view2;
 
 DROP TABLE invites;
-DROP VIEW view2;
\ No newline at end of file
+DROP VIEW tv2.view2;
+
+DROP DATABASE tv1;
+DROP DATABASE tv2;

Modified: hive/branches/spark/ql/src/test/queries/clientpositive/archive_multi.q
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/test/queries/clientpositive/archive_multi.q?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/test/queries/clientpositive/archive_multi.q (original)
+++ hive/branches/spark/ql/src/test/queries/clientpositive/archive_multi.q Sat Aug 30 06:44:46 2014
@@ -1,42 +1,41 @@
 set hive.archive.enabled = true;
 set hive.enforce.bucketing = true;
 
-drop table tstsrc;
-drop table tstsrcpart;
+create database ac_test;
 
-create table tstsrc like src;
-insert overwrite table tstsrc select key, value from src;
+create table ac_test.tstsrc like default.src;
+insert overwrite table ac_test.tstsrc select key, value from default.src;
 
-create table tstsrcpart like srcpart;
+create table ac_test.tstsrcpart like default.srcpart;
 
-insert overwrite table tstsrcpart partition (ds='2008-04-08', hr='11')
-select key, value from srcpart where ds='2008-04-08' and hr='11';
+insert overwrite table ac_test.tstsrcpart partition (ds='2008-04-08', hr='11')
+select key, value from default.srcpart where ds='2008-04-08' and hr='11';
 
-insert overwrite table tstsrcpart partition (ds='2008-04-08', hr='12')
-select key, value from srcpart where ds='2008-04-08' and hr='12';
+insert overwrite table ac_test.tstsrcpart partition (ds='2008-04-08', hr='12')
+select key, value from default.srcpart where ds='2008-04-08' and hr='12';
 
-insert overwrite table tstsrcpart partition (ds='2008-04-09', hr='11')
-select key, value from srcpart where ds='2008-04-09' and hr='11';
+insert overwrite table ac_test.tstsrcpart partition (ds='2008-04-09', hr='11')
+select key, value from default.srcpart where ds='2008-04-09' and hr='11';
 
-insert overwrite table tstsrcpart partition (ds='2008-04-09', hr='12')
-select key, value from srcpart where ds='2008-04-09' and hr='12';
+insert overwrite table ac_test.tstsrcpart partition (ds='2008-04-09', hr='12')
+select key, value from default.srcpart where ds='2008-04-09' and hr='12';
 
 -- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19)
 
 SELECT SUM(hash(col)) FROM (SELECT transform(*) using 'tr "\t" "_"' AS col
-FROM (SELECT * FROM tstsrcpart WHERE ds='2008-04-08') subq1) subq2;
+FROM (SELECT * FROM ac_test.tstsrcpart WHERE ds='2008-04-08') subq1) subq2;
 
-ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08');
+ALTER TABLE ac_test.tstsrcpart ARCHIVE PARTITION (ds='2008-04-08');
 
 SELECT SUM(hash(col)) FROM (SELECT transform(*) using 'tr "\t" "_"' AS col
-FROM (SELECT * FROM tstsrcpart WHERE ds='2008-04-08') subq1) subq2;
+FROM (SELECT * FROM ac_test.tstsrcpart WHERE ds='2008-04-08') subq1) subq2;
 
-SELECT key, count(1) FROM tstsrcpart WHERE ds='2008-04-08' AND hr='12' AND key='0' GROUP BY key;
+SELECT key, count(1) FROM ac_test.tstsrcpart WHERE ds='2008-04-08' AND hr='12' AND key='0' GROUP BY key;
 
-SELECT * FROM tstsrcpart a JOIN tstsrc b ON a.key=b.key
+SELECT * FROM ac_test.tstsrcpart a JOIN ac_test.tstsrc b ON a.key=b.key
 WHERE a.ds='2008-04-08' AND a.hr='12' AND a.key='0';
 
-ALTER TABLE tstsrcpart UNARCHIVE PARTITION (ds='2008-04-08');
+ALTER TABLE ac_test.tstsrcpart UNARCHIVE PARTITION (ds='2008-04-08');
 
 SELECT SUM(hash(col)) FROM (SELECT transform(*) using 'tr "\t" "_"' AS col
-FROM (SELECT * FROM tstsrcpart WHERE ds='2008-04-08') subq1) subq2;
+FROM (SELECT * FROM ac_test.tstsrcpart WHERE ds='2008-04-08') subq1) subq2;

Modified: hive/branches/spark/ql/src/test/queries/clientpositive/authorization_1.q
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/test/queries/clientpositive/authorization_1.q?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/test/queries/clientpositive/authorization_1.q (original)
+++ hive/branches/spark/ql/src/test/queries/clientpositive/authorization_1.q Sat Aug 30 06:44:46 2014
@@ -57,33 +57,33 @@ show grant group hive_test_group1 on tab
 show grant group hive_test_group1 on table src_autho_test(key);
 
 --role
-create role src_role;
-grant role src_role to user hive_test_user;
+create role sRc_roLE;
+grant role sRc_roLE to user hive_test_user;
 show role grant user hive_test_user;
 
 --column grant to role
 
-grant select(key) on table src_autho_test to role src_role;
+grant select(key) on table src_autho_test to role sRc_roLE;
 
-show grant role src_role on table src_autho_test;
-show grant role src_role on table src_autho_test(key);
+show grant role sRc_roLE on table src_autho_test;
+show grant role sRc_roLE on table src_autho_test(key);
 
 select key from src_autho_test order by key limit 20;
 
-revoke select(key) on table src_autho_test from role src_role;
+revoke select(key) on table src_autho_test from role sRc_roLE;
 
 --table grant to role
 
-grant select on table src_autho_test to role src_role;
+grant select on table src_autho_test to role sRc_roLE;
 
 select key from src_autho_test order by key limit 20;
 
-show grant role src_role on table src_autho_test;
-show grant role src_role on table src_autho_test(key);
-revoke select on table src_autho_test from role src_role;
+show grant role sRc_roLE on table src_autho_test;
+show grant role sRc_roLE on table src_autho_test(key);
+revoke select on table src_autho_test from role sRc_roLE;
 
 -- drop role
-drop role src_role;
+drop role sRc_roLE;
 
 set hive.security.authorization.enabled=false;
 drop table src_autho_test;
\ No newline at end of file

Modified: hive/branches/spark/ql/src/test/queries/clientpositive/authorization_5.q
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/test/queries/clientpositive/authorization_5.q?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/test/queries/clientpositive/authorization_5.q (original)
+++ hive/branches/spark/ql/src/test/queries/clientpositive/authorization_5.q Sat Aug 30 06:44:46 2014
@@ -8,13 +8,13 @@ GRANT select ON DATABASE test_db TO USER
 
 SHOW GRANT USER hive_test_user ON DATABASE test_db;
 
-CREATE ROLE db_test_role;
-GRANT ROLE db_test_role TO USER hive_test_user;
+CREATE ROLE db_TEST_Role;
+GRANT ROLE db_TEST_Role TO USER hive_test_user;
 SHOW ROLE GRANT USER hive_test_user;
 
-GRANT drop ON DATABASE test_db TO ROLE db_test_role;
-GRANT select ON DATABASE test_db TO ROLE db_test_role;
+GRANT drop ON DATABASE test_db TO ROLE db_TEST_Role;
+GRANT select ON DATABASE test_db TO ROLE db_TEST_Role;
 
-SHOW GRANT ROLE db_test_role ON DATABASE test_db;
+SHOW GRANT ROLE db_TEST_Role ON DATABASE test_db;
 
 DROP DATABASE IF EXISTS test_db;

Modified: hive/branches/spark/ql/src/test/queries/clientpositive/authorization_grant_public_role.q
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/test/queries/clientpositive/authorization_grant_public_role.q?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/test/queries/clientpositive/authorization_grant_public_role.q (original)
+++ hive/branches/spark/ql/src/test/queries/clientpositive/authorization_grant_public_role.q Sat Aug 30 06:44:46 2014
@@ -9,10 +9,10 @@ CREATE TABLE  t_gpr1(i int);
 
 -- all privileges should have been set for user
 
-GRANT ALL ON t_gpr1 TO ROLE public;
+GRANT ALL ON t_gpr1 TO ROLE pubLic;
 
 SHOW GRANT USER user1 ON TABLE t_gpr1;
-SHOW GRANT ROLE public ON TABLE t_gpr1;
+SHOW GRANT ROLE pubLic ON TABLE t_gpr1;
 
 set user.name=user2;
 SHOW CURRENT ROLES;

Modified: hive/branches/spark/ql/src/test/queries/clientpositive/authorization_role_grant2.q
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/test/queries/clientpositive/authorization_role_grant2.q?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/test/queries/clientpositive/authorization_role_grant2.q (original)
+++ hive/branches/spark/ql/src/test/queries/clientpositive/authorization_role_grant2.q Sat Aug 30 06:44:46 2014
@@ -31,7 +31,7 @@ set user.name=user2;
 grant src_role_wadmin to role sRc_role2;
 
 set user.name=hive_admin_user;
-set role ADMIN;
+set role ADMIn;
 grant src_role2 to user user3;
 
 set user.name=user3;

Modified: hive/branches/spark/ql/src/test/queries/clientpositive/create_or_replace_view.q
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/test/queries/clientpositive/create_or_replace_view.q?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/test/queries/clientpositive/create_or_replace_view.q (original)
+++ hive/branches/spark/ql/src/test/queries/clientpositive/create_or_replace_view.q Sat Aug 30 06:44:46 2014
@@ -1,32 +1,39 @@
-drop view v;
-create view v as select * from srcpart;
-describe formatted v;
+create database vt;
+
+create view vt.v as select * from srcpart;
+describe formatted vt.v;
 
 -- modifying definition of unpartitioned view
-create or replace view v partitioned on (ds, hr) as select * from srcpart;
-alter view v add partition (ds='2008-04-08',hr='11');
-alter view v add partition (ds='2008-04-08',hr='12');
-select * from v where value='val_409' and ds='2008-04-08' and hr='11';
-describe formatted v;
-show partitions v;
+create or replace view vt.v partitioned on (ds, hr) as select * from srcpart;
+alter view vt.v add partition (ds='2008-04-08',hr='11');
+alter view vt.v add partition (ds='2008-04-08',hr='12');
+select * from vt.v where value='val_409' and ds='2008-04-08' and hr='11';
+describe formatted vt.v;
+show partitions vt.v;
+
+alter view vt.v drop partition (ds='2008-04-08',hr='11');
+alter view vt.v drop partition (ds='2008-04-08',hr='12');
+show partitions vt.v;
 
 -- altering partitioned view 1
-create or replace view v partitioned on (ds, hr) as select value, ds, hr from srcpart;
-select * from v where value='val_409' and ds='2008-04-08' and hr='11';
-describe formatted v;
-show partitions v;
+create or replace view vt.v partitioned on (ds, hr) as select value, ds, hr from srcpart;
+select * from vt.v where value='val_409' and ds='2008-04-08' and hr='11';
+describe formatted vt.v;
+show partitions vt.v;
 
 -- altering partitioned view 2
-create or replace view v partitioned on (ds, hr) as select key, value, ds, hr from srcpart;
-select * from v where value='val_409' and ds='2008-04-08' and hr='11';
-describe formatted v;
-show partitions v;
-drop view v;
+create or replace view vt.v partitioned on (ds, hr) as select key, value, ds, hr from srcpart;
+select * from vt.v where value='val_409' and ds='2008-04-08' and hr='11';
+describe formatted vt.v;
+show partitions vt.v;
+drop view vt.v;
 
 -- updating to fix view with invalid definition
 create table srcpart_temp like srcpart;
-create view v partitioned on (ds, hr) as select * from srcpart_temp;
-drop table srcpart_temp; -- v is now invalid
-create or replace view v partitioned on (ds, hr) as select * from srcpart;
-describe formatted v;
-drop view v;
\ No newline at end of file
+create view vt.v partitioned on (ds, hr) as select * from srcpart_temp;
+drop table srcpart_temp; -- vt.v is now invalid
+create or replace view vt.v partitioned on (ds, hr) as select * from srcpart;
+describe formatted vt.v;
+drop view vt.v;
+
+drop database vt;
\ No newline at end of file

Modified: hive/branches/spark/ql/src/test/queries/clientpositive/drop_multi_partitions.q
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/test/queries/clientpositive/drop_multi_partitions.q?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/test/queries/clientpositive/drop_multi_partitions.q (original)
+++ hive/branches/spark/ql/src/test/queries/clientpositive/drop_multi_partitions.q Sat Aug 30 06:44:46 2014
@@ -1,17 +1,23 @@
-create table mp (a string) partitioned by (b string, c string);
+create database dmp;
 
-alter table mp add partition (b='1', c='1');
-alter table mp add partition (b='1', c='2');
-alter table mp add partition (b='2', c='2');
+create table dmp.mp (a string) partitioned by (b string, c string);
 
-show partitions mp;
+alter table dmp.mp add partition (b='1', c='1');
+alter table dmp.mp add partition (b='1', c='2');
+alter table dmp.mp add partition (b='2', c='2');
 
-explain extended alter table mp drop partition (b='1');
-alter table mp drop partition (b='1');
+show partitions dmp.mp;
 
-show partitions mp;
+explain extended alter table dmp.mp drop partition (b='1');
+alter table dmp.mp drop partition (b='1');
+
+show partitions dmp.mp;
 
 set hive.exec.drop.ignorenonexistent=false;
-alter table mp drop if exists partition (b='3');
+alter table dmp.mp drop if exists partition (b='3');
+
+show partitions dmp.mp;
+
+drop table dmp.mp;
 
-show partitions mp;
+drop database dmp;

Modified: hive/branches/spark/ql/src/test/queries/clientpositive/exchange_partition.q
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/test/queries/clientpositive/exchange_partition.q?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/test/queries/clientpositive/exchange_partition.q (original)
+++ hive/branches/spark/ql/src/test/queries/clientpositive/exchange_partition.q Sat Aug 30 06:44:46 2014
@@ -1,12 +1,15 @@
-CREATE TABLE exchange_part_test1 (f1 string) PARTITIONED BY (ds STRING);
-CREATE TABLE exchange_part_test2 (f1 string) PARTITIONED BY (ds STRING);
-SHOW PARTITIONS exchange_part_test1;
-SHOW PARTITIONS exchange_part_test2;
+create database ex1;
+create database ex2;
 
-ALTER TABLE exchange_part_test2 ADD PARTITION (ds='2013-04-05');
-SHOW PARTITIONS exchange_part_test1;
-SHOW PARTITIONS exchange_part_test2;
+CREATE TABLE ex1.exchange_part_test1 (f1 string) PARTITIONED BY (ds STRING);
+CREATE TABLE ex2.exchange_part_test2 (f1 string) PARTITIONED BY (ds STRING);
+SHOW PARTITIONS ex1.exchange_part_test1;
+SHOW PARTITIONS ex2.exchange_part_test2;
 
-ALTER TABLE exchange_part_test1 EXCHANGE PARTITION (ds='2013-04-05') WITH TABLE exchange_part_test2;
-SHOW PARTITIONS exchange_part_test1;
-SHOW PARTITIONS exchange_part_test2;
+ALTER TABLE ex2.exchange_part_test2 ADD PARTITION (ds='2013-04-05');
+SHOW PARTITIONS ex1.exchange_part_test1;
+SHOW PARTITIONS ex2.exchange_part_test2;
+
+ALTER TABLE ex1.exchange_part_test1 EXCHANGE PARTITION (ds='2013-04-05') WITH TABLE ex2.exchange_part_test2;
+SHOW PARTITIONS ex1.exchange_part_test1;
+SHOW PARTITIONS ex2.exchange_part_test2;

Modified: hive/branches/spark/ql/src/test/queries/clientpositive/index_auto_empty.q
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/test/queries/clientpositive/index_auto_empty.q?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/test/queries/clientpositive/index_auto_empty.q (original)
+++ hive/branches/spark/ql/src/test/queries/clientpositive/index_auto_empty.q Sat Aug 30 06:44:46 2014
@@ -1,22 +1,25 @@
 -- Test to ensure that an empty index result is propagated correctly
 
+CREATE DATABASE it;
 -- Create temp, and populate it with some values in src.
-CREATE TABLE temp(key STRING, val STRING) STORED AS TEXTFILE;
+CREATE TABLE it.temp(key STRING, val STRING) STORED AS TEXTFILE;
 
 set hive.stats.dbclass=fs;
--- Build an index on temp.
-CREATE INDEX temp_index ON TABLE temp(key) as 'COMPACT' WITH DEFERRED REBUILD;
-ALTER INDEX temp_index ON temp REBUILD;
+-- Build an index on it.temp.
+CREATE INDEX temp_index ON TABLE it.temp(key) as 'COMPACT' WITH DEFERRED REBUILD;
+ALTER INDEX temp_index ON it.temp REBUILD;
 
 SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
 SET hive.optimize.index.filter=true;
 SET hive.optimize.index.filter.compact.minsize=0;
 
 -- query should not return any values
-SELECT * FROM default__temp_temp_index__ WHERE key = 86;
-EXPLAIN SELECT * FROM temp WHERE key  = 86;
-SELECT * FROM temp WHERE key  = 86;
+SELECT * FROM it.it__temp_temp_index__ WHERE key = 86;
+EXPLAIN SELECT * FROM it.temp WHERE key  = 86;
+SELECT * FROM it.temp WHERE key  = 86;
 
 SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
 SET hive.optimize.index.filter=false;
-DROP table temp;
+DROP table it.temp;
+
+DROP DATABASE it;

Modified: hive/branches/spark/ql/src/test/queries/clientpositive/input46.q
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/test/queries/clientpositive/input46.q?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/test/queries/clientpositive/input46.q (original)
+++ hive/branches/spark/ql/src/test/queries/clientpositive/input46.q Sat Aug 30 06:44:46 2014
@@ -1,6 +1,6 @@
 create database if not exists table_in_database_creation;
 create table table_in_database_creation.test1  as select * from src limit 1;
-create table `table_in_database_creation.test2` as select * from src limit 1;
+create table `table_in_database_creation`.`test2` as select * from src limit 1;
 create table table_in_database_creation.test3 (a string);
-create table `table_in_database_creation.test4` (a string);
+create table `table_in_database_creation`.`test4` (a string);
 drop database table_in_database_creation cascade;
\ No newline at end of file

Modified: hive/branches/spark/ql/src/test/queries/clientpositive/touch.q
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/test/queries/clientpositive/touch.q?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/test/queries/clientpositive/touch.q (original)
+++ hive/branches/spark/ql/src/test/queries/clientpositive/touch.q Sat Aug 30 06:44:46 2014
@@ -1,17 +1,17 @@
-drop table tstsrc;
-drop table tstsrcpart;
+create database tc;
 
-create table tstsrc like src;
-insert overwrite table tstsrc select key, value from src;
+create table tc.tstsrc like default.src;
+insert overwrite table tc.tstsrc select key, value from default.src;
 
-create table tstsrcpart like srcpart;
-insert overwrite table tstsrcpart partition (ds='2008-04-08', hr='12')
-select key, value from srcpart where ds='2008-04-08' and hr='12';
+create table tc.tstsrcpart like default.srcpart;
+insert overwrite table tc.tstsrcpart partition (ds='2008-04-08', hr='12')
+select key, value from default.srcpart where ds='2008-04-08' and hr='12';
 
+ALTER TABLE tc.tstsrc TOUCH;
+ALTER TABLE tc.tstsrcpart TOUCH;
+ALTER TABLE tc.tstsrcpart TOUCH PARTITION (ds='2008-04-08', hr='12');
 
-ALTER TABLE tstsrc TOUCH;
-ALTER TABLE tstsrcpart TOUCH;
-ALTER TABLE tstsrcpart TOUCH PARTITION (ds='2008-04-08', hr='12');
+drop table tc.tstsrc;
+drop table tc.tstsrcpart;
 
-drop table tstsrc;
-drop table tstsrcpart;
+drop database tc;

Modified: hive/branches/spark/ql/src/test/queries/clientpositive/unset_table_view_property.q
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/test/queries/clientpositive/unset_table_view_property.q?rev=1621416&r1=1621415&r2=1621416&view=diff
==============================================================================
--- hive/branches/spark/ql/src/test/queries/clientpositive/unset_table_view_property.q (original)
+++ hive/branches/spark/ql/src/test/queries/clientpositive/unset_table_view_property.q Sat Aug 30 06:44:46 2014
@@ -1,64 +1,71 @@
-CREATE TABLE testTable(col1 INT, col2 INT);
-SHOW TBLPROPERTIES testTable;
+CREATE DATABASE vt;
+
+CREATE TABLE vt.testTable(col1 INT, col2 INT);
+SHOW TBLPROPERTIES vt.testTable;
 
 -- UNSET TABLE PROPERTIES
-ALTER TABLE testTable SET TBLPROPERTIES ('a'='1', 'c'='3');
-SHOW TBLPROPERTIES testTable;
+ALTER TABLE vt.testTable SET TBLPROPERTIES ('a'='1', 'c'='3');
+SHOW TBLPROPERTIES vt.testTable;
 
 -- UNSET all the properties
-ALTER TABLE testTable UNSET TBLPROPERTIES ('a', 'c');
-SHOW TBLPROPERTIES testTable;
+ALTER TABLE vt.testTable UNSET TBLPROPERTIES ('a', 'c');
+SHOW TBLPROPERTIES vt.testTable;
 
-ALTER TABLE testTable SET TBLPROPERTIES ('a'='1', 'c'='3', 'd'='4');
-SHOW TBLPROPERTIES testTable;
+ALTER TABLE vt.testTable SET TBLPROPERTIES ('a'='1', 'c'='3', 'd'='4');
+SHOW TBLPROPERTIES vt.testTable;
 
 -- UNSET a subset of the properties
-ALTER TABLE testTable UNSET TBLPROPERTIES ('a', 'd');
-SHOW TBLPROPERTIES testTable;
+ALTER TABLE vt.testTable UNSET TBLPROPERTIES ('a', 'd');
+SHOW TBLPROPERTIES vt.testTable;
 
 -- the same property being UNSET multiple times
-ALTER TABLE testTable UNSET TBLPROPERTIES ('c', 'c', 'c');
-SHOW TBLPROPERTIES testTable;
+ALTER TABLE vt.testTable UNSET TBLPROPERTIES ('c', 'c', 'c');
+SHOW TBLPROPERTIES vt.testTable;
 
-ALTER TABLE testTable SET TBLPROPERTIES ('a'='1', 'b' = '2', 'c'='3', 'd'='4');
-SHOW TBLPROPERTIES testTable;
+ALTER TABLE vt.testTable SET TBLPROPERTIES ('a'='1', 'b' = '2', 'c'='3', 'd'='4');
+SHOW TBLPROPERTIES vt.testTable;
 
 -- UNSET a subset of the properties and some non-existed properties using IF EXISTS
-ALTER TABLE testTable UNSET TBLPROPERTIES IF EXISTS ('b', 'd', 'b', 'f');
-SHOW TBLPROPERTIES testTable;
+ALTER TABLE vt.testTable UNSET TBLPROPERTIES IF EXISTS ('b', 'd', 'b', 'f');
+SHOW TBLPROPERTIES vt.testTable;
 
 -- UNSET a subset of the properties and some non-existed properties using IF EXISTS
-ALTER TABLE testTable UNSET TBLPROPERTIES IF EXISTS ('b', 'd', 'c', 'f', 'x', 'y', 'z');
-SHOW TBLPROPERTIES testTable;
+ALTER TABLE vt.testTable UNSET TBLPROPERTIES IF EXISTS ('b', 'd', 'c', 'f', 'x', 'y', 'z');
+SHOW TBLPROPERTIES vt.testTable;
+
+DROP TABLE vt.testTable;
 
 -- UNSET VIEW PROPERTIES
-CREATE VIEW testView AS SELECT value FROM src WHERE key=86;
-ALTER VIEW testView SET TBLPROPERTIES ('propA'='100', 'propB'='200');
-SHOW TBLPROPERTIES testView;
+CREATE VIEW vt.testView AS SELECT value FROM src WHERE key=86;
+ALTER VIEW vt.testView SET TBLPROPERTIES ('propA'='100', 'propB'='200');
+SHOW TBLPROPERTIES vt.testView;
 
 -- UNSET all the properties
-ALTER VIEW testView UNSET TBLPROPERTIES ('propA', 'propB');
-SHOW TBLPROPERTIES testView;
+ALTER VIEW vt.testView UNSET TBLPROPERTIES ('propA', 'propB');
+SHOW TBLPROPERTIES vt.testView;
 
-ALTER VIEW testView SET TBLPROPERTIES ('propA'='100', 'propC'='300', 'propD'='400');
-SHOW TBLPROPERTIES testView;
+ALTER VIEW vt.testView SET TBLPROPERTIES ('propA'='100', 'propC'='300', 'propD'='400');
+SHOW TBLPROPERTIES vt.testView;
 
 -- UNSET a subset of the properties
-ALTER VIEW testView UNSET TBLPROPERTIES ('propA', 'propC');
-SHOW TBLPROPERTIES testView;
+ALTER VIEW vt.testView UNSET TBLPROPERTIES ('propA', 'propC');
+SHOW TBLPROPERTIES vt.testView;
 
 -- the same property being UNSET multiple times
-ALTER VIEW testView UNSET TBLPROPERTIES ('propD', 'propD', 'propD');
-SHOW TBLPROPERTIES testView;
+ALTER VIEW vt.testView UNSET TBLPROPERTIES ('propD', 'propD', 'propD');
+SHOW TBLPROPERTIES vt.testView;
 
-ALTER VIEW testView SET TBLPROPERTIES ('propA'='100', 'propB' = '200', 'propC'='300', 'propD'='400');
-SHOW TBLPROPERTIES testView;
+ALTER VIEW vt.testView SET TBLPROPERTIES ('propA'='100', 'propB' = '200', 'propC'='300', 'propD'='400');
+SHOW TBLPROPERTIES vt.testView;
 
 -- UNSET a subset of the properties and some non-existed properties using IF EXISTS
-ALTER VIEW testView UNSET TBLPROPERTIES IF EXISTS ('propC', 'propD', 'propD', 'propC', 'propZ');
-SHOW TBLPROPERTIES testView;
+ALTER VIEW vt.testView UNSET TBLPROPERTIES IF EXISTS ('propC', 'propD', 'propD', 'propC', 'propZ');
+SHOW TBLPROPERTIES vt.testView;
 
 -- UNSET a subset of the properties and some non-existed properties using IF EXISTS
-ALTER VIEW testView UNSET TBLPROPERTIES IF EXISTS ('propB', 'propC', 'propD', 'propF');
-SHOW TBLPROPERTIES testView;
+ALTER VIEW vt.testView UNSET TBLPROPERTIES IF EXISTS ('propB', 'propC', 'propD', 'propF');
+SHOW TBLPROPERTIES vt.testView;
+
+DROP VIEW vt.testView;
 
+DROP DATABASE vt;
\ No newline at end of file