You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sentry.apache.org by pr...@apache.org on 2014/06/23 23:21:00 UTC

git commit: SENTRY-310: Make Hive operation to required privileges more granular (Sravya Tirukkovalur via Prasad Mujumdar)

Repository: incubator-sentry
Updated Branches:
  refs/heads/master b11f5aab9 -> b41d9ee37


SENTRY-310: Make Hive operation to required privileges more granular (Sravya Tirukkovalur via Prasad Mujumdar)


Project: http://git-wip-us.apache.org/repos/asf/incubator-sentry/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-sentry/commit/b41d9ee3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-sentry/tree/b41d9ee3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-sentry/diff/b41d9ee3

Branch: refs/heads/master
Commit: b41d9ee37cc705cc4a5eb86a9a7d04b2082a6458
Parents: b11f5aa
Author: Prasad Mujumdar <pr...@cloudera.com>
Authored: Mon Jun 23 14:20:17 2014 -0700
Committer: Prasad Mujumdar <pr...@cloudera.com>
Committed: Mon Jun 23 14:20:17 2014 -0700

----------------------------------------------------------------------
 .../binding/hive/HiveAuthzBindingHook.java      | 187 +++---
 .../hive/HiveAuthzBindingPreExecHook.java       |   2 +-
 .../binding/hive/authz/HiveAuthzBinding.java    |  86 ++-
 .../hive/authz/HiveAuthzPrivilegesMap.java      | 141 ++--
 .../binding/hive/TestHiveAuthzBindings.java     |   8 +-
 sentry-tests/sentry-tests-hive/pom.xml          |   2 +
 .../tests/e2e/dbprovider/TestDbOperations.java  |  37 +
 .../AbstractTestWithStaticConfiguration.java    |  12 +-
 .../apache/sentry/tests/e2e/hive/Context.java   |  11 +
 .../sentry/tests/e2e/hive/TestCrossDbOps.java   |   4 -
 .../sentry/tests/e2e/hive/TestOperations.java   | 672 +++++++++++++++++++
 .../e2e/hive/TestPrivilegeAtTransform.java      |   2 +-
 .../e2e/hive/TestPrivilegesAtDatabaseScope.java |  18 +-
 .../sentry/tests/e2e/hive/TestSandboxOps.java   |   4 -
 .../tests/e2e/hive/TestUriPermissions.java      |  18 +-
 15 files changed, 984 insertions(+), 220 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/b41d9ee3/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingHook.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingHook.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingHook.java
index 6c507b8..44c0d20 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingHook.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingHook.java
@@ -16,6 +16,7 @@
  */
 package org.apache.sentry.binding.hive;
 
+import org.apache.commons.lang.StringUtils;
 import static org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME;
 
 import java.io.Serializable;
@@ -42,6 +43,7 @@ import org.apache.hadoop.hive.ql.hooks.Entity.Type;
 import org.apache.hadoop.hive.ql.hooks.Hook;
 import org.apache.hadoop.hive.ql.hooks.ReadEntity;
 import org.apache.hadoop.hive.ql.hooks.WriteEntity;
+import org.apache.hadoop.hive.ql.lib.Node;
 import org.apache.hadoop.hive.ql.metadata.AuthorizationException;
 import org.apache.hadoop.hive.ql.parse.ASTNode;
 import org.apache.hadoop.hive.ql.parse.AbstractSemanticAnalyzerHook;
