You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ha...@apache.org on 2014/01/28 20:24:46 UTC

svn commit: r1562165 - in /hive/trunk/ql/src: java/org/apache/hadoop/hive/ql/ java/org/apache/hadoop/hive/ql/exec/ java/org/apache/hadoop/hive/ql/metadata/ java/org/apache/hadoop/hive/ql/parse/authorization/ java/org/apache/hadoop/hive/ql/security/auth...

Author: hashutosh
Date: Tue Jan 28 19:24:45 2014
New Revision: 1562165

URL: http://svn.apache.org/r1562165
Log:
HIVE-5929 : SQL std auth - Access control statement updates (Thejas Nair via Ashutosh Chauhan)

Added:
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationUtils.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizationPluginException.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HivePrivilegeInfo.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessController.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizationValidator.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizerFactory.java
    hive/trunk/ql/src/test/queries/clientpositive/authorization_1_sql_std.q
    hive/trunk/ql/src/test/results/clientpositive/authorization_1_sql_std.q.out
Removed:
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/DefaultHiveAuthorizerFactory.java
Modified:
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/HiveAuthorizationTaskFactoryImpl.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAccessController.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizationValidator.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizer.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizerImpl.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
    hive/trunk/ql/src/test/queries/clientpositive/authorization_role_grant1.q
    hive/trunk/ql/src/test/results/clientpositive/authorization_role_grant1.q.out

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java?rev=1562165&r1=1562164&r2=1562165&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java Tue Jan 28 19:24:45 2014
@@ -55,7 +55,6 @@ import org.apache.hadoop.hive.ql.exec.Ta
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.history.HiveHistory.Keys;
 import org.apache.hadoop.hive.ql.hooks.Entity;
-import org.apache.hadoop.hive.ql.hooks.Entity.Type;
 import org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext;
 import org.apache.hadoop.hive.ql.hooks.Hook;
 import org.apache.hadoop.hive.ql.hooks.HookContext;
@@ -102,6 +101,7 @@ import org.apache.hadoop.hive.ql.plan.Op
 import org.apache.hadoop.hive.ql.plan.TableDesc;
 import org.apache.hadoop.hive.ql.processors.CommandProcessor;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
+import org.apache.hadoop.hive.ql.security.authorization.AuthorizationUtils;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType;
@@ -703,7 +703,7 @@ public class Driver implements CommandPr
   }
 
   private void doAuthorizationV2(SessionState ss, HiveOperation op, HashSet<ReadEntity> inputs,
-      HashSet<WriteEntity> outputs) {
+      HashSet<WriteEntity> outputs) throws HiveException {
     HiveOperationType hiveOpType = getHiveOperationType(op);
     List<HivePrivilegeObject> inputsHObjs = getHivePrivObjects(inputs);
     List<HivePrivilegeObject> outputHObjs = getHivePrivObjects(outputs);
@@ -711,35 +711,25 @@ public class Driver implements CommandPr
     return;
   }
 
