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

svn commit: r1626027 - in /hive/trunk: itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/ ql/src/java/org/apache/hadoop/hive/ql/ ql/src/java/org/apache/hadoop/hive/ql/parse/ ql/src/java/org/apache/hadoop/hive/ql/sec...

Author: gates
Date: Thu Sep 18 16:57:15 2014
New Revision: 1626027

URL: http://svn.apache.org/r1626027
Log:
HIVE-7790 Update privileges to check for update and delete (Alan Gates, reviewed by Thejas Nair)

Added:
    hive/trunk/ql/src/test/queries/clientnegative/authorization_delete_nodeletepriv.q
    hive/trunk/ql/src/test/queries/clientnegative/authorization_update_noupdatepriv.q
    hive/trunk/ql/src/test/queries/clientpositive/authorization_delete.q
    hive/trunk/ql/src/test/queries/clientpositive/authorization_delete_own_table.q
    hive/trunk/ql/src/test/queries/clientpositive/authorization_update.q
    hive/trunk/ql/src/test/queries/clientpositive/authorization_update_own_table.q
    hive/trunk/ql/src/test/results/clientnegative/authorization_delete_nodeletepriv.q.out
    hive/trunk/ql/src/test/results/clientnegative/authorization_update_noupdatepriv.q.out
    hive/trunk/ql/src/test/results/clientpositive/authorization_delete.q.out
    hive/trunk/ql/src/test/results/clientpositive/authorization_delete_own_table.q.out
    hive/trunk/ql/src/test/results/clientpositive/authorization_update.q.out
    hive/trunk/ql/src/test/results/clientpositive/authorization_update_own_table.q.out
Modified:
    hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.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/ColumnAccessInfo.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
    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/HivePrivilegeObject.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java

Modified: hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java
URL: http://svn.apache.org/viewvc/hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java?rev=1626027&r1=1626026&r2=1626027&view=diff
==============================================================================
--- hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java (original)
+++ hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java Thu Sep 18 16:57:15 2014
@@ -33,10 +33,13 @@ import java.util.List;
 
 import org.apache.commons.lang3.tuple.ImmutablePair;
 import org.apache.commons.lang3.tuple.Pair;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
 import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider;
 import org.apache.hadoop.hive.ql.security.SessionStateUserAuthenticator;
@@ -52,10 +55,12 @@ import org.mockito.Mockito;
  * Test HiveAuthorizer api invocation
  */
 public class TestHiveAuthorizerCheckInvocation {
+  private final Log LOG = LogFactory.getLog(this.getClass().getName());;
   protected static HiveConf conf;
   protected static Driver driver;
   private static final String tableName = TestHiveAuthorizerCheckInvocation.class.getSimpleName()
       + "Table";
+  private static final String acidTableName = tableName + "_acid";
   private static final String dbName = TestHiveAuthorizerCheckInvocation.class.getSimpleName()
       + "Db";
   static HiveAuthorizer mockedAuthorizer;
@@ -82,14 +87,18 @@ public class TestHiveAuthorizerCheckInvo
     conf.setVar(ConfVars.HIVE_AUTHORIZATION_MANAGER, MockedHiveAuthorizerFactory.class.getName());
     conf.setVar(ConfVars.HIVE_AUTHENTICATOR_MANAGER, SessionStateUserAuthenticator.class.getName());
     conf.setBoolVar(ConfVars.HIVE_AUTHORIZATION_ENABLED, true);
-    conf.setBoolVar(ConfVars.HIVE_SUPPORT_CONCURRENCY, false);
     conf.setBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS, false);
+    conf.setBoolVar(ConfVars.HIVE_SUPPORT_CONCURRENCY, true);
+    conf.setVar(ConfVars.HIVE_TXN_MANAGER, DbTxnManager.class.getName());
 
     SessionState.start(conf);
     driver = new Driver(conf);
     runCmd("create table " + tableName
         + " (i int, j int, k string) partitioned by (city string, date string) ");
     runCmd("create database " + dbName);
+    // Need a separate table for ACID testing since it has to be bucketed and it has to be Acid
+    runCmd("create table " + acidTableName + " (i int, j int) clustered by (i) into 2 buckets " +
+        "stored as orc");
   }
 
   private static void runCmd(String cmd) throws CommandNeedRetryException {
@@ -99,6 +108,10 @@ public class TestHiveAuthorizerCheckInvo
 
   @AfterClass
   public static void afterTests() throws Exception {
+    // Drop the tables when we're done.  This makes the test work inside an IDE
+    runCmd("drop table if exists " + acidTableName);
+    runCmd("drop table if exists " + tableName);
+    runCmd("drop database if exists " + dbName);
     driver.close();
   }
 
@@ -244,6 +257,63 @@ public class TestHiveAuthorizerCheckInvo
     assertEquals("db name", null, funcObj.getDbname());
   }
 
