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

svn commit: r1556253 - in /hive/trunk: common/src/java/org/apache/hadoop/hive/conf/ ql/src/java/org/apache/hadoop/hive/ql/exec/ ql/src/java/org/apache/hadoop/hive/ql/parse/ ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/ ql/src/test/org/apac...

Author: brock
Date: Tue Jan  7 15:55:00 2014
New Revision: 1556253

URL: http://svn.apache.org/r1556253
Log:
HIVE-5946 - DDL authorization task factory should be better tested (Brock reviewed by Thejas)

Added:
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/HiveAuthorizationTaskFactory.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/HiveAuthorizationTaskFactoryImpl.java
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestHiveAuthorizationTaskFactory.java
Modified:
    hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/TaskFactory.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java

Modified: hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
URL: http://svn.apache.org/viewvc/hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java?rev=1556253&r1=1556252&r2=1556253&view=diff
==============================================================================
--- hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (original)
+++ hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java Tue Jan  7 15:55:00 2014
@@ -712,7 +712,6 @@ public class HiveConf extends Configurat
     HIVECONFVALIDATION("hive.conf.validation", true),
 
     SEMANTIC_ANALYZER_HOOK("hive.semantic.analyzer.hook", ""),
-
     HIVE_AUTHORIZATION_ENABLED("hive.security.authorization.enabled", false),
     HIVE_AUTHORIZATION_MANAGER("hive.security.authorization.manager",
         "org.apache.hadoop.hive.ql.security.authorization.DefaultHiveAuthorizationProvider"),

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/TaskFactory.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/TaskFactory.java?rev=1556253&r1=1556252&r2=1556253&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/TaskFactory.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/TaskFactory.java Tue Jan  7 15:55:00 2014
@@ -52,42 +52,42 @@ public final class TaskFactory {
    *
    * @param <T>
    */
-  public static final class taskTuple<T extends Serializable> {
+  public static final class TaskTuple<T extends Serializable> {
     public Class<T> workClass;
     public Class<? extends Task<T>> taskClass;
 
-    public taskTuple(Class<T> workClass, Class<? extends Task<T>> taskClass) {
+    public TaskTuple(Class<T> workClass, Class<? extends Task<T>> taskClass) {
       this.workClass = workClass;
       this.taskClass = taskClass;
     }
   }
 
-  public static ArrayList<taskTuple<? extends Serializable>> taskvec;
+  public static ArrayList<TaskTuple<? extends Serializable>> taskvec;
   static {
-    taskvec = new ArrayList<taskTuple<? extends Serializable>>();
-    taskvec.add(new taskTuple<MoveWork>(MoveWork.class, MoveTask.class));
-    taskvec.add(new taskTuple<FetchWork>(FetchWork.class, FetchTask.class));
-    taskvec.add(new taskTuple<CopyWork>(CopyWork.class, CopyTask.class));
-    taskvec.add(new taskTuple<DDLWork>(DDLWork.class, DDLTask.class));
-    taskvec.add(new taskTuple<FunctionWork>(FunctionWork.class,
+    taskvec = new ArrayList<TaskTuple<? extends Serializable>>();
+    taskvec.add(new TaskTuple<MoveWork>(MoveWork.class, MoveTask.class));
+    taskvec.add(new TaskTuple<FetchWork>(FetchWork.class, FetchTask.class));
+    taskvec.add(new TaskTuple<CopyWork>(CopyWork.class, CopyTask.class));
+    taskvec.add(new TaskTuple<DDLWork>(DDLWork.class, DDLTask.class));
+    taskvec.add(new TaskTuple<FunctionWork>(FunctionWork.class,
         FunctionTask.class));
     taskvec
-        .add(new taskTuple<ExplainWork>(ExplainWork.class, ExplainTask.class));
-    taskvec.add(new taskTuple<ConditionalWork>(ConditionalWork.class,
+        .add(new TaskTuple<ExplainWork>(ExplainWork.class, ExplainTask.class));
+    taskvec.add(new TaskTuple<ConditionalWork>(ConditionalWork.class,
         ConditionalTask.class));
-    taskvec.add(new taskTuple<MapredWork>(MapredWork.class,
+    taskvec.add(new TaskTuple<MapredWork>(MapredWork.class,
                                           MapRedTask.class));
 
-    taskvec.add(new taskTuple<MapredLocalWork>(MapredLocalWork.class,
+    taskvec.add(new TaskTuple<MapredLocalWork>(MapredLocalWork.class,
         MapredLocalTask.class));
-    taskvec.add(new taskTuple<StatsWork>(StatsWork.class,
+    taskvec.add(new TaskTuple<StatsWork>(StatsWork.class,
         StatsTask.class));
-    taskvec.add(new taskTuple<ColumnStatsWork>(ColumnStatsWork.class, ColumnStatsTask.class));
-    taskvec.add(new taskTuple<MergeWork>(MergeWork.class,
+    taskvec.add(new TaskTuple<ColumnStatsWork>(ColumnStatsWork.class, ColumnStatsTask.class));
+    taskvec.add(new TaskTuple<MergeWork>(MergeWork.class,
         BlockMergeTask.class));
-    taskvec.add(new taskTuple<DependencyCollectionWork>(DependencyCollectionWork.class,
+    taskvec.add(new TaskTuple<DependencyCollectionWork>(DependencyCollectionWork.class,
         DependencyCollectionTask.class));
-    taskvec.add(new taskTuple<PartialScanWork>(PartialScanWork.class,
+    taskvec.add(new TaskTuple<PartialScanWork>(PartialScanWork.class,
         PartialScanTask.class));
 
   }
@@ -113,7 +113,7 @@ public final class TaskFactory {
   public static <T extends Serializable> Task<T> get(Class<T> workClass,
       HiveConf conf) {
 
-    for (taskTuple<? extends Serializable> t : taskvec) {
+    for (TaskTuple<? extends Serializable> t : taskvec) {
       if (t.workClass == workClass) {
         try {
           Task<T> ret = (Task<T>) t.taskClass.newInstance();

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java?rev=1556253&r1=1556252&r2=1556253&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java Tue Jan  7 15:55:00 2014
@@ -261,13 +261,16 @@ public abstract class BaseSemanticAnalyz
         }
       }
     }
-
   }
 
   public BaseSemanticAnalyzer(HiveConf conf) throws SemanticException {
+   this(conf, createHiveDB(conf));
+  }
+
+  public BaseSemanticAnalyzer(HiveConf conf, Hive db) throws SemanticException {
     try {
       this.conf = conf;
-      db = Hive.get(conf);
+      this.db = db;
       rootTasks = new ArrayList<Task<? extends Serializable>>();
       LOG = LogFactory.getLog(this.getClass().getName());
       console = new LogHelper(LOG);
@@ -279,6 +282,14 @@ public abstract class BaseSemanticAnalyz
     }
   }
 
+  protected static Hive createHiveDB(HiveConf conf) throws SemanticException {
+    try {
+      return Hive.get(conf);
+    } catch (HiveException e) {
+      throw new SemanticException(e);
+    }
+  }
+
   public HashMap<String, String> getIdToTableNameMap() {
     return idToTableNameMap;
   }
@@ -615,7 +626,7 @@ public abstract class BaseSemanticAnalyz
     return colList;
   }
 
-  protected List<String> getColumnNames(ASTNode ast) {
+  public static List<String> getColumnNames(ASTNode ast) {
     List<String> colList = new ArrayList<String>();
     int numCh = ast.getChildCount();
     for (int i = 0; i < numCh; i++) {

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java?rev=1556253&r1=1556252&r2=1556253&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java Tue Jan  7 15:55:00 2014
@@ -51,7 +51,6 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.api.Index;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Order;
-import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.hadoop.hive.metastore.api.SkewedInfo;
 import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.ql.ErrorMsg;
@@ -73,6 +72,8 @@ import org.apache.hadoop.hive.ql.metadat
 import org.apache.hadoop.hive.ql.metadata.HiveUtils;
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.parse.authorization.HiveAuthorizationTaskFactory;
+import org.apache.hadoop.hive.ql.parse.authorization.HiveAuthorizationTaskFactoryImpl;
 import org.apache.hadoop.hive.ql.plan.AddPartitionDesc;
 import org.apache.hadoop.hive.ql.plan.AlterDatabaseDesc;
 import org.apache.hadoop.hive.ql.plan.AlterIndexDesc;
@@ -92,8 +93,6 @@ import org.apache.hadoop.hive.ql.plan.Dr
 import org.apache.hadoop.hive.ql.plan.DropIndexDesc;
 import org.apache.hadoop.hive.ql.plan.DropTableDesc;
 import org.apache.hadoop.hive.ql.plan.FetchWork;
-import org.apache.hadoop.hive.ql.plan.GrantDesc;
-import org.apache.hadoop.hive.ql.plan.GrantRevokeRoleDDL;
 import org.apache.hadoop.hive.ql.plan.ListBucketingCtx;
 import org.apache.hadoop.hive.ql.plan.LoadTableDesc;
 import org.apache.hadoop.hive.ql.plan.LockDatabaseDesc;
@@ -102,11 +101,7 @@ import org.apache.hadoop.hive.ql.plan.Mo
 import org.apache.hadoop.hive.ql.plan.MsckDesc;
 import org.apache.hadoop.hive.ql.plan.PartitionSpec;
 import org.apache.hadoop.hive.ql.plan.PlanUtils;
-import org.apache.hadoop.hive.ql.plan.PrincipalDesc;
-import org.apache.hadoop.hive.ql.plan.PrivilegeDesc;
-import org.apache.hadoop.hive.ql.plan.PrivilegeObjectDesc;
 import org.apache.hadoop.hive.ql.plan.RenamePartitionDesc;
-import org.apache.hadoop.hive.ql.plan.RevokeDesc;
 import org.apache.hadoop.hive.ql.plan.RoleDDLDesc;
 import org.apache.hadoop.hive.ql.plan.ShowColumnsDesc;
 import org.apache.hadoop.hive.ql.plan.ShowCreateTableDesc;
@@ -125,8 +120,6 @@ import org.apache.hadoop.hive.ql.plan.Ta
 import org.apache.hadoop.hive.ql.plan.TruncateTableDesc;
 import org.apache.hadoop.hive.ql.plan.UnlockDatabaseDesc;
 import org.apache.hadoop.hive.ql.plan.UnlockTableDesc;
-import org.apache.hadoop.hive.ql.security.authorization.Privilege;
-import org.apache.hadoop.hive.ql.security.authorization.PrivilegeRegistry;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
@@ -145,6 +138,8 @@ public class DDLSemanticAnalyzer extends
   private static final Map<Integer, String> TokenToTypeName = new HashMap<Integer, String>();
 
   private final Set<String> reservedPartitionValues;
+  private final HiveAuthorizationTaskFactory hiveAuthorizationTaskFactory;
+
   static {
     TokenToTypeName.put(HiveParser.TOK_BOOLEAN, serdeConstants.BOOLEAN_TYPE_NAME);
     TokenToTypeName.put(HiveParser.TOK_TINYINT, serdeConstants.TINYINT_TYPE_NAME);
@@ -210,7 +205,11 @@ public class DDLSemanticAnalyzer extends
   }
 
   public DDLSemanticAnalyzer(HiveConf conf) throws SemanticException {
-    super(conf);
+    this(conf, createHiveDB(conf));
+  }
+
+  public DDLSemanticAnalyzer(HiveConf conf, Hive db) throws SemanticException {
+    super(conf, db);
     reservedPartitionValues = new HashSet<String>();
     // Partition can't have this name
     reservedPartitionValues.add(HiveConf.getVar(conf, ConfVars.DEFAULTPARTITIONNAME));
@@ -219,6 +218,7 @@ public class DDLSemanticAnalyzer extends
     reservedPartitionValues.add(HiveConf.getVar(conf, ConfVars.METASTORE_INT_ORIGINAL));
     reservedPartitionValues.add(HiveConf.getVar(conf, ConfVars.METASTORE_INT_ARCHIVED));
     reservedPartitionValues.add(HiveConf.getVar(conf, ConfVars.METASTORE_INT_EXTRACTED));
+    hiveAuthorizationTaskFactory = new HiveAuthorizationTaskFactoryImpl(conf, db);
   }
 
   @Override
@@ -453,248 +453,66 @@ public class DDLSemanticAnalyzer extends
     }
   }
 
-  private void analyzeGrantRevokeRole(boolean grant, ASTNode ast) {
-    List<PrincipalDesc> principalDesc = analyzePrincipalListDef(
-        (ASTNode) ast.getChild(0));
-
-    //check if admin option has been specified
-    int rolesStartPos = 1;
-    ASTNode wAdminOption = (ASTNode) ast.getChild(1);
-    if(wAdminOption.getToken().getType() == HiveParser.TOK_GRANT_WITH_ADMIN_OPTION){
-      rolesStartPos = 2; //start reading role names from next postion
-      //TODO: use the admin option
-    }
-
-    List<String> roles = new ArrayList<String>();
-    for (int i = rolesStartPos; i < ast.getChildCount(); i++) {
-      roles.add(unescapeIdentifier(ast.getChild(i).getText()));
-    }
-    String roleOwnerName = "";
-    if (SessionState.get() != null
-        && SessionState.get().getAuthenticator() != null) {
-      roleOwnerName = SessionState.get().getAuthenticator().getUserName();
+  private void analyzeGrantRevokeRole(boolean grant, ASTNode ast) throws SemanticException {
+    Task<? extends Serializable> task;
+    if(grant) {
+      task = hiveAuthorizationTaskFactory.createGrantRoleTask(ast, getInputs(), getOutputs());
+    } else {
+      task = hiveAuthorizationTaskFactory.createRevokeRoleTask(ast, getInputs(), getOutputs());
+    }
+    if(task != null) {
+      rootTasks.add(task);
     }
-    GrantRevokeRoleDDL grantRevokeRoleDDL = new GrantRevokeRoleDDL(grant,
-        roles, principalDesc, roleOwnerName, PrincipalType.USER, true);
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        grantRevokeRoleDDL), conf));
   }
 
   private void analyzeShowGrant(ASTNode ast) throws SemanticException {
-    PrivilegeObjectDesc privHiveObj = null;
-
-    ASTNode principal = (ASTNode) ast.getChild(0);
-    PrincipalType type = PrincipalType.USER;
-    switch (principal.getType()) {
-    case HiveParser.TOK_USER:
-      type = PrincipalType.USER;
-      break;
-    case HiveParser.TOK_GROUP:
-      type = PrincipalType.GROUP;
-      break;
-    case HiveParser.TOK_ROLE:
-      type = PrincipalType.ROLE;
-      break;
+    Task<? extends Serializable> task = hiveAuthorizationTaskFactory.
+        createShowGrantTask(ast, ctx.getResFile(), getInputs(), getOutputs());
+    if(task != null) {
+      rootTasks.add(task);
+      setFetchTask(createFetchTask(ShowGrantDesc.getSchema()));
     }
-    String principalName = unescapeIdentifier(principal.getChild(0).getText());
-    PrincipalDesc principalDesc = new PrincipalDesc(principalName, type);
-
-    List<String> cols = null;
-    if (ast.getChildCount() > 1) {
-      ASTNode child = (ASTNode) ast.getChild(1);
-      if (child.getToken().getType() == HiveParser.TOK_PRIV_OBJECT_COL) {
-        privHiveObj = new PrivilegeObjectDesc();
-        //set object name
-        privHiveObj.setObject(unescapeIdentifier(child.getChild(0).getText()));
-        //set object type
-        ASTNode objTypeNode = (ASTNode) child.getChild(1);
-        privHiveObj.setTable(objTypeNode.getToken().getType() == HiveParser.TOK_TABLE_TYPE);
-
-        //set col and partition spec if specified
-        if (child.getChildCount() > 2) {
-          for (int i = 2; i < child.getChildCount(); i++) {
-            ASTNode grandChild = (ASTNode) child.getChild(i);
-            if (grandChild.getToken().getType() == HiveParser.TOK_PARTSPEC) {
-              privHiveObj.setPartSpec(DDLSemanticAnalyzer.getPartSpec(grandChild));
-            } else if (grandChild.getToken().getType() == HiveParser.TOK_TABCOLNAME) {
-              cols = getColumnNames((ASTNode) grandChild);
-            }
-          }
-        }
-      }
-    }
-
-    if (privHiveObj == null && cols != null) {
-      throw new SemanticException(
-          "For user-level privileges, column sets should be null. columns="
-              + cols.toString());
-    }
-
-    ShowGrantDesc showGrant = new ShowGrantDesc(ctx.getResFile().toString(),
-        principalDesc, privHiveObj, cols);
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        showGrant), conf));
-    setFetchTask(createFetchTask(ShowGrantDesc.getSchema()));
   }
 
   private void analyzeGrant(ASTNode ast) throws SemanticException {
-    List<PrivilegeDesc> privilegeDesc = analyzePrivilegeListDef(
-        (ASTNode) ast.getChild(0));
-    List<PrincipalDesc> principalDesc = analyzePrincipalListDef(
-        (ASTNode) ast.getChild(1));
-    boolean grantOption = false;
-    PrivilegeObjectDesc privilegeObj = null;
-
-    if (ast.getChildCount() > 2) {
-      for (int i = 2; i < ast.getChildCount(); i++) {
-        ASTNode astChild = (ASTNode) ast.getChild(i);
-        if (astChild.getType() == HiveParser.TOK_GRANT_WITH_OPTION) {
-          grantOption = true;
-        } else if (astChild.getType() == HiveParser.TOK_PRIV_OBJECT) {
-          privilegeObj = analyzePrivilegeObject(astChild, getOutputs());
-        }
-      }
-    }
-
-    String userName = null;
-    if (SessionState.get() != null
-        && SessionState.get().getAuthenticator() != null) {
-      userName = SessionState.get().getAuthenticator().getUserName();
+    Task<? extends Serializable> task = hiveAuthorizationTaskFactory.
+        createGrantTask(ast, getInputs(), getOutputs());
+    if(task != null) {
+      rootTasks.add(task);
     }
-
-    GrantDesc grantDesc = new GrantDesc(privilegeObj, privilegeDesc,
-        principalDesc, userName, PrincipalType.USER, grantOption);
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        grantDesc), conf));
   }
 
   private void analyzeRevoke(ASTNode ast) throws SemanticException {
-    List<PrivilegeDesc> privilegeDesc = analyzePrivilegeListDef(
-        (ASTNode) ast.getChild(0));
-    List<PrincipalDesc> principalDesc = analyzePrincipalListDef(
-        (ASTNode) ast.getChild(1));
-    PrivilegeObjectDesc hiveObj = null;
-    if (ast.getChildCount() > 2) {
-      ASTNode astChild = (ASTNode) ast.getChild(2);
-      hiveObj = analyzePrivilegeObject(astChild, getOutputs());
-    }
-
-    RevokeDesc revokeDesc = new RevokeDesc(privilegeDesc, principalDesc, hiveObj);
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        revokeDesc), conf));
-  }
-
-  private PrivilegeObjectDesc analyzePrivilegeObject(ASTNode ast,
-      HashSet<WriteEntity> outputs)
-      throws SemanticException {
-    PrivilegeObjectDesc subject = new PrivilegeObjectDesc();
-    //set object identifier
-    subject.setObject(unescapeIdentifier(ast.getChild(0).getText()));
-    //set object type
-    ASTNode objTypeNode =  (ASTNode) ast.getChild(1);
-    subject.setTable(objTypeNode.getToken().getType() == HiveParser.TOK_TABLE_TYPE);
-    if (ast.getChildCount() == 3) {
-      //if partition spec node is present, set partition spec
-      ASTNode partSpecNode = (ASTNode) ast.getChild(2);
-      subject.setPartSpec(DDLSemanticAnalyzer.getPartSpec(partSpecNode));
-    }
-
-    if (subject.getTable()) {
-      Table tbl = getTable(subject.getObject(), true);
-      if (subject.getPartSpec() != null) {
-        Partition part = getPartition(tbl, subject.getPartSpec(), true);
-        outputs.add(new WriteEntity(part));
-      } else {
-        outputs.add(new WriteEntity(tbl));
-      }
+    Task<? extends Serializable> task = hiveAuthorizationTaskFactory.
+        createRevokeTask(ast, getInputs(), getOutputs());
+    if(task != null) {
+      rootTasks.add(task);
     }
-
-    return subject;
   }
 
-  private List<PrincipalDesc> analyzePrincipalListDef(ASTNode node) {
-    List<PrincipalDesc> principalList = new ArrayList<PrincipalDesc>();
-
-    for (int i = 0; i < node.getChildCount(); i++) {
-      ASTNode child = (ASTNode) node.getChild(i);
-      PrincipalType type = null;
-      switch (child.getType()) {
-      case HiveParser.TOK_USER:
-        type = PrincipalType.USER;
-        break;
-      case HiveParser.TOK_GROUP:
-        type = PrincipalType.GROUP;
-        break;
-      case HiveParser.TOK_ROLE:
-        type = PrincipalType.ROLE;
-        break;
-      }
-      String principalName = unescapeIdentifier(child.getChild(0).getText());
-      PrincipalDesc principalDesc = new PrincipalDesc(principalName, type);
-      principalList.add(principalDesc);
+  private void analyzeCreateRole(ASTNode ast) throws SemanticException {
+    Task<? extends Serializable> task = hiveAuthorizationTaskFactory.
+        createCreateRoleTask(ast, getInputs(), getOutputs());
+    if(task != null) {
+      rootTasks.add(task);
     }
-
-    return principalList;
   }
 
-  private List<PrivilegeDesc> analyzePrivilegeListDef(ASTNode node)
-      throws SemanticException {
-    List<PrivilegeDesc> ret = new ArrayList<PrivilegeDesc>();
-    for (int i = 0; i < node.getChildCount(); i++) {
-      ASTNode privilegeDef = (ASTNode) node.getChild(i);
-      ASTNode privilegeType = (ASTNode) privilegeDef.getChild(0);
-      Privilege privObj = PrivilegeRegistry.getPrivilege(privilegeType.getType());
-
-      if (privObj == null) {
-        throw new SemanticException("undefined privilege " + privilegeType.getType());
-      }
-      List<String> cols = null;
-      if (privilegeDef.getChildCount() > 1) {
-        cols = getColumnNames((ASTNode) privilegeDef.getChild(1));
-      }
-      PrivilegeDesc privilegeDesc = new PrivilegeDesc(privObj, cols);
-      ret.add(privilegeDesc);
+  private void analyzeDropRole(ASTNode ast) throws SemanticException {
+    Task<? extends Serializable> task = hiveAuthorizationTaskFactory.
+        createDropRoleTask(ast, getInputs(), getOutputs());
+    if(task != null) {
+      rootTasks.add(task);
     }
-    return ret;
   }
 
-  private void analyzeCreateRole(ASTNode ast) {
-    String roleName = unescapeIdentifier(ast.getChild(0).getText());
-    RoleDDLDesc createRoleDesc = new RoleDDLDesc(roleName,
-        RoleDDLDesc.RoleOperation.CREATE_ROLE);
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        createRoleDesc), conf));
-  }
-
-  private void analyzeDropRole(ASTNode ast) {
-    String roleName = unescapeIdentifier(ast.getChild(0).getText());
-    RoleDDLDesc createRoleDesc = new RoleDDLDesc(roleName,
-        RoleDDLDesc.RoleOperation.DROP_ROLE);
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        createRoleDesc), conf));
-  }
-
-  private void analyzeShowRoleGrant(ASTNode ast) {
-    ASTNode child = (ASTNode) ast.getChild(0);
-    PrincipalType principalType = PrincipalType.USER;
-    switch (child.getType()) {
-    case HiveParser.TOK_USER:
-      principalType = PrincipalType.USER;
-      break;
-    case HiveParser.TOK_GROUP:
-      principalType = PrincipalType.GROUP;
-      break;
-    case HiveParser.TOK_ROLE:
-      principalType = PrincipalType.ROLE;
-      break;
+  private void analyzeShowRoleGrant(ASTNode ast) throws SemanticException {
+    Task<? extends Serializable> task = hiveAuthorizationTaskFactory.
+        createShowRoleGrantTask(ast, ctx.getResFile(), getInputs(), getOutputs());
+    if(task != null) {
+      rootTasks.add(task);
+      setFetchTask(createFetchTask(RoleDDLDesc.getSchema()));
     }
-    String principalName = unescapeIdentifier(child.getChild(0).getText());
-    RoleDDLDesc createRoleDesc = new RoleDDLDesc(principalName, principalType,
-        RoleDDLDesc.RoleOperation.SHOW_ROLE_GRANT, null);
-    createRoleDesc.setResFile(ctx.getResFile().toString());
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        createRoleDesc), conf));
-    setFetchTask(createFetchTask(RoleDDLDesc.getSchema()));
   }
 
   private void analyzeAlterDatabase(ASTNode ast) throws SemanticException {
@@ -2049,7 +1867,7 @@ public class DDLSemanticAnalyzer extends
     setFetchTask(createFetchTask(descDbDesc.getSchema()));
   }
 
-  private static HashMap<String, String> getPartSpec(ASTNode partspec)
+  public static HashMap<String, String> getPartSpec(ASTNode partspec)
       throws SemanticException {
     if (partspec == null) {
       return null;

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/HiveAuthorizationTaskFactory.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/HiveAuthorizationTaskFactory.java?rev=1556253&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/HiveAuthorizationTaskFactory.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/HiveAuthorizationTaskFactory.java Tue Jan  7 15:55:00 2014
@@ -0,0 +1,59 @@
+/**
+ * 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;
+
+import java.io.Serializable;
+import java.util.HashSet;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.exec.Task;
+import org.apache.hadoop.hive.ql.hooks.ReadEntity;
+import org.apache.hadoop.hive.ql.hooks.WriteEntity;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+
+/**
+ * HiveAuthorizationTaskFactory creates DDL authorization related
+ * tasks. Every method in this class may return null, indicating no task
+ * needs to be executed or can throw a SemanticException.
+ */
+public interface HiveAuthorizationTaskFactory {
+  public Task<? extends Serializable> createCreateRoleTask(ASTNode node, HashSet<ReadEntity> inputs,
+      HashSet<WriteEntity> outputs) throws SemanticException;
+
+  public Task<? extends Serializable> createDropRoleTask(ASTNode node, HashSet<ReadEntity> inputs,
+      HashSet<WriteEntity> outputs) throws SemanticException;
+
+  public Task<? extends Serializable> createShowRoleGrantTask(ASTNode node, Path resultFile,
+      HashSet<ReadEntity> inputs, HashSet<WriteEntity> outputs) throws SemanticException;
+
+  public Task<? extends Serializable> createGrantRoleTask(ASTNode node, HashSet<ReadEntity> inputs,
+      HashSet<WriteEntity> outputs) throws SemanticException;
+
+  public Task<? extends Serializable> createRevokeRoleTask(ASTNode node, HashSet<ReadEntity> inputs,
+      HashSet<WriteEntity> outputs) throws SemanticException;
+
+  public Task<? extends Serializable> createGrantTask(ASTNode node, HashSet<ReadEntity> inputs,
+      HashSet<WriteEntity> outputs) throws SemanticException;
+
+  public Task<? extends Serializable> createShowGrantTask(ASTNode node, Path resultFile, HashSet<ReadEntity> inputs,
+      HashSet<WriteEntity> outputs) throws SemanticException;
+
+  public Task<? extends Serializable> createRevokeTask(ASTNode node, HashSet<ReadEntity> inputs,
+      HashSet<WriteEntity> outputs) throws SemanticException;
+}

Added: 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=1556253&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/HiveAuthorizationTaskFactoryImpl.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/HiveAuthorizationTaskFactoryImpl.java Tue Jan  7 15:55:00 2014
@@ -0,0 +1,354 @@
+/**
+ * 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;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.PrincipalType;
+import org.apache.hadoop.hive.ql.ErrorMsg;
+import org.apache.hadoop.hive.ql.exec.Task;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.hooks.ReadEntity;
+import org.apache.hadoop.hive.ql.hooks.WriteEntity;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.Partition;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.DDLSemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.plan.DDLWork;
+import org.apache.hadoop.hive.ql.plan.GrantDesc;
+import org.apache.hadoop.hive.ql.plan.GrantRevokeRoleDDL;
+import org.apache.hadoop.hive.ql.plan.PrincipalDesc;
+import org.apache.hadoop.hive.ql.plan.PrivilegeDesc;
+import org.apache.hadoop.hive.ql.plan.PrivilegeObjectDesc;
+import org.apache.hadoop.hive.ql.plan.RevokeDesc;
+import org.apache.hadoop.hive.ql.plan.RoleDDLDesc;
+import org.apache.hadoop.hive.ql.plan.ShowGrantDesc;
+import org.apache.hadoop.hive.ql.security.authorization.Privilege;
+import org.apache.hadoop.hive.ql.security.authorization.PrivilegeRegistry;
+import org.apache.hadoop.hive.ql.session.SessionState;
+/**
+ * Default implementation of HiveAuthorizationTaskFactory
+ */
+@SuppressWarnings("unchecked")
+public class HiveAuthorizationTaskFactoryImpl implements HiveAuthorizationTaskFactory {
+
+  private final HiveConf conf;
+  private final Hive db;
+
+  public HiveAuthorizationTaskFactoryImpl(HiveConf conf, Hive db) {
+    this.conf = conf;
+    this.db = db;
+  }
+
+  @Override
+  public Task<? extends Serializable> createCreateRoleTask(ASTNode ast, HashSet<ReadEntity> inputs,
+      HashSet<WriteEntity> outputs) {
+    String roleName = BaseSemanticAnalyzer.unescapeIdentifier(ast.getChild(0).getText());
+    RoleDDLDesc roleDesc = new RoleDDLDesc(roleName, RoleDDLDesc.RoleOperation.CREATE_ROLE);
+    return TaskFactory.get(new DDLWork(inputs, outputs, roleDesc), conf);
+  }
+  @Override
+  public Task<? extends Serializable> createDropRoleTask(ASTNode ast, HashSet<ReadEntity> inputs,
+      HashSet<WriteEntity> outputs) {
+    String roleName = BaseSemanticAnalyzer.unescapeIdentifier(ast.getChild(0).getText());
+    RoleDDLDesc roleDesc = new RoleDDLDesc(roleName, RoleDDLDesc.RoleOperation.DROP_ROLE);
+    return TaskFactory.get(new DDLWork(inputs, outputs, roleDesc), conf);
+  }
+  @Override
+  public Task<? extends Serializable> createShowRoleGrantTask(ASTNode ast, Path resultFile,
+      HashSet<ReadEntity> inputs, HashSet<WriteEntity> outputs) {
+    ASTNode child = (ASTNode) ast.getChild(0);
+    PrincipalType principalType = PrincipalType.USER;
+    switch (child.getType()) {
+    case HiveParser.TOK_USER:
+      principalType = PrincipalType.USER;
+      break;
+    case HiveParser.TOK_GROUP:
+      principalType = PrincipalType.GROUP;
+      break;
+    case HiveParser.TOK_ROLE:
+      principalType = PrincipalType.ROLE;
+      break;
+    }
+    String principalName = BaseSemanticAnalyzer.unescapeIdentifier(child.getChild(0).getText());
+    RoleDDLDesc roleDesc = new RoleDDLDesc(principalName, principalType,
+        RoleDDLDesc.RoleOperation.SHOW_ROLE_GRANT, null);
+    roleDesc.setResFile(resultFile.toString());
+    return TaskFactory.get(new DDLWork(inputs, outputs,  roleDesc), conf);
+  }
+  @Override
+  public Task<? extends Serializable> createGrantTask(ASTNode ast, HashSet<ReadEntity> inputs,
+      HashSet<WriteEntity> outputs) throws SemanticException {
+    List<PrivilegeDesc> privilegeDesc = analyzePrivilegeListDef(
+        (ASTNode) ast.getChild(0));
+    List<PrincipalDesc> principalDesc = analyzePrincipalListDef(
+        (ASTNode) ast.getChild(1));
+    boolean grantOption = false;
+    PrivilegeObjectDesc privilegeObj = null;
+
+    if (ast.getChildCount() > 2) {
+      for (int i = 2; i < ast.getChildCount(); i++) {
+        ASTNode astChild = (ASTNode) ast.getChild(i);
+        if (astChild.getType() == HiveParser.TOK_GRANT_WITH_OPTION) {
+          grantOption = true;
+        } else if (astChild.getType() == HiveParser.TOK_PRIV_OBJECT) {
+          privilegeObj = analyzePrivilegeObject(astChild, outputs);
+        }
+      }
+    }
+
+    String userName = null;
+    if (SessionState.get() != null
+        && SessionState.get().getAuthenticator() != null) {
+      userName = SessionState.get().getAuthenticator().getUserName();
+    }
+
+    GrantDesc grantDesc = new GrantDesc(privilegeObj, privilegeDesc,
+        principalDesc, userName, PrincipalType.USER, grantOption);
+    return TaskFactory.get(new DDLWork(inputs, outputs, grantDesc), conf);
+  }
+  @Override
+  public Task<? extends Serializable> createRevokeTask(ASTNode ast, HashSet<ReadEntity> inputs,
+      HashSet<WriteEntity> outputs) throws SemanticException {
+    List<PrivilegeDesc> privilegeDesc = analyzePrivilegeListDef((ASTNode) ast.getChild(0));
+    List<PrincipalDesc> principalDesc = analyzePrincipalListDef((ASTNode) ast.getChild(1));
+    PrivilegeObjectDesc hiveObj = null;
+    if (ast.getChildCount() > 2) {
+      ASTNode astChild = (ASTNode) ast.getChild(2);
+      hiveObj = analyzePrivilegeObject(astChild, outputs);
+    }
+    RevokeDesc revokeDesc = new RevokeDesc(privilegeDesc, principalDesc, hiveObj);
+    return TaskFactory.get(new DDLWork(inputs, outputs, revokeDesc), conf);
+  }
+  @Override
+  public Task<? extends Serializable> createGrantRoleTask(ASTNode ast, HashSet<ReadEntity> inputs,
+      HashSet<WriteEntity> outputs) {
+    return analyzeGrantRevokeRole(true, ast, inputs, outputs);
+  }
+  @Override
+  public Task<? extends Serializable> createShowGrantTask(ASTNode ast, Path resultFile, HashSet<ReadEntity> inputs,
+      HashSet<WriteEntity> outputs) throws SemanticException {
+    PrivilegeObjectDesc privHiveObj = null;
+
+    ASTNode principal = (ASTNode) ast.getChild(0);
+    PrincipalType type = PrincipalType.USER;
+    switch (principal.getType()) {
+    case HiveParser.TOK_USER:
+      type = PrincipalType.USER;
+      break;
+    case HiveParser.TOK_GROUP:
+      type = PrincipalType.GROUP;
+      break;
+    case HiveParser.TOK_ROLE:
+      type = PrincipalType.ROLE;
+      break;
+    }
+    String principalName = BaseSemanticAnalyzer.unescapeIdentifier(principal.getChild(0).getText());
+    PrincipalDesc principalDesc = new PrincipalDesc(principalName, type);
+
+    List<String> cols = null;
+    if (ast.getChildCount() > 1) {
+      ASTNode child = (ASTNode) ast.getChild(1);
+      if (child.getToken().getType() == HiveParser.TOK_PRIV_OBJECT_COL) {
+        privHiveObj = new PrivilegeObjectDesc();
+        //set object name
+        privHiveObj.setObject(BaseSemanticAnalyzer.unescapeIdentifier(child.getChild(0).getText()));
+        //set object type
+        ASTNode objTypeNode = (ASTNode) child.getChild(1);
+        privHiveObj.setTable(objTypeNode.getToken().getType() == HiveParser.TOK_TABLE_TYPE);
+
+        //set col and partition spec if specified
+        if (child.getChildCount() > 2) {
+          for (int i = 2; i < child.getChildCount(); i++) {
+            ASTNode grandChild = (ASTNode) child.getChild(i);
+            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);
+            }
+          }
+        }
+      }
+    }
+
+    if (privHiveObj == null && cols != null) {
+      throw new SemanticException(
+          "For user-level privileges, column sets should be null. columns="
+              + cols.toString());
+    }
+
+    ShowGrantDesc showGrant = new ShowGrantDesc(resultFile.toString(),
+        principalDesc, privHiveObj, cols);
+    return TaskFactory.get(new DDLWork(inputs, outputs, showGrant), conf);
+  }
+  @Override
+  public Task<? extends Serializable> createRevokeRoleTask(ASTNode ast, HashSet<ReadEntity> inputs,
+      HashSet<WriteEntity> outputs) {
+    return analyzeGrantRevokeRole(false, ast, inputs, outputs);
+  }
+  private Task<? extends Serializable> analyzeGrantRevokeRole(boolean isGrant, ASTNode ast,
+      HashSet<ReadEntity> inputs, HashSet<WriteEntity> outputs) {
+    List<PrincipalDesc> principalDesc = analyzePrincipalListDef(
+        (ASTNode) ast.getChild(0));
+
+    //check if admin option has been specified
+    int rolesStartPos = 1;
+    ASTNode wAdminOption = (ASTNode) ast.getChild(1);
+    if(wAdminOption.getToken().getType() == HiveParser.TOK_GRANT_WITH_ADMIN_OPTION){
+      rolesStartPos = 2; //start reading role names from next postion
+      //TODO: use the admin option
+    }
+
+    List<String> roles = new ArrayList<String>();
+    for (int i = rolesStartPos; i < ast.getChildCount(); i++) {
+      roles.add(BaseSemanticAnalyzer.unescapeIdentifier(ast.getChild(i).getText()));
+    }
+
+    String roleOwnerName = "";
+    if (SessionState.get() != null
+        && SessionState.get().getAuthenticator() != null) {
+      roleOwnerName = SessionState.get().getAuthenticator().getUserName();
+    }
+    GrantRevokeRoleDDL grantRevokeRoleDDL = new GrantRevokeRoleDDL(isGrant,
+        roles, principalDesc, roleOwnerName, PrincipalType.USER, true);
+    return TaskFactory.get(new DDLWork(inputs, outputs, grantRevokeRoleDDL), conf);
+  }
+
+  private PrivilegeObjectDesc analyzePrivilegeObject(ASTNode ast,
+      HashSet<WriteEntity> outputs)
+      throws SemanticException {
+
+    PrivilegeObjectDesc subject = new PrivilegeObjectDesc();
+    //set object identifier
+    subject.setObject(BaseSemanticAnalyzer.unescapeIdentifier(ast.getChild(0).getText()));
+    //set object type
+    ASTNode objTypeNode =  (ASTNode) ast.getChild(1);
+    subject.setTable(objTypeNode.getToken().getType() == HiveParser.TOK_TABLE_TYPE);
+    if (ast.getChildCount() == 3) {
+      //if partition spec node is present, set partition spec
+      ASTNode partSpecNode = (ASTNode) ast.getChild(2);
+      subject.setPartSpec(DDLSemanticAnalyzer.getPartSpec(partSpecNode));
+    }
+
+    if (subject.getTable()) {
+      Table tbl = getTable(SessionState.get().getCurrentDatabase(), subject.getObject());
+      if (subject.getPartSpec() != null) {
+        Partition part = getPartition(tbl, subject.getPartSpec());
+        outputs.add(new WriteEntity(part));
+      } else {
+        outputs.add(new WriteEntity(tbl));
+      }
+    }
+
+    return subject;
+  }
+
+  private List<PrincipalDesc> analyzePrincipalListDef(ASTNode node) {
+    List<PrincipalDesc> principalList = new ArrayList<PrincipalDesc>();
+
+    for (int i = 0; i < node.getChildCount(); i++) {
+      ASTNode child = (ASTNode) node.getChild(i);
+      PrincipalType type = null;
+      switch (child.getType()) {
+      case HiveParser.TOK_USER:
+        type = PrincipalType.USER;
+        break;
+      case HiveParser.TOK_GROUP:
+        type = PrincipalType.GROUP;
+        break;
+      case HiveParser.TOK_ROLE:
+        type = PrincipalType.ROLE;
+        break;
+      }
+      String principalName = BaseSemanticAnalyzer.unescapeIdentifier(child.getChild(0).getText());
+      PrincipalDesc principalDesc = new PrincipalDesc(principalName, type);
+      principalList.add(principalDesc);
+    }
+
+    return principalList;
+  }
+
+  private List<PrivilegeDesc> analyzePrivilegeListDef(ASTNode node)
+      throws SemanticException {
+    List<PrivilegeDesc> ret = new ArrayList<PrivilegeDesc>();
+    for (int i = 0; i < node.getChildCount(); i++) {
+      ASTNode privilegeDef = (ASTNode) node.getChild(i);
+      ASTNode privilegeType = (ASTNode) privilegeDef.getChild(0);
+      Privilege privObj = PrivilegeRegistry.getPrivilege(privilegeType.getType());
+
+      if (privObj == null) {
+        throw new SemanticException("undefined privilege " + privilegeType.getType());
+      }
+      List<String> cols = null;
+      if (privilegeDef.getChildCount() > 1) {
+        cols = BaseSemanticAnalyzer.getColumnNames((ASTNode) privilegeDef.getChild(1));
+      }
+      PrivilegeDesc privilegeDesc = new PrivilegeDesc(privObj, cols);
+      ret.add(privilegeDesc);
+    }
+    return ret;
+  }
+
+  private Table getTable(String database, String tblName)
+      throws SemanticException {
+    try {
+      Table tab = database == null ? db.getTable(tblName, false)
+          : db.getTable(database, tblName, false);
+      if (tab == null) {
+        throw new SemanticException(ErrorMsg.INVALID_TABLE.getMsg(tblName));
+      }
+      return tab;
+    } catch (HiveException e) {
+      if(e instanceof SemanticException) {
+        throw (SemanticException)e;
+      }
+      throw new SemanticException(ErrorMsg.INVALID_TABLE.getMsg(tblName), e);
+    }
+  }
+
+  private Partition getPartition(Table table, Map<String, String> partSpec)
+      throws SemanticException {
+    try {
+      Partition partition = db.getPartition(table, partSpec, false);
+      if (partition == null) {
+        throw new SemanticException(toMessage(ErrorMsg.INVALID_PARTITION, partSpec));
+      }
+      return partition;
+    } catch (HiveException e) {
+      if(e instanceof SemanticException) {
+        throw (SemanticException)e;
+      }
+      throw new SemanticException(toMessage(ErrorMsg.INVALID_PARTITION, partSpec), e);
+    }
+
+  }
+  private String toMessage(ErrorMsg message, Object detail) {
+    return detail == null ? message.getMsg() : message.getMsg(detail.toString());
+  }
+}

Added: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestHiveAuthorizationTaskFactory.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestHiveAuthorizationTaskFactory.java?rev=1556253&view=auto
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestHiveAuthorizationTaskFactory.java (added)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestHiveAuthorizationTaskFactory.java Tue Jan  7 15:55:00 2014
@@ -0,0 +1,441 @@
+/**
+ * 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;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.List;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.PrincipalType;
+import org.apache.hadoop.hive.ql.Context;
+import org.apache.hadoop.hive.ql.exec.Task;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.metadata.Partition;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.DDLSemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.ParseDriver;
+import org.apache.hadoop.hive.ql.parse.ParseUtils;
+import org.apache.hadoop.hive.ql.plan.DDLWork;
+import org.apache.hadoop.hive.ql.plan.GrantDesc;
+import org.apache.hadoop.hive.ql.plan.GrantRevokeRoleDDL;
+import org.apache.hadoop.hive.ql.plan.PrincipalDesc;
+import org.apache.hadoop.hive.ql.plan.PrivilegeDesc;
+import org.apache.hadoop.hive.ql.plan.RevokeDesc;
+import org.apache.hadoop.hive.ql.plan.RoleDDLDesc;
+import org.apache.hadoop.hive.ql.plan.RoleDDLDesc.RoleOperation;
+import org.apache.hadoop.hive.ql.plan.ShowGrantDesc;
+import org.apache.hadoop.hive.ql.security.HadoopDefaultAuthenticator;
+import org.apache.hadoop.hive.ql.security.authorization.Privilege;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+public class TestHiveAuthorizationTaskFactory {
+
+  private static final String SELECT = "SELECT";
+  private static final String DB = "default";
+  private static final String TABLE = "table1";
+  private static final String GROUP = "group1";
+  private static final String ROLE = "role1";
+  private static final String USER = "user1";
+
+  private ParseDriver parseDriver;
+  private DDLSemanticAnalyzer analyzer;
+  private HiveConf conf;
+  private Context context;
+  private String currentUser;
+  private Hive db;
+  private Table table;
+  private Partition partition;
+
+  @Before
+  public void setup() throws Exception {
+    conf = new HiveConf();
+    db = Mockito.mock(Hive.class);
+    table = new Table(DB, TABLE);
+    partition = new Partition(table);
+    context = new Context(conf);
+    parseDriver = new ParseDriver();
+    analyzer = new DDLSemanticAnalyzer(conf, db);
+    SessionState.start(conf);
+    Mockito.when(db.getTable(DB, TABLE, false)).thenReturn(table);
+    Mockito.when(db.getPartition(table, new HashMap<String, String>(), false))
+      .thenReturn(partition);
+    HadoopDefaultAuthenticator auth = new HadoopDefaultAuthenticator();
+    auth.setConf(conf);
+    currentUser = auth.getUserName();
+
+  }
+  /**
+   * CREATE ROLE ...
+   */
+  @Test
+  public void testCreateRole() throws Exception {
+    DDLWork work = analyze(parse("CREATE ROLE " + ROLE));
+    RoleDDLDesc roleDesc = work.getRoleDDLDesc();
+    Assert.assertNotNull("Role should not be null", roleDesc);
+    Assert.assertEquals(RoleOperation.CREATE_ROLE, roleDesc.getOperation());
+    Assert.assertFalse("Did not expect a group", roleDesc.getGroup());
+    Assert.assertEquals(ROLE, roleDesc.getName());
+  }
+  /**
+   * DROP ROLE ...
+   */
+  @Test
+  public void testDropRole() throws Exception {
+    DDLWork work = analyze(parse("DROp ROLE " + ROLE));
+    RoleDDLDesc roleDesc = work.getRoleDDLDesc();
+    Assert.assertNotNull("Role should not be null", roleDesc);
+    Assert.assertEquals(RoleOperation.DROP_ROLE, roleDesc.getOperation());
+    Assert.assertFalse("Did not expect a group", roleDesc.getGroup());
+    Assert.assertEquals(ROLE, roleDesc.getName());
+  }
+  /**
+   * GRANT ... ON TABLE ... TO USER ...
+   */
+  @Test
+  public void testGrantUserTable() throws Exception {
+    DDLWork work = analyze(parse("GRANT " + SELECT + " ON TABLE " + TABLE + " TO USER " + USER));
+    GrantDesc grantDesc = work.getGrantDesc();
+    Assert.assertNotNull("Grant should not be null", grantDesc);
+    for(PrincipalDesc principal : inList(grantDesc.getPrincipals()).ofSize(1)) {
+      Assert.assertEquals(PrincipalType.USER, principal.getType());
+      Assert.assertEquals(USER, principal.getName());
+    }
+    for(PrivilegeDesc privilege : inList(grantDesc.getPrivileges()).ofSize(1)) {
+      Assert.assertEquals(Privilege.SELECT, privilege.getPrivilege());
+    }
+    Assert.assertTrue("Expected table", grantDesc.getPrivilegeSubjectDesc().getTable());
+    Assert.assertEquals(TABLE, grantDesc.getPrivilegeSubjectDesc().getObject());
+  }
+  /**
+   * GRANT ... ON TABLE ... TO ROLE ...
+   */
+  @Test
+  public void testGrantRoleTable() throws Exception {
+    DDLWork work = analyze(parse("GRANT " + SELECT + " ON TABLE " + TABLE + " TO ROLE " + ROLE));
+    GrantDesc grantDesc = work.getGrantDesc();
+    Assert.assertNotNull("Grant should not be null", grantDesc);
+    for(PrincipalDesc principal : inList(grantDesc.getPrincipals()).ofSize(1)) {
+      Assert.assertEquals(PrincipalType.ROLE, principal.getType());
+      Assert.assertEquals(ROLE, principal.getName());
+    }
+    for(PrivilegeDesc privilege : inList(grantDesc.getPrivileges()).ofSize(1)) {
+      Assert.assertEquals(Privilege.SELECT, privilege.getPrivilege());
+    }
+    Assert.assertTrue("Expected table", grantDesc.getPrivilegeSubjectDesc().getTable());
+    Assert.assertEquals(TABLE, grantDesc.getPrivilegeSubjectDesc().getObject());
+  }
+  /**
+   * GRANT ... ON TABLE ... TO GROUP ...
+   */
+  @Test
+  public void testGrantGroupTable() throws Exception {
+    DDLWork work = analyze(parse("GRANT " + SELECT + " ON TABLE " + TABLE + " TO GROUP " + GROUP));
+    GrantDesc grantDesc = work.getGrantDesc();
+    Assert.assertNotNull("Grant should not be null", grantDesc);
+    for(PrincipalDesc principal : inList(grantDesc.getPrincipals()).ofSize(1)) {
+      Assert.assertEquals(PrincipalType.GROUP, principal.getType());
+      Assert.assertEquals(GROUP, principal.getName());
+    }
+    for(PrivilegeDesc privilege : inList(grantDesc.getPrivileges()).ofSize(1)) {
+      Assert.assertEquals(Privilege.SELECT, privilege.getPrivilege());
+    }
+    Assert.assertTrue("Expected table", grantDesc.getPrivilegeSubjectDesc().getTable());
+    Assert.assertEquals(TABLE, grantDesc.getPrivilegeSubjectDesc().getObject());
+  }
+  /**
+   * REVOKE ... ON TABLE ... FROM USER ...
+   */
+  @Test
+  public void testRevokeUserTable() throws Exception {
+    DDLWork work = analyze(parse("REVOKE " + SELECT + " ON TABLE " + TABLE + " FROM USER " + USER));
+    RevokeDesc grantDesc = work.getRevokeDesc();
+    Assert.assertNotNull("Revoke should not be null", grantDesc);
+    for(PrincipalDesc principal : inList(grantDesc.getPrincipals()).ofSize(1)) {
+      Assert.assertEquals(PrincipalType.USER, principal.getType());
+      Assert.assertEquals(USER, principal.getName());
+    }
+    for(PrivilegeDesc privilege : inList(grantDesc.getPrivileges()).ofSize(1)) {
+      Assert.assertEquals(Privilege.SELECT, privilege.getPrivilege());
+    }
+    Assert.assertTrue("Expected table", grantDesc.getPrivilegeSubjectDesc().getTable());
+    Assert.assertEquals(TABLE, grantDesc.getPrivilegeSubjectDesc().getObject());
+  }
+  /**
+   * REVOKE ... ON TABLE ... FROM ROLE ...
+   */
+  @Test
+  public void testRevokeRoleTable() throws Exception {
+    DDLWork work = analyze(parse("REVOKE " + SELECT + " ON TABLE " + TABLE + " FROM ROLE " + ROLE));
+    RevokeDesc grantDesc = work.getRevokeDesc();
+    Assert.assertNotNull("Revoke should not be null", grantDesc);
+    for(PrincipalDesc principal : inList(grantDesc.getPrincipals()).ofSize(1)) {
+      Assert.assertEquals(PrincipalType.ROLE, principal.getType());
+      Assert.assertEquals(ROLE, principal.getName());
+    }
+    for(PrivilegeDesc privilege : inList(grantDesc.getPrivileges()).ofSize(1)) {
+      Assert.assertEquals(Privilege.SELECT, privilege.getPrivilege());
+    }
+    Assert.assertTrue("Expected table", grantDesc.getPrivilegeSubjectDesc().getTable());
+    Assert.assertEquals(TABLE, grantDesc.getPrivilegeSubjectDesc().getObject());
+  }
+  /**
+   * REVOKE ... ON TABLE ... FROM GROUP ...
+   */
+  @Test
+  public void testRevokeGroupTable() throws Exception {
+    DDLWork work = analyze(parse("REVOKE " + SELECT + " ON TABLE " + TABLE + " FROM GROUP " + GROUP));
+    RevokeDesc grantDesc = work.getRevokeDesc();
+    Assert.assertNotNull("Revoke should not be null", grantDesc);
+    for(PrincipalDesc principal : inList(grantDesc.getPrincipals()).ofSize(1)) {
+      Assert.assertEquals(PrincipalType.GROUP, principal.getType());
+      Assert.assertEquals(GROUP, principal.getName());
+    }
+    for(PrivilegeDesc privilege : inList(grantDesc.getPrivileges()).ofSize(1)) {
+      Assert.assertEquals(Privilege.SELECT, privilege.getPrivilege());
+    }
+    Assert.assertTrue("Expected table", grantDesc.getPrivilegeSubjectDesc().getTable());
+    Assert.assertEquals(TABLE, grantDesc.getPrivilegeSubjectDesc().getObject());
+  }
+  /**
+   * GRANT ROLE ... TO USER ...
+   */
+  @Test
+  public void testGrantRoleUser() throws Exception {
+    DDLWork work = analyze(parse("GRANT ROLE " + ROLE + " TO USER " + USER));
+    GrantRevokeRoleDDL grantDesc = work.getGrantRevokeRoleDDL();
+    Assert.assertNotNull("Grant should not be null", grantDesc);
+    Assert.assertTrue("Expected grant ", grantDesc.getGrant());
+    Assert.assertTrue("Grant option is always true ", grantDesc.isGrantOption());
+    Assert.assertEquals(currentUser, grantDesc.getGrantor());
+    Assert.assertEquals(PrincipalType.USER, grantDesc.getGrantorType());
+    for(String role : inList(grantDesc.getRoles()).ofSize(1)) {
+      Assert.assertEquals(ROLE, role);
+    }
+    for(PrincipalDesc principal : inList(grantDesc.getPrincipalDesc()).ofSize(1)) {
+      Assert.assertEquals(PrincipalType.USER, principal.getType());
+      Assert.assertEquals(USER, principal.getName());
+    }
+  }
+  /**
+   * GRANT ROLE ... TO ROLE ...
+   */
+  @Test
+  public void testGrantRoleRole() throws Exception {
+    DDLWork work = analyze(parse("GRANT ROLE " + ROLE + " TO ROLE " + ROLE));
+    GrantRevokeRoleDDL grantDesc = work.getGrantRevokeRoleDDL();
+    Assert.assertNotNull("Grant should not be null", grantDesc);
+    Assert.assertTrue("Expected grant ", grantDesc.getGrant());
+    Assert.assertTrue("Grant option is always true ", grantDesc.isGrantOption());
+    Assert.assertEquals(currentUser, grantDesc.getGrantor());
+    Assert.assertEquals(PrincipalType.USER, grantDesc.getGrantorType());
+    for(String role : inList(grantDesc.getRoles()).ofSize(1)) {
+      Assert.assertEquals(ROLE, role);
+    }
+    for(PrincipalDesc principal : inList(grantDesc.getPrincipalDesc()).ofSize(1)) {
+      Assert.assertEquals(PrincipalType.ROLE, principal.getType());
+      Assert.assertEquals(ROLE, principal.getName());
+    }
+  }
+  /**
+   * GRANT ROLE ... TO GROUP ...
+   */
+  @Test
+  public void testGrantRoleGroup() throws Exception {
+    DDLWork work = analyze(parse("GRANT ROLE " + ROLE + " TO GROUP " + GROUP));
+    GrantRevokeRoleDDL grantDesc = work.getGrantRevokeRoleDDL();
+    Assert.assertNotNull("Grant should not be null", grantDesc);
+    Assert.assertTrue("Expected grant ", grantDesc.getGrant());
+    Assert.assertTrue("Grant option is always true ", grantDesc.isGrantOption());
+    Assert.assertEquals(currentUser, grantDesc.getGrantor());
+    Assert.assertEquals(PrincipalType.USER, grantDesc.getGrantorType());
+    for(String role : inList(grantDesc.getRoles()).ofSize(1)) {
+      Assert.assertEquals(ROLE, role);
+    }
+    for(PrincipalDesc principal : inList(grantDesc.getPrincipalDesc()).ofSize(1)) {
+      Assert.assertEquals(PrincipalType.GROUP, principal.getType());
+      Assert.assertEquals(GROUP, principal.getName());
+    }
+  }
+  /**
+   * REVOKE ROLE ... FROM USER ...
+   */
+  @Test
+  public void testRevokeRoleUser() throws Exception {
+    DDLWork work = analyze(parse("REVOKE ROLE " + ROLE + " FROM USER " + USER));
+    GrantRevokeRoleDDL grantDesc = work.getGrantRevokeRoleDDL();
+    Assert.assertNotNull("Grant should not be null", grantDesc);
+    Assert.assertFalse("Did not expect grant ", grantDesc.getGrant());
+    Assert.assertTrue("Grant option is always true ", grantDesc.isGrantOption());
+    Assert.assertEquals(currentUser, grantDesc.getGrantor());
+    Assert.assertEquals(PrincipalType.USER, grantDesc.getGrantorType());
+    for(String role : inList(grantDesc.getRoles()).ofSize(1)) {
+      Assert.assertEquals(ROLE, role);
+    }
+    for(PrincipalDesc principal : inList(grantDesc.getPrincipalDesc()).ofSize(1)) {
+      Assert.assertEquals(PrincipalType.USER, principal.getType());
+      Assert.assertEquals(USER, principal.getName());
+    }
+  }
+  /**
+   * REVOKE ROLE ... FROM ROLE ...
+   */
+  @Test
+  public void testRevokeRoleRole() throws Exception {
+    DDLWork work = analyze(parse("REVOKE ROLE " + ROLE + " FROM ROLE " + ROLE));
+    GrantRevokeRoleDDL grantDesc = work.getGrantRevokeRoleDDL();
+    Assert.assertNotNull("Grant should not be null", grantDesc);
+    Assert.assertFalse("Did not expect grant ", grantDesc.getGrant());
+    Assert.assertTrue("Grant option is always true ", grantDesc.isGrantOption());
+    Assert.assertEquals(currentUser, grantDesc.getGrantor());
+    Assert.assertEquals(PrincipalType.USER, grantDesc.getGrantorType());
+    for(String role : inList(grantDesc.getRoles()).ofSize(1)) {
+      Assert.assertEquals(ROLE, role);
+    }
+    for(PrincipalDesc principal : inList(grantDesc.getPrincipalDesc()).ofSize(1)) {
+      Assert.assertEquals(PrincipalType.ROLE, principal.getType());
+      Assert.assertEquals(ROLE, principal.getName());
+    }
+  }
+  /**
+   * REVOKE ROLE ... FROM GROUP ...
+   */
+  @Test
+  public void testRevokeRoleGroup() throws Exception {
+    DDLWork work = analyze(parse("REVOKE ROLE " + ROLE + " FROM GROUP " + GROUP));
+    GrantRevokeRoleDDL grantDesc = work.getGrantRevokeRoleDDL();
+    Assert.assertNotNull("Grant should not be null", grantDesc);
+    Assert.assertFalse("Did not expect grant ", grantDesc.getGrant());
+    Assert.assertTrue("Grant option is always true ", grantDesc.isGrantOption());
+    Assert.assertEquals(currentUser, grantDesc.getGrantor());
+    Assert.assertEquals(PrincipalType.USER, grantDesc.getGrantorType());
+    for(String role : inList(grantDesc.getRoles()).ofSize(1)) {
+      Assert.assertEquals(ROLE, role);
+    }
+    for(PrincipalDesc principal : inList(grantDesc.getPrincipalDesc()).ofSize(1)) {
+      Assert.assertEquals(PrincipalType.GROUP, principal.getType());
+      Assert.assertEquals(GROUP, principal.getName());
+    }
+  }
+  /**
+   * SHOW ROLE GRANT USER ...
+   */
+  @Test
+  public void testShowRoleGrantUser() throws Exception {
+    DDLWork work = analyze(parse("SHOW ROLE GRANT USER " + USER));
+    RoleDDLDesc roleDesc = work.getRoleDDLDesc();
+    Assert.assertNotNull("Role should not be null", roleDesc);
+    Assert.assertEquals(RoleOperation.SHOW_ROLE_GRANT, roleDesc.getOperation());
+    Assert.assertEquals(PrincipalType.USER, roleDesc.getPrincipalType());
+    Assert.assertEquals(USER, roleDesc.getName());
+  }
+  /**
+   * SHOW ROLE GRANT ROLE ...
+   */
+  @Test
+  public void testShowRoleGrantRole() throws Exception {
+    DDLWork work = analyze(parse("SHOW ROLE GRANT ROLE " + ROLE));
+    RoleDDLDesc roleDesc = work.getRoleDDLDesc();
+    Assert.assertNotNull("Role should not be null", roleDesc);
+    Assert.assertEquals(RoleOperation.SHOW_ROLE_GRANT, roleDesc.getOperation());
+    Assert.assertEquals(PrincipalType.ROLE, roleDesc.getPrincipalType());
+    Assert.assertEquals(ROLE, roleDesc.getName());
+  }
+  /**
+   * SHOW ROLE GRANT GROUP ...
+   */
+  @Test
+  public void testShowRoleGrantGroup() throws Exception {
+    DDLWork work = analyze(parse("SHOW ROLE GRANT GROUP " + GROUP));
+    RoleDDLDesc roleDesc = work.getRoleDDLDesc();
+    Assert.assertNotNull("Role should not be null", roleDesc);
+    Assert.assertEquals(RoleOperation.SHOW_ROLE_GRANT, roleDesc.getOperation());
+    Assert.assertEquals(PrincipalType.GROUP, roleDesc.getPrincipalType());
+    Assert.assertEquals(GROUP, roleDesc.getName());
+  }
+  /**
+   * SHOW GRANT USER ... ON TABLE ...
+   */
+  @Test
+  public void testShowGrantUserOnTable() throws Exception {
+    DDLWork work = analyze(parse("SHOW GRANT USER " + USER + " ON TABLE " + TABLE));
+    ShowGrantDesc grantDesc = work.getShowGrantDesc();
+    Assert.assertNotNull("Show grant should not be null", grantDesc);
+    Assert.assertEquals(PrincipalType.USER, grantDesc.getPrincipalDesc().getType());
+    Assert.assertEquals(USER, grantDesc.getPrincipalDesc().getName());
+    Assert.assertTrue("Expected table", grantDesc.getHiveObj().getTable());
+    Assert.assertEquals(TABLE, grantDesc.getHiveObj().getObject());
+    Assert.assertTrue("Expected table", grantDesc.getHiveObj().getTable());
+  }
+  /**
+   * SHOW GRANT ROLE ... ON TABLE ...
+   */
+  @Test
+  public void testShowGrantRoleOnTable() throws Exception {
+    DDLWork work = analyze(parse("SHOW GRANT ROLE " + ROLE + " ON TABLE " + TABLE));
+    ShowGrantDesc grantDesc = work.getShowGrantDesc();
+    Assert.assertNotNull("Show grant should not be null", grantDesc);
+    Assert.assertEquals(PrincipalType.ROLE, grantDesc.getPrincipalDesc().getType());
+    Assert.assertEquals(ROLE, grantDesc.getPrincipalDesc().getName());
+    Assert.assertTrue("Expected table", grantDesc.getHiveObj().getTable());
+    Assert.assertEquals(TABLE, grantDesc.getHiveObj().getObject());
+    Assert.assertTrue("Expected table", grantDesc.getHiveObj().getTable());
+  }
+  /**
+   * SHOW GRANT GROUP ... ON TABLE ...
+   */
+  @Test
+  public void testShowGrantGroupOnTable() throws Exception {
+    DDLWork work = analyze(parse("SHOW GRANT GROUP " + GROUP + " ON TABLE " + TABLE));
+    ShowGrantDesc grantDesc = work.getShowGrantDesc();
+    Assert.assertNotNull("Show grant should not be null", grantDesc);
+    Assert.assertEquals(PrincipalType.GROUP, grantDesc.getPrincipalDesc().getType());
+    Assert.assertEquals(GROUP, grantDesc.getPrincipalDesc().getName());
+    Assert.assertTrue("Expected table", grantDesc.getHiveObj().getTable());
+    Assert.assertEquals(TABLE, grantDesc.getHiveObj().getObject());
+    Assert.assertTrue("Expected table", grantDesc.getHiveObj().getTable());
+  }
+  private ASTNode parse(String command) throws Exception {
+    return ParseUtils.findRootNonNullToken(parseDriver.parse(command));
+  }
+
+  private DDLWork analyze(ASTNode ast) throws Exception {
+    analyzer.analyze(ast, context);
+    List<Task<? extends Serializable>> rootTasks = analyzer.getRootTasks();
+    return (DDLWork) inList(rootTasks).ofSize(1).get(0).getWork();
+  }
+  private static class ListSizeMatcher<E> {
+    private final List<E> list;
+    private ListSizeMatcher(List<E> list) {
+      this.list = list;
+    }
+    private List<E> ofSize(int size) {
+      Assert.assertEquals(list.toString(),  size, list.size());
+      return list;
+    }
+  }
+  private static <E> ListSizeMatcher<E> inList(List<E> list) {
+    return new ListSizeMatcher<E>(list);
+  }
+}