-  private List<HivePrivilegeObject> getHivePrivObjects(HashSet<? extends Entity> inputs) {
+  private List<HivePrivilegeObject> getHivePrivObjects(HashSet<? extends Entity> privObjects) {
     List<HivePrivilegeObject> hivePrivobjs = new ArrayList<HivePrivilegeObject>();
-    for(Entity input : inputs){
-      HivePrivilegeObjectType privObjType = getHivePrivilegeObjectType(input.getType());
+    if(privObjects == null){
+      return hivePrivobjs;
+    }
+    for(Entity privObject : privObjects){
+      HivePrivilegeObjectType privObjType =
+          AuthorizationUtils.getHivePrivilegeObjectType(privObject.getType());
+
       //support for authorization on partitions or uri needs to be added
       HivePrivilegeObject hPrivObject = new HivePrivilegeObject(privObjType,
-          input.getDatabase().getName(),
-          input.getTable().getTableName());
+          privObject.getDatabase() == null ? null : privObject.getDatabase().getName(),
+              privObject.getTable() == null ? null : privObject.getTable().getTableName());
       hivePrivobjs.add(hPrivObject);
     }
     return hivePrivobjs;
   }
 
-  private HivePrivilegeObjectType getHivePrivilegeObjectType(Type type) {
-    switch(type){
-    case DATABASE:
-      return HivePrivilegeObjectType.DATABASE;
-    case TABLE:
-      return HivePrivilegeObjectType.TABLE;
-    case LOCAL_DIR:
-    case DFS_DIR:
-      return HivePrivilegeObjectType.URI;
-    case PARTITION:
-    case DUMMYPARTITION: //need to determine if a different type is needed for dummy partitions
-      return HivePrivilegeObjectType.PARTITION;
-    default:
-      return null;
-    }
-  }
+
 
   private HiveOperationType getHiveOperationType(HiveOperation op) {
     return HiveOperationType.valueOf(op.name());

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java?rev=1562165&r1=1562164&r2=1562165&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java Tue Jan 28 19:24:45 2014
@@ -155,11 +155,13 @@ import org.apache.hadoop.hive.ql.plan.Tr
 import org.apache.hadoop.hive.ql.plan.UnlockDatabaseDesc;
 import org.apache.hadoop.hive.ql.plan.UnlockTableDesc;
 import org.apache.hadoop.hive.ql.plan.api.StageType;
+import org.apache.hadoop.hive.ql.security.authorization.AuthorizationUtils;
 import org.apache.hadoop.hive.ql.security.authorization.Privilege;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthorizer;
 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.HivePrivilege;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeInfo;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType;
 import org.apache.hadoop.hive.ql.session.SessionState;
@@ -541,6 +543,10 @@ public class DDLTask extends Task<DDLWor
   }
 
   private int showGrants(ShowGrantDesc showGrantDesc) throws HiveException {
+
+    if(SessionState.get().isAuthorizationModeV2()){
+      return showGrantsV2(showGrantDesc);
+    }
     StringBuilder builder = new StringBuilder();
     try {
       PrincipalDesc principalDesc = showGrantDesc.getPrincipalDesc();
@@ -639,6 +645,44 @@ public class DDLTask extends Task<DDLWor
     return 0;
   }
 
+  private int showGrantsV2(ShowGrantDesc showGrantDesc) throws HiveException {
+    HiveAuthorizer authorizer = SessionState.get().getAuthorizerV2();
+    StringBuilder builder = new StringBuilder();
+    try {
+      List<HivePrivilegeInfo> privInfos = authorizer.showPrivileges(
+          getHivePrincipal(showGrantDesc.getPrincipalDesc()),
+          getHivePrivilegeObject(showGrantDesc.getHiveObj())
+          );
+      for(HivePrivilegeInfo privInfo : privInfos){
+        HivePrincipal principal = privInfo.getPrincipal();
+        HivePrivilegeObject privObj = privInfo.getObject();
+        HivePrivilege priv = privInfo.getPrivilege();
+
+        PrivilegeGrantInfo grantInfo =
+            AuthorizationUtils.getThriftPrivilegeGrantInfo(priv, privInfo.getGrantorPrincipal(),
+                privInfo.isGrantOption());
+
+        //only grantInfo is used
+        HiveObjectPrivilege thriftObjectPriv = new HiveObjectPrivilege(null, null, null, grantInfo);
+        List<HiveObjectPrivilege> privList = new ArrayList<HiveObjectPrivilege>();
+        privList.add(thriftObjectPriv);
+        writeGrantInfo(builder,
+            AuthorizationUtils.getThriftPrincipalType(principal.getType()),
+            principal.getName(),
+            privObj.getDbname(),
+            privObj.getTableviewname(),
+            null,
+            null,
+            privList
+            );
+      }
+      writeToFile(builder.toString(), showGrantDesc.getResFile());
+    } catch (IOException e) {
+      throw new HiveException("Error in show grant statement", e);
+    }
+    return 0;
+  }
+
   private static void sortPrivileges(List<HiveObjectPrivilege> privileges) {
     Collections.sort(privileges, new Comparator<HiveObjectPrivilege>() {
 
@@ -823,6 +867,24 @@ public class DDLTask extends Task<DDLWor
     return new HivePrivilegeObject(getPrivObjectType(privSubjectDesc), dbTable[0], dbTable[1]);
   }
 
+  private HivePrincipalType getHivePrincipalType(PrincipalType type) throws HiveException {
+    if(type == null){
+      return null;
+    }
+
+    switch(type){
+    case USER:
+      return HivePrincipalType.USER;
+    case ROLE:
+      return HivePrincipalType.ROLE;
+    case GROUP:
+      throw new HiveException(ErrorMsg.UNNSUPPORTED_AUTHORIZATION_PRINCIPAL_TYPE_GROUP);
+    default:
+      //should not happen as we take care of all existing types
+      throw new AssertionError("Unsupported authorization type specified");
+    }
+  }
+
   private HivePrivilegeObjectType getPrivObjectType(PrivilegeObjectDesc privSubjectDesc) {
     //TODO: This needs to change to support view once view grant/revoke is supported as
     // part of HIVE-6181
@@ -841,24 +903,14 @@ public class DDLTask extends Task<DDLWor
   private List<HivePrincipal> getHivePrincipals(List<PrincipalDesc> principals) throws HiveException {
     ArrayList<HivePrincipal> hivePrincipals = new ArrayList<HivePrincipal>();
     for(PrincipalDesc principal : principals){
-      hivePrincipals.add(
-          new HivePrincipal(principal.getName(), getHivePrincipalType(principal.getType())));
+      hivePrincipals.add(getHivePrincipal(principal));
     }
     return hivePrincipals;
   }
 
-  private HivePrincipalType getHivePrincipalType(PrincipalType type) throws HiveException {
-    switch(type){
-    case USER:
-      return HivePrincipalType.USER;
-    case ROLE:
-      return HivePrincipalType.ROLE;
-    case GROUP:
-      throw new HiveException(ErrorMsg.UNNSUPPORTED_AUTHORIZATION_PRINCIPAL_TYPE_GROUP);
-    default:
-      //should not happen as we take care of all existing types
-      throw new HiveException("Unsupported authorization type specified");
-    }
+  private HivePrincipal getHivePrincipal(PrincipalDesc principal) throws HiveException {
+    return new HivePrincipal(principal.getName(),
+        AuthorizationUtils.getHivePrincipalType(principal.getType()));
   }
 
   private void throwNotFound(String objType, String objName) throws HiveException {
@@ -915,7 +967,7 @@ public class DDLTask extends Task<DDLWor
       LOG.info("role ddl exception: " + stringifyException(e));
       return 1;
     } finally {
-      IOUtils.closeStream((FSDataOutputStream) outStream);
+      IOUtils.closeStream(outStream);
     }
 
     return 0;
@@ -945,6 +997,7 @@ public class DDLTask extends Task<DDLWor
       throw new HiveException("Unkown role operation "
           + operation.getOperationName());
     }
+
     return 0;
   }
 
@@ -955,10 +1008,10 @@ public class DDLTask extends Task<DDLWor
    * @throws IOException
    */
   private void writeListToFile(List<String> entries, String resFile) throws IOException {
-    StringBuilder sb = new StringBuilder(entries.size()*2);
+    StringBuilder sb = new StringBuilder();
     for(String entry : entries){
       sb.append(entry);
-      sb.append(terminator);
+      sb.append((char)terminator);
     }
     writeToFile(sb.toString(), resFile);
   }
@@ -2191,7 +2244,7 @@ public class DDLTask extends Task<DDLWor
     } catch (Exception e) {
       throw new HiveException(e);
     } finally {
-      IOUtils.closeStream((FSDataOutputStream) outStream);
+      IOUtils.closeStream(outStream);
     }
 
     return 0;
@@ -2249,7 +2302,7 @@ public class DDLTask extends Task<DDLWor
     } catch (Exception e) {
       throw new HiveException(e.toString());
     } finally {
-      IOUtils.closeStream((FSDataOutputStream) outStream);
+      IOUtils.closeStream(outStream);
     }
 
     return 0;
@@ -2368,7 +2421,7 @@ public class DDLTask extends Task<DDLWor
     } catch (IOException e) {
       throw new HiveException(e, ErrorMsg.GENERIC_ERROR);
     } finally {
-      IOUtils.closeStream((FSDataOutputStream) outStream);
+      IOUtils.closeStream(outStream);
     }
     return 0;
   }
@@ -2420,7 +2473,7 @@ public class DDLTask extends Task<DDLWor
     } catch (Exception e) {
       throw new HiveException(e);
     } finally {
-      IOUtils.closeStream((FSDataOutputStream) outStream);
+      IOUtils.closeStream(outStream);
     }
     return 0;
   }
@@ -2512,7 +2565,7 @@ public class DDLTask extends Task<DDLWor
     } catch (Exception e) {
       throw new HiveException(e.toString());
     } finally {
-      IOUtils.closeStream((FSDataOutputStream) outStream);
+      IOUtils.closeStream(outStream);
     }
     return 0;
   }
@@ -2753,7 +2806,7 @@ public class DDLTask extends Task<DDLWor
     } catch (Exception e) {
       throw new HiveException(e);
     } finally {
-      IOUtils.closeStream((FSDataOutputStream) outStream);
+      IOUtils.closeStream(outStream);
     }
     return 0;
   }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java?rev=1562165&r1=1562164&r2=1562165&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java Tue Jan 28 19:24:45 2014
@@ -29,9 +29,9 @@ import org.apache.hadoop.hive.ql.securit
 import org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider;
 import org.apache.hadoop.hive.ql.security.authorization.DefaultHiveAuthorizationProvider;
 import org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider;
-import org.apache.hadoop.hive.ql.security.authorization.plugin.DefaultHiveAuthorizerFactory;
 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.sqlstd.SQLStdHiveAuthorizerFactory;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.util.ReflectionUtils;
 