+  @Test
+  public void testUpdateSomeColumnsUsed() throws HiveAuthzPluginException,
+      HiveAccessControlException, CommandNeedRetryException {
+    reset(mockedAuthorizer);
+    int status = driver.compile("update " + acidTableName + " set i = 5 where j = 3");
+    assertEquals(0, status);
+
+    Pair<List<HivePrivilegeObject>, List<HivePrivilegeObject>> io = getHivePrivilegeObjectInputs();
+    List<HivePrivilegeObject> outputs = io.getRight();
+    HivePrivilegeObject tableObj = outputs.get(0);
+    LOG.debug("Got privilege object " + tableObj);
+    assertEquals("no of columns used", 1, tableObj.getColumns().size());
+    assertEquals("Column used", "i", tableObj.getColumns().get(0));
+    List<HivePrivilegeObject> inputs = io.getLeft();
+    assertEquals(1, inputs.size());
+    tableObj = inputs.get(0);
+    assertEquals(1, tableObj.getColumns().size());
+    assertEquals("j", tableObj.getColumns().get(0));
+  }
+
+  @Test
+  public void testUpdateSomeColumnsUsedExprInSet() throws HiveAuthzPluginException,
+      HiveAccessControlException, CommandNeedRetryException {
+    reset(mockedAuthorizer);
+    int status = driver.compile("update " + acidTableName + " set i = 5, l = k where j = 3");
+    assertEquals(0, status);
+
+    Pair<List<HivePrivilegeObject>, List<HivePrivilegeObject>> io = getHivePrivilegeObjectInputs();
+    List<HivePrivilegeObject> outputs = io.getRight();
+    HivePrivilegeObject tableObj = outputs.get(0);
+    LOG.debug("Got privilege object " + tableObj);
+    assertEquals("no of columns used", 2, tableObj.getColumns().size());
+    assertEquals("Columns used", Arrays.asList("i", "l"),
+        getSortedList(tableObj.getColumns()));
+    List<HivePrivilegeObject> inputs = io.getLeft();
+    assertEquals(1, inputs.size());
+    tableObj = inputs.get(0);
+    assertEquals(2, tableObj.getColumns().size());
+    assertEquals("Columns used", Arrays.asList("j", "k"),
+        getSortedList(tableObj.getColumns()));
+  }
+
+  @Test
+  public void testDelete() throws HiveAuthzPluginException,
+      HiveAccessControlException, CommandNeedRetryException {
+    reset(mockedAuthorizer);
+    int status = driver.compile("delete from " + acidTableName + " where j = 3");
+    assertEquals(0, status);
+
+    Pair<List<HivePrivilegeObject>, List<HivePrivilegeObject>> io = getHivePrivilegeObjectInputs();
+    List<HivePrivilegeObject> inputs = io.getLeft();
+    assertEquals(1, inputs.size());
+    HivePrivilegeObject tableObj = inputs.get(0);
+    assertEquals(1, tableObj.getColumns().size());
+    assertEquals("j", tableObj.getColumns().get(0));
+  }
+
   private void checkSingleTableInput(List<HivePrivilegeObject> inputs) {
     assertEquals("number of inputs", 1, inputs.size());
 

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=1626027&r1=1626026&r2=1626027&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 Thu Sep 18 16:57:15 2014
@@ -503,9 +503,11 @@ public class Driver implements CommandPr
       // get mapping of tables to columns used
       ColumnAccessInfo colAccessInfo = sem.getColumnAccessInfo();
       // colAccessInfo is set only in case of SemanticAnalyzer
-      Map<String, List<String>> tab2Cols = colAccessInfo != null ? colAccessInfo
+      Map<String, List<String>> selectTab2Cols = colAccessInfo != null ? colAccessInfo
           .getTableToColumnAccessMap() : null;
-      doAuthorizationV2(ss, op, inputs, outputs, command, tab2Cols);
+      Map<String, List<String>> updateTab2Cols = sem.getUpdateColumnAccessInfo() != null ?
+          sem.getUpdateColumnAccessInfo().getTableToColumnAccessMap() : null;
+      doAuthorizationV2(ss, op, inputs, outputs, command, selectTab2Cols, updateTab2Cols);
      return;
     }
     if (op == null) {
@@ -696,7 +698,13 @@ public class Driver implements CommandPr
   }
 
   private static void doAuthorizationV2(SessionState ss, HiveOperation op, HashSet<ReadEntity> inputs,
-      HashSet<WriteEntity> outputs, String command, Map<String, List<String>> tab2cols) throws HiveException {
+      HashSet<WriteEntity> outputs, String command, Map<String, List<String>> tab2cols,
+      Map<String, List<String>> updateTab2Cols) throws HiveException {
+
+    /* comment for reviewers -> updateTab2Cols needed to be separate from tab2cols because if I
+    pass tab2cols to getHivePrivObjects for the output case it will trip up insert/selects,
+    since the insert will get passed the columns from the select.
+     */
 
     HiveAuthzContext.Builder authzContextBuilder = new HiveAuthzContext.Builder();
     authzContextBuilder.setUserIpAddress(ss.getUserIpAddress());
@@ -704,7 +712,7 @@ public class Driver implements CommandPr
 
     HiveOperationType hiveOpType = getHiveOperationType(op);
     List<HivePrivilegeObject> inputsHObjs = getHivePrivObjects(inputs, tab2cols);
-    List<HivePrivilegeObject> outputHObjs = getHivePrivObjects(outputs, null);
+    List<HivePrivilegeObject> outputHObjs = getHivePrivObjects(outputs, updateTab2Cols);
 
     ss.getAuthorizerV2().checkPrivileges(hiveOpType, inputsHObjs, outputHObjs, authzContextBuilder.build());
   }
