You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by th...@apache.org on 2014/02/09 21:43:38 UTC

svn commit: r1566401 [2/3] - in /hive/trunk: common/src/java/org/apache/hadoop/hive/conf/ conf/ data/conf/ itests/util/src/main/java/org/apache/hadoop/hive/ql/security/ metastore/src/java/org/apache/hadoop/hive/metastore/ ql/src/java/org/apache/hadoop/...

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/RequiredPrivileges.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/RequiredPrivileges.java?rev=1566401&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/RequiredPrivileges.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/RequiredPrivileges.java Sun Feb  9 20:43:37 2014
@@ -0,0 +1,112 @@
+/**
+ * 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.security.authorization.plugin.sqlstd;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzPluginException;
+
+/**
+ * Captures privilege sets, and can be used to compare required and available privileges
+ * to find missing privileges (if any).
+ */
+public class RequiredPrivileges {
+
+  private final Set<SQLPrivTypeGrant> privilegeGrantSet = new HashSet<SQLPrivTypeGrant>();
+
+  public void addPrivilege(String priv, boolean withGrant) throws HiveAuthzPluginException {
+    SQLPrivTypeGrant privType = SQLPrivTypeGrant.getSQLPrivTypeGrant(priv, withGrant);
+    addPrivilege(privType);
+    privilegeGrantSet.add(privType);
+    if(withGrant){
+      //as with grant also implies without grant privilege, add without privilege as well
+      addPrivilege(priv, false);
+    }
+  }
+
+  public Set<SQLPrivTypeGrant> getRequiredPrivilegeSet() {
+    return privilegeGrantSet;
+  }
+
+  /**
+   * Find the missing privileges in availPrivs
+   *
+   * @param availPrivs
+   *          - available privileges
+   * @return missing privileges as RequiredPrivileges object
+   */
+  public Collection<SQLPrivTypeGrant> findMissingPrivs(RequiredPrivileges availPrivs) {
+    MissingPrivilegeCapturer missingPrivCapturer = new MissingPrivilegeCapturer();
+    for (SQLPrivTypeGrant requiredPriv : privilegeGrantSet) {
+      if (!availPrivs.privilegeGrantSet.contains(requiredPriv)) {
+        missingPrivCapturer.addMissingPrivilege(requiredPriv);
+      }
+    }
+    return missingPrivCapturer.getMissingPrivileges();
+  }
+
+  void addPrivilege(SQLPrivTypeGrant requiredPriv) {
+    privilegeGrantSet.add(requiredPriv);
+  }
+
+  Set<SQLPrivTypeGrant> getPrivilegeWithGrants() {
+    return privilegeGrantSet;
+  }
+
+  /**
+   * Capture privileges that are missing. If privilege "X with grant" and "X without grant"
+   * are reported missing, capture only "X with grant". This is useful for better error messages.
+   */
+  class MissingPrivilegeCapturer {
+
+    private final Map<SQLPrivilegeType, SQLPrivTypeGrant> priv2privWithGrant = new HashMap<SQLPrivilegeType, SQLPrivTypeGrant>();
+
+    void addMissingPrivilege(SQLPrivTypeGrant newPrivWGrant) {
+      SQLPrivTypeGrant matchingPrivWGrant = priv2privWithGrant.get(newPrivWGrant.getPrivType());
+      if (matchingPrivWGrant != null) {
+        if (matchingPrivWGrant.isWithGrant() || !newPrivWGrant.isWithGrant()) {
+          // the existing entry already has grant, or new priv does not have
+          // grant
+          // no update needs to be done.
+          return;
+        }
+      }
+      // add the new entry
+      priv2privWithGrant.put(newPrivWGrant.getPrivType(), newPrivWGrant);
+    }
+
+    Collection<SQLPrivTypeGrant> getMissingPrivileges() {
+      return priv2privWithGrant.values();
+    }
+
+  }
+
+  public void addAll(SQLPrivTypeGrant[] inputPrivs) {
+    if (inputPrivs == null) {
+      return;
+    }
+    for (SQLPrivTypeGrant privType : inputPrivs) {
+      addPrivilege(privType);
+    }
+  }
+
+}

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/RevokePrivAuthUtils.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/RevokePrivAuthUtils.java?rev=1566401&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/RevokePrivAuthUtils.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/RevokePrivAuthUtils.java Sun Feb  9 20:43:37 2014
@@ -0,0 +1,84 @@
+package org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.PrincipalType;
+import org.apache.hadoop.hive.metastore.api.PrivilegeGrantInfo;
+import org.apache.hadoop.hive.ql.security.authorization.AuthorizationUtils;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAccessControlException;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzPluginException;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrincipal;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilege;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject;
+import org.apache.thrift.TException;
+
+public class RevokePrivAuthUtils {
+
+  public static List<HiveObjectPrivilege> authorizeAndGetRevokePrivileges(List<HivePrincipal> principals,
+      List<HivePrivilege> hivePrivileges, HivePrivilegeObject hivePrivObject, boolean grantOption,
+      IMetaStoreClient mClient, String userName)
+          throws HiveAuthzPluginException, HiveAccessControlException {
+
+    List<HiveObjectPrivilege> matchingPrivs = new ArrayList<HiveObjectPrivilege>();
+
+    StringBuilder errMsg = new StringBuilder();
+    for (HivePrincipal principal : principals) {
+
+      // get metastore/thrift privilege object for this principal and object, not looking at
+      // privileges obtained indirectly via roles
+      List<HiveObjectPrivilege> msObjPrivs;
+      try {
+        msObjPrivs = mClient.list_privileges(principal.getName(),
+            AuthorizationUtils.getThriftPrincipalType(principal.getType()),
+            SQLAuthorizationUtils.getThriftHiveObjectRef(hivePrivObject));
+      } catch (MetaException e) {
+        throw new HiveAuthzPluginException(e);
+      } catch (TException e) {
+        throw new HiveAuthzPluginException(e);
+      }
+
+      // the resulting privileges need to be filtered on privilege type and
+      // username
+
+      // create a Map to capture object privileges corresponding to privilege
+      // type
+      Map<String, HiveObjectPrivilege> priv2privObj = new HashMap<String, HiveObjectPrivilege>();
+
+      for (HiveObjectPrivilege msObjPriv : msObjPrivs) {
+        PrivilegeGrantInfo grantInfo = msObjPriv.getGrantInfo();
+        // check if the grantor matches current user
+        if (grantInfo.getGrantor() != null && grantInfo.getGrantor().equals(userName)
+            && grantInfo.getGrantorType() == PrincipalType.USER) {
+          // add to the map
+          priv2privObj.put(grantInfo.getPrivilege(), msObjPriv);
+        }
+        // else skip this one
+      }
+
+      // find the privileges that we are looking for
+      for (HivePrivilege hivePrivilege : hivePrivileges) {
+        HiveObjectPrivilege matchedPriv = priv2privObj.get(hivePrivilege.getName());
+        if (matchedPriv != null) {
+          matchingPrivs.add(matchedPriv);
+        } else {
+          errMsg.append("Cannot find privilege ").append(hivePrivilege).append(" for ")
+              .append(principal).append(" on ").append(hivePrivObject).append(" granted by ")
+              .append(userName).append(System.getProperty("line.separator"));
+        }
+      }
+
+    }
+
+    if (errMsg.length() != 0) {
+      throw new HiveAccessControlException(errMsg.toString());
+    }
+    return matchingPrivs;
+  }
+
+}

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLAuthorizationUtils.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLAuthorizationUtils.java?rev=1566401&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLAuthorizationUtils.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLAuthorizationUtils.java Sun Feb  9 20:43:37 2014
@@ -0,0 +1,285 @@
+/**
+ * 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.security.authorization.plugin.sqlstd;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege;
+import org.apache.hadoop.hive.metastore.api.HiveObjectRef;
+import org.apache.hadoop.hive.metastore.api.HiveObjectType;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet;
+import org.apache.hadoop.hive.metastore.api.PrivilegeBag;
+import org.apache.hadoop.hive.metastore.api.PrivilegeGrantInfo;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.security.authorization.AuthorizationUtils;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAccessControlException;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzPluginException;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrincipal;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilege;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType;
+import org.apache.thrift.TException;
+
+public class SQLAuthorizationUtils {
+
+  private static final String[] SUPPORTED_PRIVS = { "INSERT", "UPDATE", "DELETE", "SELECT" };
+  private static final Set<String> SUPPORTED_PRIVS_SET = new HashSet<String>(
+      Arrays.asList(SUPPORTED_PRIVS));
+
+  /**
+   * Create thrift privileges bag
+   *
+   * @param hivePrincipals
+   * @param hivePrivileges
+   * @param hivePrivObject
+   * @param grantorPrincipal
+   * @param grantOption
+   * @return
+   * @throws HiveAuthzPluginException
+   */
+  static PrivilegeBag getThriftPrivilegesBag(List<HivePrincipal> hivePrincipals,
+      List<HivePrivilege> hivePrivileges, HivePrivilegeObject hivePrivObject,
+      HivePrincipal grantorPrincipal, boolean grantOption) throws HiveAuthzPluginException {
+    HiveObjectRef privObj = getThriftHiveObjectRef(hivePrivObject);
+    PrivilegeBag privBag = new PrivilegeBag();
+    for (HivePrivilege privilege : hivePrivileges) {
+      if (privilege.getColumns() != null && privilege.getColumns().size() > 0) {
+        throw new HiveAuthzPluginException("Privileges on columns not supported currently"
+            + " in sql standard authorization mode");
+      }
+      if (!SUPPORTED_PRIVS_SET.contains(privilege.getName().toUpperCase(Locale.US))) {
+        throw new HiveAuthzPluginException("Privilege: " + privilege.getName()
+            + " is not supported in sql standard authorization mode");
+      }
+      PrivilegeGrantInfo grantInfo = getThriftPrivilegeGrantInfo(privilege, grantorPrincipal,
+          grantOption);
+      for (HivePrincipal principal : hivePrincipals) {
+        HiveObjectPrivilege objPriv = new HiveObjectPrivilege(privObj, principal.getName(),
+            AuthorizationUtils.getThriftPrincipalType(principal.getType()), grantInfo);
+        privBag.addToPrivileges(objPriv);
+      }
+    }
+    return privBag;
+  }
+
+  static PrivilegeGrantInfo getThriftPrivilegeGrantInfo(HivePrivilege privilege,
+      HivePrincipal grantorPrincipal, boolean grantOption) throws HiveAuthzPluginException {
+    try {
+      return AuthorizationUtils.getThriftPrivilegeGrantInfo(privilege, grantorPrincipal,
+          grantOption);
+    } catch (HiveException e) {
+      throw new HiveAuthzPluginException(e);
+    }
+  }
+
+  /**
+   * Create a thrift privilege object from the plugin interface privilege object
+   *
+   * @param privObj
+   * @return
+   * @throws HiveAuthzPluginException
+   */
+  static HiveObjectRef getThriftHiveObjectRef(HivePrivilegeObject privObj)
+      throws HiveAuthzPluginException {
+    try {
+      return AuthorizationUtils.getThriftHiveObjectRef(privObj);
+    } catch (HiveException e) {
+      throw new HiveAuthzPluginException(e);
+    }
+  }
+
+  static HivePrivilegeObjectType getPluginObjType(HiveObjectType objectType)
+      throws HiveAuthzPluginException {
+    switch (objectType) {
+    case DATABASE:
+      return HivePrivilegeObjectType.DATABASE;
+    case TABLE:
+      return HivePrivilegeObjectType.TABLE;
+    case COLUMN:
+    case GLOBAL:
+    case PARTITION:
+      throw new HiveAuthzPluginException("Unsupported object type " + objectType);
+    default:
+      throw new AssertionError("Unexpected object type " + objectType);
+    }
+  }
+
+  /**
+   * Check if the privileges are acceptable for SQL Standard authorization implementation
+   * @param hivePrivileges
+   * @throws HiveAuthzPluginException
+   */
+  public static void validatePrivileges(List<HivePrivilege> hivePrivileges) throws HiveAuthzPluginException {
+    for (HivePrivilege hivePrivilege : hivePrivileges) {
+      if (hivePrivilege.getColumns() != null && hivePrivilege.getColumns().size() != 0) {
+        throw new HiveAuthzPluginException(
+            "Privilege with columns are not currently supported with sql standard authorization:"
+                + hivePrivilege);
+      }
+      //try converting to the enum to verify that this is a valid privilege type
+      SQLPrivilegeType.getRequirePrivilege(hivePrivilege.getName());
+
+    }
+  }
+
+  /**
+   * Get the privileges this user(userName argument) has on the object
+   * (hivePrivObject argument)
+   *
+   * @param metastoreClient
+   * @param userName
+   * @param hivePrivObject
+   * @return
+   * @throws HiveAuthzPluginException
+   */
+  static RequiredPrivileges getPrivilegesFromMetaStore(IMetaStoreClient metastoreClient,
+      String userName, HivePrivilegeObject hivePrivObject) throws HiveAuthzPluginException {
+
+    // get privileges for this user and its role on this object
+    PrincipalPrivilegeSet thrifPrivs = null;
+    try {
+      thrifPrivs = metastoreClient.get_privilege_set(
+          AuthorizationUtils.getThriftHiveObjectRef(hivePrivObject), userName, null);
+    } catch (MetaException e) {
+      throwGetPrivErr(e, hivePrivObject, userName);
+    } catch (TException e) {
+      throwGetPrivErr(e, hivePrivObject, userName);
+    } catch (HiveException e) {
+      throwGetPrivErr(e, hivePrivObject, userName);
+    }
+
+    // convert to RequiredPrivileges
+    RequiredPrivileges privs = getRequiredPrivsFromThrift(thrifPrivs);
+
+    // add owner privilege if user is owner of the object
+    if (isOwner(metastoreClient, userName, hivePrivObject)) {
+      privs.addPrivilege(SQLPrivTypeGrant.OWNER_PRIV);
+    }
+
+    return privs;
+  }
+
+  /**
+   * Check if user is owner of the given object
+   *
+   * @param metastoreClient
+   * @param userName
+   *          user
+   * @param hivePrivObject
+   *          given object
+   * @return true if user is owner
+   * @throws HiveAuthzPluginException
+   */
+  private static boolean isOwner(IMetaStoreClient metastoreClient, String userName,
+      HivePrivilegeObject hivePrivObject) throws HiveAuthzPluginException {
+    //for now, check only table
+    if(hivePrivObject.getType() == HivePrivilegeObjectType.TABLE){
+      Table thriftTableObj = null;
+      try {
+        thriftTableObj = metastoreClient.getTable(hivePrivObject.getDbname(), hivePrivObject.getTableviewname());
+      } catch (MetaException e) {
+        throwGetTableErr(e, hivePrivObject);
+      } catch (NoSuchObjectException e) {
+        throwGetTableErr(e, hivePrivObject);
+      } catch (TException e) {
+        throwGetTableErr(e, hivePrivObject);
+      }
+      return userName.equals(thriftTableObj.getOwner());
+    }
+    return false;
+  }
+
+  private static void throwGetTableErr(Exception e, HivePrivilegeObject hivePrivObject)
+      throws HiveAuthzPluginException {
+    String msg = "Error getting table object from metastore for" + hivePrivObject;
+    throw new HiveAuthzPluginException(msg, e);
+  }
+
+  private static void throwGetPrivErr(Exception e, HivePrivilegeObject hivePrivObject,
+      String userName) throws HiveAuthzPluginException {
+    String msg = "Error getting privileges on " + hivePrivObject + " for " + userName;
+    throw new HiveAuthzPluginException(msg, e);
+  }
+
+  private static RequiredPrivileges getRequiredPrivsFromThrift(PrincipalPrivilegeSet thrifPrivs)
+      throws HiveAuthzPluginException {
+
+    RequiredPrivileges reqPrivs = new RequiredPrivileges();
+    // add user privileges
+    Map<String, List<PrivilegeGrantInfo>> userPrivs = thrifPrivs.getUserPrivileges();
+    if (userPrivs != null && userPrivs.size() != 1) {
+      throw new HiveAuthzPluginException("Invalid number of user privilege objects: "
+          + userPrivs.size());
+    }
+    addRequiredPrivs(reqPrivs, userPrivs);
+
+    // add role privileges
+    Map<String, List<PrivilegeGrantInfo>> rolePrivs = thrifPrivs.getRolePrivileges();
+    addRequiredPrivs(reqPrivs, rolePrivs);
+    return reqPrivs;
+  }
+
+  /**
+   * Add privileges to RequiredPrivileges object reqPrivs from thrift availPrivs
+   * object
+   * @param reqPrivs
+   * @param availPrivs
+   * @throws HiveAuthzPluginException
+   */
+  private static void addRequiredPrivs(RequiredPrivileges reqPrivs,
+      Map<String, List<PrivilegeGrantInfo>> availPrivs) throws HiveAuthzPluginException {
+    if(availPrivs == null){
+      return;
+    }
+    for (Map.Entry<String, List<PrivilegeGrantInfo>> userPriv : availPrivs.entrySet()) {
+      List<PrivilegeGrantInfo> userPrivGInfos = userPriv.getValue();
+      for (PrivilegeGrantInfo userPrivGInfo : userPrivGInfos) {
+        reqPrivs.addPrivilege(userPrivGInfo.getPrivilege(), userPrivGInfo.isGrantOption());
+      }
+    }
+  }
+
+  public static void assertNoMissingPrivilege(Collection<SQLPrivTypeGrant> missingPrivs,
+      HivePrincipal hivePrincipal, HivePrivilegeObject hivePrivObject)
+      throws HiveAccessControlException {
+    if (missingPrivs.size() != 0) {
+      // there are some required privileges missing, create error message
+      // sort the privileges so that error message is deterministic (for tests)
+      List<SQLPrivTypeGrant> sortedmissingPrivs = new ArrayList<SQLPrivTypeGrant>(missingPrivs);
+      Collections.sort(sortedmissingPrivs);
+
+      String errMsg = "Permission denied. " + hivePrincipal
+          + " does not have following privileges on " + hivePrivObject + " : " + sortedmissingPrivs;
+      throw new HiveAccessControlException(errMsg.toString());
+    }
+  }
+
+
+}

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLPrivTypeGrant.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLPrivTypeGrant.java?rev=1566401&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLPrivTypeGrant.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLPrivTypeGrant.java Sun Feb  9 20:43:37 2014
@@ -0,0 +1,99 @@
+/**
+ * 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.security.authorization.plugin.sqlstd;
+
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzPluginException;
+
+
+public enum SQLPrivTypeGrant {
+  SELECT_NOGRANT(SQLPrivilegeType.SELECT, false),
+  SELECT_WGRANT(SQLPrivilegeType.SELECT, true),
+  INSERT_NOGRANT(SQLPrivilegeType.INSERT, false),
+  INSERT_WGRANT(SQLPrivilegeType.INSERT, true),
+  UPDATE_NOGRANT(SQLPrivilegeType.UPDATE, false),
+  UPDATE_WGRANT(SQLPrivilegeType.UPDATE, true),
+  DELETE_NOGRANT(SQLPrivilegeType.DELETE, false),
+  DELETE_WGRANT(SQLPrivilegeType.DELETE, true),
+  OWNER_PRIV("Object ownership"),
+  ADMIN_PRIV("Admin privilege"); // This one can be used to deny permission for performing the operation
+
+  private final SQLPrivilegeType privType;
+  private final boolean withGrant;
+
+  private final String privDesc;
+  SQLPrivTypeGrant(SQLPrivilegeType privType, boolean isGrant){
+    this.privType = privType;
+    this.withGrant = isGrant;
+    this.privDesc = privType.toString() + (withGrant ? " with grant" : "");
+  }
+
+  /**
+   * Constructor for privileges that are not the standard sql types, but are used by
+   * authorization rules
+   * @param privDesc
+   */
+  SQLPrivTypeGrant(String privDesc){
+    this.privDesc = privDesc;
+    this.privType = null;
+    this.withGrant = false;
+  }
+
+  /**
+   * Find matching enum
+   * @param privType
+   * @param isGrant
+   * @return
+   */
+  public static SQLPrivTypeGrant getSQLPrivTypeGrant(
+      SQLPrivilegeType privType, boolean isGrant) {
+    String typeName = privType.name() + (isGrant ? "_WGRANT" : "_NOGRANT");
+    return SQLPrivTypeGrant.valueOf(typeName);
+  }
+
+  /**
+   * Find matching enum
+   *
+   * @param privTypeStr
+   *          privilege type string
+   * @param isGrant
+   * @return
+   * @throws HiveAuthzPluginException
+   */
+  public static SQLPrivTypeGrant getSQLPrivTypeGrant(String privTypeStr, boolean isGrant)
+      throws HiveAuthzPluginException {
+    SQLPrivilegeType ptype = SQLPrivilegeType.getRequirePrivilege(privTypeStr);
+    return getSQLPrivTypeGrant(ptype, isGrant);
+  }
+
+  public SQLPrivilegeType getPrivType() {
+    return privType;
+  }
+
+  public boolean isWithGrant() {
+    return withGrant;
+  }
+
+  /**
+   * @return String representation for use in error messages
+   */
+  @Override
+  public String toString(){
+    return privDesc;
+  }
+
+};

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLPrivilegeType.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLPrivilegeType.java?rev=1566401&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLPrivilegeType.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLPrivilegeType.java Sun Feb  9 20:43:37 2014
@@ -0,0 +1,43 @@
+/**
+ * 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.security.authorization.plugin.sqlstd;
+
+import java.util.Locale;
+
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzPluginException;
+
+public enum SQLPrivilegeType {
+  //ALL privilege is expanded to these, so it is not needed here
+  SELECT, INSERT, UPDATE, DELETE;
+
+  public static SQLPrivilegeType getRequirePrivilege(String priv)
+      throws HiveAuthzPluginException {
+    SQLPrivilegeType reqPriv;
+    if(priv == null){
+      throw new HiveAuthzPluginException("Null privilege obtained");
+    }
+    try {
+      reqPriv = SQLPrivilegeType.valueOf(priv.toUpperCase(Locale.US));
+    } catch (IllegalArgumentException e) {
+      throw new HiveAuthzPluginException("Unsupported privilege type " + priv, e);
+    }
+    return reqPriv;
+  }
+
+
+};

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessController.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessController.java?rev=1566401&r1=1566400&r2=1566401&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessController.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessController.java Sun Feb  9 20:43:37 2014
@@ -18,15 +18,14 @@
 package org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd;
 
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.HashSet;
 import java.util.List;
-import java.util.Locale;
 import java.util.Set;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.HiveMetaStore;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege;
 import org.apache.hadoop.hive.metastore.api.HiveObjectRef;
 import org.apache.hadoop.hive.metastore.api.HiveObjectType;
@@ -36,9 +35,11 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.api.PrivilegeGrantInfo;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider;
 import org.apache.hadoop.hive.ql.security.authorization.AuthorizationUtils;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAccessControlException;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAccessController;
-import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthorizationPluginException;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzPluginException;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveMetastoreClientFactory;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrincipal;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilege;
@@ -48,31 +49,45 @@ import org.apache.hadoop.hive.ql.securit
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveRole;
 
 /**
- * Implements functionality of access control statements for sql standard based authorization
+ * Implements functionality of access control statements for sql standard based
+ * authorization
  */
 @Private
 public class SQLStdHiveAccessController implements HiveAccessController {
 
   private final HiveMetastoreClientFactory metastoreClientFactory;
-  private static final String [] SUPPORTED_PRIVS = {"INSERT", "UPDATE", "DELETE", "SELECT", "ALL"};
-  private static final Set<String> SUPPORTED_PRIVS_SET
-    = new HashSet<String>(Arrays.asList(SUPPORTED_PRIVS));
-  private final List<HiveRole> currentRoles;
-  private final String currentUserName;
+  private final HiveConf conf;
+  private final HiveAuthenticationProvider authenticator;
+  private String currentUserName;
+  private List<HiveRole> currentRoles;
   private HiveRole adminRole;
 
-  SQLStdHiveAccessController(HiveMetastoreClientFactory metastoreClientFactory,
-      HiveConf conf, String hiveCurrentUser) throws HiveAuthorizationPluginException {
-    this.currentUserName = hiveCurrentUser;
-    try {
-      this.metastoreClientFactory = metastoreClientFactory;
-      this.currentRoles = getRolesFromMS();
-    } catch (HiveAuthorizationPluginException e) {
-      throw e;
+  SQLStdHiveAccessController(HiveMetastoreClientFactory metastoreClientFactory, HiveConf conf,
+      HiveAuthenticationProvider authenticator) throws HiveAuthzPluginException {
+    this.metastoreClientFactory = metastoreClientFactory;
+    this.conf = conf;
+    this.authenticator = authenticator;
+    initUserRoles();
+  }
+
+  /**
+   * (Re-)initialize currentRoleNames if necessary.
+   * @throws HiveAuthzPluginException
+   */
+  private void initUserRoles() throws HiveAuthzPluginException {
+    //to aid in testing through .q files, authenticator is passed as argument to
+    // the interface. this helps in being able to switch the user within a session.
+    // so we need to check if the user has changed
+    String newUserName = authenticator.getUserName();
+    if(currentUserName == newUserName){
+      //no need to (re-)initialize the currentUserName, currentRoles fields
+      return;
     }
+    this.currentUserName = newUserName;
+    this.currentRoles = getRolesFromMS();
   }
 
-  private List<HiveRole> getRolesFromMS() throws HiveAuthorizationPluginException {
+  private List<HiveRole> getRolesFromMS() throws HiveAuthzPluginException {
     List<Role> roles;
     try {
       roles = metastoreClientFactory.getHiveMetastoreClient().
@@ -87,52 +102,78 @@ public class SQLStdHiveAccessController 
       }
       return currentRoles;
     } catch (Exception e) {
-        throw new HiveAuthorizationPluginException("Failed to retrieve roles for "+
-        currentUserName, e);
+        throw new HiveAuthzPluginException("Failed to retrieve roles for "+
+            currentUserName, e);
     }
   }
 
   @Override
   public void grantPrivileges(List<HivePrincipal> hivePrincipals,
-    List<HivePrivilege> hivePrivileges, HivePrivilegeObject hivePrivObject,
-    HivePrincipal grantorPrincipal, boolean grantOption) throws HiveAuthorizationPluginException {
+      List<HivePrivilege> hivePrivileges, HivePrivilegeObject hivePrivObject,
+      HivePrincipal grantorPrincipal, boolean grantOption)
+          throws HiveAuthzPluginException, HiveAccessControlException {
+
+    // expand ALL privileges, if any
+    hivePrivileges = expandAllPrivileges(hivePrivileges);
+
+    SQLAuthorizationUtils.validatePrivileges(hivePrivileges);
 
-    PrivilegeBag privBag =
-        getThriftPrivilegesBag(hivePrincipals, hivePrivileges, hivePrivObject, grantorPrincipal,
-            grantOption);
+    IMetaStoreClient metastoreClient = metastoreClientFactory.getHiveMetastoreClient();
+    // authorize the grant
+    GrantPrivAuthUtils.authorize(hivePrincipals, hivePrivileges, hivePrivObject, grantOption,
+        metastoreClient, authenticator.getUserName());
+
+    // grant
+    PrivilegeBag privBag = getThriftPrivilegesBag(hivePrincipals, hivePrivileges, hivePrivObject,
+        grantorPrincipal, grantOption);
     try {
-      metastoreClientFactory.getHiveMetastoreClient().grant_privileges(privBag);
+      metastoreClient.grant_privileges(privBag);
     } catch (Exception e) {
-      throw new HiveAuthorizationPluginException("Error granting privileges", e);
+      throw new HiveAuthzPluginException("Error granting privileges", e);
     }
   }
 
+  private List<HivePrivilege> expandAllPrivileges(List<HivePrivilege> hivePrivileges) {
+    Set<HivePrivilege> hivePrivSet = new HashSet<HivePrivilege>();
+    for (HivePrivilege hivePrivilege : hivePrivileges) {
+      if (hivePrivilege.getName().equals("ALL")) {
+        // expand to all supported privileges
+        for (SQLPrivilegeType privType : SQLPrivilegeType.values()) {
+          hivePrivSet.add(new HivePrivilege(privType.name(), hivePrivilege.getColumns()));
+        }
+      } else {
+        hivePrivSet.add(hivePrivilege);
+      }
+    }
+    return new ArrayList<HivePrivilege>(hivePrivSet);
+  }
+
   /**
    * Create thrift privileges bag
+   *
    * @param hivePrincipals
    * @param hivePrivileges
    * @param hivePrivObject
    * @param grantorPrincipal
    * @param grantOption
    * @return
-   * @throws HiveAuthorizationPluginException
+   * @throws HiveAuthzPluginException
    */
   private PrivilegeBag getThriftPrivilegesBag(List<HivePrincipal> hivePrincipals,
-    List<HivePrivilege> hivePrivileges, HivePrivilegeObject hivePrivObject,
-    HivePrincipal grantorPrincipal, boolean grantOption) throws HiveAuthorizationPluginException {
-    HiveObjectRef privObj = getThriftHiveObjectRef(hivePrivObject);
+      List<HivePrivilege> hivePrivileges, HivePrivilegeObject hivePrivObject,
+      HivePrincipal grantorPrincipal, boolean grantOption) throws HiveAuthzPluginException {
+
+    HiveObjectRef privObj = SQLAuthorizationUtils.getThriftHiveObjectRef(hivePrivObject);
     PrivilegeBag privBag = new PrivilegeBag();
-    for(HivePrivilege privilege : hivePrivileges){
-      if(privilege.getColumns() != null && privilege.getColumns().size() > 0){
-        throw new HiveAuthorizationPluginException("Privileges on columns not supported currently"
+    for (HivePrivilege privilege : hivePrivileges) {
+      if (privilege.getColumns() != null && privilege.getColumns().size() > 0) {
+        throw new HiveAuthzPluginException("Privileges on columns not supported currently"
             + " in sql standard authorization mode");
       }
-      if(!SUPPORTED_PRIVS_SET.contains(privilege.getName().toUpperCase(Locale.US))){
-        throw new HiveAuthorizationPluginException("Privilege: " + privilege.getName() +
-            " is not supported in sql standard authorization mode");
-      }
-      PrivilegeGrantInfo grantInfo = getThriftPrivilegeGrantInfo(privilege, grantorPrincipal, grantOption);
-      for(HivePrincipal principal : hivePrincipals){
+
+      PrivilegeGrantInfo grantInfo = getThriftPrivilegeGrantInfo(privilege, grantorPrincipal,
+          grantOption);
+      for (HivePrincipal principal : hivePrincipals) {
         HiveObjectPrivilege objPriv = new HiveObjectPrivilege(privObj, principal.getName(),
             AuthorizationUtils.getThriftPrincipalType(principal.getType()), grantInfo);
         privBag.addToPrivileges(objPriv);
@@ -142,99 +183,95 @@ public class SQLStdHiveAccessController 
   }
 
   private PrivilegeGrantInfo getThriftPrivilegeGrantInfo(HivePrivilege privilege,
-      HivePrincipal grantorPrincipal, boolean grantOption) throws HiveAuthorizationPluginException {
-    try {
-      return  AuthorizationUtils.getThriftPrivilegeGrantInfo(privilege, grantorPrincipal, grantOption);
-    } catch (HiveException e) {
-      throw new HiveAuthorizationPluginException(e);
-    }
-  }
-
-  /**
-   * Create a thrift privilege object from the plugin interface privilege object
-   * @param privObj
-   * @return
-   * @throws HiveAuthorizationPluginException
-   */
-  private HiveObjectRef getThriftHiveObjectRef(HivePrivilegeObject privObj)
-      throws HiveAuthorizationPluginException {
+      HivePrincipal grantorPrincipal, boolean grantOption) throws HiveAuthzPluginException {
     try {
-      return  AuthorizationUtils.getThriftHiveObjectRef(privObj);
+      return AuthorizationUtils.getThriftPrivilegeGrantInfo(privilege, grantorPrincipal,
+          grantOption);
     } catch (HiveException e) {
-      throw new HiveAuthorizationPluginException(e);
+      throw new HiveAuthzPluginException(e);
     }
   }
 
   @Override
   public void revokePrivileges(List<HivePrincipal> hivePrincipals,
       List<HivePrivilege> hivePrivileges, HivePrivilegeObject hivePrivObject,
-      HivePrincipal grantorPrincipal, boolean grantOption) throws HiveAuthorizationPluginException {
-
-    PrivilegeBag privBag =
-        getThriftPrivilegesBag(hivePrincipals, hivePrivileges, hivePrivObject, grantorPrincipal,
-            grantOption);
-    try {
-      metastoreClientFactory.getHiveMetastoreClient().revoke_privileges(privBag);
+      HivePrincipal grantorPrincipal, boolean grantOption)
+          throws HiveAuthzPluginException, HiveAccessControlException {
+    SQLAuthorizationUtils.validatePrivileges(hivePrivileges);
+
+    IMetaStoreClient metastoreClient = metastoreClientFactory.getHiveMetastoreClient();
+    // authorize the revoke, and get the set of privileges to be revoked
+    List<HiveObjectPrivilege> revokePrivs = RevokePrivAuthUtils
+        .authorizeAndGetRevokePrivileges(hivePrincipals, hivePrivileges, hivePrivObject,
+            grantOption, metastoreClient, authenticator.getUserName());
+
+    try {
+      // unfortunately, the metastore api revokes all privileges that match on
+      // principal, privilege object type it does not filter on the grator
+      // username.
+      // So this will revoke privileges that are granted by other users.This is
+      // not SQL compliant behavior. Need to change/add a metastore api
+      // that has desired behavior.
+      metastoreClient.revoke_privileges(new PrivilegeBag(revokePrivs));
     } catch (Exception e) {
-      throw new HiveAuthorizationPluginException("Error revoking privileges", e);
+      throw new HiveAuthzPluginException("Error revoking privileges", e);
     }
   }
 
   @Override
   public void createRole(String roleName, HivePrincipal adminGrantor)
-      throws HiveAuthorizationPluginException {
+      throws HiveAuthzPluginException {
     try {
       String grantorName = adminGrantor == null ? null : adminGrantor.getName();
       metastoreClientFactory.getHiveMetastoreClient().create_role(
         new Role(roleName, 0, grantorName));
     } catch (Exception e) {
-      throw new HiveAuthorizationPluginException("Error create role", e);
+      throw new HiveAuthzPluginException("Error create role", e);
     }
   }
 
   @Override
-  public void dropRole(String roleName) throws HiveAuthorizationPluginException {
+  public void dropRole(String roleName) throws HiveAuthzPluginException {
     try {
       metastoreClientFactory.getHiveMetastoreClient().drop_role(roleName);
     } catch (Exception e) {
-      throw new HiveAuthorizationPluginException("Error dropping role", e);
+      throw new HiveAuthzPluginException("Error dropping role", e);
     }
   }
 
   @Override
-  public List<HiveRole> getRoles(HivePrincipal hivePrincipal) throws HiveAuthorizationPluginException {
+  public List<HiveRole> getRoles(HivePrincipal hivePrincipal) throws HiveAuthzPluginException {
     try {
       List<Role> roles = metastoreClientFactory.getHiveMetastoreClient().list_roles(
-        hivePrincipal.getName(), AuthorizationUtils.getThriftPrincipalType(hivePrincipal.getType()));
-      List<HiveRole> roleNames = new ArrayList<HiveRole>(roles.size());
+          hivePrincipal.getName(), AuthorizationUtils.getThriftPrincipalType(hivePrincipal.getType()));
+      List<HiveRole> hiveRoles = new ArrayList<HiveRole>(roles.size());
       for (Role role : roles){
-        ;
-        roleNames.add(new HiveRole(role));
+        hiveRoles.add(new HiveRole(role));
       }
-      return roleNames;
+      return hiveRoles;
     } catch (Exception e) {
-      throw new HiveAuthorizationPluginException(
-          "Error listing roles for user" + hivePrincipal.getName(), e);
+      throw new HiveAuthzPluginException("Error listing roles for user "
+          + hivePrincipal.getName(), e);
     }
   }
 
   @Override
   public void grantRole(List<HivePrincipal> hivePrincipals, List<String> roleNames,
-      boolean grantOption, HivePrincipal grantorPrinc) throws HiveAuthorizationPluginException {
-    for(HivePrincipal hivePrincipal : hivePrincipals){
-      for(String roleName : roleNames){
+      boolean grantOption, HivePrincipal grantorPrinc) throws HiveAuthzPluginException {
+    for (HivePrincipal hivePrincipal : hivePrincipals) {
+      for (String roleName : roleNames) {
         try {
-          metastoreClientFactory.getHiveMetastoreClient().grant_role(
-            roleName, hivePrincipal.getName(),
-            AuthorizationUtils.getThriftPrincipalType(hivePrincipal.getType()),
-            grantorPrinc.getName(),
-            AuthorizationUtils.getThriftPrincipalType(grantorPrinc.getType()),grantOption);
+          IMetaStoreClient mClient = metastoreClientFactory.getHiveMetastoreClient();
+          mClient.grant_role(roleName, hivePrincipal.getName(),
+              AuthorizationUtils.getThriftPrincipalType(hivePrincipal.getType()),
+              grantorPrinc.getName(),
+              AuthorizationUtils.getThriftPrincipalType(grantorPrinc.getType()), grantOption);
         } catch (MetaException e) {
-          throw new HiveAuthorizationPluginException(e.getMessage(), e);
+          throw new HiveAuthzPluginException(e.getMessage(), e);
         } catch (Exception e) {
-          String msg = "Error granting roles for " + hivePrincipal.getName() +  " to role "
+          String msg = "Error granting roles for " + hivePrincipal.getName() + " to role "
               + roleName + ": " + e.getMessage();
-          throw new HiveAuthorizationPluginException(msg, e);
+          throw new HiveAuthzPluginException(msg, e);
         }
       }
     }
@@ -242,73 +279,67 @@ public class SQLStdHiveAccessController 
 
   @Override
   public void revokeRole(List<HivePrincipal> hivePrincipals, List<String> roleNames,
-      boolean grantOption, HivePrincipal grantorPrinc) throws HiveAuthorizationPluginException {
-    if(grantOption){
-      //removing grant privileges only is not supported in metastore api
-      throw new HiveAuthorizationPluginException("Revoking only the admin privileges on "
+      boolean grantOption, HivePrincipal grantorPrinc) throws HiveAuthzPluginException {
+    if (grantOption) {
+      // removing grant privileges only is not supported in metastore api
+      throw new HiveAuthzPluginException("Revoking only the admin privileges on "
           + "role is not currently supported");
     }
-    for(HivePrincipal hivePrincipal : hivePrincipals){
-      for(String roleName : roleNames){
+    for (HivePrincipal hivePrincipal : hivePrincipals) {
+      for (String roleName : roleNames) {
         try {
-          metastoreClientFactory.getHiveMetastoreClient().revoke_role(roleName,
-            hivePrincipal.getName(), AuthorizationUtils.getThriftPrincipalType(hivePrincipal.getType()));
-        }  catch (Exception e) {
-          String msg = "Error revoking roles for " + hivePrincipal.getName() +  " to role " + roleName
-              + hivePrincipal.getName();
-          throw new HiveAuthorizationPluginException(msg, e);
+          IMetaStoreClient mClient = metastoreClientFactory.getHiveMetastoreClient();
+          mClient.revoke_role(roleName, hivePrincipal.getName(),
+              AuthorizationUtils.getThriftPrincipalType(hivePrincipal.getType()));
+        } catch (Exception e) {
+          String msg = "Error revoking roles for " + hivePrincipal.getName() + " to role "
+              + roleName;
+          throw new HiveAuthzPluginException(msg, e);
         }
       }
     }
   }
 
   @Override
-  public List<String> getAllRoles() throws HiveAuthorizationPluginException {
+  public List<String> getAllRoles() throws HiveAuthzPluginException {
     try {
       return metastoreClientFactory.getHiveMetastoreClient().listRoleNames();
     } catch (Exception e) {
-      throw new HiveAuthorizationPluginException("Error listing all roles", e);
+      throw new HiveAuthzPluginException("Error listing all roles", e);
     }
   }
 
-
   @Override
   public List<HivePrivilegeInfo> showPrivileges(HivePrincipal principal, HivePrivilegeObject privObj)
-      throws HiveAuthorizationPluginException {
+      throws HiveAuthzPluginException {
     try {
-
+      IMetaStoreClient mClient = metastoreClientFactory.getHiveMetastoreClient();
       List<HivePrivilegeInfo> resPrivInfos = new ArrayList<HivePrivilegeInfo>();
+      // get metastore/thrift privilege object using metastore api
+      List<HiveObjectPrivilege> msObjPrivs = mClient.list_privileges(principal.getName(),
+          AuthorizationUtils.getThriftPrincipalType(principal.getType()),
+          SQLAuthorizationUtils.getThriftHiveObjectRef(privObj));
 
-      //get metastore/thrift privilege object using metastore api
-      List<HiveObjectPrivilege> msObjPrivs
-        = metastoreClientFactory.getHiveMetastoreClient().list_privileges(principal.getName(),
-            AuthorizationUtils.getThriftPrincipalType(principal.getType()),
-            getThriftHiveObjectRef(privObj));
-
-      //convert the metastore thrift objects to result objects
-      for(HiveObjectPrivilege msObjPriv : msObjPrivs){
-        //result principal
-        HivePrincipal resPrincipal =
-            new HivePrincipal(msObjPriv.getPrincipalName(),
-                AuthorizationUtils.getHivePrincipalType(msObjPriv.getPrincipalType()));
 
-        //result privilege
+      // convert the metastore thrift objects to result objects
+      for (HiveObjectPrivilege msObjPriv : msObjPrivs) {
+        // result principal
+        HivePrincipal resPrincipal = new HivePrincipal(msObjPriv.getPrincipalName(),
+            AuthorizationUtils.getHivePrincipalType(msObjPriv.getPrincipalType()));
+
+        // result privilege
         PrivilegeGrantInfo msGrantInfo = msObjPriv.getGrantInfo();
         HivePrivilege resPrivilege = new HivePrivilege(msGrantInfo.getPrivilege(), null);
 
-        //result object
+        // result object
         HiveObjectRef msObjRef = msObjPriv.getHiveObject();
         HivePrivilegeObject resPrivObj = new HivePrivilegeObject(
-            getPluginObjType(msObjRef.getObjectType()),
-            msObjRef.getDbName(),
-            msObjRef.getObjectName()
-            );
-
-        //result grantor principal
-        HivePrincipal grantorPrincipal =
-            new HivePrincipal(msGrantInfo.getGrantor(),
-                AuthorizationUtils.getHivePrincipalType(msGrantInfo.getGrantorType()));
+            getPluginObjType(msObjRef.getObjectType()), msObjRef.getDbName(),
+            msObjRef.getObjectName());
 
+        // result grantor principal
+        HivePrincipal grantorPrincipal = new HivePrincipal(msGrantInfo.getGrantor(),
+            AuthorizationUtils.getHivePrincipalType(msGrantInfo.getGrantorType()));
 
         HivePrivilegeInfo resPrivInfo = new HivePrivilegeInfo(resPrincipal, resPrivilege,
             resPrivObj, grantorPrincipal, msGrantInfo.isGrantOption());
@@ -316,16 +347,15 @@ public class SQLStdHiveAccessController 
       }
       return resPrivInfos;
 
-    }
-    catch (Exception e) {
-      throw new HiveAuthorizationPluginException("Error showing privileges", e);
+    } catch (Exception e) {
+      throw new HiveAuthzPluginException("Error showing privileges", e);
     }
 
   }
 
   private HivePrivilegeObjectType getPluginObjType(HiveObjectType objectType)
-      throws HiveAuthorizationPluginException {
-    switch(objectType){
+      throws HiveAuthzPluginException {
+    switch (objectType) {
     case DATABASE:
       return HivePrivilegeObjectType.DATABASE;
     case TABLE:
@@ -333,14 +363,15 @@ public class SQLStdHiveAccessController 
     case COLUMN:
     case GLOBAL:
     case PARTITION:
-      throw new HiveAuthorizationPluginException("Unsupported object type " + objectType);
+      throw new HiveAuthzPluginException("Unsupported object type " + objectType);
     default:
       throw new AssertionError("Unexpected object type " + objectType);
     }
   }
 
   @Override
-  public void setCurrentRole(String roleName) throws HiveAuthorizationPluginException {
+  public void setCurrentRole(String roleName) throws HiveAuthzPluginException {
+
     if ("NONE".equalsIgnoreCase(roleName)) {
       // for set role NONE, reset roles to default roles.
       currentRoles.clear();
@@ -362,12 +393,13 @@ public class SQLStdHiveAccessController 
       return;
     }
     // If we are here it means, user is requesting a role he doesn't belong to.
-    throw new HiveAuthorizationPluginException(currentUserName +" doesn't belong to role "
+    throw new HiveAuthzPluginException(currentUserName +" doesn't belong to role "
       +roleName);
   }
 
   @Override
-  public List<HiveRole> getCurrentRoles() throws HiveAuthorizationPluginException {
+  public List<HiveRole> getCurrentRoles() throws HiveAuthzPluginException {
+    initUserRoles();
     return currentRoles;
   }
 }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizationValidator.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizationValidator.java?rev=1566401&r1=1566400&r2=1566401&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizationValidator.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizationValidator.java Sun Feb  9 20:43:37 2014
@@ -17,18 +17,66 @@
  */
 package org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd;
 
+import java.util.Collection;
 import java.util.List;
 
-import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthorizationPluginException;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthorizationValidator;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAccessControlException;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzPluginException;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveMetastoreClientFactory;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrincipal;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrincipal.HivePrincipalType;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject;
 
 public class SQLStdHiveAuthorizationValidator implements HiveAuthorizationValidator {
 
+  private final HiveMetastoreClientFactory metastoreClientFactory;
+  private final HiveConf conf;
+  private final HiveAuthenticationProvider authenticator;
+
+  public SQLStdHiveAuthorizationValidator(HiveMetastoreClientFactory metastoreClientFactory,
+      HiveConf conf, HiveAuthenticationProvider authenticator) {
+    this.metastoreClientFactory = metastoreClientFactory;
+    this.conf = conf;
+    this.authenticator = authenticator;
+  }
+
   @Override
   public void checkPrivileges(HiveOperationType hiveOpType, List<HivePrivilegeObject> inputHObjs,
-      List<HivePrivilegeObject> outputHObjs) throws HiveAuthorizationPluginException {
+      List<HivePrivilegeObject> outputHObjs) throws HiveAuthzPluginException, HiveAccessControlException {
+    String userName = authenticator.getUserName();
+    IMetaStoreClient metastoreClient = metastoreClientFactory.getHiveMetastoreClient();
+
+    // get privileges required on input and check
+    SQLPrivTypeGrant[] inputPrivs = Operation2Privilege.getInputPrivs(hiveOpType);
+    checkPrivileges(inputPrivs, inputHObjs, metastoreClient, userName);
+
+    // get privileges required on input and check
+    SQLPrivTypeGrant[] outputPrivs = Operation2Privilege.getOutputPrivs(hiveOpType);
+    checkPrivileges(outputPrivs, outputHObjs, metastoreClient, userName);
+
+  }
+
+  private void checkPrivileges(SQLPrivTypeGrant[] reqPrivs,
+      List<HivePrivilegeObject> hObjs, IMetaStoreClient metastoreClient, String userName)
+          throws HiveAuthzPluginException, HiveAccessControlException {
+    RequiredPrivileges requiredInpPrivs = new RequiredPrivileges();
+    requiredInpPrivs.addAll(reqPrivs);
+
+    // check if this user has these privileges on the objects
+    for (HivePrivilegeObject hObj : hObjs) {
+      // get the privileges that this user has on the object
+      RequiredPrivileges availPrivs = SQLAuthorizationUtils.getPrivilegesFromMetaStore(
+          metastoreClient, userName, hObj);
+      Collection<SQLPrivTypeGrant> missingPriv = requiredInpPrivs
+          .findMissingPrivs(availPrivs);
+      SQLAuthorizationUtils.assertNoMissingPrivilege(missingPriv, new HivePrincipal(userName,
+          HivePrincipalType.USER), hObj);
+    }
   }
 
 }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizerFactory.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizerFactory.java?rev=1566401&r1=1566400&r2=1566401&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizerFactory.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizerFactory.java Sun Feb  9 20:43:37 2014
@@ -19,20 +19,22 @@ package org.apache.hadoop.hive.ql.securi
 
 import org.apache.hadoop.hive.common.classification.InterfaceAudience.Private;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthorizationPluginException;
+import org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthorizer;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthorizerFactory;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthorizerImpl;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzPluginException;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveMetastoreClientFactory;
 
 @Private
 public class SQLStdHiveAuthorizerFactory implements HiveAuthorizerFactory{
   @Override
   public HiveAuthorizer createHiveAuthorizer(HiveMetastoreClientFactory metastoreClientFactory,
-      HiveConf conf, String hiveCurrentUser) throws HiveAuthorizationPluginException {
+      HiveConf conf, HiveAuthenticationProvider authenticator) throws HiveAuthzPluginException {
+
     return new HiveAuthorizerImpl(
-        new SQLStdHiveAccessController(metastoreClientFactory, conf, hiveCurrentUser),
-        new SQLStdHiveAuthorizationValidator()
+        new SQLStdHiveAccessController(metastoreClientFactory, conf, authenticator),
+        new SQLStdHiveAuthorizationValidator(metastoreClientFactory, conf, authenticator)
         );
   }
 }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java?rev=1566401&r1=1566400&r2=1566401&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java Sun Feb  9 20:43:37 2014
@@ -54,6 +54,7 @@ import org.apache.hadoop.hive.ql.metadat
 import org.apache.hadoop.hive.ql.metadata.HiveUtils;
 import org.apache.hadoop.hive.ql.plan.HiveOperation;
 import org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider;
+import org.apache.hadoop.hive.ql.security.SessionStateUserAuthenticator;
 import org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthorizer;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthorizerFactory;
@@ -340,29 +341,39 @@ public class SessionState {
    */
   private void setupAuth() {
 
-    if(authenticator != null){
-      //auth has been initialized
+    if (authenticator != null) {
+      // auth has been initialized
       return;
     }
 
     try {
-      authenticator = HiveUtils.getAuthenticator(
-          getConf(),HiveConf.ConfVars.HIVE_AUTHENTICATOR_MANAGER);
-      authorizer = HiveUtils.getAuthorizeProviderManager(
-          getConf(), HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER,
-          authenticator, true);
-
-      if(authorizer == null){
-        //if it was null, the new authorization plugin must be specified in config
-        HiveAuthorizerFactory authorizerFactory =
-            HiveUtils.getAuthorizerFactory(getConf(), HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER);
-        String authUser = userName == null ? authenticator.getUserName() : userName;
+      authenticator = HiveUtils.getAuthenticator(getConf(),
+          HiveConf.ConfVars.HIVE_AUTHENTICATOR_MANAGER);
+
+      if (userName != null) {
+        // if username is set through the session, use an authenticator that
+        // just returns the sessionstate user
+        authenticator = new SessionStateUserAuthenticator(this);
+      }
+      authenticator.setSessionState(this);
+
+      authorizer = HiveUtils.getAuthorizeProviderManager(getConf(),
+          HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, authenticator, true);
+
+      if (authorizer == null) {
+        // if it was null, the new authorization plugin must be specified in
+        // config
+        HiveAuthorizerFactory authorizerFactory = HiveUtils.getAuthorizerFactory(getConf(),
+            HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER);
+
         authorizerV2 = authorizerFactory.createHiveAuthorizer(new HiveMetastoreClientFactoryImpl(),
-            getConf(), authUser);
-      }
-      else{
-        createTableGrants = CreateTableAutomaticGrant.create(getConf());
+            getConf(), authenticator);
+        // grant all privileges for table to its owner
+        getConf().setVar(ConfVars.HIVE_AUTHORIZATION_TABLE_OWNER_GRANTS, "insert,select,update,delete");
       }
+
+      createTableGrants = CreateTableAutomaticGrant.create(getConf());
+
     } catch (HiveException e) {
       throw new RuntimeException(e);
     }
@@ -594,16 +605,19 @@ public class SessionState {
    */
   public static enum ResourceType {
     FILE(new ResourceHook() {
+      @Override
       public String preHook(Set<String> cur, String s) {
         return validateFile(cur, s);
       }
 
+      @Override
       public boolean postHook(Set<String> cur, String s) {
         return true;
       }
     }),
 
     JAR(new ResourceHook() {
+      @Override
       public String preHook(Set<String> cur, String s) {
         String newJar = validateFile(cur, s);
         if (newJar != null) {
@@ -613,16 +627,19 @@ public class SessionState {
         }
       }
 
+      @Override
       public boolean postHook(Set<String> cur, String s) {
         return unregisterJar(s);
       }
     }),
 
     ARCHIVE(new ResourceHook() {
+      @Override
       public String preHook(Set<String> cur, String s) {
         return validateFile(cur, s);
       }
 
+      @Override
       public boolean postHook(Set<String> cur, String s) {
         return true;
       }
@@ -821,6 +838,7 @@ public class SessionState {
   }
 
   public CreateTableAutomaticGrant getCreateTableGrants() {
+    setupAuth();
     return createTableGrants;
   }
 
@@ -947,4 +965,9 @@ public class SessionState {
   public void setTezSession(TezSessionState session) {
     this.tezSessionState = session;
   }
+
+  public String getUserName() {
+    return userName;
+  }
+
 }

Modified: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java?rev=1566401&r1=1566400&r2=1566401&view=diff
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java (original)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java Sun Feb  9 20:43:37 2014
@@ -232,6 +232,11 @@ public class TestHive extends TestCase {
     Warehouse wh = new Warehouse(hiveConf);
     Table ft = null;
     try {
+      // hm.getTable result will not have privileges set (it does not retrieve
+      // that part from metastore), so unset privileges to null before comparing
+      // (create table sets it to empty (non null) structures)
+      tbl.getTTable().setPrivilegesIsSet(false);
+
       ft = hm.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName);
       assertNotNull("Unable to fetch table", ft);
       ft.checkValidity();

Modified: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestSessionUserName.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestSessionUserName.java?rev=1566401&r1=1566400&r2=1566401&view=diff
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestSessionUserName.java (original)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestSessionUserName.java Sun Feb  9 20:43:37 2014
@@ -22,6 +22,7 @@ import junit.framework.Assert;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthorizer;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthorizerFactory;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthorizerImpl;
@@ -106,8 +107,8 @@ public class TestSessionUserName {
 
     @Override
     public HiveAuthorizer createHiveAuthorizer(HiveMetastoreClientFactory metastoreClientFactory,
-        HiveConf conf, String hiveCurrentUser) {
-      username = hiveCurrentUser;
+        HiveConf conf, HiveAuthenticationProvider authenticator) {
+      username = authenticator.getUserName();
       return new HiveAuthorizerImpl(null, null);
     }
   }

Added: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/plugin/sqlstd/TestOperation2Privilege.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/plugin/sqlstd/TestOperation2Privilege.java?rev=1566401&view=auto
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/plugin/sqlstd/TestOperation2Privilege.java (added)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/plugin/sqlstd/TestOperation2Privilege.java Sun Feb  9 20:43:37 2014
@@ -0,0 +1,51 @@
+/**
+ * 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.parse.authorization.plugin.sqlstd;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.util.Set;
+
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.Operation2Privilege;
+import org.junit.Test;
+
+/**
+ * Test HiveOperationType
+ */
+public class TestOperation2Privilege {
+
+  /**
+   * test that all enums in {@link HiveOperationType} match one map entry in
+   * Operation2Privilege
+   */
+  @Test
+  public void checkHiveOperationTypeMatch() {
+    Set<HiveOperationType> operationMapKeys = Operation2Privilege.getOperationTypes();
+    for (HiveOperationType operationType : HiveOperationType.values()) {
+      if (!operationMapKeys.contains(operationType)) {
+        fail("Unable to find corresponding entry in Operation2Privilege map for HiveOperationType "
+            + operationType);
+      }
+    }
+    assertEquals("Check if Operation2Privilege, HiveOperationType have same number of instances",
+        operationMapKeys.size(), HiveOperationType.values().length);
+  }
+
+}

Added: hive/trunk/ql/src/test/queries/clientnegative/authorization_grant_table_allpriv.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/authorization_grant_table_allpriv.q?rev=1566401&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/authorization_grant_table_allpriv.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/authorization_grant_table_allpriv.q Sun Feb  9 20:43:37 2014
@@ -0,0 +1,14 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+
+set user.name=user1;
+-- current user has been set (comment line before the set cmd is resulting in parse error!!)
+
+CREATE TABLE table_priv_allf(i int);
+
+-- grant insert to user2 WITH grant option
+GRANT INSERT ON table_priv_allf TO USER user2 with grant option;
+
+set user.name=user2;
+-- try grant all to user3, without having all privileges
+GRANT ALL ON table_priv_allf TO USER user3;

Added: hive/trunk/ql/src/test/queries/clientnegative/authorization_grant_table_fail1.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/authorization_grant_table_fail1.q?rev=1566401&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/authorization_grant_table_fail1.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/authorization_grant_table_fail1.q Sun Feb  9 20:43:37 2014
@@ -0,0 +1,11 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+
+set user.name=user1;
+-- current user has been set (comment line before the set cmd is resulting in parse error!!)
+
+CREATE TABLE  table_priv_gfail1(i int);
+
+set user.name=user2;
+-- try grant insert to user3 as user2
+GRANT INSERT ON table_priv_gfail1 TO USER user3;

Added: hive/trunk/ql/src/test/queries/clientnegative/authorization_grant_table_fail_nogrant.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/authorization_grant_table_fail_nogrant.q?rev=1566401&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/authorization_grant_table_fail_nogrant.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/authorization_grant_table_fail_nogrant.q Sun Feb  9 20:43:37 2014
@@ -0,0 +1,14 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+
+set user.name=user1;
+-- current user has been set (comment line before the set cmd is resulting in parse error!!)
+
+CREATE TABLE table_priv_gfail1(i int);
+
+-- grant insert to user2 WITHOUT grant option
+GRANT INSERT ON table_priv_gfail1 TO USER user2;
+
+set user.name=user2;
+-- try grant insert to user3
+GRANT INSERT ON table_priv_gfail1 TO USER user3;

Added: hive/trunk/ql/src/test/queries/clientnegative/authorization_revoke_table_fail1.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/authorization_revoke_table_fail1.q?rev=1566401&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/authorization_revoke_table_fail1.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/authorization_revoke_table_fail1.q Sun Feb  9 20:43:37 2014
@@ -0,0 +1,14 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+
+set user.name=user1;
+-- current user has been set (comment line before the set cmd is resulting in parse error!!)
+
+CREATE TABLE table_priv_rfail1(i int);
+
+-- grant insert to user2
+GRANT INSERT ON table_priv_rfail1 TO USER user2;
+
+set user.name=user3;
+-- try dropping the privilege as user3
+REVOKE INSERT ON TABLE table_priv_rfail1 FROM USER user2;

Added: hive/trunk/ql/src/test/queries/clientnegative/authorization_revoke_table_fail2.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/authorization_revoke_table_fail2.q?rev=1566401&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/authorization_revoke_table_fail2.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/authorization_revoke_table_fail2.q Sun Feb  9 20:43:37 2014
@@ -0,0 +1,18 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+
+set user.name=user1;
+-- current user has been set (comment line before the set cmd is resulting in parse error!!)
+
+CREATE TABLE table_priv_rfai2(i int);
+
+-- grant insert to user2
+GRANT INSERT ON table_priv_rfai2 TO USER user2;
+GRANT SELECT ON table_priv_rfai2 TO USER user3 WITH GRANT OPTION;
+
+set user.name=user3;
+-- grant select as user3 to user 2
+GRANT SELECT ON table_priv_rfai2 TO USER user2;
+
+-- try dropping the privilege as user3
+REVOKE INSERT ON TABLE table_priv_rfai2 FROM USER user2;

Modified: hive/trunk/ql/src/test/queries/clientpositive/authorization_1_sql_std.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/authorization_1_sql_std.q?rev=1566401&r1=1566400&r2=1566401&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/authorization_1_sql_std.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/authorization_1_sql_std.q Sun Feb  9 20:43:37 2014
@@ -6,18 +6,18 @@ set hive.security.authorization.enabled=
 
 --table grant to user
 
-grant select on table src_autho_test to user hive_test_user;
+grant select on table src_autho_test to user user_sauth;
 
-show grant user hive_test_user on table src_autho_test;
+show grant user user_sauth on table src_autho_test;
 
 
-revoke select on table src_autho_test from user hive_test_user;
-show grant user hive_test_user on table src_autho_test;
+revoke select on table src_autho_test from user user_sauth;
+show grant user user_sauth on table src_autho_test;
 
 --role
 create role src_role;
-grant role src_role to user hive_test_user;
-show role grant user hive_test_user;
+grant role src_role to user user_sauth;
+show role grant user user_sauth;
 
 --table grant to role
 

Added: hive/trunk/ql/src/test/queries/clientpositive/authorization_create_table_owner_privs.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/authorization_create_table_owner_privs.q?rev=1566401&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/authorization_create_table_owner_privs.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/authorization_create_table_owner_privs.q Sun Feb  9 20:43:37 2014
@@ -0,0 +1,10 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+
+set user.name=user1;
+
+create table create_table_creator_priv_test(i int);
+
+-- all privileges should have been set for user
+
+show grant user user1 on table create_table_creator_priv_test;

Added: hive/trunk/ql/src/test/queries/clientpositive/authorization_grant_table_priv.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/authorization_grant_table_priv.q?rev=1566401&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/authorization_grant_table_priv.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/authorization_grant_table_priv.q Sun Feb  9 20:43:37 2014
@@ -0,0 +1,43 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+
+set user.name=user1;
+-- current user has been set (comment line before the set cmd is resulting in parse error!!)
+
+CREATE TABLE  table_priv1(i int);
+
+-- all privileges should have been set for user
+
+-- grant insert privilege to another user
+GRANT INSERT ON table_priv1 TO USER user2;
+SHOW GRANT USER user2 ON TABLE table_priv1;
+
+-- grant select privilege to another user with grant
+GRANT SELECT ON table_priv1 TO USER user2 with grant option;
+SHOW GRANT USER user2 ON TABLE table_priv1;
+
+set user.name=user2;
+-- change to other user - user2
+-- grant permissions to another user as user2
+GRANT SELECT ON table_priv1 TO USER user3 with grant option;
+SHOW GRANT USER user3 ON TABLE table_priv1;
+
+set user.name=user3;
+-- change to other user - user3
+-- grant permissions to another user as user3
+GRANT SELECT ON table_priv1 TO USER user4 with grant option;
+SHOW GRANT USER user4 ON TABLE table_priv1;
+
+set user.name=user1;
+-- switched back to table owner
+
+-- grant all with grant to user22
+GRANT ALL ON table_priv1 TO USER user22 with grant option;
+SHOW GRANT USER user22 ON TABLE table_priv1;
+
+set user.name=user22;
+
+-- grant all without grant to user33
+GRANT ALL ON table_priv1 TO USER user33 with grant option;
+SHOW GRANT USER user33 ON TABLE table_priv1;
+

Added: hive/trunk/ql/src/test/queries/clientpositive/authorization_revoke_table_priv.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/authorization_revoke_table_priv.q?rev=1566401&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/authorization_revoke_table_priv.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/authorization_revoke_table_priv.q Sun Feb  9 20:43:37 2014
@@ -0,0 +1,50 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+
+set user.name=user1;
+-- current user has been set (comment line before the set cmd is resulting in parse error!!)
+
+CREATE TABLE  table_priv_rev(i int);
+
+-- grant insert privilege to user2
+GRANT INSERT ON table_priv_rev TO USER user2;
+SHOW GRANT USER user2 ON TABLE table_priv_rev;
+
+-- revoke insert privilege from user2
+REVOKE INSERT ON TABLE table_priv_rev FROM USER user2;
+SHOW GRANT USER user2 ON TABLE table_priv_rev;
+
+-- grant all privileges one at a time --
+-- grant insert privilege to user2
+GRANT INSERT ON table_priv_rev TO USER user2;
+SHOW GRANT USER user2 ON TABLE table_priv_rev;
+
+-- grant select privilege to user2, with grant option
+GRANT SELECT ON table_priv_rev TO USER user2 WITH GRANT OPTION;
+SHOW GRANT USER user2 ON TABLE table_priv_rev;
+
+-- grant update privilege to user2
+GRANT UPDATE ON table_priv_rev TO USER user2;
+SHOW GRANT USER user2 ON TABLE table_priv_rev;
+
+-- grant delete privilege to user2
+GRANT DELETE ON table_priv_rev TO USER user2;
+SHOW GRANT USER user2 ON TABLE table_priv_rev;
+
+-- start revoking --
+-- revoke update privilege from user2
+REVOKE UPDATE ON TABLE table_priv_rev FROM USER user2;
+SHOW GRANT USER user2 ON TABLE table_priv_rev;
+
+-- revoke DELETE privilege from user2
+REVOKE DELETE ON TABLE table_priv_rev FROM USER user2;
+SHOW GRANT USER user2 ON TABLE table_priv_rev;
+
+-- revoke insert privilege from user2
+REVOKE INSERT ON TABLE table_priv_rev FROM USER user2;
+SHOW GRANT USER user2 ON TABLE table_priv_rev;
+
+-- revoke select privilege from user2
+REVOKE SELECT ON TABLE table_priv_rev FROM USER user2;
+SHOW GRANT USER user2 ON TABLE table_priv_rev;
+

Added: hive/trunk/ql/src/test/results/clientnegative/authorization_grant_table_allpriv.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/authorization_grant_table_allpriv.q.out?rev=1566401&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/authorization_grant_table_allpriv.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/authorization_grant_table_allpriv.q.out Sun Feb  9 20:43:37 2014
@@ -0,0 +1,22 @@
+PREHOOK: query: -- current user has been set (comment line before the set cmd is resulting in parse error!!)
+
+CREATE TABLE table_priv_allf(i int)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: -- current user has been set (comment line before the set cmd is resulting in parse error!!)
+
+CREATE TABLE table_priv_allf(i int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@table_priv_allf
+PREHOOK: query: -- grant insert to user2 WITH grant option
+GRANT INSERT ON table_priv_allf TO USER user2 with grant option
+PREHOOK: type: GRANT_PRIVILEGE
+PREHOOK: Output: default@table_priv_allf
+POSTHOOK: query: -- grant insert to user2 WITH grant option
+GRANT INSERT ON table_priv_allf TO USER user2 with grant option
+POSTHOOK: type: GRANT_PRIVILEGE
+POSTHOOK: Output: default@table_priv_allf
+PREHOOK: query: -- try grant all to user3, without having all privileges
+GRANT ALL ON table_priv_allf TO USER user3
+PREHOOK: type: GRANT_PRIVILEGE
+PREHOOK: Output: default@table_priv_allf
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Permission denied. Principal [name=user2, type=USER] does not have following privileges on Hive Object [type=TABLE, dbname=default, table/viewname=table_priv_allf] : [SELECT with grant, UPDATE with grant, DELETE with grant]

Added: hive/trunk/ql/src/test/results/clientnegative/authorization_grant_table_fail1.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/authorization_grant_table_fail1.q.out?rev=1566401&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/authorization_grant_table_fail1.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/authorization_grant_table_fail1.q.out Sun Feb  9 20:43:37 2014
@@ -0,0 +1,14 @@
+PREHOOK: query: -- current user has been set (comment line before the set cmd is resulting in parse error!!)
+
+CREATE TABLE  table_priv_gfail1(i int)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: -- current user has been set (comment line before the set cmd is resulting in parse error!!)
+
+CREATE TABLE  table_priv_gfail1(i int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@table_priv_gfail1
+PREHOOK: query: -- try grant insert to user3 as user2
+GRANT INSERT ON table_priv_gfail1 TO USER user3
+PREHOOK: type: GRANT_PRIVILEGE
+PREHOOK: Output: default@table_priv_gfail1
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Permission denied. Principal [name=user2, type=USER] does not have following privileges on Hive Object [type=TABLE, dbname=default, table/viewname=table_priv_gfail1] : [INSERT with grant]

Added: hive/trunk/ql/src/test/results/clientnegative/authorization_grant_table_fail_nogrant.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/authorization_grant_table_fail_nogrant.q.out?rev=1566401&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/authorization_grant_table_fail_nogrant.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/authorization_grant_table_fail_nogrant.q.out Sun Feb  9 20:43:37 2014
@@ -0,0 +1,22 @@
+PREHOOK: query: -- current user has been set (comment line before the set cmd is resulting in parse error!!)
+
+CREATE TABLE table_priv_gfail1(i int)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: -- current user has been set (comment line before the set cmd is resulting in parse error!!)
+
+CREATE TABLE table_priv_gfail1(i int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@table_priv_gfail1
+PREHOOK: query: -- grant insert to user2 WITHOUT grant option
+GRANT INSERT ON table_priv_gfail1 TO USER user2
+PREHOOK: type: GRANT_PRIVILEGE
+PREHOOK: Output: default@table_priv_gfail1
+POSTHOOK: query: -- grant insert to user2 WITHOUT grant option
+GRANT INSERT ON table_priv_gfail1 TO USER user2
+POSTHOOK: type: GRANT_PRIVILEGE
+POSTHOOK: Output: default@table_priv_gfail1
+PREHOOK: query: -- try grant insert to user3
+GRANT INSERT ON table_priv_gfail1 TO USER user3
+PREHOOK: type: GRANT_PRIVILEGE
+PREHOOK: Output: default@table_priv_gfail1
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Permission denied. Principal [name=user2, type=USER] does not have following privileges on Hive Object [type=TABLE, dbname=default, table/viewname=table_priv_gfail1] : [INSERT with grant]

Modified: hive/trunk/ql/src/test/results/clientnegative/authorization_invalid_priv_v2.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/authorization_invalid_priv_v2.q.out?rev=1566401&r1=1566400&r2=1566401&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/authorization_invalid_priv_v2.q.out (original)
+++ hive/trunk/ql/src/test/results/clientnegative/authorization_invalid_priv_v2.q.out Sun Feb  9 20:43:37 2014
@@ -6,4 +6,4 @@ POSTHOOK: Output: default@authorization_
 PREHOOK: query: grant index on table authorization_invalid_v2 to user hive_test_user
 PREHOOK: type: GRANT_PRIVILEGE
 PREHOOK: Output: default@authorization_invalid_v2
-FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Privilege: Index is not supported in sql standard authorization mode
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Unsupported privilege type INDEX

Added: hive/trunk/ql/src/test/results/clientnegative/authorization_revoke_table_fail1.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/authorization_revoke_table_fail1.q.out?rev=1566401&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/authorization_revoke_table_fail1.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/authorization_revoke_table_fail1.q.out Sun Feb  9 20:43:37 2014
@@ -0,0 +1,23 @@
+PREHOOK: query: -- current user has been set (comment line before the set cmd is resulting in parse error!!)
+
+CREATE TABLE table_priv_rfail1(i int)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: -- current user has been set (comment line before the set cmd is resulting in parse error!!)
+
+CREATE TABLE table_priv_rfail1(i int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@table_priv_rfail1
+PREHOOK: query: -- grant insert to user2
+GRANT INSERT ON table_priv_rfail1 TO USER user2
+PREHOOK: type: GRANT_PRIVILEGE
+PREHOOK: Output: default@table_priv_rfail1
+POSTHOOK: query: -- grant insert to user2
+GRANT INSERT ON table_priv_rfail1 TO USER user2
+POSTHOOK: type: GRANT_PRIVILEGE
+POSTHOOK: Output: default@table_priv_rfail1
+PREHOOK: query: -- try dropping the privilege as user3
+REVOKE INSERT ON TABLE table_priv_rfail1 FROM USER user2
+PREHOOK: type: REVOKE_PRIVILEGE
+PREHOOK: Output: default@table_priv_rfail1
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Cannot find privilege Privilege [name=INSERT, columns=null] for Principal [name=user2, type=USER] on Hive Object [type=TABLE, dbname=default, table/viewname=table_priv_rfail1] granted by user3
+