@@ -392,7 +392,7 @@ public final class HiveUtils {
           throws HiveException {
 
     Class<? extends HiveAuthorizerFactory> cls = conf.getClass(authorizationProviderConfKey.varname,
-        DefaultHiveAuthorizerFactory.class, HiveAuthorizerFactory.class);
+        SQLStdHiveAuthorizerFactory.class, HiveAuthorizerFactory.class);
 
     if(cls == null){
       //should not happen as default value is set

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/HiveAuthorizationTaskFactoryImpl.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/HiveAuthorizationTaskFactoryImpl.java?rev=1562165&r1=1562164&r2=1562165&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/HiveAuthorizationTaskFactoryImpl.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/HiveAuthorizationTaskFactoryImpl.java Tue Jan 28 19:24:45 2014
@@ -190,7 +190,7 @@ public class HiveAuthorizationTaskFactor
             if (grandChild.getToken().getType() == HiveParser.TOK_PARTSPEC) {
               privHiveObj.setPartSpec(DDLSemanticAnalyzer.getPartSpec(grandChild));
             } else if (grandChild.getToken().getType() == HiveParser.TOK_TABCOLNAME) {
-              cols = BaseSemanticAnalyzer.getColumnNames((ASTNode) grandChild);
+              cols = BaseSemanticAnalyzer.getColumnNames(grandChild);
             }
           }
         }
@@ -235,8 +235,12 @@ public class HiveAuthorizationTaskFactor
         && SessionState.get().getAuthenticator() != null) {
       roleOwnerName = SessionState.get().getAuthenticator().getUserName();
     }
+
+    //until change is made to use the admin option. Default to false with V2 authorization
+    boolean isAdmin = SessionState.get().isAuthorizationModeV2() ? false : true;
+
     GrantRevokeRoleDDL grantRevokeRoleDDL = new GrantRevokeRoleDDL(isGrant,
-        roles, principalDesc, roleOwnerName, PrincipalType.USER, true);
+        roles, principalDesc, roleOwnerName, PrincipalType.USER, isAdmin);
     return TaskFactory.get(new DDLWork(inputs, outputs, grantRevokeRoleDDL), conf);
   }
 

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationUtils.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationUtils.java?rev=1562165&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationUtils.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationUtils.java Tue Jan 28 19:24:45 2014
@@ -0,0 +1,156 @@
+/**
+ * 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;
+
+import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
+import org.apache.hadoop.hive.metastore.api.HiveObjectRef;
+import org.apache.hadoop.hive.metastore.api.HiveObjectType;
+import org.apache.hadoop.hive.metastore.api.PrincipalType;
+import org.apache.hadoop.hive.metastore.api.PrivilegeGrantInfo;
+import org.apache.hadoop.hive.ql.ErrorMsg;
+import org.apache.hadoop.hive.ql.hooks.Entity.Type;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+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.HivePrivilege;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType;
+
+/**
+ * Utility code shared by hive internal code and sql standard authorization plugin implementation
+ */
+@LimitedPrivate(value = { "Sql standard authorization plugin" })
+public class AuthorizationUtils {
+
+  /**
+   * Convert thrift principal type to authorization plugin principal type
+   * @param type - thrift principal type
+   * @return
+   * @throws HiveException
+   */
+  public static HivePrincipalType getHivePrincipalType(PrincipalType type) throws HiveException {
+    switch(type){
+    case USER:
+      return HivePrincipalType.USER;
+    case ROLE:
+      return HivePrincipalType.ROLE;
+    case GROUP:
+      throw new HiveException(ErrorMsg.UNNSUPPORTED_AUTHORIZATION_PRINCIPAL_TYPE_GROUP);
+    default:
+      //should not happen as we take care of all existing types
+      throw new AssertionError("Unsupported authorization type specified");
+    }
+  }
+
+
+  /**
+   * Convert thrift object type to hive authorization plugin object type
+   * @param type - thrift object type
+   * @return
+   */
+  public static HivePrivilegeObjectType getHivePrivilegeObjectType(Type type) {
+    switch(type){
+    case DATABASE:
+      return HivePrivilegeObjectType.DATABASE;
+    case TABLE:
+      return HivePrivilegeObjectType.TABLE;
+    case LOCAL_DIR:
+    case DFS_DIR:
+      return HivePrivilegeObjectType.URI;
+    case PARTITION:
+    case DUMMYPARTITION: //need to determine if a different type is needed for dummy partitions
+      return HivePrivilegeObjectType.PARTITION;
+    default:
+      return null;
+    }
+  }
+
+
+  /**
+   * Convert authorization plugin principal type to thrift principal type
+   * @param type
+   * @return
+   * @throws HiveException
+   */
+  public static PrincipalType getThriftPrincipalType(HivePrincipalType type) {
+    if(type == null){
+      return null;
+    }
+    switch(type){
+    case USER:
+      return PrincipalType.USER;
+    case ROLE:
+      return PrincipalType.ROLE;
+    default:
+      throw new AssertionError("Invalid principal type " + type);
+    }
+  }
+
+
+  /**
+   * Get thrift privilege grant info
+   * @param privilege
+   * @param grantorPrincipal
+   * @param grantOption
+   * @return
+   * @throws HiveException
+   */
+  public static PrivilegeGrantInfo getThriftPrivilegeGrantInfo(HivePrivilege privilege,
+      HivePrincipal grantorPrincipal, boolean grantOption) throws HiveException {
+    return new PrivilegeGrantInfo(privilege.getName(), 0 /* time gets added by server */,
+        grantorPrincipal.getName(), getThriftPrincipalType(grantorPrincipal.getType()), grantOption);
+  }
+
+
+  /**
+   * Convert plugin privilege object type to thrift type
+   * @param type
+   * @return
+   * @throws HiveException
+   */
+  public static HiveObjectType getThriftHiveObjType(HivePrivilegeObjectType type) throws HiveException {
+    switch(type){
+    case DATABASE:
+      return HiveObjectType.DATABASE;
+    case TABLE:
+      return HiveObjectType.TABLE;
+    case PARTITION:
+      return HiveObjectType.PARTITION;
+    case URI:
+    case VIEW:
+      throw new HiveException("Unsupported type " + type);
+    default:
+      //should not happen as we have accounted for all types
+      throw new AssertionError("Unsupported type " + type);
+    }
+  }
+
+
+  /**
+   * Convert thrift HiveObjectRef to plugin HivePrivilegeObject
+   * @param privObj
+   * @return
+   * @throws HiveException
+   */
+  public static HiveObjectRef getThriftHiveObjectRef(HivePrivilegeObject privObj) throws HiveException {
+    HiveObjectType objType = getThriftHiveObjType(privObj.getType());
+    return new HiveObjectRef(objType, privObj.getDbname(), privObj.getTableviewname(), null, null);
+  }
+
+
+}

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAccessController.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAccessController.java?rev=1562165&r1=1562164&r2=1562165&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAccessController.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAccessController.java Tue Jan 28 19:24:45 2014
@@ -27,23 +27,34 @@ import java.util.List;
 public interface HiveAccessController {
 
   void grantPrivileges(List<HivePrincipal> hivePrincipals, List<HivePrivilege> hivePrivileges,
-      HivePrivilegeObject hivePrivObject, HivePrincipal grantorPrincipal, boolean grantOption);
+      HivePrivilegeObject hivePrivObject, HivePrincipal grantorPrincipal, boolean grantOption)
+          throws HiveAuthorizationPluginException;;
 
   void revokePrivileges(List<HivePrincipal> hivePrincipals, List<HivePrivilege> hivePrivileges,
-      HivePrivilegeObject hivePrivObject, HivePrincipal grantorPrincipal, boolean grantOption);
+      HivePrivilegeObject hivePrivObject, HivePrincipal grantorPrincipal, boolean grantOption)
+          throws HiveAuthorizationPluginException;;
 