@@ -730,12 +738,6 @@ public class Driver implements CommandPr
         //do not authorize temporary uris
         continue;
       }
-      if (privObject instanceof ReadEntity && ((ReadEntity)privObject).isUpdateOrDelete()) {
-        // Skip this one, as we don't want to check select privileges for the table we're reading
-        // for an update or delete.
-        continue;
-      }
-
       //support for authorization on partitions needs to be added
       String dbname = null;
       String objName = null;

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=1626027&r1=1626026&r2=1626027&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 Thu Sep 18 16:57:15 2014
@@ -115,6 +115,10 @@ public abstract class BaseSemanticAnalyz
   protected LineageInfo linfo;
   protected TableAccessInfo tableAccessInfo;
   protected ColumnAccessInfo columnAccessInfo;
+  /**
+   * Columns accessed by updates
+   */
+  protected ColumnAccessInfo updateColumnAccessInfo;
 
 
   public boolean skipAuthorization() {
@@ -866,6 +870,14 @@ public abstract class BaseSemanticAnalyz
     this.columnAccessInfo = columnAccessInfo;
   }
 
+  public ColumnAccessInfo getUpdateColumnAccessInfo() {
+    return updateColumnAccessInfo;
+  }
+
+  public void setUpdateColumnAccessInfo(ColumnAccessInfo updateColumnAccessInfo) {
+    this.updateColumnAccessInfo = updateColumnAccessInfo;
+  }
+
   protected LinkedHashMap<String, String> extractPartitionSpecs(Tree partspec)
       throws SemanticException {
     LinkedHashMap<String, String> partSpec = new LinkedHashMap<String, String>();

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnAccessInfo.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnAccessInfo.java?rev=1626027&r1=1626026&r2=1626027&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnAccessInfo.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnAccessInfo.java Thu Sep 18 16:57:15 2014
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.hive.ql.parse;
 
+import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
+
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -54,4 +56,21 @@ public class ColumnAccessInfo {
     }
     return mapping;
   }
+
+  /**
+   * Strip a virtual column out of the set of columns.  This is useful in cases where we do not
+   * want to be checking against the user reading virtual columns, namely update and delete.
+   * @param vc
+   */
+  public void stripVirtualColumn(VirtualColumn vc) {
+    for (Map.Entry<String, Set<String>> e : tableToColumnAccessMap.entrySet()) {
+      for (String columnName : e.getValue()) {
+        if (vc.getName().equalsIgnoreCase(columnName)) {
+          e.getValue().remove(columnName);
+          break;
+        }
+      }
+    }
+
+  }
 }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java?rev=1626027&r1=1626026&r2=1626027&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java Thu Sep 18 16:57:15 2014