@@ -136,87 +138,88 @@ implements HiveDriverFilterHook {
 
     switch (ast.getToken().getType()) {
     // Hive parser doesn't capture the database name in output entity, so we store it here for now
-    case HiveParser.TOK_CREATEDATABASE:
-    case HiveParser.TOK_ALTERDATABASE_PROPERTIES:
-    case HiveParser.TOK_DROPDATABASE:
-    case HiveParser.TOK_SWITCHDATABASE:
-      currDB = new Database(BaseSemanticAnalyzer.unescapeIdentifier(ast.getChild(0).getText()));
-      break;
-    case HiveParser.TOK_DESCDATABASE:
-      currDB = new Database(BaseSemanticAnalyzer.unescapeIdentifier(ast.getChild(0).getText()));
-      break;
-    case HiveParser.TOK_CREATETABLE:
-    case HiveParser.TOK_DROPTABLE:
-    case HiveParser.TOK_ALTERTABLE_ADDCOLS:
-    case HiveParser.TOK_ALTERTABLE_RENAMECOL:
-    case HiveParser.TOK_ALTERTABLE_REPLACECOLS:
-    case HiveParser.TOK_ALTERTABLE_RENAME:
-    case HiveParser.TOK_ALTERTABLE_DROPPARTS:
-    case HiveParser.TOK_ALTERTABLE_PROPERTIES:
-    case HiveParser.TOK_ALTERTABLE_SERIALIZER:
-    case HiveParser.TOK_CREATEVIEW:
-    case HiveParser.TOK_DROPVIEW:
-    case HiveParser.TOK_ALTERVIEW_ADDPARTS:
-    case HiveParser.TOK_ALTERVIEW_DROPPARTS:
-    case HiveParser.TOK_ALTERVIEW_PROPERTIES:
-    case HiveParser.TOK_ALTERVIEW_RENAME:
-      /*
-       * Compiler doesn't create read/write entities for create table.
-       * Hence we need extract dbname from db.tab format, if applicable
-       */
-      currDB = extractDatabase(ast);
-      break;
-    case HiveParser.TOK_ALTERTABLE_ADDPARTS:
-      /*
-       * Compiler doesn't create read/write entities for create table.
-       * Hence we need extract dbname from db.tab format, if applicable
-       */
-      currDB = extractDatabase(ast);
-      partitionURI = extractPartition(ast);
-      break;
-    case HiveParser.TOK_CREATEFUNCTION:
-      String udfClassName = BaseSemanticAnalyzer.unescapeSQLString(ast.getChild(1).getText());
-      try {
-        CodeSource udfSrc = Class.forName(udfClassName).getProtectionDomain().getCodeSource();
-        if (udfSrc == null) {
-          throw new SemanticException("Could not resolve the jar for UDF class " + udfClassName);
-        }
-        String udfJar = udfSrc.getLocation().getPath();
-        if (udfJar == null || udfJar.isEmpty()) {
-          throw new SemanticException("Could not find the jar for UDF class " + udfClassName +
-              "to validate privileges");
+      case HiveParser.TOK_CREATEDATABASE:
+      case HiveParser.TOK_ALTERDATABASE_PROPERTIES:
+      case HiveParser.TOK_DROPDATABASE:
+      case HiveParser.TOK_SWITCHDATABASE:
+      case HiveParser.TOK_DESCDATABASE:
+        currDB = new Database(BaseSemanticAnalyzer.unescapeIdentifier(ast.getChild(0).getText()));
+        break;
+      case HiveParser.TOK_CREATETABLE:
+      case HiveParser.TOK_CREATEVIEW:
+        /*
+         * Compiler doesn't create read/write entities for create table.
+         * Hence we need extract dbname from db.tab format, if applicable
+         */
+        currDB = extractDatabase((ASTNode)ast.getChild(0));
+        break;
+      case HiveParser.TOK_DROPTABLE:
+      case HiveParser.TOK_DROPVIEW:
+      case HiveParser.TOK_SHOW_TABLESTATUS:
+      case HiveParser.TOK_SHOW_CREATETABLE:
+      case HiveParser.TOK_ALTERTABLE_SERIALIZER:
+      case HiveParser.TOK_ALTERVIEW_ADDPARTS:
+      case HiveParser.TOK_ALTERVIEW_DROPPARTS:
+      case HiveParser.TOK_ALTERVIEW_PROPERTIES:
+      case HiveParser.TOK_ALTERVIEW_RENAME:
+      case HiveParser.TOK_CREATEINDEX:
+      case HiveParser.TOK_DROPINDEX:
+        currTab = extractTable((ASTNode)ast.getFirstChildWithType(HiveParser.TOK_TABNAME));
+        currDB = extractDatabase((ASTNode) ast.getChild(0));
+        break;
+      case HiveParser.TOK_ALTERTABLE_RENAME:
+      case HiveParser.TOK_ALTERTABLE_PROPERTIES:
+      case HiveParser.TOK_ALTERTABLE_DROPPARTS:
+      case HiveParser.TOK_ALTERTABLE_RENAMECOL:
+      case HiveParser.TOK_ALTERTABLE_ADDCOLS:
+      case HiveParser.TOK_ALTERTABLE_REPLACECOLS:
+      case HiveParser.TOK_SHOW_TBLPROPERTIES:
+      case HiveParser.TOK_SHOWINDEXES:
+      case HiveParser.TOK_SHOWPARTITIONS:
+        //token name TOK_TABNAME is not properly set in this case
+        currTab = extractTable((ASTNode)ast.getChild(0));
+        currDB = extractDatabase((ASTNode)ast.getChild(0));
+        break;
+      case HiveParser.TOK_ALTERTABLE_ADDPARTS:
+        /*
+         * Compiler doesn't create read/write entities for create table.
+         * Hence we need extract dbname from db.tab format, if applicable
+         */
+        currTab = extractTable((ASTNode)ast.getChild(0));
+        currDB = extractDatabase((ASTNode)ast.getChild(0));
+        partitionURI = extractPartition(ast);
+        break;
+      case HiveParser.TOK_CREATEFUNCTION:
+        String udfClassName = BaseSemanticAnalyzer.unescapeSQLString(ast.getChild(1).getText());
+        try {
+          CodeSource udfSrc = Class.forName(udfClassName).getProtectionDomain().getCodeSource();
+          if (udfSrc == null) {
+            throw new SemanticException("Could not resolve the jar for UDF class " + udfClassName);
+          }
+          String udfJar = udfSrc.getLocation().getPath();
+          if (udfJar == null || udfJar.isEmpty()) {
+            throw new SemanticException("Could not find the jar for UDF class " + udfClassName +
+                "to validate privileges");
+          }
+          udfURI = parseURI(udfSrc.getLocation().toString(), true);
+        } catch (ClassNotFoundException e) {
+          throw new SemanticException("Error retrieving udf class", e);
         }
-        udfURI = parseURI(udfSrc.getLocation().toString(), true);
-      } catch (ClassNotFoundException e) {
-        throw new SemanticException("Error retrieving udf class", e);
-      }
-      // create/drop function is allowed with any database
-      currDB = Database.ALL;
-      break;
-    case HiveParser.TOK_DROPFUNCTION:
-      // create/drop function is allowed with any database
-      currDB = Database.ALL;
-      break;
-    case HiveParser.TOK_SHOW_TABLESTATUS:
-    case HiveParser.TOK_SHOW_CREATETABLE:
-    case HiveParser.TOK_SHOWINDEXES:
-    case HiveParser.TOK_SHOWPARTITIONS:
-      // Find the target table for metadata operations, these are not covered in the read entities by the compiler
-      currTab = new Table(BaseSemanticAnalyzer.getUnescapedName((ASTNode) ast.getChild(0)));
-      currDB = getCanonicalDb();
-      break;
-    case HiveParser.TOK_SHOW_TBLPROPERTIES:
-      currTab = new Table(BaseSemanticAnalyzer.
-          getUnescapedName((ASTNode) ast.getChild(0)));
-      currDB = getCanonicalDb();
-      break;
-    case HiveParser.TOK_LOAD:
-      String dbName = BaseSemanticAnalyzer.unescapeIdentifier(ast.getChild(1).getChild(0).getChild(0).getText());
-      currDB = new Database(dbName);
-      break;
-    default:
-      currDB = getCanonicalDb();
-      break;
+        // create/drop function is allowed with any database
+        currDB = Database.ALL;
+        break;
+      case HiveParser.TOK_DROPFUNCTION:
+        // create/drop function is allowed with any database
+        currDB = Database.ALL;
+        break;
+
+      case HiveParser.TOK_LOAD:
+        String dbName = BaseSemanticAnalyzer.unescapeIdentifier(ast.getChild(1).getChild(0).getChild(0).getText());
+        currDB = new Database(dbName);
+        break;
+      default:
+        currDB = getCanonicalDb();
+        break;
     }
     return ast;
   }
@@ -227,13 +230,21 @@ implements HiveDriverFilterHook {
   }
 
   private Database extractDatabase(ASTNode ast) throws SemanticException {
-    String tableName = BaseSemanticAnalyzer.getUnescapedName((ASTNode)ast.getChild(0));
+    String tableName = BaseSemanticAnalyzer.getUnescapedName(ast);
     if (tableName.contains(".")) {
       return new Database((tableName.split("\\."))[0]);
     } else {
       return getCanonicalDb();
     }
   }
+  private Table extractTable(ASTNode ast) throws SemanticException {
+    String tableName = BaseSemanticAnalyzer.getUnescapedName(ast);
+    if (tableName.contains(".")) {
+      return new Table((tableName.split("\\."))[1]);
+    } else {
+      return new Table(tableName);
+    }
+  }
 
   @VisibleForTesting
   protected static AccessURI extractPartition(ASTNode ast) throws SemanticException {
@@ -273,8 +284,9 @@ implements HiveDriverFilterHook {
   public void postAnalyze(HiveSemanticAnalyzerHookContext context,
       List<Task<? extends Serializable>> rootTasks) throws SemanticException {
     HiveOperation stmtOperation = getCurrentHiveStmtOp();
-    HiveAuthzPrivileges stmtAuthObject =
-        HiveAuthzPrivilegesMap.getHiveAuthzPrivileges(stmtOperation);
+    HiveAuthzPrivileges stmtAuthObject;
+
+    stmtAuthObject = HiveAuthzPrivilegesMap.getHiveAuthzPrivileges(stmtOperation);
 
     // must occur above the null check on stmtAuthObject
     // since GRANT/REVOKE/etc are not authorized by binding layer at present
@@ -382,10 +394,6 @@ implements HiveDriverFilterHook {
       dbHierarchy.add(currDB);
       inputHierarchy.add(dbHierarchy);
       outputHierarchy.add(dbHierarchy);
-      // workaround for add partitions
-      if(partitionURI != null) {
-        inputHierarchy.add(ImmutableList.of(hiveAuthzBinding.getAuthServer(), partitionURI));
-      }
 
       for(ReadEntity readEntity:inputs) {
       	 // If this is a UDF, then check whether its allowed to be executed
@@ -403,6 +411,11 @@ implements HiveDriverFilterHook {
       }
       break;
     case TABLE:
+      // workaround for add partitions
+      if(partitionURI != null) {
+        inputHierarchy.add(ImmutableList.of(hiveAuthzBinding.getAuthServer(), partitionURI));
+      }
+
       for (ReadEntity readEntity: inputs) {
         // skip the tables/view that are part of expanded view definition.
         if (isChildTabForView(readEntity)) {

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/b41d9ee3/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingPreExecHook.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingPreExecHook.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingPreExecHook.java
index 7859521..813200a 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingPreExecHook.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingPreExecHook.java
@@ -57,7 +57,7 @@ public class HiveAuthzBindingPreExecHook implements ExecuteWithHookContext {
         List<DBModelAuthorizable> serverHierarchy = new ArrayList<DBModelAuthorizable>();
 
         serverHierarchy.add(hiveAuthzBinding.getAuthServer());
-        outputHierarchy.add(serverHierarchy);
+        inputHierarchy.add(serverHierarchy);
         hiveAuthzBinding.authorize(HiveOperation.QUERY,
           HiveAuthzPrivilegesMap.getHiveExtendedAuthzPrivileges(HiveExtendedOperation.TRANSFORM),
           new Subject(hookContext.getUserName()), inputHierarchy, outputHierarchy);

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/b41d9ee3/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzBinding.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzBinding.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzBinding.java
index cedf368..5a8ca6f 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzBinding.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzBinding.java
@@ -16,6 +16,7 @@
  */
 package org.apache.sentry.binding.hive.authz;
 
+import java.io.Serializable;
 import java.lang.reflect.Constructor;
 import java.util.EnumSet;
 import java.util.HashSet;
@@ -285,50 +286,71 @@ public class HiveAuthzBinding {
           " for subject " + subject.getName());
     }
 
-      /* for each read and write entity captured by the compiler -
-       *    check if that object type is part of the input/output privilege list
-       *    If it is, then validate the access.
-       * Note the hive compiler gathers information on additional entities like partitions,
-       * etc which are not of our interest at this point. Hence its very
-       * much possible that the we won't be validating all the entities in the given list
-       */
-
-      // Check read entities
-      Map<AuthorizableType, EnumSet<DBModelAction>> requiredInputPrivileges =
-          stmtAuthPrivileges.getInputPrivileges();
+    /* for each read and write entity captured by the compiler -
+     *    check if that object type is part of the input/output privilege list
+     *    If it is, then validate the access.
+     * Note the hive compiler gathers information on additional entities like partitions,
+     * etc which are not of our interest at this point. Hence its very
+     * much possible that the we won't be validating all the entities in the given list
+     */
+
+    // Check read entities
+    Map<AuthorizableType, EnumSet<DBModelAction>> requiredInputPrivileges =
+        stmtAuthPrivileges.getInputPrivileges();
+    if(isDebug) {
+      LOG.debug("requiredInputPrivileges = " + requiredInputPrivileges);
+      LOG.debug("inputHierarchyList = " + inputHierarchyList);
+    }
+    Map<AuthorizableType, EnumSet<DBModelAction>> requiredOutputPrivileges =
+        stmtAuthPrivileges.getOutputPrivileges();
+    if(isDebug) {
+      LOG.debug("requiredOuputPrivileges = " + requiredOutputPrivileges);
+      LOG.debug("outputHierarchyList = " + outputHierarchyList);
+    }
+
+    boolean found = false;
+    for(AuthorizableType key: requiredInputPrivileges.keySet()) {
       for (List<DBModelAuthorizable> inputHierarchy : inputHierarchyList) {
-        if(isDebug) {
-          LOG.debug("requiredInputPrivileges = " + requiredInputPrivileges);
-          LOG.debug("inputHierarchy = " + inputHierarchy);
-          LOG.debug("getAuthzType(inputHierarchy) = " + getAuthzType(inputHierarchy));
-        }
-        if (requiredInputPrivileges.containsKey(getAuthzType(inputHierarchy))) {
-          EnumSet<DBModelAction> inputPrivSet =
-            requiredInputPrivileges.get(getAuthzType(inputHierarchy));
-          if (!authProvider.hasAccess(subject, inputHierarchy, inputPrivSet, activeRoleSet)) {
+        if (getAuthzType(inputHierarchy).equals(key)) {
+          found = true;
+          if (!authProvider.hasAccess(subject, inputHierarchy, requiredInputPrivileges.get(key), activeRoleSet)) {
             throw new AuthorizationException("User " + subject.getName() +
                 " does not have privileges for " + hiveOp.name());
           }
         }
       }
-      // Check write entities
-      Map<AuthorizableType, EnumSet<DBModelAction>> requiredOutputPrivileges =
-          stmtAuthPrivileges.getOutputPrivileges();
+      if(!found && !(key.equals(AuthorizableType.URI)) &&  !(hiveOp.equals(HiveOperation.QUERY))) {
+        //URI privileges are optional for some privileges: anyPrivilege, tableDDLAndOptionalUriPrivilege
+        //Query can mean select/insert/analyze where all of them have different required privileges.
+        //For these alone we skip if there is no equivalent input privilege
+        //TODO: Even this case should be handled to make sure we do not skip the privilege check if we did not build
+        //the input privileges correctly
+        throw new AuthorizationException("Required privilege( " + key.name() + ") not available in input privileges");
+      }
+      found = false;
+    }
+
+    for(AuthorizableType key: requiredOutputPrivileges.keySet()) {
       for (List<DBModelAuthorizable> outputHierarchy : outputHierarchyList) {
-        if(isDebug) {
-          LOG.debug("requiredOutputPrivileges = " + requiredOutputPrivileges);
-          LOG.debug("outputHierarchy = " + outputHierarchy);
-          LOG.debug("getAuthzType(outputHierarchy) = " + getAuthzType(outputHierarchy));
-        }
-        if (requiredOutputPrivileges.containsKey(getAuthzType(outputHierarchy))) {
-          EnumSet<DBModelAction> outputPrivSet =
-            requiredOutputPrivileges.get(getAuthzType(outputHierarchy));
-          if (!authProvider.hasAccess(subject, outputHierarchy, outputPrivSet, activeRoleSet)) {
+        if (getAuthzType(outputHierarchy).equals(key)) {
+          found = true;
+          if (!authProvider.hasAccess(subject, outputHierarchy, requiredOutputPrivileges.get(key), activeRoleSet)) {
             throw new AuthorizationException("User " + subject.getName() +
                 " does not have privileges for " + hiveOp.name());
           }
         }
       }
+      if(!found && !(key.equals(AuthorizableType.URI)) &&  !(hiveOp.equals(HiveOperation.QUERY))) {
+        //URI privileges are optional for some privileges: tableInsertPrivilege
+        //Query can mean select/insert/analyze where all of them have different required privileges.
+        //For these alone we skip if there is no equivalent output privilege
+        //TODO: Even this case should be handled to make sure we do not skip the privilege check if we did not build
+        //the output privileges correctly
+        throw new AuthorizationException("Required privilege( " + key.name() + ") not available in output privileges");
+      }
+      found = false;
+    }
+
   }
 
   public void setActiveRoleSet(String activeRoleSet,

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/b41d9ee3/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzPrivilegesMap.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzPrivilegesMap.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzPrivilegesMap.java
index 7d241d0..4b2723a 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzPrivilegesMap.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzPrivilegesMap.java
@@ -32,26 +32,40 @@ public class HiveAuthzPrivilegesMap {
     new HashMap<HiveOperation, HiveAuthzPrivileges>();
   private static final Map <HiveExtendedOperation, HiveAuthzPrivileges> hiveAuthzExtendedPrivMap =
     new HashMap<HiveExtendedOperation, HiveAuthzPrivileges>();
-
   static {
     HiveAuthzPrivileges tableDDLPrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
-        addOutputObjectPriviledge(AuthorizableType.Table, EnumSet.of(DBModelAction.ALL)).
+        addInputObjectPriviledge(AuthorizableType.Table, EnumSet.of(DBModelAction.ALL)).
+        setOperationScope(HiveOperationScope.TABLE).
+        setOperationType(HiveOperationType.DDL).
+        build();
+    HiveAuthzPrivileges tableDDLAndUriPrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
+        addInputObjectPriviledge(AuthorizableType.Table, EnumSet.of(DBModelAction.ALL)).
         addInputObjectPriviledge(AuthorizableType.URI, EnumSet.of(DBModelAction.SELECT)).
         setOperationScope(HiveOperationScope.TABLE).
         setOperationType(HiveOperationType.DDL).
         build();
-    /* Currently Hive treats both select and insert as Query
-     * The difference is that the insert also has output table entities
+    HiveAuthzPrivileges tableDDLAndOptionalUriPrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
+        addInputObjectPriviledge(AuthorizableType.Table, EnumSet.of(DBModelAction.ALL)).
+        addInputObjectPriviledge(AuthorizableType.URI, EnumSet.of(DBModelAction.SELECT)).//TODO: make it optional
+        setOperationScope(HiveOperationScope.TABLE).
+        setOperationType(HiveOperationType.DDL).
+        build();
+
+    /* Currently Hive treats select/insert/analyze as Query
+     * select = select on table
+     * insert = insert on table /all on uri
+     * analyze = select + insert on table
      */
     HiveAuthzPrivileges tableQueryPrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
         addInputObjectPriviledge(AuthorizableType.Table, EnumSet.of(DBModelAction.SELECT)).
         addOutputObjectPriviledge(AuthorizableType.Table, EnumSet.of(DBModelAction.INSERT)).
-        addOutputObjectPriviledge(AuthorizableType.URI, EnumSet.of(DBModelAction.INSERT)).
+        addOutputObjectPriviledge(AuthorizableType.URI, EnumSet.of(DBModelAction.ALL)).
         setOperationScope(HiveOperationScope.TABLE).
         setOperationType(HiveOperationType.QUERY).
         build();
+
     HiveAuthzPrivileges tableLoadPrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
-        addInputObjectPriviledge(AuthorizableType.URI, EnumSet.of(DBModelAction.SELECT)).
+        addInputObjectPriviledge(AuthorizableType.URI, EnumSet.of(DBModelAction.ALL)).
         addOutputObjectPriviledge(AuthorizableType.Table, EnumSet.of(DBModelAction.INSERT)).
         setOperationScope(HiveOperationScope.TABLE).
         setOperationType(HiveOperationType.DATA_LOAD).
@@ -59,7 +73,7 @@ public class HiveAuthzPrivilegesMap {
 
     HiveAuthzPrivileges tableExportPrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
         addInputObjectPriviledge(AuthorizableType.Table, EnumSet.of(DBModelAction.SELECT)).
-        addOutputObjectPriviledge(AuthorizableType.URI, EnumSet.of(DBModelAction.INSERT)).
+        addOutputObjectPriviledge(AuthorizableType.URI, EnumSet.of(DBModelAction.ALL)).
         setOperationScope(HiveOperationScope.TABLE).
         setOperationType(HiveOperationType.DATA_UNLOAD).
         build();
@@ -71,15 +85,21 @@ public class HiveAuthzPrivilegesMap {
         build();
 
     HiveAuthzPrivileges dbDDLPrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
-        addOutputObjectPriviledge(AuthorizableType.Db, EnumSet.of(DBModelAction.ALL)).
-        addInputObjectPriviledge(AuthorizableType.URI, EnumSet.of(DBModelAction.ALL)).
+        addInputObjectPriviledge(AuthorizableType.Db, EnumSet.of(DBModelAction.ALL)).
+        setOperationScope(HiveOperationScope.DATABASE).
+        setOperationType(HiveOperationType.DDL).
+        build();
+
+    HiveAuthzPrivileges createTablePrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
+        addInputObjectPriviledge(AuthorizableType.Db, EnumSet.of(DBModelAction.ALL)).
+        addInputObjectPriviledge(AuthorizableType.URI, EnumSet.of(DBModelAction.ALL)).//TODO: make it optional
         setOperationScope(HiveOperationScope.DATABASE).
         setOperationType(HiveOperationType.DDL).
         build();
 
     HiveAuthzPrivileges dbImportPrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
         addOutputObjectPriviledge(AuthorizableType.Db, EnumSet.of(DBModelAction.ALL)).
-        addInputObjectPriviledge(AuthorizableType.URI, EnumSet.of(DBModelAction.SELECT)).
+        addInputObjectPriviledge(AuthorizableType.URI, EnumSet.of(DBModelAction.ALL)).
         setOperationScope(HiveOperationScope.DATABASE).
         setOperationType(HiveOperationType.DDL).
         build();
@@ -87,13 +107,13 @@ public class HiveAuthzPrivilegesMap {
     HiveAuthzPrivileges createViewPrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
     addOutputObjectPriviledge(AuthorizableType.Db, EnumSet.of(DBModelAction.ALL)).
     addInputObjectPriviledge(AuthorizableType.Table, EnumSet.of(DBModelAction.SELECT)).
-    addInputObjectPriviledge(AuthorizableType.URI, EnumSet.of(DBModelAction.SELECT)).
+    addInputObjectPriviledge(AuthorizableType.URI, EnumSet.of(DBModelAction.ALL)).//TODO: This should not be required
     setOperationScope(HiveOperationScope.DATABASE).
     setOperationType(HiveOperationType.DDL).
     build();
 
     HiveAuthzPrivileges dbMetaDataPrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
-      addInputObjectPriviledge(AuthorizableType.Db, EnumSet.of(DBModelAction.SELECT)).
+      addInputObjectPriviledge(AuthorizableType.Db, EnumSet.of(DBModelAction.SELECT, DBModelAction.INSERT)).
       setOperationScope(HiveOperationScope.DATABASE).
       setOperationType(HiveOperationType.INFO).
       build();
@@ -105,36 +125,73 @@ public class HiveAuthzPrivilegesMap {
         build();
     HiveAuthzPrivileges serverPrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
         addInputObjectPriviledge(AuthorizableType.Server, EnumSet.of(DBModelAction.ALL)).
-        addOutputObjectPriviledge(AuthorizableType.Server, EnumSet.of(DBModelAction.ALL)).
         setOperationScope(HiveOperationScope.SERVER).
         setOperationType(HiveOperationType.DDL).
         build();
 
+
     HiveAuthzPrivileges anyPrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
         addInputObjectPriviledge(AuthorizableType.Table, EnumSet.of(DBModelAction.SELECT, DBModelAction.INSERT)).
-        addInputObjectPriviledge(AuthorizableType.URI, EnumSet.of(DBModelAction.SELECT)).
+        addInputObjectPriviledge(AuthorizableType.URI, EnumSet.of(DBModelAction.ALL)). //TODO: make them ||
         setOperationScope(HiveOperationScope.CONNECT).
         setOperationType(HiveOperationType.QUERY).
         build();
 
-    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_ADDCOLS, dbDDLPrivilege);
-    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_REPLACECOLS, dbDDLPrivilege);
-    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_RENAMECOL, dbDDLPrivilege);
-    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_RENAMEPART, dbDDLPrivilege);
-    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_RENAME, dbDDLPrivilege);
-    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_DROPPARTS, dbDDLPrivilege);
-    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_ADDPARTS, dbDDLPrivilege);
-    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_TOUCH, dbDDLPrivilege);
-    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_ARCHIVE, dbDDLPrivilege);
-    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_UNARCHIVE, dbDDLPrivilege);
-    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_SERIALIZER, dbDDLPrivilege);
-    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_PROPERTIES, dbDDLPrivilege);
-    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERPARTITION_SERIALIZER, dbDDLPrivilege);
-    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_SERDEPROPERTIES, dbDDLPrivilege);
-    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERPARTITION_SERDEPROPERTIES, dbDDLPrivilege);
-    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_CLUSTER_SORT, dbDDLPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.CREATEDATABASE, serverPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.DROPDATABASE, dbDDLPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.CREATETABLE, createTablePrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.DROPTABLE, tableDDLPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.CREATEVIEW, createViewPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.DROPVIEW, tableDDLPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.CREATEINDEX, tableDDLPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.DROPINDEX, tableDDLPrivilege);
+
+
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_RENAME, tableDDLPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_PROPERTIES, tableDDLPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_SERDEPROPERTIES, tableDDLPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_CLUSTER_SORT, tableDDLPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_FILEFORMAT, tableDDLPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_TOUCH, tableDDLPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_PROTECTMODE, tableDDLPrivilege);
+
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_RENAMECOL, tableDDLPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_ADDCOLS, tableDDLPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_REPLACECOLS, tableDDLPrivilege);
+
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_ADDPARTS, tableDDLAndOptionalUriPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_RENAMEPART, tableDDLPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_DROPPARTS, tableDDLPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_ARCHIVE, tableDDLPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_UNARCHIVE, tableDDLPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERPARTITION_FILEFORMAT, tableDDLPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERPARTITION_PROTECTMODE, tableDDLPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERPARTITION_SERDEPROPERTIES, tableDDLPrivilege);
+
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_SERIALIZER, tableDDLPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_MERGEFILES, tableDDLPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_SKEWED, tableDDLPrivilege);
+
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERPARTITION_SERIALIZER, tableDDLPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERPARTITION_MERGEFILES, tableDDLPrivilege);
+
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERINDEX_PROPS, tableDDLPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERINDEX_REBUILD, tableDDLPrivilege);
+
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERVIEW_PROPERTIES, tableDDLPrivilege);
+
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_LOCATION, tableDDLAndUriPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERPARTITION_LOCATION, tableDDLAndUriPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTBLPART_SKEWED_LOCATION, tableDDLAndUriPrivilege);
+
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERDATABASE, dbDDLPrivilege);
+
     hiveAuthzStmtPrivMap.put(HiveOperation.ANALYZE_TABLE, tableQueryPrivilege);