-  void createRole(String roleName, HivePrincipal adminGrantor);
+  void createRole(String roleName, HivePrincipal adminGrantor)
+      throws HiveAuthorizationPluginException;
 
-  void dropRole(String roleName);
+  void dropRole(String roleName)
+      throws HiveAuthorizationPluginException;
 
-  List<String> getRoles(HivePrincipal hivePrincipal);
+  List<String> getRoles(HivePrincipal hivePrincipal)
+      throws HiveAuthorizationPluginException;
 
   void grantRole(List<HivePrincipal> hivePrincipals, List<String> roles, boolean grantOption,
-      HivePrincipal grantorPrinc);
+      HivePrincipal grantorPrinc)
+          throws HiveAuthorizationPluginException;
 
   void revokeRole(List<HivePrincipal> hivePrincipals, List<String> roles, boolean grantOption,
-      HivePrincipal grantorPrinc);
+      HivePrincipal grantorPrinc)
+          throws HiveAuthorizationPluginException;
 
-  List<String> getAllRoles();
+  List<String> getAllRoles()
+      throws HiveAuthorizationPluginException;
+
+  List<HivePrivilegeInfo> showPrivileges(HivePrincipal principal, HivePrivilegeObject privObj)
+      throws HiveAuthorizationPluginException;
 
 }

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizationPluginException.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizationPluginException.java?rev=1562165&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizationPluginException.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizationPluginException.java Tue Jan 28 19:24:45 2014
@@ -0,0 +1,47 @@
+/**
+ * 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;
+
+import org.apache.hadoop.hive.common.classification.InterfaceAudience.Public;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+
+/**
+ * Exception thrown by the Authorization plugin api (v2)
+ */
+@Public
+public class HiveAuthorizationPluginException extends HiveException{
+
+  private static final long serialVersionUID = 1L;
+
+  public HiveAuthorizationPluginException(){
+  }
+
+  public HiveAuthorizationPluginException(String msg){
+    super(msg);
+  }
+
+  public HiveAuthorizationPluginException(String msg, Throwable cause){
+    super(msg, cause);
+  }
+
+  public HiveAuthorizationPluginException(Throwable cause){
+    super(cause);
+  }
+
+}

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizationValidator.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizationValidator.java?rev=1562165&r1=1562164&r2=1562165&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizationValidator.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizationValidator.java Tue Jan 28 19:24:45 2014
@@ -37,6 +37,6 @@ public interface HiveAuthorizationValida
    * @param outputHObjs
    */
   void checkPrivileges(HiveOperationType hiveOpType, List<HivePrivilegeObject> inputHObjs,
-      List<HivePrivilegeObject> outputHObjs);
+      List<HivePrivilegeObject> outputHObjs) throws HiveAuthorizationPluginException;
 
 }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizer.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizer.java?rev=1562165&r1=1562164&r2=1562165&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizer.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizer.java Tue Jan 28 19:24:45 2014
@@ -37,6 +37,13 @@ import org.apache.hadoop.hive.ql.securit
 @Evolving
 public interface HiveAuthorizer {
 
+  public enum VERSION { V1 };
+
+  /**
+   * @return version of HiveAuthorizer interface that is implemented by this instance
+   */
+  public VERSION getVersion();
+
   /**
    * Grant privileges for principals on the object
    * @param hivePrincipals
@@ -44,9 +51,11 @@ public interface HiveAuthorizer {
    * @param hivePrivObject
    * @param grantorPrincipal
    * @param grantOption
+   * @throws HiveAuthorizationPluginException
    */
   void grantPrivileges(List<HivePrincipal> hivePrincipals, List<HivePrivilege> hivePrivileges,
-      HivePrivilegeObject hivePrivObject, HivePrincipal grantorPrincipal, boolean grantOption);
+      HivePrivilegeObject hivePrivObject, HivePrincipal grantorPrincipal, boolean grantOption)
+      throws HiveAuthorizationPluginException;
 
   /**
    * Revoke privileges for principals on the object
@@ -55,29 +64,38 @@ public interface HiveAuthorizer {
    * @param hivePrivObject
    * @param grantorPrincipal
    * @param grantOption
+   * @throws HiveAuthorizationPluginException
    */
   void revokePrivileges(List<HivePrincipal> hivePrincipals, List<HivePrivilege> hivePrivileges,
-      HivePrivilegeObject hivePrivObject, HivePrincipal grantorPrincipal, boolean grantOption);
+      HivePrivilegeObject hivePrivObject, HivePrincipal grantorPrincipal, boolean grantOption)
+      throws HiveAuthorizationPluginException;
+
 
   /**
    * Create role
    * @param roleName
    * @param adminGrantor - The user in "[ WITH ADMIN <user> ]" clause of "create role"
+   * @throws HiveAuthorizationPluginException
    */
-  void createRole(String roleName, HivePrincipal adminGrantor);
+  void createRole(String roleName, HivePrincipal adminGrantor)
+      throws HiveAuthorizationPluginException;
 
   /**
    * Drop role
    * @param roleName
+   * @throws HiveAuthorizationPluginException
    */
-  void dropRole(String roleName);
+  void dropRole(String roleName)
+      throws HiveAuthorizationPluginException;
 
   /**
    * Get roles that this user/role belongs to
    * @param hivePrincipal - user or role
    * @return list of roles
+   * @throws HiveAuthorizationPluginException
    */
-  List<String> getRoles(HivePrincipal hivePrincipal);
+  List<String> getRoles(HivePrincipal hivePrincipal)
+      throws HiveAuthorizationPluginException;
 
   /**
    * Grant roles in given roles list to principals in given hivePrincipals list
@@ -85,9 +103,12 @@ public interface HiveAuthorizer {
    * @param roles
    * @param grantOption
    * @param grantorPrinc
+   * @throws HiveAuthorizationPluginException
    */
   void grantRole(List<HivePrincipal> hivePrincipals, List<String> roles, boolean grantOption,
-      HivePrincipal grantorPrinc);
+      HivePrincipal grantorPrinc)
+      throws HiveAuthorizationPluginException;
+
 
   /**
    * Revoke roles in given roles list to principals in given hivePrincipals list
@@ -95,29 +116,43 @@ public interface HiveAuthorizer {
    * @param roles
    * @param grantOption
    * @param grantorPrinc
+   * @throws HiveAuthorizationPluginException
    */
   void revokeRole(List<HivePrincipal> hivePrincipals, List<String> roles, boolean grantOption,
-      HivePrincipal grantorPrinc);
+      HivePrincipal grantorPrinc)
+      throws HiveAuthorizationPluginException;
 
   /**
    * Check if user has privileges to do this action on these objects
    * @param hiveOpType
    * @param inputsHObjs
    * @param outputHObjs
+   * @throws HiveAuthorizationPluginException
    */
   void checkPrivileges(HiveOperationType hiveOpType, List<HivePrivilegeObject> inputsHObjs,
-      List<HivePrivilegeObject> outputHObjs);
+      List<HivePrivilegeObject> outputHObjs)
+      throws HiveAuthorizationPluginException;
 
   /**
    * @return all existing roles
+   * @throws HiveAuthorizationPluginException
+   */
+  List<String> getAllRoles()
+      throws HiveAuthorizationPluginException;
+
+  /**
+   * Show privileges for given principal on given object
+   * @param principal
+   * @param privObj
+   * @return
+   * @throws HiveAuthorizationPluginException
    */