@@ -28,11 +28,13 @@ import org.apache.hadoop.hive.ql.io.Acid
 import org.apache.hadoop.hive.ql.lib.Node;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
 import org.apache.hadoop.hive.ql.session.SessionState;
 
 
 import java.io.IOException;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -148,6 +150,8 @@ public class UpdateDeleteSemanticAnalyze
 
     rewrittenQueryStr.append(" select ROW__ID");
     Map<Integer, ASTNode> setColExprs = null;
+    Map<String, ASTNode> setCols = null;
+    Set<String> setRCols = new HashSet<String>();
     if (updating()) {
       // An update needs to select all of the columns, as we rewrite the entire row.  Also,
       // we need to figure out which columns we are going to replace.  We won't write the set
@@ -160,7 +164,7 @@ public class UpdateDeleteSemanticAnalyze
 
       // Get the children of the set clause, each of which should be a column assignment
       List<? extends Node> assignments = setClause.getChildren();
-      Map<String, ASTNode> setCols = new HashMap<String, ASTNode>(assignments.size());
+      setCols = new HashMap<String, ASTNode>(assignments.size());
       setColExprs = new HashMap<Integer, ASTNode>(assignments.size());
       for (Node a : assignments) {
         ASTNode assignment = (ASTNode)a;
@@ -173,6 +177,8 @@ public class UpdateDeleteSemanticAnalyze
         assert colName.getToken().getType() == HiveParser.Identifier :
             "Expected column name";
 
+        addSetRCols((ASTNode) assignment.getChildren().get(1), setRCols);
+
         String columnName = colName.getText();
 
         // Make sure this isn't one of the partitioning columns, that's not supported.
@@ -323,6 +329,28 @@ public class UpdateDeleteSemanticAnalyze
             WriteEntity.WriteType.UPDATE);
       }
     }
+
+    // For updates, we need to set the column access info so that it contains information on
+    // the columns we are updating.
+    if (updating()) {
+      ColumnAccessInfo cai = new ColumnAccessInfo();
+      for (String colName : setCols.keySet()) {
+        cai.add(Table.getCompleteName(mTable.getDbName(), mTable.getTableName()), colName);
+      }
+      setUpdateColumnAccessInfo(cai);
+
+      // Add the setRCols to the input list
+      for (String colName : setRCols) {
+        columnAccessInfo.add(Table.getCompleteName(mTable.getDbName(), mTable.getTableName()),
+            colName);
+      }
+    }
+
+    // We need to weed ROW__ID out of the input column info, as it doesn't make any sense to
+    // require the user to have authorization on that column.
+    if (columnAccessInfo != null) {
+      columnAccessInfo.stripVirtualColumn(VirtualColumn.ROWID);
+    }
   }
 
   private String operation() {
@@ -342,4 +370,22 @@ public class UpdateDeleteSemanticAnalyze
     }
     return false;
   }
+
+  // This method find any columns on the right side of a set statement (thus rcols) and puts them
+  // in a set so we can add them to the list of input cols to check.
+  private void addSetRCols(ASTNode node, Set<String> setRCols) {
+
+    // See if this node is a TOK_TABLE_OR_COL.  If so, find the value and put it in the list.  If
+    // not, recurse on any children
+    if (node.getToken().getType() == HiveParser.TOK_TABLE_OR_COL) {
+      ASTNode colName = (ASTNode)node.getChildren().get(0);
+      assert colName.getToken().getType() == HiveParser.Identifier :
+          "Expected column name";
+      setRCols.add(colName.getText());
+    } else if (node.getChildren() != null) {
+      for (Node n : node.getChildren()) {
+        addSetRCols(node, setRCols);
+      }
+    }
+  }
 }

Modified: 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=1626027&r1=1626026&r2=1626027&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationUtils.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationUtils.java Thu Sep 18 16:57:15 2014
@@ -310,9 +310,12 @@ public class AuthorizationUtils {
         return HivePrivObjectActionType.INSERT;
       case INSERT_OVERWRITE:
         return HivePrivObjectActionType.INSERT_OVERWRITE;
+      case UPDATE:
+        return HivePrivObjectActionType.UPDATE;
+      case DELETE:
+        return HivePrivObjectActionType.DELETE;
       default:
-        // Ignore other types for purposes of authorization, we are interested only
-        // in INSERT vs INSERT_OVERWRITE as of now
+        // Ignore other types for purposes of authorization
         break;
       }
     }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HivePrivilegeObject.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HivePrivilegeObject.java?rev=1626027&r1=1626026&r2=1626027&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HivePrivilegeObject.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HivePrivilegeObject.java Thu Sep 18 16:57:15 2014