+
     hiveAuthzStmtPrivMap.put(HiveOperation.SWITCHDATABASE, anyPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.CREATEFUNCTION, anyPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.DROPFUNCTION, anyPrivilege);
+
     // SHOWDATABASES
     // SHOWTABLES
     hiveAuthzStmtPrivMap.put(HiveOperation.SHOWCOLUMNS, tableMetaDataPrivilege);
@@ -145,19 +202,9 @@ public class HiveAuthzPrivilegesMap {
     hiveAuthzStmtPrivMap.put(HiveOperation.SHOWINDEXES, tableMetaDataPrivilege);
     hiveAuthzStmtPrivMap.put(HiveOperation.SHOWPARTITIONS, tableMetaDataPrivilege);
     // SHOWLOCKS
-    hiveAuthzStmtPrivMap.put(HiveOperation.CREATEFUNCTION, anyPrivilege);
-    hiveAuthzStmtPrivMap.put(HiveOperation.DROPFUNCTION, anyPrivilege);
-    hiveAuthzStmtPrivMap.put(HiveOperation.CREATEVIEW, createViewPrivilege);
-    hiveAuthzStmtPrivMap.put(HiveOperation.DROPVIEW, dbDDLPrivilege);
-    hiveAuthzStmtPrivMap.put(HiveOperation.CREATEINDEX, dbDDLPrivilege);
-    hiveAuthzStmtPrivMap.put(HiveOperation.DROPINDEX, dbDDLPrivilege);
-    hiveAuthzStmtPrivMap.put(HiveOperation.DROPDATABASE, serverPrivilege);
-    hiveAuthzStmtPrivMap.put(HiveOperation.DROPTABLE, dbDDLPrivilege);
     hiveAuthzStmtPrivMap.put(HiveOperation.EXPORT, tableExportPrivilege);
     hiveAuthzStmtPrivMap.put(HiveOperation.IMPORT, dbImportPrivilege);
     hiveAuthzStmtPrivMap.put(HiveOperation.LOAD, tableLoadPrivilege);
-    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERINDEX_REBUILD, tableDDLPrivilege);
-    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERVIEW_PROPERTIES, tableDDLPrivilege);
     hiveAuthzStmtPrivMap.put(HiveOperation.LOCKTABLE, tableDMLPrivilege);
     hiveAuthzStmtPrivMap.put(HiveOperation.UNLOCKTABLE, tableDMLPrivilege);
     // CREATEROLE