-  List<String> getAllRoles();
+  List<HivePrivilegeInfo> showPrivileges(HivePrincipal principal, HivePrivilegeObject privObj)
+      throws HiveAuthorizationPluginException;
 
 
   //other functions to be added -
-  //showAllRoles()
   //showUsersInRole(rolename)
-  //showgrants(username)
   //isSuperuser(username)
 
 

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizerImpl.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizerImpl.java?rev=1562165&r1=1562164&r2=1562165&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizerImpl.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizerImpl.java Tue Jan 28 19:24:45 2014
@@ -34,7 +34,7 @@ public class HiveAuthorizerImpl implemen
   HiveAccessController accessController;
   HiveAuthorizationValidator authValidator;
 
-   HiveAuthorizerImpl(HiveAccessController accessController, HiveAuthorizationValidator authValidator){
+   public HiveAuthorizerImpl(HiveAccessController accessController, HiveAuthorizationValidator authValidator){
      this.accessController = accessController;
      this.authValidator = authValidator;
    }
@@ -42,7 +42,7 @@ public class HiveAuthorizerImpl implemen
   @Override
   public void grantPrivileges(List<HivePrincipal> hivePrincipals,
       List<HivePrivilege> hivePrivileges, HivePrivilegeObject hivePrivObject,
-      HivePrincipal grantorPrincipal, boolean grantOption) {
+      HivePrincipal grantorPrincipal, boolean grantOption) throws HiveAuthorizationPluginException {
     accessController.grantPrivileges(hivePrincipals, hivePrivileges, hivePrivObject,
         grantorPrincipal, grantOption);
   }
@@ -50,49 +50,60 @@ public class HiveAuthorizerImpl implemen
   @Override
   public void revokePrivileges(List<HivePrincipal> hivePrincipals,
       List<HivePrivilege> hivePrivileges, HivePrivilegeObject hivePrivObject,
-      HivePrincipal grantorPrincipal, boolean grantOption) {
+      HivePrincipal grantorPrincipal, boolean grantOption) throws HiveAuthorizationPluginException {
     accessController.revokePrivileges(hivePrincipals, hivePrivileges, hivePrivObject,
         grantorPrincipal, grantOption);
   }
 
   @Override
-  public void createRole(String roleName, HivePrincipal adminGrantor) {
+  public void createRole(String roleName, HivePrincipal adminGrantor) throws HiveAuthorizationPluginException {
     accessController.createRole(roleName, adminGrantor);
   }
 
   @Override
-  public void dropRole(String roleName) {
+  public void dropRole(String roleName) throws HiveAuthorizationPluginException {
     accessController.dropRole(roleName);
   }
 
   @Override
-  public List<String> getRoles(HivePrincipal hivePrincipal) {
+  public List<String> getRoles(HivePrincipal hivePrincipal) throws HiveAuthorizationPluginException {
     return accessController.getRoles(hivePrincipal);
   }
 
   @Override
   public void grantRole(List<HivePrincipal> hivePrincipals, List<String> roles,
-      boolean grantOption, HivePrincipal grantorPrinc) {
+      boolean grantOption, HivePrincipal grantorPrinc) throws HiveAuthorizationPluginException {
     accessController.grantRole(hivePrincipals, roles, grantOption, grantorPrinc);
   }
 
   @Override
   public void revokeRole(List<HivePrincipal> hivePrincipals, List<String> roles,
-      boolean grantOption, HivePrincipal grantorPrinc) {
+      boolean grantOption, HivePrincipal grantorPrinc) throws HiveAuthorizationPluginException {
     accessController.revokeRole(hivePrincipals, roles, grantOption, grantorPrinc);
   }
 
   @Override
   public void checkPrivileges(HiveOperationType hiveOpType, List<HivePrivilegeObject> inputHObjs,
-      List<HivePrivilegeObject> outputHObjs) {
+      List<HivePrivilegeObject> outputHObjs) throws HiveAuthorizationPluginException {
     authValidator.checkPrivileges(hiveOpType, inputHObjs, outputHObjs);
   }
 
   @Override
-  public List<String> getAllRoles() {
+  public List<String> getAllRoles() throws HiveAuthorizationPluginException {
     return accessController.getAllRoles();
   }
 
+  @Override
+  public List<HivePrivilegeInfo> showPrivileges(HivePrincipal principal,
+      HivePrivilegeObject privObj) throws HiveAuthorizationPluginException {
+    return accessController.showPrivileges(principal, privObj);
+  }
+
+  @Override
+  public VERSION getVersion() {
+    return VERSION.V1;
+  }
+
 
  // other access control functions
 

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HivePrivilegeInfo.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HivePrivilegeInfo.java?rev=1562165&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HivePrivilegeInfo.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HivePrivilegeInfo.java Tue Jan 28 19:24:45 2014
@@ -0,0 +1,66 @@
+/**
+ * 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;
+
+import org.apache.hadoop.hive.common.classification.InterfaceAudience.Public;
+import org.apache.hadoop.hive.common.classification.InterfaceStability.Evolving;
+
+/**
+ * Represents a privilege granted for an object to a principal
+ */
+@Public
+@Evolving
+
+public class HivePrivilegeInfo{
+  private final HivePrincipal principal;
+  private final HivePrivilege privilege;
+  private final HivePrivilegeObject object;
+  private final HivePrincipal grantorPrincipal;
+  private final boolean grantOption;
+
+  public HivePrivilegeInfo(HivePrincipal principal, HivePrivilege privilege,
+      HivePrivilegeObject object, HivePrincipal grantorPrincipal, boolean grantOption){
+    this.principal = principal;
+    this.privilege = privilege;
+    this.object = object;
+    this.grantorPrincipal = grantorPrincipal;
+    this.grantOption = grantOption;
+  }
+
+  public HivePrincipal getPrincipal() {
+    return principal;
+  }
+
+  public HivePrivilege getPrivilege() {
+    return privilege;
+  }
+
+  public HivePrivilegeObject getObject() {
+    return object;
+  }
+
+  public HivePrincipal getGrantorPrincipal() {
+    return grantorPrincipal;
+  }
+
+  public boolean isGrantOption() {
+    return grantOption;
+  }
+
+
+}
\ No newline at end of file

Added: 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=1562165&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessController.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAccessController.java Tue Jan 28 19:24:45 2014
@@ -0,0 +1,307 @@
+/**
+ * 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.List;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.hive.conf.HiveConf;
+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.PrivilegeBag;
+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.authorization.AuthorizationUtils;
+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.HiveMetastoreClientFactory;
+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.HivePrivilegeInfo;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType;
+
+
+/**
+ * Implements functionality of access control statements for sql standard based authorization
+ */
+@Private
+public class SQLStdHiveAccessController implements HiveAccessController {
+
+  private HiveMetastoreClientFactory metastoreClientFactory;
+
+
+  SQLStdHiveAccessController(HiveMetastoreClientFactory metastoreClientFactory,
+      HiveConf conf, String hiveCurrentUser){
+    this.metastoreClientFactory = metastoreClientFactory;
+  }
+
+
+  @Override
+  public void grantPrivileges(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().grant_privileges(privBag);
+    } catch (Exception e) {
+      throw new HiveAuthorizationPluginException("Error granting privileges", e);
+    }
+  }
+
+  /**
+   * Create thrift privileges bag
+   * @param hivePrincipals
+   * @param hivePrivileges
+   * @param hivePrivObject
+   * @param grantorPrincipal
+   * @param grantOption
+   * @return
+   * @throws HiveAuthorizationPluginException
+   */
+  private PrivilegeBag getThriftPrivilegesBag(List<HivePrincipal> hivePrincipals,
+      List<HivePrivilege> hivePrivileges, HivePrivilegeObject hivePrivObject,
+      HivePrincipal grantorPrincipal, boolean grantOption) throws HiveAuthorizationPluginException {
+    HiveObjectRef privObj = 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"
+            + " 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;
+  }
+
+  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 {
+    try {
+      return  AuthorizationUtils.getThriftHiveObjectRef(privObj);
+    } catch (HiveException e) {
+      throw new HiveAuthorizationPluginException(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);
+    } catch (Exception e) {
+      throw new HiveAuthorizationPluginException("Error revoking privileges", e);
+    }
+  }
+
+  @Override
+  public void createRole(String roleName, HivePrincipal adminGrantor)
+      throws HiveAuthorizationPluginException {
+    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);
+    }
+  }
+
+  @Override
+  public void dropRole(String roleName) throws HiveAuthorizationPluginException {
+    try {
+      metastoreClientFactory.getHiveMetastoreClient().drop_role(roleName);
+    } catch (Exception e) {
+      throw new HiveAuthorizationPluginException("Error dropping role", e);
+    }
+  }
+
+  @Override
+  public List<String> getRoles(HivePrincipal hivePrincipal) throws HiveAuthorizationPluginException {
+    try {
+      List<Role> roles = metastoreClientFactory.getHiveMetastoreClient().list_roles(
+          hivePrincipal.getName(), AuthorizationUtils.getThriftPrincipalType(hivePrincipal.getType()));
+      List<String> roleNames = new ArrayList<String>(roles.size());
+      for(Role role : roles){
+        roleNames.add(role.getRoleName());
+      }
+      return roleNames;
+    } catch (Exception e) {
+      throw new HiveAuthorizationPluginException(
+          "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){
+        try {
+          IMetaStoreClient mClient = metastoreClientFactory.getHiveMetastoreClient();
+          mClient.grant_role(roleName,
+              hivePrincipal.getName(),
+              AuthorizationUtils.getThriftPrincipalType(hivePrincipal.getType()),
+              grantorPrinc.getName(),
+              AuthorizationUtils.getThriftPrincipalType(grantorPrinc.getType()),
+              grantOption
+              );
+        }  catch (Exception e) {
+          String msg = "Error granting roles for " + hivePrincipal.getName() +  " to role " + roleName
+              + hivePrincipal.getName();
+          throw new HiveAuthorizationPluginException(msg, e);
+        }
+      }
+    }
+  }
+
+  @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 "
+          + "role is not currently supported");
+    }
+    for(HivePrincipal hivePrincipal : hivePrincipals){
+      for(String roleName : roleNames){
+        try {
+          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
+              + hivePrincipal.getName();
+          throw new HiveAuthorizationPluginException(msg, e);
+        }
+      }
+    }
+  }
+
+  @Override
+  public List<String> getAllRoles() throws HiveAuthorizationPluginException {
+    try {
+      return metastoreClientFactory.getHiveMetastoreClient().listRoleNames();
+    } catch (Exception e) {
+      throw new HiveAuthorizationPluginException("Error listing all roles", e);
+    }
+  }
+
+
+  @Override
+  public List<HivePrivilegeInfo> showPrivileges(HivePrincipal principal, HivePrivilegeObject privObj)
+      throws HiveAuthorizationPluginException {
+    try {
+
+      List<HivePrivilegeInfo> resPrivInfos = new ArrayList<HivePrivilegeInfo>();
+      IMetaStoreClient mClient = metastoreClientFactory.getHiveMetastoreClient();
+
+      //get metastore/thrift privilege object using metastore api
+      List<HiveObjectPrivilege> msObjPrivs
+        = mClient.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
+        PrivilegeGrantInfo msGrantInfo = msObjPriv.getGrantInfo();
+        HivePrivilege resPrivilege = new HivePrivilege(msGrantInfo.getPrivilege(), null);
+
+        //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()));
+
+
+        HivePrivilegeInfo resPrivInfo = new HivePrivilegeInfo(resPrincipal, resPrivilege,
+            resPrivObj, grantorPrincipal, msGrantInfo.isGrantOption());
+        resPrivInfos.add(resPrivInfo);
+      }
+      return resPrivInfos;
+
+    }
+    catch (Exception e) {
+      throw new HiveAuthorizationPluginException("Error showing privileges", e);
+    }
+
+  }
+
+
+  private HivePrivilegeObjectType getPluginObjType(HiveObjectType objectType)
+      throws HiveAuthorizationPluginException {
+    switch(objectType){
+    case DATABASE:
+      return HivePrivilegeObjectType.DATABASE;
+    case TABLE:
+      return HivePrivilegeObjectType.TABLE;
+    case COLUMN:
+    case GLOBAL:
+    case PARTITION:
+      throw new HiveAuthorizationPluginException("Unsupported object type " + objectType);
+    default:
+      throw new AssertionError("Unexpected object type " + objectType);
+    }
+  }
+
+}