@@ -81,7 +81,7 @@ public class HivePrivilegeObject impleme
     GLOBAL, DATABASE, TABLE_OR_VIEW, PARTITION, COLUMN, LOCAL_URI, DFS_URI, COMMAND_PARAMS, FUNCTION
   } ;
   public enum HivePrivObjectActionType {
-    OTHER, INSERT, INSERT_OVERWRITE
+    OTHER, INSERT, INSERT_OVERWRITE, UPDATE, DELETE
   };
 
   private final HivePrivilegeObjectType type;

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java?rev=1626027&r1=1626026&r2=1626027&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java Thu Sep 18 16:57:15 2014
@@ -118,6 +118,7 @@ public class Operation2Privilege {
   private static SQLPrivTypeGrant[] ADMIN_PRIV_AR = arr(SQLPrivTypeGrant.ADMIN_PRIV);
   private static SQLPrivTypeGrant[] INS_NOGRANT_AR = arr(SQLPrivTypeGrant.INSERT_NOGRANT);
   private static SQLPrivTypeGrant[] DEL_NOGRANT_AR = arr(SQLPrivTypeGrant.DELETE_NOGRANT);
+  private static SQLPrivTypeGrant[] UPD_NOGRANT_AR = arr(SQLPrivTypeGrant.UPDATE_NOGRANT);
   private static SQLPrivTypeGrant[] OWNER_INS_SEL_DEL_NOGRANT_AR =
       arr(SQLPrivTypeGrant.OWNER_PRIV,
           SQLPrivTypeGrant.INSERT_NOGRANT,
@@ -287,8 +288,14 @@ public class Operation2Privilege {
     op2Priv.put(HiveOperationType.QUERY,
         arr(
             new PrivRequirement(SEL_NOGRANT_AR, IOType.INPUT),
-            new PrivRequirement(INS_NOGRANT_AR, IOType.OUTPUT, null),
-            new PrivRequirement(DEL_NOGRANT_AR, IOType.OUTPUT, HivePrivObjectActionType.INSERT_OVERWRITE)
+            new PrivRequirement(INS_NOGRANT_AR, IOType.OUTPUT, HivePrivObjectActionType.INSERT),
+            new PrivRequirement(
+                arr(SQLPrivTypeGrant.INSERT_NOGRANT, SQLPrivTypeGrant.DELETE_NOGRANT),
+                IOType.OUTPUT,
+                HivePrivObjectActionType.INSERT_OVERWRITE),
+            new PrivRequirement(DEL_NOGRANT_AR, IOType.OUTPUT, HivePrivObjectActionType.DELETE),
+            new PrivRequirement(UPD_NOGRANT_AR, IOType.OUTPUT, HivePrivObjectActionType.UPDATE),
+            new PrivRequirement(INS_NOGRANT_AR, IOType.OUTPUT, HivePrivObjectActionType.OTHER)
             )
         );
 

Added: hive/trunk/ql/src/test/queries/clientnegative/authorization_delete_nodeletepriv.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/authorization_delete_nodeletepriv.q?rev=1626027&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/authorization_delete_nodeletepriv.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/authorization_delete_nodeletepriv.q Thu Sep 18 16:57:15 2014
@@ -0,0 +1,17 @@
+set hive.test.authz.sstd.hs2.mode=true;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+set hive.enforce.bucketing=true;
+
+
+-- check update without update priv
+create table auth_nodel(i int) clustered by (i) into 2 buckets stored as orc;;
+
+set user.name=user1;
+delete from auth_nodel where i > 0;
+

Added: hive/trunk/ql/src/test/queries/clientnegative/authorization_update_noupdatepriv.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/authorization_update_noupdatepriv.q?rev=1626027&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/authorization_update_noupdatepriv.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/authorization_update_noupdatepriv.q Thu Sep 18 16:57:15 2014
@@ -0,0 +1,17 @@
+set hive.test.authz.sstd.hs2.mode=true;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+set hive.enforce.bucketing=true;
+
+
+-- check update without update priv
+create table auth_noupd(i int) clustered by (i) into 2 buckets stored as orc;;
+
+set user.name=user1;
+update auth_noupd set i = 0 where i > 0;
+

Added: hive/trunk/ql/src/test/queries/clientpositive/authorization_delete.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/authorization_delete.q?rev=1626027&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/authorization_delete.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/authorization_delete.q Thu Sep 18 16:57:15 2014
@@ -0,0 +1,25 @@
+set hive.test.authz.sstd.hs2.mode=true;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+set hive.enforce.bucketing=true;
+
+set user.name=user1;
+-- current user has been set (comment line before the set cmd is resulting in parse error!!)
+
+CREATE TABLE t_auth_del(i int) clustered by (i) into 2 buckets stored as orc;
+
+-- grant update privilege to another user
+GRANT DELETE ON t_auth_del TO USER userWIns;
+GRANT SELECT ON t_auth_del TO USER userWIns;
+
+set user.name=hive_admin_user;
+set role admin;
+SHOW GRANT ON TABLE t_auth_del;
+
+
+set user.name=userWIns;
+delete from t_auth_del where i > 0;

Added: hive/trunk/ql/src/test/queries/clientpositive/authorization_delete_own_table.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/authorization_delete_own_table.q?rev=1626027&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/authorization_delete_own_table.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/authorization_delete_own_table.q Thu Sep 18 16:57:15 2014
@@ -0,0 +1,17 @@
+set hive.test.authz.sstd.hs2.mode=true;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+set hive.enforce.bucketing=true;
+
+
+set user.name=user1;
+create table auth_noupd(i int) clustered by (i) into 2 buckets stored as orc;;
+delete from auth_noupd where i > 0;
+
+set user.name=hive_admin_user;
+set role admin;

Added: hive/trunk/ql/src/test/queries/clientpositive/authorization_update.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/authorization_update.q?rev=1626027&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/authorization_update.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/authorization_update.q Thu Sep 18 16:57:15 2014
@@ -0,0 +1,28 @@
+set hive.test.authz.sstd.hs2.mode=true;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+set hive.enforce.bucketing=true;
+
+set user.name=user1;
+-- current user has been set (comment line before the set cmd is resulting in parse error!!)
+
+CREATE TABLE t_auth_up(i int) clustered by (i) into 2 buckets stored as orc;
+
+CREATE TABLE t_select(i int);
+GRANT ALL ON TABLE t_select TO ROLE public;
+
+-- grant update privilege to another user
+GRANT UPDATE ON t_auth_up TO USER userWIns;
+GRANT SELECT ON t_auth_up TO USER userWIns;
+
+set user.name=hive_admin_user;
+set role admin;
+SHOW GRANT ON TABLE t_auth_up;
+
+
+set user.name=userWIns;
+update t_auth_up set i = 0 where i > 0;

Added: hive/trunk/ql/src/test/queries/clientpositive/authorization_update_own_table.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/authorization_update_own_table.q?rev=1626027&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/authorization_update_own_table.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/authorization_update_own_table.q Thu Sep 18 16:57:15 2014
@@ -0,0 +1,17 @@
+set hive.test.authz.sstd.hs2.mode=true;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+set hive.enforce.bucketing=true;
+
+
+set user.name=user1;
+create table auth_noupd(i int) clustered by (i) into 2 buckets stored as orc;;
+update auth_noupd set i = 0 where i > 0;
+
+set user.name=hive_admin_user;
+set role admin;

Added: hive/trunk/ql/src/test/results/clientnegative/authorization_delete_nodeletepriv.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/authorization_delete_nodeletepriv.q.out?rev=1626027&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/authorization_delete_nodeletepriv.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/authorization_delete_nodeletepriv.q.out Thu Sep 18 16:57:15 2014
@@ -0,0 +1,11 @@
+PREHOOK: query: -- check update without update priv
+create table auth_nodel(i int) clustered by (i) into 2 buckets stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@auth_nodel
+POSTHOOK: query: -- check update without update priv
+create table auth_nodel(i int) clustered by (i) into 2 buckets stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@auth_nodel
+FAILED: HiveAccessControlException Permission denied: Principal [name=user1, type=USER] does not have following privileges for operation QUERY [[DELETE] on Object [type=TABLE_OR_VIEW, name=default.auth_nodel], [SELECT] on Object [type=TABLE_OR_VIEW, name=default.auth_nodel]]

Added: hive/trunk/ql/src/test/results/clientnegative/authorization_update_noupdatepriv.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/authorization_update_noupdatepriv.q.out?rev=1626027&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/authorization_update_noupdatepriv.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/authorization_update_noupdatepriv.q.out Thu Sep 18 16:57:15 2014
@@ -0,0 +1,11 @@
+PREHOOK: query: -- check update without update priv
+create table auth_noupd(i int) clustered by (i) into 2 buckets stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@auth_noupd
+POSTHOOK: query: -- check update without update priv
+create table auth_noupd(i int) clustered by (i) into 2 buckets stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@auth_noupd
+FAILED: HiveAccessControlException Permission denied: Principal [name=user1, type=USER] does not have following privileges for operation QUERY [[SELECT] on Object [type=TABLE_OR_VIEW, name=default.auth_noupd], [UPDATE] on Object [type=TABLE_OR_VIEW, name=default.auth_noupd]]

Added: hive/trunk/ql/src/test/results/clientpositive/authorization_delete.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/authorization_delete.q.out?rev=1626027&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/authorization_delete.q.out (added)
+++ hive/trunk/ql/src/test/results/clientpositive/authorization_delete.q.out Thu Sep 18 16:57:15 2014
@@ -0,0 +1,48 @@
+PREHOOK: query: -- current user has been set (comment line before the set cmd is resulting in parse error!!)
+
+CREATE TABLE t_auth_del(i int) clustered by (i) into 2 buckets stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t_auth_del
+POSTHOOK: query: -- current user has been set (comment line before the set cmd is resulting in parse error!!)
+
+CREATE TABLE t_auth_del(i int) clustered by (i) into 2 buckets stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t_auth_del
+PREHOOK: query: -- grant update privilege to another user
+GRANT DELETE ON t_auth_del TO USER userWIns
+PREHOOK: type: GRANT_PRIVILEGE
+PREHOOK: Output: default@t_auth_del
+POSTHOOK: query: -- grant update privilege to another user
+GRANT DELETE ON t_auth_del TO USER userWIns
+POSTHOOK: type: GRANT_PRIVILEGE
+POSTHOOK: Output: default@t_auth_del
+PREHOOK: query: GRANT SELECT ON t_auth_del TO USER userWIns
+PREHOOK: type: GRANT_PRIVILEGE
+PREHOOK: Output: default@t_auth_del
+POSTHOOK: query: GRANT SELECT ON t_auth_del TO USER userWIns
+POSTHOOK: type: GRANT_PRIVILEGE
+POSTHOOK: Output: default@t_auth_del
+PREHOOK: query: set role admin
+PREHOOK: type: SHOW_ROLES
+POSTHOOK: query: set role admin
+POSTHOOK: type: SHOW_ROLES
+PREHOOK: query: SHOW GRANT ON TABLE t_auth_del
+PREHOOK: type: SHOW_GRANT
+POSTHOOK: query: SHOW GRANT ON TABLE t_auth_del
+POSTHOOK: type: SHOW_GRANT
+default	t_auth_del			user1	USER	DELETE	true	-1	user1
+default	t_auth_del			user1	USER	INSERT	true	-1	user1
+default	t_auth_del			user1	USER	SELECT	true	-1	user1
+default	t_auth_del			user1	USER	UPDATE	true	-1	user1
+default	t_auth_del			userWIns	USER	DELETE	false	-1	user1
+default	t_auth_del			userWIns	USER	SELECT	false	-1	user1
+PREHOOK: query: delete from t_auth_del where i > 0
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t_auth_del
+PREHOOK: Output: default@t_auth_del
+POSTHOOK: query: delete from t_auth_del where i > 0
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t_auth_del
+POSTHOOK: Output: default@t_auth_del

Added: hive/trunk/ql/src/test/results/clientpositive/authorization_delete_own_table.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/authorization_delete_own_table.q.out?rev=1626027&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/authorization_delete_own_table.q.out (added)
+++ hive/trunk/ql/src/test/results/clientpositive/authorization_delete_own_table.q.out Thu Sep 18 16:57:15 2014
@@ -0,0 +1,20 @@
+PREHOOK: query: create table auth_noupd(i int) clustered by (i) into 2 buckets stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@auth_noupd
+POSTHOOK: query: create table auth_noupd(i int) clustered by (i) into 2 buckets stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@auth_noupd
+PREHOOK: query: delete from auth_noupd where i > 0
+PREHOOK: type: QUERY
+PREHOOK: Input: default@auth_noupd
+PREHOOK: Output: default@auth_noupd
+POSTHOOK: query: delete from auth_noupd where i > 0
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@auth_noupd
+POSTHOOK: Output: default@auth_noupd
+PREHOOK: query: set role admin
+PREHOOK: type: SHOW_ROLES
+POSTHOOK: query: set role admin
+POSTHOOK: type: SHOW_ROLES

Added: hive/trunk/ql/src/test/results/clientpositive/authorization_update.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/authorization_update.q.out?rev=1626027&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/authorization_update.q.out (added)
+++ hive/trunk/ql/src/test/results/clientpositive/authorization_update.q.out Thu Sep 18 16:57:15 2014
@@ -0,0 +1,62 @@
+PREHOOK: query: -- current user has been set (comment line before the set cmd is resulting in parse error!!)
+
+CREATE TABLE t_auth_up(i int) clustered by (i) into 2 buckets stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t_auth_up
+POSTHOOK: query: -- current user has been set (comment line before the set cmd is resulting in parse error!!)
+
+CREATE TABLE t_auth_up(i int) clustered by (i) into 2 buckets stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t_auth_up
+PREHOOK: query: CREATE TABLE t_select(i int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t_select
+POSTHOOK: query: CREATE TABLE t_select(i int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t_select
+PREHOOK: query: GRANT ALL ON TABLE t_select TO ROLE public
+PREHOOK: type: GRANT_PRIVILEGE
+PREHOOK: Output: default@t_select
+POSTHOOK: query: GRANT ALL ON TABLE t_select TO ROLE public
+POSTHOOK: type: GRANT_PRIVILEGE
+POSTHOOK: Output: default@t_select
+PREHOOK: query: -- grant update privilege to another user
+GRANT UPDATE ON t_auth_up TO USER userWIns
+PREHOOK: type: GRANT_PRIVILEGE
+PREHOOK: Output: default@t_auth_up
+POSTHOOK: query: -- grant update privilege to another user
+GRANT UPDATE ON t_auth_up TO USER userWIns
+POSTHOOK: type: GRANT_PRIVILEGE
+POSTHOOK: Output: default@t_auth_up
+PREHOOK: query: GRANT SELECT ON t_auth_up TO USER userWIns
+PREHOOK: type: GRANT_PRIVILEGE
+PREHOOK: Output: default@t_auth_up
+POSTHOOK: query: GRANT SELECT ON t_auth_up TO USER userWIns
+POSTHOOK: type: GRANT_PRIVILEGE
+POSTHOOK: Output: default@t_auth_up
+PREHOOK: query: set role admin
+PREHOOK: type: SHOW_ROLES
+POSTHOOK: query: set role admin
+POSTHOOK: type: SHOW_ROLES
+PREHOOK: query: SHOW GRANT ON TABLE t_auth_up
+PREHOOK: type: SHOW_GRANT
+POSTHOOK: query: SHOW GRANT ON TABLE t_auth_up
+POSTHOOK: type: SHOW_GRANT
+default	t_auth_up			user1	USER	DELETE	true	-1	user1
+default	t_auth_up			user1	USER	INSERT	true	-1	user1
+default	t_auth_up			user1	USER	SELECT	true	-1	user1
+default	t_auth_up			user1	USER	UPDATE	true	-1	user1
+default	t_auth_up			userWIns	USER	SELECT	false	-1	user1
+default	t_auth_up			userWIns	USER	UPDATE	false	-1	user1
+PREHOOK: query: update t_auth_up set i = 0 where i > 0
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t_auth_up
+PREHOOK: Output: default@t_auth_up
+POSTHOOK: query: update t_auth_up set i = 0 where i > 0
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t_auth_up
+POSTHOOK: Output: default@t_auth_up

Added: hive/trunk/ql/src/test/results/clientpositive/authorization_update_own_table.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/authorization_update_own_table.q.out?rev=1626027&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/authorization_update_own_table.q.out (added)
+++ hive/trunk/ql/src/test/results/clientpositive/authorization_update_own_table.q.out Thu Sep 18 16:57:15 2014
@@ -0,0 +1,20 @@
+PREHOOK: query: create table auth_noupd(i int) clustered by (i) into 2 buckets stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@auth_noupd
+POSTHOOK: query: create table auth_noupd(i int) clustered by (i) into 2 buckets stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@auth_noupd
+PREHOOK: query: update auth_noupd set i = 0 where i > 0
+PREHOOK: type: QUERY
+PREHOOK: Input: default@auth_noupd
+PREHOOK: Output: default@auth_noupd
+POSTHOOK: query: update auth_noupd set i = 0 where i > 0
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@auth_noupd
+POSTHOOK: Output: default@auth_noupd
+PREHOOK: query: set role admin
+PREHOOK: type: SHOW_ROLES
+POSTHOOK: query: set role admin
+POSTHOOK: type: SHOW_ROLES