@@ -168,14 +215,6 @@ public class HiveAuthzPrivilegesMap {
     // GRANT_ROLE
     // REVOKE_ROLE
     // SHOW_ROLE_GRANT
-    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_PROTECTMODE, dbDDLPrivilege);
-    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERPARTITION_PROTECTMODE, dbDDLPrivilege);
-    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_FILEFORMAT, dbDDLPrivilege);
-    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERPARTITION_FILEFORMAT, dbDDLPrivilege);
-    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_LOCATION, serverPrivilege);
-    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERPARTITION_LOCATION, serverPrivilege);
-    hiveAuthzStmtPrivMap.put(HiveOperation.CREATEDATABASE, serverPrivilege);
-    hiveAuthzStmtPrivMap.put(HiveOperation.CREATETABLE, dbDDLPrivilege);
     hiveAuthzStmtPrivMap.put(HiveOperation.CREATETABLE_AS_SELECT,
         new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
         addInputObjectPriviledge(AuthorizableType.Table, EnumSet.of(DBModelAction.SELECT)).
@@ -184,14 +223,8 @@ public class HiveAuthzPrivilegesMap {
         setOperationType(HiveOperationType.DDL).
         build());
     hiveAuthzStmtPrivMap.put(HiveOperation.QUERY, tableQueryPrivilege);
-    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERINDEX_PROPS, dbDDLPrivilege);
-    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERDATABASE, dbDDLPrivilege);
     hiveAuthzStmtPrivMap.put(HiveOperation.DESCDATABASE, dbMetaDataPrivilege);
     hiveAuthzStmtPrivMap.put(HiveOperation.DESCTABLE, tableMetaDataPrivilege);
-    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_MERGEFILES, dbDDLPrivilege);
-    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERPARTITION_MERGEFILES, dbDDLPrivilege);
-    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_SKEWED, dbDDLPrivilege);
-    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTBLPART_SKEWED_LOCATION, dbDDLPrivilege);
 
     hiveAuthzExtendedPrivMap.put(HiveExtendedOperation.TRANSFORM, serverPrivilege);
   }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/b41d9ee3/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/TestHiveAuthzBindings.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/TestHiveAuthzBindings.java b/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/TestHiveAuthzBindings.java