Added: 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=1562165&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizationValidator.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizationValidator.java Tue Jan 28 19:24:45 2014
@@ -0,0 +1,34 @@
+/**
+ * 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.List;
+
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthorizationPluginException;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthorizationValidator;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject;
+
+public class SQLStdHiveAuthorizationValidator implements HiveAuthorizationValidator {
+
+  @Override
+  public void checkPrivileges(HiveOperationType hiveOpType, List<HivePrivilegeObject> inputHObjs,
+      List<HivePrivilegeObject> outputHObjs) throws HiveAuthorizationPluginException {
+  }
+
+}

Added: 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=1562165&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizerFactory.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/SQLStdHiveAuthorizerFactory.java Tue Jan 28 19:24:45 2014
@@ -0,0 +1,37 @@
+/**
+ * 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.common.classification.InterfaceAudience.Private;
+import org.apache.hadoop.hive.conf.HiveConf;
+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.HiveMetastoreClientFactory;
+
+@Private
+public class SQLStdHiveAuthorizerFactory implements HiveAuthorizerFactory{
+  @Override
+  public HiveAuthorizer createHiveAuthorizer(HiveMetastoreClientFactory metastoreClientFactory,
+      HiveConf conf, String hiveCurrentUser) {
+    return new HiveAuthorizerImpl(
+        new SQLStdHiveAccessController(metastoreClientFactory, conf, hiveCurrentUser),
+        new SQLStdHiveAuthorizationValidator()
+        );
+  }
+}

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=1562165&r1=1562164&r2=1562165&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 Tue Jan 28 19:24:45 2014
@@ -54,7 +54,6 @@ 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.authorization.AuthorizationPreEventListener;
 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;
@@ -299,7 +298,10 @@ public class SessionState {
     // Get the following out of the way when you start the session these take a
     // while and should be done when we start up.
     try {
-      Hive.get(startSs.conf).getMSC();
+      //Hive object instance should be created with a copy of the conf object. If the conf is
+      // shared with SessionState, other parts of the code might update the config, but
+      // Hive.get(HiveConf) would not recognize the case when it needs refreshing
+      Hive.get(new HiveConf(startSs.conf)).getMSC();
       ShimLoader.getHadoopShims().getUGIForConf(startSs.conf);
       FileSystem.get(startSs.conf);
     } catch (Exception e) {
@@ -307,7 +309,20 @@ public class SessionState {
       // that would cause ClassNoFoundException otherwise
       throw new RuntimeException(e);
     }
-    setupAuth(startSs);
+
+    if (HiveConf.getVar(startSs.getConf(), HiveConf.ConfVars.HIVE_EXECUTION_ENGINE)
+        .equals("tez")) {
+      try {
+        if (startSs.tezSessionState == null) {
+          startSs.tezSessionState = new TezSessionState();
+        }
+        startSs.tezSessionState.open(startSs.getSessionId(), startSs.conf);
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    } else {
+       LOG.info("No Tez session required at this point. hive.execution.engine=mr.");
+    }
     return startSs;
   }
 
@@ -315,41 +330,38 @@ public class SessionState {
    * Setup authentication and authorization plugins for this session.
    * @param startSs
    */
-  private static void setupAuth(SessionState startSs) {
+  private void setupAuth() {
+
+    if(authenticator != null){
+      //auth has been initialized
+      return;
+    }
+
     try {
-      startSs.authenticator = HiveUtils.getAuthenticator(
-          startSs.getConf(),HiveConf.ConfVars.HIVE_AUTHENTICATOR_MANAGER);
-      startSs.authorizer = HiveUtils.getAuthorizeProviderManager(
-          startSs.getConf(), HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER,
-          startSs.authenticator, true);
+        authenticator = HiveUtils.getAuthenticator(
+          getConf(),HiveConf.ConfVars.HIVE_AUTHENTICATOR_MANAGER);
+      authorizer = HiveUtils.getAuthorizeProviderManager(
+          getConf(), HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER,
+          authenticator, true);
 
-      if(startSs.authorizer == null){
+      if(authorizer == null){
         //if it was null, the new authorization plugin must be specified in config
         HiveAuthorizerFactory authorizerFactory =
-            HiveUtils.getAuthorizerFactory(startSs.getConf(), HiveConf.ConfVars.HIVE_AUTHENTICATOR_MANAGER);
-        startSs.authorizerV2 = authorizerFactory.createHiveAuthorizer(new HiveMetastoreClientFactoryImpl(),
-            startSs.getConf(), startSs.authenticator.getUserName());
+            HiveUtils.getAuthorizerFactory(getConf(), HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER);
+        authorizerV2 = authorizerFactory.createHiveAuthorizer(new HiveMetastoreClientFactoryImpl(),
+            getConf(), authenticator.getUserName());
       }
       else{
-        startSs.createTableGrants = CreateTableAutomaticGrant.create(startSs
-            .getConf());
+        createTableGrants = CreateTableAutomaticGrant.create(getConf());
       }
     } catch (HiveException e) {
       throw new RuntimeException(e);
     }
 
-    if (HiveConf.getVar(startSs.getConf(), HiveConf.ConfVars.HIVE_EXECUTION_ENGINE)
-        .equals("tez")) {
-      try {
-        if (startSs.tezSessionState == null) {
-          startSs.tezSessionState = new TezSessionState();
-        }
-        startSs.tezSessionState.open(startSs.getSessionId(), startSs.conf);
-      } catch (Exception e) {
-        throw new RuntimeException(e);
-      }
-    } else {
-       LOG.info("No Tez session required at this point. hive.execution.engine=mr.");
+    if(LOG.isDebugEnabled()){
+      Object authorizationClass = getAuthorizationMode() == AuthorizationMode.V1 ?
+          getAuthorizer() : getAuthorizerV2();
+      LOG.debug("Session is using authorization class " + authorizationClass.getClass());
     }
     return;
   }
@@ -777,6 +789,7 @@ public class SessionState {
   }
 
   public HiveAuthorizationProvider getAuthorizer() {
+    setupAuth();
     return authorizer;
   }
 
@@ -785,10 +798,12 @@ public class SessionState {
   }
 
   public HiveAuthorizer getAuthorizerV2() {
+    setupAuth();
     return authorizerV2;
   }
 
   public HiveAuthenticationProvider getAuthenticator() {
+    setupAuth();
     return authenticator;
   }
 
@@ -882,6 +897,7 @@ public class SessionState {
   }
 
   public AuthorizationMode getAuthorizationMode(){
+    setupAuth();
     if(authorizer != null){
       return AuthorizationMode.V1;
     }else if(authorizerV2 != null){

Added: 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=1562165&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/authorization_1_sql_std.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/authorization_1_sql_std.q Tue Jan 28 19:24:45 2014
@@ -0,0 +1,33 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+
+create table src_autho_test (key STRING, value STRING) ;
+
+set hive.security.authorization.enabled=true;
+
+--table grant to user
+
+grant select on table src_autho_test to user hive_test_user;
+
+show grant user hive_test_user 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;
+
+--role
+create role src_role;
+grant role src_role to user hive_test_user;
+show role grant user hive_test_user;
+
+--table grant to role
+
+grant select on table src_autho_test to role src_role;
+
+show grant role src_role on table src_autho_test;
+revoke select on table src_autho_test from role src_role;
+
+-- drop role
+drop role src_role;
+
+set hive.security.authorization.enabled=false;
+drop table src_autho_test;
\ No newline at end of file

Modified: hive/trunk/ql/src/test/queries/clientpositive/authorization_role_grant1.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/authorization_role_grant1.q?rev=1562165&r1=1562164&r2=1562165&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/authorization_role_grant1.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/authorization_role_grant1.q Tue Jan 28 19:24:45 2014
@@ -1,11 +1,16 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+-- enable sql standard authorization
+
 -- role granting without role keyword
 create role src_role2;
 grant  src_role2 to user user2 ;
 show role grant user user2;
+show roles;
 
 -- revoke role without role keyword
 revoke src_role2 from user user2;
 show role grant user user2;
+show roles;
 
 ----------------------------------------
 -- role granting without role keyword, with admin option (syntax check)
@@ -18,3 +23,12 @@ show role grant user user2;
 -- revoke role without role keyword
 revoke src_role_wadmin from user user2 with admin option;
 show role grant user user2;
+
+
+
+-- drop roles
+show roles;
+drop role src_role2;
+show roles;
+drop role src_role_wadmin;
+show roles;

Added: hive/trunk/ql/src/test/results/clientpositive/authorization_1_sql_std.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/authorization_1_sql_std.q.out?rev=1562165&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/authorization_1_sql_std.q.out (added)
+++ hive/trunk/ql/src/test/results/clientpositive/authorization_1_sql_std.q.out Tue Jan 28 19:24:45 2014
@@ -0,0 +1,93 @@
+PREHOOK: query: create table src_autho_test (key STRING, value STRING)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table src_autho_test (key STRING, value STRING)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@src_autho_test
+PREHOOK: query: --table grant to user
+
+grant select on table src_autho_test to user hive_test_user
+PREHOOK: type: GRANT_PRIVILEGE
+PREHOOK: Output: default@src_autho_test
+POSTHOOK: query: --table grant to user
+
+grant select on table src_autho_test to user hive_test_user
+POSTHOOK: type: GRANT_PRIVILEGE
+POSTHOOK: Output: default@src_autho_test
+PREHOOK: query: show grant user hive_test_user on table src_autho_test
+PREHOOK: type: SHOW_GRANT
+POSTHOOK: query: show grant user hive_test_user on table src_autho_test
+POSTHOOK: type: SHOW_GRANT
+database	default
+table	src_autho_test
+principalName	hive_test_user
+principalType	USER
+privilege	Select
+#### A masked pattern was here ####
+grantor	hive_test_user
+PREHOOK: query: revoke select on table src_autho_test from user hive_test_user
+PREHOOK: type: REVOKE_PRIVILEGE
+PREHOOK: Output: default@src_autho_test
+POSTHOOK: query: revoke select on table src_autho_test from user hive_test_user
+POSTHOOK: type: REVOKE_PRIVILEGE
+POSTHOOK: Output: default@src_autho_test
+PREHOOK: query: show grant user hive_test_user on table src_autho_test
+PREHOOK: type: SHOW_GRANT
+POSTHOOK: query: show grant user hive_test_user on table src_autho_test
+POSTHOOK: type: SHOW_GRANT
+PREHOOK: query: --role
+create role src_role
+PREHOOK: type: CREATEROLE
+POSTHOOK: query: --role
+create role src_role
+POSTHOOK: type: CREATEROLE
+PREHOOK: query: grant role src_role to user hive_test_user
+PREHOOK: type: GRANT_ROLE
+POSTHOOK: query: grant role src_role to user hive_test_user
+POSTHOOK: type: GRANT_ROLE
+PREHOOK: query: show role grant user hive_test_user
+PREHOOK: type: SHOW_ROLE_GRANT
+POSTHOOK: query: show role grant user hive_test_user
+POSTHOOK: type: SHOW_ROLE_GRANT
+src_role
+
+PREHOOK: query: --table grant to role
+
+grant select on table src_autho_test to role src_role
+PREHOOK: type: GRANT_PRIVILEGE
+PREHOOK: Output: default@src_autho_test
+POSTHOOK: query: --table grant to role
+
+grant select on table src_autho_test to role src_role
+POSTHOOK: type: GRANT_PRIVILEGE
+POSTHOOK: Output: default@src_autho_test
+PREHOOK: query: show grant role src_role on table src_autho_test
+PREHOOK: type: SHOW_GRANT
+POSTHOOK: query: show grant role src_role on table src_autho_test
+POSTHOOK: type: SHOW_GRANT
+database	default
+table	src_autho_test
+principalName	src_role
+principalType	ROLE
+privilege	Select
+#### A masked pattern was here ####
+grantor	hive_test_user
+PREHOOK: query: revoke select on table src_autho_test from role src_role
+PREHOOK: type: REVOKE_PRIVILEGE
+PREHOOK: Output: default@src_autho_test
+POSTHOOK: query: revoke select on table src_autho_test from role src_role
+POSTHOOK: type: REVOKE_PRIVILEGE
+POSTHOOK: Output: default@src_autho_test
+PREHOOK: query: -- drop role
+drop role src_role
+PREHOOK: type: DROPROLE
+POSTHOOK: query: -- drop role
+drop role src_role
+POSTHOOK: type: DROPROLE
+PREHOOK: query: drop table src_autho_test
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@src_autho_test
+PREHOOK: Output: default@src_autho_test
+POSTHOOK: query: drop table src_autho_test
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@src_autho_test
+POSTHOOK: Output: default@src_autho_test

Modified: hive/trunk/ql/src/test/results/clientpositive/authorization_role_grant1.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/authorization_role_grant1.q.out?rev=1562165&r1=1562164&r2=1562165&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/authorization_role_grant1.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/authorization_role_grant1.q.out Tue Jan 28 19:24:45 2014
@@ -1,7 +1,11 @@
-PREHOOK: query: -- role granting without role keyword
+PREHOOK: query: -- enable sql standard authorization
+
+-- role granting without role keyword
 create role src_role2
 PREHOOK: type: CREATEROLE
-POSTHOOK: query: -- role granting without role keyword
+POSTHOOK: query: -- enable sql standard authorization
+
+-- role granting without role keyword
 create role src_role2
 POSTHOOK: type: CREATEROLE
 PREHOOK: query: grant  src_role2 to user user2
@@ -13,6 +17,13 @@ PREHOOK: type: SHOW_ROLE_GRANT
 POSTHOOK: query: show role grant user user2
 POSTHOOK: type: SHOW_ROLE_GRANT
 src_role2
+
+PREHOOK: query: show roles
+PREHOOK: type: SHOW_ROLES
+POSTHOOK: query: show roles
+POSTHOOK: type: SHOW_ROLES
+src_role2
+
 PREHOOK: query: -- revoke role without role keyword
 revoke src_role2 from user user2
 PREHOOK: type: REVOKE_ROLE
@@ -23,6 +34,12 @@ PREHOOK: query: show role grant user use
 PREHOOK: type: SHOW_ROLE_GRANT
 POSTHOOK: query: show role grant user user2
 POSTHOOK: type: SHOW_ROLE_GRANT
+PREHOOK: query: show roles
+PREHOOK: type: SHOW_ROLES
+POSTHOOK: query: show roles
+POSTHOOK: type: SHOW_ROLES
+src_role2
+
 PREHOOK: query: ----------------------------------------
 -- role granting without role keyword, with admin option (syntax check)
 ----------------------------------------
@@ -44,6 +61,7 @@ PREHOOK: type: SHOW_ROLE_GRANT
 POSTHOOK: query: show role grant user user2
 POSTHOOK: type: SHOW_ROLE_GRANT
 src_role_wadmin
+
 PREHOOK: query: -- revoke role without role keyword
 revoke src_role_wadmin from user user2 with admin option
 PREHOOK: type: REVOKE_ROLE
@@ -54,3 +72,30 @@ PREHOOK: query: show role grant user use
 PREHOOK: type: SHOW_ROLE_GRANT
 POSTHOOK: query: show role grant user user2
 POSTHOOK: type: SHOW_ROLE_GRANT
+PREHOOK: query: -- drop roles
+show roles
+PREHOOK: type: SHOW_ROLES
+POSTHOOK: query: -- drop roles
+show roles
+POSTHOOK: type: SHOW_ROLES
+src_role2
+src_role_wadmin
+
+PREHOOK: query: drop role src_role2
+PREHOOK: type: DROPROLE
+POSTHOOK: query: drop role src_role2
+POSTHOOK: type: DROPROLE
+PREHOOK: query: show roles
+PREHOOK: type: SHOW_ROLES
+POSTHOOK: query: show roles
+POSTHOOK: type: SHOW_ROLES
+src_role_wadmin
+
+PREHOOK: query: drop role src_role_wadmin
+PREHOOK: type: DROPROLE
+POSTHOOK: query: drop role src_role_wadmin
+POSTHOOK: type: DROPROLE
+PREHOOK: query: show roles
+PREHOOK: type: SHOW_ROLES
+POSTHOOK: query: show roles
+POSTHOOK: type: SHOW_ROLES