index 1f9d1ec..6e40a5c 100644
--- a/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/TestHiveAuthzBindings.java
+++ b/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/TestHiveAuthzBindings.java
@@ -156,7 +156,7 @@ public class TestHiveAuthzBindings {
    */
   @Test
   public void testValidateCreateTabPrivilegesForAdmin() throws Exception {
-    outputTabHierarcyList.add(buildObjectHierarchy(SERVER1, CUSTOMER_DB, PAYMENT_TAB));
+    inputTabHierarcyList.add(buildObjectHierarchy(SERVER1, CUSTOMER_DB, null));
     testAuth.authorize(HiveOperation.CREATETABLE, createTabPrivileges, ADMIN_SUBJECT,
         inputTabHierarcyList, outputTabHierarcyList);
   }
@@ -166,7 +166,7 @@ public class TestHiveAuthzBindings {
    */
   @Test
   public void testValidateCreateTabPrivilegesForUser() throws Exception {
-    outputTabHierarcyList.add(buildObjectHierarchy(SERVER1, JUNIOR_ANALYST_DB, PAYMENT_TAB));
+    inputTabHierarcyList.add(buildObjectHierarchy(SERVER1, JUNIOR_ANALYST_DB, null));
     testAuth.authorize(HiveOperation.CREATETABLE, createTabPrivileges, MANAGER_SUBJECT,
         inputTabHierarcyList, outputTabHierarcyList);
   }
@@ -225,7 +225,7 @@ public class TestHiveAuthzBindings {
    */
   @Test
   public void testValidateCreateDbForAdmin() throws Exception {
-    outputTabHierarcyList.add(buildObjectHierarchy(SERVER1, null, null));
+    inputTabHierarcyList.add(buildObjectHierarchy(SERVER1, null, null));
     testAuth.authorize(HiveOperation.CREATEDATABASE, createDbPrivileges, ADMIN_SUBJECT,
         inputTabHierarcyList, outputTabHierarcyList);
   }
@@ -246,7 +246,7 @@ public class TestHiveAuthzBindings {
    */
   @Test
   public void testValidateCreateFunctionForAdmin() throws Exception {
-    inputTabHierarcyList.add(buildObjectHierarchy(SERVER1, null, null));
+    inputTabHierarcyList.add(buildObjectHierarchy(SERVER1, CUSTOMER_DB, PURCHASES_TAB));
     inputTabHierarcyList.add(Arrays.asList(new DBModelAuthorizable[] {
         new Server(SERVER1), new AccessURI("file:///some/path/to/a/jar")
     }));

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/b41d9ee3/sentry-tests/sentry-tests-hive/pom.xml
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/pom.xml b/sentry-tests/sentry-tests-hive/pom.xml
index d66627f..1c95a8c 100644
--- a/sentry-tests/sentry-tests-hive/pom.xml
+++ b/sentry-tests/sentry-tests-hive/pom.xml
@@ -442,6 +442,7 @@ limitations under the License.
           <include>**/TestUriPermissions.java</include>
           <include>**/TestRuntimeMetadataRetrieval.java</include>
           <include>**/TestPrivilegesAtFunctionScope.java</include>
+          <include>**/TestOperations.java</include>
         </includes>
         <argLine>-Dsentry.e2etest.hiveServer2Type=UnmanagedHiveServer2 -Dsentry.e2etest.DFSType=ClusterDFS</argLine>
        </configuration>
@@ -506,6 +507,7 @@ limitations under the License.
           <include>**/TestDbRuntimeMetadataRetrieval.java</include>
           <include>**/TestDbPrivilegesAtFunctionScope.java</include>
           <include>**/TestDatabaseProvider.java</include>
+          <include>**/TestDbOperations.java</include>
         </includes>
         <argLine>-Dsentry.e2etest.hiveServer2Type=UnmanagedHiveServer2 -Dsentry.e2etest.DFSType=ClusterDFS -Dsentry.e2etest.external.sentry=true</argLine>
        </configuration>

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/b41d9ee3/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbOperations.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbOperations.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbOperations.java
new file mode 100644
index 0000000..3fab344
--- /dev/null
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbOperations.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.sentry.tests.e2e.dbprovider;
+
+import org.apache.sentry.tests.e2e.hive.AbstractTestWithStaticConfiguration;
+import org.apache.sentry.tests.e2e.hive.TestOperations;
+import org.junit.Before;
+import org.junit.BeforeClass;
+
+public class TestDbOperations extends TestOperations{
+  @Override
+  @Before
+  public void setup() throws Exception {
+    super.setupAdmin();
+    super.setup();
+  }
+  @BeforeClass
+  public static void setupTestStaticConfiguration() throws Exception {
+    useSentryService = true;
+    AbstractTestWithStaticConfiguration.setupTestStaticConfiguration();
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/b41d9ee3/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/AbstractTestWithStaticConfiguration.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/AbstractTestWithStaticConfiguration.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/AbstractTestWithStaticConfiguration.java
index b92ca97..422c623 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/AbstractTestWithStaticConfiguration.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/AbstractTestWithStaticConfiguration.java
@@ -154,11 +154,13 @@ public abstract class AbstractTestWithStaticConfiguration {
       statement.execute("DROP TABLE IF EXISTS " + table);
       statement.execute("create table " + table
           + " (under_col int comment 'the under column', value string)");
-      statement.execute("load data local inpath '" + dataFile.getPath()
-          + "' into table " + table);
-      ResultSet res = statement.executeQuery("select * from " + table);
-      Assert.assertTrue("Table should have data after load", res.next());
-      res.close();
+      if(dataFile != null) {
+        statement.execute("load data local inpath '" + dataFile.getPath()
+            + "' into table " + table);
+        ResultSet res = statement.executeQuery("select * from " + table);
+        Assert.assertTrue("Table should have data after load", res.next());
+        res.close();
+      }
     }
     statement.close();
     connection.close();

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/b41d9ee3/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/Context.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/Context.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/Context.java
index e725eb0..f609c46 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/Context.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/Context.java
@@ -195,6 +195,17 @@ public class Context {
     }
   }
 
+  public void assertSentrySemanticException(Statement statement, String query, String exceptionType)
+      throws SQLException {
+    try {
+      statement.execute(query);
+      Assert.fail("Expected SQLException for '" + query + "'");
+    } catch (SQLException e) {
+      verifyAuthzExceptionForState(e, AUTHZ_EXCEPTION_SQL_STATE);
+      Assert.assertTrue("Expected " + exceptionType + " : " + e.getMessage(),
+          Strings.nullToEmpty(e.getMessage()).contains(exceptionType));
+    }
+  }
   // verify that the sqlexception is due to authorization failure
   public void verifyAuthzException(SQLException sqlException) throws SQLException{
     verifyAuthzExceptionForState(sqlException, AUTHZ_EXCEPTION_SQL_STATE);

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/b41d9ee3/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestCrossDbOps.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestCrossDbOps.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestCrossDbOps.java
index 8552cc0..447f4d9 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestCrossDbOps.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestCrossDbOps.java
@@ -605,10 +605,6 @@ public class TestCrossDbOps extends AbstractTestWithStaticConfiguration {
     statement.execute("load data local inpath '" + dataFile.getPath()
         + "' into table " + DB2 + "." + TBL3);
 
-    // c
-    context.assertAuthzException(statement, "DROP DATABASE IF EXISTS " + DB1);
-    context.assertAuthzException(statement, "DROP DATABASE IF EXISTS " + DB2);
-
     policyFile.removePermissionsFromRole(GROUP1_ROLE, ALL_DB2);
     writePolicyFile(policyFile);
     // create db1.view1 as select from db2.tbl2

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/b41d9ee3/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestOperations.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestOperations.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestOperations.java
new file mode 100644
index 0000000..22f7dce
--- /dev/null
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestOperations.java
@@ -0,0 +1,672 @@
+/**
+ * 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.sentry.tests.e2e.hive;
+
+import com.google.common.io.Resources;
+import org.apache.sentry.provider.file.PolicyFile;
+import org.apache.sentry.tests.e2e.hive.hiveserver.HiveServerFactory;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.HashMap;
+import java.util.Map;
+
+public class TestOperations extends AbstractTestWithStaticConfiguration {
+  private PolicyFile policyFile;
+  final String dbName = "db1";
+  final String tableName = "tb1";
+  static String warehouseDir = hiveServer.getProperty(HiveServerFactory.WAREHOUSE_DIR);
+  final String semanticException = "SemanticException No valid privileges";
+
+  static final Map<String, String> privileges = new HashMap<String, String>();
+  static {
+    privileges.put("all_server", "server=server1->action=all");
+    privileges.put("all_db1", "server=server1->db=db1->action=all");
+    privileges.put("select_db1", "server=server1->db=db1->action=select");
+    privileges.put("insert_db1", "server=server1->db=db1->action=insert");
+    privileges.put("all_db2", "server=server1->db=db2->action=all");
+    privileges.put("all_db1_tb1", "server=server1->db=db1->table=tb1->action=all");
+    privileges.put("select_db1_tb1", "server=server1->db=db1->table=tb1->action=select");
+    privileges.put("insert_db1_tb1", "server=server1->db=db1->table=tb1->action=insert");
+    privileges.put("insert_db2_tb2", "server=server1->db=db2->table=tb2->action=insert");
+
+  }
+
+  @Before
+  public void setup() throws Exception{
+    policyFile = PolicyFile.setAdminOnServer1(ADMINGROUP)
+        .setUserGroupMapping(StaticUserGroup.getStaticMapping());
+    writePolicyFile(policyFile);
+  }
+
+  private void adminCreate(String db, String table) throws Exception{
+    adminCreate(db, table, false);
+  }
+
+  private void adminCreate(String db, String table, boolean partitioned) throws Exception{
+    Connection connection = context.createConnection(ADMIN1);
+    Statement statement = context.createStatement(connection);
+    statement.execute("DROP DATABASE IF EXISTS " + db + " CASCADE");
+    statement.execute("CREATE DATABASE " + db);
+    if(table !=null) {
+      if (partitioned) {
+        statement.execute("CREATE table  " + db + "." + table + " (a string) PARTITIONED BY (b string)");
+      } else{
+        statement.execute("CREATE table  " + db + "." + table + " (a string)");
+      }
+
+    }
+    statement.close();
+    connection.close();
+  }
+
+  private void adminCreatePartition() throws Exception{
+    Connection connection = context.createConnection(ADMIN1);
+    Statement statement = context.createStatement(connection);
+    statement.execute("USE db1");
+    statement.execute("ALTER TABLE tb1 ADD IF NOT EXISTS PARTITION (b = '1') ");
+    statement.close();
+    connection.close();
+  }
+
+  /* Test all operations that require all on Database alone
+  1. Create table : HiveOperation.CREATETABLE
+  2. Alter database : HiveOperation.ALTERDATABASE
+  3. Drop database : HiveOperation.DROPDATABASE
+   */
+  @Test
+  public void testAllOnDatabase() throws Exception{
+    adminCreate(dbName, null);
+    policyFile
+        .addPermissionsToRole("all_db1", privileges.get("all_db1"))
+        .addRolesToGroup(USERGROUP1, "all_db1");
+
+    writePolicyFile(policyFile);
+
+    Connection connection = context.createConnection(USER1_1);
+    Statement statement = context.createStatement(connection);
+    statement.execute("CREATE TABLE db1.tb1(a int)");
+    statement.execute("ALTER DATABASE db1 SET DBPROPERTIES ('comment'='comment')");
+    statement.execute("DROP database db1 cascade");
+    statement.close();
+    connection.close();
+
+    //Negative case
+    adminCreate(dbName, null);
+    policyFile
+        .addPermissionsToRole("select_db1", privileges.get("select_db1"))
+        .addRolesToGroup(USERGROUP2, "select_db1");
+    writePolicyFile(policyFile);
+
+    connection = context.createConnection(USER2_1);
+    statement = context.createStatement(connection);
+    context.assertSentrySemanticException(statement, "CREATE TABLE db1.tb1(a int)", semanticException);
+    context.assertSentrySemanticException(statement, "ALTER DATABASE db1 SET DBPROPERTIES ('comment'='comment')", semanticException);
+    context.assertSentrySemanticException(statement, "DROP database db1 cascade", semanticException);
+    statement.close();
+    connection.close();
+
+  }
+  /* SELECT/INSERT on DATABASE
+   1. HiveOperation.DESCDATABASE
+   */
+  @Test
+  public void testDescDB() throws Exception {
+    adminCreate(dbName, null);
+    policyFile
+        .addPermissionsToRole("select_db1", privileges.get("select_db1"))
+        .addPermissionsToRole("insert_db1", privileges.get("insert_db1"))
+        .addRolesToGroup(USERGROUP1, "select_db1")
+        .addRolesToGroup(USERGROUP2, "insert_db1");
+    writePolicyFile(policyFile);
+
+    Connection connection = context.createConnection(USER1_1);
+    Statement statement = context.createStatement(connection);
+    statement.execute("describe database db1");
+    statement.close();
+    connection.close();
+
+    connection = context.createConnection(USER2_1);
+    statement = context.createStatement(connection);
+    statement.execute("describe database db1");
+    statement.close();
+    connection.close();
+
+    //Negative case
+    policyFile
+        .addPermissionsToRole("all_db1_tb1", privileges.get("all_db1_tb1"))
+        .addRolesToGroup(USERGROUP3, "all_db1_tb1");
+    writePolicyFile(policyFile);
+    connection = context.createConnection(USER3_1);
+    statement = context.createStatement(connection);
+    context.assertSentrySemanticException(statement, "describe database db1", semanticException);
+    statement.close();
+    connection.close();
+
+  }
+
+  private void assertSemanticException(Statement stmt, String command) throws SQLException{
+    context.assertSentrySemanticException(stmt,command, semanticException);
+  }
+  /* Test all operations that require all on table alone
+  1. Create index : HiveOperation.CREATEINDEX
+  2. Drop index : HiveOperation.DROPINDEX
+  3. Alter table add partition : HiveOperation.ALTERTABLE_ADDPARTS
+  4. HiveOperation.ALTERTABLE_PROPERTIES
+  5. HiveOperation.ALTERTABLE_SERDEPROPERTIES
+  6. HiveOperation.ALTERTABLE_CLUSTER_SORT
+  7. HiveOperation.ALTERTABLE_TOUCH
+  8. HiveOperation.ALTERTABLE_PROTECTMODE
+  9. HiveOperation.ALTERTABLE_FILEFORMAT
+  10. HiveOperation.ALTERTABLE_RENAMEPART
+  11. HiveOperation.ALTERPARTITION_SERDEPROPERTIES
+  12. TODO: archive partition
+  13. TODO: unarchive partition
+  14. HiveOperation.ALTERPARTITION_FILEFORMAT
+  15. TODO: partition touch (is it same as  HiveOperation.ALTERTABLE_TOUCH?)
+  16. HiveOperation.ALTERPARTITION_PROTECTMODE
+  17. HiveOperation.ALTERTABLE_DROPPARTS
+  18. HiveOperation.ALTERTABLE_RENAMECOL
+  19. HiveOperation.ALTERTABLE_ADDCOLS
+  20. HiveOperation.ALTERTABLE_REPLACECOLS
+  21. TODO: HiveOperation.ALTERVIEW_PROPERTIES
+  22. HiveOperation.CREATEINDEX
+  23. TODO: HiveOperation.ALTERINDEX_REBUILD
+  21. HiveOperation.ALTERTABLE_RENAME
+  22. HiveOperation.DROPTABLE
+  23. TODO: HiveOperation.ALTERTABLE_SERIALIZER
+  24. TODO: HiveOperation.ALTERPARTITION_SERIALIZER
+  25. TODO: HiveOperation.ALTERINDEX_PROPS
+  */
+  @Test
+  public void testAllOnTable() throws Exception{
+    adminCreate(dbName, tableName, true);
+    policyFile
+        .addPermissionsToRole("all_db1_tb1", privileges.get("all_db1_tb1"))
+        .addRolesToGroup(USERGROUP1, "all_db1_tb1")
+        .addPermissionsToRole("insert_db1_tb1", privileges.get("insert_db1_tb1"))
+        .addRolesToGroup(USERGROUP2, "insert_db1_tb1");
+    writePolicyFile(policyFile);
+
+    Connection connection;
+    Statement statement;
+    //Negative test cases
+    connection = context.createConnection(USER2_1);
+    statement = context.createStatement(connection);
+    statement.execute("Use db1");
+    assertSemanticException(statement, "CREATE INDEX table01_index ON TABLE tb1 (a) AS 'COMPACT' WITH DEFERRED REBUILD");
+    assertSemanticException(statement, "DROP INDEX table01_index ON tb1");
+    assertSemanticException(statement, "ALTER TABLE tb1 ADD IF NOT EXISTS PARTITION (b = '1') ");
+    assertSemanticException(statement, "ALTER TABLE tb1 SET TBLPROPERTIES ('comment' = 'new_comment')");
+    assertSemanticException(statement, "ALTER TABLE tb1 SET SERDEPROPERTIES ('field.delim' = ',')");
+    assertSemanticException(statement, "ALTER TABLE tb1 CLUSTERED BY (a) SORTED BY (a) INTO 1 BUCKETS");
+    assertSemanticException(statement, "ALTER TABLE tb1 TOUCH");
+    assertSemanticException(statement, "ALTER TABLE tb1 ENABLE NO_DROP");
+    assertSemanticException(statement, "ALTER TABLE tb1 DISABLE OFFLINE");
+    assertSemanticException(statement, "ALTER TABLE tb1 SET FILEFORMAT RCFILE");
+
+    //Setup
+    connection = context.createConnection(USER1_1);
+    statement = context.createStatement(connection);
+    statement.execute("Use db1");
+    statement.execute("ALTER TABLE tb1 ADD IF NOT EXISTS PARTITION (b = '10') ");
+
+    //Negative test cases
+    connection = context.createConnection(USER2_1);
+    statement = context.createStatement(connection);
+    statement.execute("Use db1");
+    assertSemanticException(statement, "ALTER TABLE tb1 PARTITION (b = 10) RENAME TO PARTITION (b = 2)");
+    assertSemanticException(statement, "ALTER TABLE tb1 PARTITION (b = 10) SET SERDEPROPERTIES ('field.delim' = ',')");
+    //assertSemanticException(statement, "ALTER TABLE tb1 ARCHIVE PARTITION (b = 2)");
+    //assertSemanticException(statement, "ALTER TABLE tb1 UNARCHIVE PARTITION (b = 2)");
+    assertSemanticException(statement, "ALTER TABLE tb1 PARTITION (b = 10) SET FILEFORMAT RCFILE");
+    assertSemanticException(statement, "ALTER TABLE tb1 TOUCH PARTITION (b = 10)");
+    assertSemanticException(statement, "ALTER TABLE tb1 PARTITION (b = 10) DISABLE NO_DROP");
+    assertSemanticException(statement, "ALTER TABLE tb1 PARTITION (b = 10) DISABLE OFFLINE");
+    assertSemanticException(statement, "ALTER TABLE tb1 DROP PARTITION (b = 10)");
+
+    assertSemanticException(statement, "ALTER TABLE tb1 CHANGE COLUMN a c int");
+    assertSemanticException(statement, "ALTER TABLE tb1 ADD COLUMNS (a int)");
+    assertSemanticException(statement, "ALTER TABLE tb1 REPLACE COLUMNS (a int, c int)");
+
+    //assertSemanticException(statement, "ALTER VIEW view1 SET TBLPROPERTIES ('comment' = 'new_comment')");
+
+    assertSemanticException(statement, "CREATE INDEX tb1_index ON TABLE tb1 (a) AS 'COMPACT' WITH DEFERRED REBUILD");
+    //assertSemanticException(statement, "ALTER INDEX tb1_index ON tb1 REBUILD");
+    assertSemanticException(statement, "ALTER TABLE tb1 RENAME TO tb2");
+
+    assertSemanticException(statement, "DROP TABLE db1.tb1");
+
+    //Positive cases
+    connection = context.createConnection(USER1_1);
+    statement = context.createStatement(connection);
+    statement.execute("Use db1");
+    statement.execute("CREATE INDEX table01_index ON TABLE tb1 (a) AS 'COMPACT' WITH DEFERRED REBUILD");
+    statement.execute("DROP INDEX table01_index ON tb1");
+    statement.execute("ALTER TABLE tb1 ADD IF NOT EXISTS PARTITION (b = '1') ");
+    statement.execute("ALTER TABLE tb1 SET TBLPROPERTIES ('comment' = 'new_comment')");
+    statement.execute("ALTER TABLE tb1 SET SERDEPROPERTIES ('field.delim' = ',')");
+    statement.execute("ALTER TABLE tb1 CLUSTERED BY (a) SORTED BY (a) INTO 1 BUCKETS");
+    statement.execute("ALTER TABLE tb1 TOUCH");
+    statement.execute("ALTER TABLE tb1 ENABLE NO_DROP");
+    statement.execute("ALTER TABLE tb1 DISABLE OFFLINE");
+    statement.execute("ALTER TABLE tb1 SET FILEFORMAT RCFILE");
+
+    statement.execute("ALTER TABLE tb1 PARTITION (b = 1) RENAME TO PARTITION (b = 2)");
+    statement.execute("ALTER TABLE tb1 PARTITION (b = 2) SET SERDEPROPERTIES ('field.delim' = ',')");
+    //statement.execute("ALTER TABLE tb1 ARCHIVE PARTITION (b = 2)");
+    //statement.execute("ALTER TABLE tb1 UNARCHIVE PARTITION (b = 2)");
+    statement.execute("ALTER TABLE tb1 PARTITION (b = 2) SET FILEFORMAT RCFILE");
+    statement.execute("ALTER TABLE tb1 TOUCH PARTITION (b = 2)");
+    statement.execute("ALTER TABLE tb1 PARTITION (b = 2) DISABLE NO_DROP");
+    statement.execute("ALTER TABLE tb1 PARTITION (b = 2) DISABLE OFFLINE");
+    statement.execute("ALTER TABLE tb1 DROP PARTITION (b = 2)");
+
+    statement.execute("ALTER TABLE tb1 CHANGE COLUMN a c int");
+    statement.execute("ALTER TABLE tb1 ADD COLUMNS (a int)");
+    statement.execute("ALTER TABLE tb1 REPLACE COLUMNS (a int, c int)");
+
+    //statement.execute("ALTER VIEW view1 SET TBLPROPERTIES ('comment' = 'new_comment')");
+
+    statement.execute("CREATE INDEX tb1_index ON TABLE tb1 (a) AS 'COMPACT' WITH DEFERRED REBUILD");
+    //statement.execute("ALTER INDEX tb1_index ON tb1 REBUILD");
+    statement.execute("ALTER TABLE tb1 RENAME TO tb2");
+    statement.execute("DROP TABLE db1.tb1");
+
+    statement.close();
+    connection.close();
+
+  }
+
+  /*
+  1. Analyze table (HiveOperation.QUERY) : select + insert on table
+   */
+  @Test
+  public void testSelectAndInsertOnTable() throws Exception {
+    adminCreate(dbName, tableName, true);
+    adminCreatePartition();
+    policyFile
+        .addPermissionsToRole("select_db1_tb1", privileges.get("select_db1_tb1"))
+        .addPermissionsToRole("insert_db1_tb1", privileges.get("insert_db1_tb1"))
+        .addRolesToGroup(USERGROUP1, "select_db1_tb1", "insert_db1_tb1");
+    writePolicyFile(policyFile);
+
+    Connection connection = context.createConnection(USER1_1);
+    Statement statement = context.createStatement(connection);
+    statement.execute("Use db1");
+    statement.execute("ANALYZE TABLE tb1 PARTITION (b='1' ) COMPUTE STATISTICS");
+    statement.close();
+    connection.close();
+  }
+
+  /* Operations which require select on table alone
+  1. HiveOperation.QUERY
+  2. HiveOperation.SHOW_TBLPROPERTIES
+  3. HiveOperation.SHOW_CREATETABLE
+  4. HiveOperation.SHOWINDEXES
+  5. HiveOperation.SHOWCOLUMNS
+  6. Describe tb1 : HiveOperation.DESCTABLE5.
+  7. HiveOperation.SHOWPARTITIONS
+  8. TODO: show functions?
+   */
+  @Test
+  public void testSelectOnTable() throws Exception {
+    adminCreate(dbName, tableName, true);
+    adminCreatePartition();
+    policyFile
+        .addPermissionsToRole("select_db1_tb1", privileges.get("select_db1_tb1"))
+        .addRolesToGroup(USERGROUP1, "select_db1_tb1");
+    writePolicyFile(policyFile);
+
+    Connection connection = context.createConnection(USER1_1);
+    Statement statement = context.createStatement(connection);
+    statement.execute("Use db1");
+    statement.execute("select * from tb1");
+
+    statement.executeQuery("SHOW Partitions tb1");
+    statement.executeQuery("SHOW TBLPROPERTIES tb1");
+    statement.executeQuery("SHOW CREATE TABLE tb1");
+    statement.executeQuery("SHOW indexes on tb1");
+    statement.executeQuery("SHOW COLUMNS from tb1");
+    statement.executeQuery("SHOW functions '.*'");
+
+    statement.executeQuery("DESCRIBE tb1");
+    statement.executeQuery("DESCRIBE tb1 PARTITION (b=1)");
+
+    statement.close();
+    connection.close();
+
+    //Negative case
+    policyFile
+        .addPermissionsToRole("insert_db1_tb1", privileges.get("insert_db1_tb1"))
+        .addRolesToGroup(USERGROUP3, "insert_db1_tb1");
+    writePolicyFile(policyFile);
+    connection = context.createConnection(USER3_1);
+    statement = context.createStatement(connection);
+    statement.execute("Use db1");
+    context.assertSentrySemanticException(statement, "select * from tb1", semanticException);
+
+    statement.close();
+    connection.close();
+
+
+  }
+
+  /* Operations which require insert on table alone
+  1. HiveOperation.SHOW_TBLPROPERTIES
+  2. HiveOperation.SHOW_CREATETABLE
+  3. HiveOperation.SHOWINDEXES
+  4. HiveOperation.SHOWCOLUMNS
+  5. HiveOperation.DESCTABLE
+  6. HiveOperation.SHOWPARTITIONS
+  7. TODO: show functions?
+  8. TODO: lock, unlock, Show locks
+   */
+  @Test
+  public void testInsertOnTable() throws Exception {
+    adminCreate(dbName, tableName, true);
+    adminCreatePartition();
+    policyFile
+        .addPermissionsToRole("insert_db1_tb1", privileges.get("insert_db1_tb1"))
+        .addRolesToGroup(USERGROUP1, "insert_db1_tb1");
+    writePolicyFile(policyFile);
+
+    Connection connection = context.createConnection(USER1_1);
+    Statement statement = context.createStatement(connection);
+    statement.execute("Use db1");
+    /*statement.execute("LOCK TABLE tb1 EXCLUSIVE");
+    statement.execute("UNLOCK TABLE tb1");
+    */
+    statement.executeQuery("SHOW TBLPROPERTIES tb1");
+    statement.executeQuery("SHOW CREATE TABLE tb1");
+    statement.executeQuery("SHOW indexes on tb1");
+    statement.executeQuery("SHOW COLUMNS from tb1");
+    statement.executeQuery("SHOW functions '.*'");
+    //statement.executeQuery("SHOW LOCKS tb1");
+
+    //NoViableAltException
+    //statement.executeQuery("SHOW transactions");
+    //statement.executeQuery("SHOW compactions");
+    statement.executeQuery("DESCRIBE tb1");
+    statement.executeQuery("DESCRIBE tb1 PARTITION (b=1)");
+    statement.executeQuery("SHOW Partitions tb1");
+
+
+    statement.close();
+    connection.close();
+  }
+
+  /* Test all operations which require all on table + all on URI
+   1. HiveOperation.ALTERTABLE_LOCATION
+   2. HiveOperation.ALTERTABLE_ADDPARTS
+   3. TODO: HiveOperation.ALTERPARTITION_LOCATION
+   4. TODO: HiveOperation.ALTERTBLPART_SKEWED_LOCATION
+   */
+  @Test
+  public void testAlterAllOnTableAndURI() throws Exception {
+    adminCreate(dbName, tableName, true);
+    String tabLocation = warehouseDir + "/blah";
+    policyFile
+        .addPermissionsToRole("all_db1_tb1", privileges.get("all_db1_tb1"))
+        .addPermissionsToRole("all_uri", "server=server1->uri=" + tabLocation)
+        .addRolesToGroup(USERGROUP1, "all_db1_tb1", "all_uri")
+        .addRolesToGroup(USERGROUP2, "all_db1_tb1");
+
+    writePolicyFile(policyFile);
+
+    Connection connection = context.createConnection(USER1_1);
+    Statement statement = context.createStatement(connection);
+    statement.execute("Use db1");
+    statement.execute("ALTER TABLE tb1 SET LOCATION '" + tabLocation + "'");
+    statement.execute("ALTER TABLE tb1 ADD IF NOT EXISTS PARTITION (b = '3') LOCATION '" + tabLocation + "/part'");
+    statement.close();
+    connection.close();
+
+    //Negative case
+    policyFile
+        .addPermissionsToRole("insert_db1_tb1", privileges.get("insert_db1_tb1"))
+        .addRolesToGroup(USERGROUP3, "insert_db1_tb1", "all_uri");
+    writePolicyFile(policyFile);
+
+    connection = context.createConnection(USER3_1);
+    statement = context.createStatement(connection);
+    statement.execute("Use db1");
+    context.assertSentrySemanticException(statement, "ALTER TABLE tb1 SET LOCATION '" + tabLocation + "'",
+        semanticException);
+    context.assertSentrySemanticException(statement, "ALTER TABLE tb1 ADD IF NOT EXISTS PARTITION (b = '3') LOCATION '"
+        + tabLocation + "/part'", semanticException);
+    statement.close();
+    connection.close();
+
+
+  }
+
+  /* All on Database and select on table
+  1. Create view :  HiveOperation.CREATEVIEW
+   */
+  @Test
+  public void testCreateView() throws Exception {
+    adminCreate(dbName, tableName);
+    adminCreate("db2", null);
+    policyFile
+        .addPermissionsToRole("select_db1_tb1", privileges.get("select_db1_tb1"))
+        .addPermissionsToRole("all_db2", privileges.get("all_db2"))
+        .addRolesToGroup(USERGROUP1, "select_db1_tb1", "all_db2");
+    writePolicyFile(policyFile);
+
+    Connection connection = context.createConnection(USER1_1);
+    Statement statement = context.createStatement(connection);
+    statement.execute("use db2");
+    statement.execute("create view view1 as select a from db1.tb1");
+    statement.close();
+    connection.close();
+
+    //Negative case
+    policyFile
+        .addPermissionsToRole("insert_db1_tb1", privileges.get("insert_db1_tb1"))
+        .addRolesToGroup(USERGROUP3, "insert_db1_tb1", "all_db2");
+    writePolicyFile(policyFile);
+
+    connection = context.createConnection(USER3_1);
+    statement = context.createStatement(connection);
+    statement.execute("Use db2");
+    context.assertSentrySemanticException(statement, "create view view1 as select a from db1.tb1",
+        semanticException);
+    statement.close();
+    connection.close();
+
+
+  }
+
+  /*
+   1. HiveOperation.IMPORT : All on db + all on URI
+   2. HiveOperation.EXPORT : SELECT on table + all on uri
+   */
+
+  @Test
+  public void testExportImport() throws Exception {
+    File dataFile;
+    dataFile = new File(dataDir, SINGLE_TYPE_DATA_FILE_NAME);
+    FileOutputStream to = new FileOutputStream(dataFile);
+    Resources.copy(Resources.getResource(SINGLE_TYPE_DATA_FILE_NAME), to);
+    to.close();
+
+    dropDb(ADMIN1, dbName);
+    createDb(ADMIN1, dbName);
+    createTable(ADMIN1, dbName, dataFile, tableName);
+    String location = warehouseDir + "/loc";
+    policyFile
+        .addPermissionsToRole("all_db1", privileges.get("all_db1"))
+        .addPermissionsToRole("all_uri", "server=server1->uri="+ location)
+        .addPermissionsToRole("select_db1_tb1", privileges.get("select_db1_tb1"))
+        .addPermissionsToRole("insert_db1", privileges.get("insert_db1"))
+        .addRolesToGroup(USERGROUP1, "select_db1_tb1", "all_uri")
+        .addRolesToGroup(USERGROUP2, "all_db1", "all_uri")
+        .addRolesToGroup(USERGROUP3, "insert_db1", "all_uri");
+    writePolicyFile(policyFile);
+    Connection connection;
+    Statement statement;
+
+    //Negative case
+    connection = context.createConnection(USER3_1);
+    statement = context.createStatement(connection);
+    statement.execute("Use db1");
+    context.assertSentrySemanticException(statement, "export table tb1 to '" + location + "'",
+        semanticException);
+    statement.close();
+    connection.close();
+
+    //Positive
+    connection = context.createConnection(USER1_1);
+    statement = context.createStatement(connection);
+    statement.execute("Use db1");
+    statement.execute("export table tb1 to '" + location + "'" );
+    statement.close();
+    connection.close();
+
+    //Negative
+    connection = context.createConnection(USER3_1);
+    statement = context.createStatement(connection);
+    statement.execute("Use db1");
+    context.assertSentrySemanticException(statement, "import table tb2 from '" + location + "'",
+        semanticException);
+    statement.close();
+    connection.close();
+
+    //Positive
+    connection = context.createConnection(USER2_1);
+    statement = context.createStatement(connection);
+    statement.execute("Use db1");
+    statement.execute("import table tb2 from '" + location + "'");
+    statement.close();
+    connection.close();
+
+  }
+
+  /*
+  1. HiveOperation.LOAD: INSERT on table + all on uri
+   */
+  @Test
+  public void testLoad() throws Exception {
+    File dataFile;
+    dataFile = new File(dataDir, SINGLE_TYPE_DATA_FILE_NAME);
+    FileOutputStream to = new FileOutputStream(dataFile);
+    Resources.copy(Resources.getResource(SINGLE_TYPE_DATA_FILE_NAME), to);
+    to.close();
+
+    adminCreate(dbName, tableName);
+
+    policyFile
+        .addPermissionsToRole("insert_db1_tb1", privileges.get("insert_db1_tb1"))
+        .addPermissionsToRole("all_uri", "server=server1->uri=file:///" + dataDir)
+        .addRolesToGroup(USERGROUP1, "insert_db1_tb1", "all_uri");
+    writePolicyFile(policyFile);
+
+    Connection connection = context.createConnection(USER1_1);
+    Statement statement = context.createStatement(connection);
+    statement.execute("Use db1");
+    statement.execute("load data local inpath '" + dataFile.getPath() + "' into table tb1" );
+    statement.close();
+    connection.close();
+  }
+
+  /*
+  1. HiveOperation.CREATETABLE_AS_SELECT : All on db + select on table
+   */
+  @Test
+  public void testCTAS() throws Exception {
+    adminCreate(dbName, tableName);
+    adminCreate("db2", null);
+
+    policyFile
+        .addPermissionsToRole("select_db1_tb1", privileges.get("select_db1_tb1"))
+        .addPermissionsToRole("all_db2", privileges.get("all_db2"))
+        .addRolesToGroup(USERGROUP1, "select_db1_tb1", "all_db2");
+    writePolicyFile(policyFile);
+
+    Connection connection = context.createConnection(USER1_1);
+    Statement statement = context.createStatement(connection);
+    statement.execute("Use db2");
+    statement.execute("create table tb2 as select a from db1.tb1" );
+    statement.close();
+    connection.close();
+  }
+
+  /*
+  1. INSERT : IP: select on table, OP: insert on table + all on uri(optional)
+   */
+  @Test
+  public void testInsert() throws Exception {
+    File dataFile;
+    dataFile = new File(dataDir, SINGLE_TYPE_DATA_FILE_NAME);
+    FileOutputStream to = new FileOutputStream(dataFile);
+    Resources.copy(Resources.getResource(SINGLE_TYPE_DATA_FILE_NAME), to);
+    to.close();
+
+    dropDb(ADMIN1, dbName);
+    dropDb(ADMIN1, "db2");
+    createDb(ADMIN1, dbName);
+    createDb(ADMIN1, "db2");
+    createTable(ADMIN1, dbName, dataFile, tableName);
+    createTable(ADMIN1, "db2", null, "tb2");
+    String location = warehouseDir + "/loc";
+
+    policyFile
+        .addPermissionsToRole("select_db1_tb1", privileges.get("select_db1_tb1"))
+        .addPermissionsToRole("insert_db2_tb2", privileges.get("insert_db2_tb2"))
+        .addRolesToGroup(USERGROUP1, "select_db1_tb1", "insert_db2_tb2")
+        .addPermissionsToRole("all_uri", "server=server1->uri=" + location)
+        .addRolesToGroup(USERGROUP2, "select_db1_tb1", "all_uri");
+    writePolicyFile(policyFile);
+
+    Connection connection = context.createConnection(USER1_1);
+    Statement statement = context.createStatement(connection);
+    assertSemanticException(statement, "insert overwrite directory '" + location + "' select * from db1.tb1" );
+    statement.execute("insert overwrite table db2.tb2 select * from db1.tb1");
+    statement.close();
+    connection.close();
+
+    connection = context.createConnection(USER2_1);
+    statement = context.createStatement(connection);
+    statement.execute("insert overwrite directory '" + location + "' select * from db1.tb1" );
+    assertSemanticException(statement,"insert overwrite table db2.tb2 select * from db1.tb1");
+    statement.close();
+    connection.close();
+  }
+
+  @Test
+  public void testFullyQualifiedTableName() throws Exception{
+    Connection connection;
+    Statement statement;
+    connection = context.createConnection(ADMIN1);
+    statement = context.createStatement(connection);
+    statement.execute("create database db1");
+    statement.execute("create table db1.tb1(a int)");
+    statement.execute("DROP table db1.tb1");
+    statement.execute("create table db1.tb1(a int)");
+    statement.execute("use db1");
+    statement.execute("drop table tb1");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/b41d9ee3/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegeAtTransform.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegeAtTransform.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegeAtTransform.java
index 1267e6b..d115cad 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegeAtTransform.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegeAtTransform.java
@@ -76,7 +76,7 @@ public class TestPrivilegeAtTransform extends AbstractTestWithStaticConfiguratio
     statement.execute("DROP TABLE IF EXISTS " + dbName1 + "." + tableName1);
     statement.execute("create table " + dbName1 + "." + tableName1
         + " (under_col int, value string)");
-     statement.execute("load data local inpath '" + dataFile.getPath()
+    statement.execute("load data local inpath '" + dataFile.getPath()
             + "' into table " + dbName1 + "." + tableName1);
     assertTrue(query, statement.execute(query));
 

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/b41d9ee3/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegesAtDatabaseScope.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegesAtDatabaseScope.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegesAtDatabaseScope.java
index c9658ab..a0669cd 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegesAtDatabaseScope.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestPrivilegesAtDatabaseScope.java
@@ -177,13 +177,8 @@ public class TestPrivilegesAtDatabaseScope extends AbstractTestWithStaticConfigu
       context.verifyAuthzException(e);
     }
 
-    //negative test case: user can't drop own database
-    try {
-      statement.execute("DROP DATABASE DB_1 CASCADE");
-      Assert.fail("Expected SQL exception");
-    } catch (SQLException e) {
-      context.verifyAuthzException(e);
-    }
+    //User can drop own database
+    statement.execute("DROP DATABASE DB_1 CASCADE");
 
     statement.close();
     connection.close();
@@ -191,7 +186,6 @@ public class TestPrivilegesAtDatabaseScope extends AbstractTestWithStaticConfigu
     //test cleanup
     connection = context.createConnection(ADMIN1);
     statement = context.createStatement(connection);
-    statement.execute("DROP DATABASE DB_1 CASCADE");
     statement.execute("DROP DATABASE DB_2 CASCADE");
     statement.close();
     connection.close();
@@ -271,14 +265,6 @@ public class TestPrivilegesAtDatabaseScope extends AbstractTestWithStaticConfigu
     // test user can drop table
     statement.execute("DROP TABLE TAB_3");
 
-    //negative test case: user can't drop db
-    try {
-      statement.execute("DROP DATABASE DB_1 CASCADE");
-      Assert.fail("Expected SQL exception");
-    } catch (SQLException e) {
-      context.verifyAuthzException(e);
-    }
-
     //negative test case: user can't create external tables
     assertTrue("Unable to create directory for external table test" , externalTblDir.mkdir());
     statement.execute("CREATE EXTERNAL TABLE EXT_TAB_1(A STRING) STORED AS TEXTFILE LOCATION 'file:"+

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/b41d9ee3/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestSandboxOps.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestSandboxOps.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestSandboxOps.java
index 0d6e0b6..021f5d2 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestSandboxOps.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestSandboxOps.java
@@ -274,9 +274,6 @@ public class TestSandboxOps  extends AbstractTestWithStaticConfiguration {
     statement.execute("CREATE VIEW " + VIEW1 + " (value) AS SELECT value from " + TBL1 + " LIMIT 10");
 
     createTable(USER1_1, DB2, dataFile, TBL2, TBL3);
-    // c
-    context.assertAuthzException(statement, "DROP DATABASE IF EXISTS " + DB1 + " CASCADE");
-    context.assertAuthzException(statement, "DROP DATABASE IF EXISTS " + DB2 + " CASCADE");
     // d
     statement.execute("USE " + DB1);
     policyFile.removePermissionsFromRole(GROUP1_ROLE, ALL_DB2);
@@ -402,7 +399,6 @@ public class TestSandboxOps  extends AbstractTestWithStaticConfiguration {
     // c
     statement.execute("USE " + DB1);
     context.assertAuthzException(statement, "CREATE DATABASE " + DB3);
-    context.assertAuthzException(statement, "DROP DATABASE " + DB1);
     ResultSet rs = statement.executeQuery("SHOW DATABASES");
     assertTrue(rs.next());
     assertEquals(DB1, rs.getString(1));

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/b41d9ee3/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestUriPermissions.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestUriPermissions.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestUriPermissions.java
index 5a620ba..99f3a77 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestUriPermissions.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestUriPermissions.java
@@ -135,8 +135,7 @@ public class TestUriPermissions extends AbstractTestWithStaticConfiguration {
     userStmt.execute("ALTER TABLE " + tabName + " DROP PARTITION (dt = '21-Dec-2012')");
     userStmt.execute("ALTER TABLE " + tabName + " ADD PARTITION (dt = '21-Dec-2012') " +
         " LOCATION '" + tabDir + "'");
-    // negative test user1 cannot alter partition location
-    context.assertAuthzException(userStmt,
+    userStmt.execute(
         "ALTER TABLE " + tabName + " PARTITION (dt = '21-Dec-2012') " + " SET LOCATION '" + tabDir + "'");
     userConn.close();
 
@@ -154,22 +153,17 @@ public class TestUriPermissions extends AbstractTestWithStaticConfiguration {
     userStmt.execute("ALTER TABLE " + tabName + " DROP PARTITION (dt = '22-Dec-2012')");
     userConn.close();
 
-    // negative test: user3 doesn't have privilege to add/drop partitions
+    // positive test: user3 has privilege to add/drop partitions
     userConn = context.createConnection(USER3_1);
     userStmt = context.createStatement(userConn);
     userStmt.execute("use " + dbName);
-    context.assertAuthzException(userStmt,
+    userStmt.execute(
         "ALTER TABLE " + tabName + " ADD PARTITION (dt = '22-Dec-2012') " +
           " LOCATION '" + tabDir + "/foo'");
-    context.assertAuthzException(userStmt,
+    userStmt.execute(
         "ALTER TABLE " + tabName + " DROP PARTITION (dt = '21-Dec-2012')");
     userConn.close();
 
-    // positive test: user1 has privilege to alter drop partition
-    userConn = context.createConnection(USER1_1);
-    userStmt = context.createStatement(userConn);
-    userStmt.execute("use " + dbName);
-    userStmt.execute("ALTER TABLE " + tabName + " DROP PARTITION (dt = '21-Dec-2012')");
     userStmt.close();
     userConn.close();
   }
@@ -202,11 +196,11 @@ public class TestUriPermissions extends AbstractTestWithStaticConfiguration {
     adminStmt.execute("CREATE TABLE " + tabName + " (id int)  PARTITIONED BY (dt string)");
     adminCon.close();
 
-    // negative test: user2 doesn't have privilege to alter table set partition
+    // positive test: user2 has privilege to alter table set partition
     userConn = context.createConnection(USER2_1);
     userStmt = context.createStatement(userConn);
     userStmt.execute("use " + dbName);
-    context.assertAuthzException(userStmt,
+    userStmt.execute(
         "ALTER TABLE " + tabName + " SET LOCATION '" + tabDir +  "'");
     userConn.close();