You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sentry.apache.org by sd...@apache.org on 2016/03/25 03:44:26 UTC

sentry git commit: SENTRY-1137: Update hive dependence to 2.0.0 (Dapeng Sun, reviewed by Colin Ma)

Repository: sentry
Updated Branches:
  refs/heads/master 3be4330c2 -> 0702f54b4


SENTRY-1137: Update hive dependence to 2.0.0 (Dapeng Sun, reviewed by Colin Ma)


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

Branch: refs/heads/master
Commit: 0702f54b4e32ad952514cc1041447632b20a8e52
Parents: 3be4330
Author: Sun Dapeng <sd...@apache.org>
Authored: Fri Mar 25 10:42:17 2016 +0800
Committer: Sun Dapeng <sd...@apache.org>
Committed: Fri Mar 25 10:42:17 2016 +0800

----------------------------------------------------------------------
 pom.xml                                         |   2 +-
 sentry-binding/sentry-binding-hive-v2/pom.xml   |  27 ++
 .../binding/hive/v2/HiveAuthzBindingHookV2.java | 103 +++++--
 .../hive/v2/HiveAuthzBindingSessionHookV2.java  |   6 +-
 .../hive/v2/HiveAuthzPrivilegesMapV2.java       | 303 +++++++++++++++++++
 .../hive/v2/SentryAuthorizerFactory.java        |   4 +-
 .../v2/authorizer/DefaultSentryValidator.java   |  11 +-
 .../v2/metastore/AuthorizingObjectStoreV2.java  |   6 +-
 .../v2/metastore/MetastoreAuthzBindingV2.java   |  49 ++-
 .../SentryMetastorePostEventListenerV2.java     |   4 +-
 .../hive/v2/util/SimpleSemanticAnalyzer.java    |  11 +-
 11 files changed, 478 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/sentry/blob/0702f54b/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 5c4dc4c..37db007 100644
--- a/pom.xml
+++ b/pom.xml
@@ -70,7 +70,7 @@ limitations under the License.
     <guava.version>11.0.2</guava.version>
     <hadoop.version>2.6.0</hadoop.version>
     <hamcrest.version>1.3</hamcrest.version>
-    <hive-v2.version>1.3.0-SNAPSHOT</hive-v2.version>
+    <hive-v2.version>2.0.0</hive-v2.version>
     <hive.version>1.1.0</hive.version>
     <jackson.version>1.8.8</jackson.version>
     <jdo-api.version>3.0.1</jdo-api.version>

http://git-wip-us.apache.org/repos/asf/sentry/blob/0702f54b/sentry-binding/sentry-binding-hive-v2/pom.xml
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive-v2/pom.xml b/sentry-binding/sentry-binding-hive-v2/pom.xml
index f33219d..f633b6b 100644
--- a/sentry-binding/sentry-binding-hive-v2/pom.xml
+++ b/sentry-binding/sentry-binding-hive-v2/pom.xml
@@ -28,6 +28,13 @@ limitations under the License.
   <artifactId>sentry-binding-hive-v2</artifactId>
   <name>Sentry Binding v2 for Hive</name>
 
+  <properties>
+    <datanucleus-api-jdo.version>4.2.1</datanucleus-api-jdo.version>
+    <datanucleus-core.version>4.1.6</datanucleus-core.version>
+    <datanucleus-rdbms.version>4.1.7</datanucleus-rdbms.version>
+    <datanucleus-jdo.version>3.2.0-m3</datanucleus-jdo.version>
+  </properties>
+
   <dependencies>
     <dependency>
       <groupId>org.apache.sentry</groupId>
@@ -138,6 +145,26 @@ limitations under the License.
       <artifactId>mockito-all</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.datanucleus</groupId>
+      <artifactId>datanucleus-core</artifactId>
+      <version>${datanucleus-core.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.datanucleus</groupId>
+      <artifactId>datanucleus-api-jdo</artifactId>
+      <version>${datanucleus-api-jdo.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.datanucleus</groupId>
+      <artifactId>datanucleus-rdbms</artifactId>
+      <version>${datanucleus-rdbms.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.datanucleus</groupId>
+      <artifactId>javax.jdo</artifactId>
+      <version>${datanucleus-jdo.version}</version>
+    </dependency>
   </dependencies>
 
 </project>

http://git-wip-us.apache.org/repos/asf/sentry/blob/0702f54b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/HiveAuthzBindingHookV2.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/HiveAuthzBindingHookV2.java b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/HiveAuthzBindingHookV2.java
index 67cf266..91a2507 100644
--- a/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/HiveAuthzBindingHookV2.java
+++ b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/HiveAuthzBindingHookV2.java
@@ -17,49 +17,81 @@
 package org.apache.sentry.binding.hive.v2;
 
 import java.io.Serializable;
+import java.security.CodeSource;
 import java.util.List;
 
-import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.exec.DDLTask;
 import org.apache.hadoop.hive.ql.exec.SentryFilterDDLTask;
 import org.apache.hadoop.hive.ql.exec.Task;
+import org.apache.hadoop.hive.ql.exec.Utilities;
+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;
+import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
 import org.apache.hadoop.hive.ql.parse.HiveSemanticAnalyzerHookContext;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.DDLWork;
 import org.apache.hadoop.hive.ql.plan.HiveOperation;
 import org.apache.hadoop.hive.ql.session.SessionState;
-import org.apache.sentry.binding.hive.HiveAuthzBindingHook;
-import org.apache.sentry.binding.hive.authz.HiveAuthzBinding;
+import org.apache.sentry.binding.hive.HiveAuthzBindingHookBase;
+import org.apache.sentry.binding.hive.authz.HiveAuthzPrivileges;
 import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
 import org.apache.sentry.core.common.Subject;
+import org.apache.sentry.core.model.db.Database;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class HiveAuthzBindingHookV2 extends AbstractSemanticAnalyzerHook {
+public class HiveAuthzBindingHookV2 extends HiveAuthzBindingHookBase {
   private static final Logger LOG = LoggerFactory
       .getLogger(HiveAuthzBindingHookV2.class);
-  private final HiveAuthzBinding hiveAuthzBinding;
-  private final HiveAuthzConf authzConf;
 
   public HiveAuthzBindingHookV2() throws Exception {
-    SessionState session = SessionState.get();
-    if(session == null) {
-      throw new IllegalStateException("Session has not been started");
-    }
-
-    HiveConf hiveConf = session.getConf();
-    if(hiveConf == null) {
-      throw new IllegalStateException("Session HiveConf is null");
-    }
-    authzConf = HiveAuthzBindingHook.loadAuthzConf(hiveConf);
-    hiveAuthzBinding = new HiveAuthzBinding(hiveConf, authzConf);
+    super();
   }
 
   @Override
   public ASTNode preAnalyze(HiveSemanticAnalyzerHookContext context, ASTNode ast)
       throws SemanticException {
+    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_CREATEFUNCTION:
+        String udfClassName = BaseSemanticAnalyzer.unescapeSQLString(ast.getChild(1).getText());
+        try {
+          CodeSource udfSrc =
+              Class.forName(udfClassName, true, Utilities.getSessionSpecifiedClassLoader())
+                  .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);
+        }
+        // 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_CREATETABLE:
+        for (Node childNode : ast.getChildren()) {
+          ASTNode childASTNode = (ASTNode) childNode;
+          if ("TOK_TABLESERIALIZER".equals(childASTNode.getText())) {
+            ASTNode serdeNode = (ASTNode) childASTNode.getChild(0);
+            String serdeClassName =
+                BaseSemanticAnalyzer.unescapeSQLString(serdeNode.getChild(0).getText());
+            setSerdeURI(serdeClassName);
+          }
+        }
+        break;
+    }
     return ast;
   }
 
@@ -80,15 +112,36 @@ public class HiveAuthzBindingHookV2 extends AbstractSemanticAnalyzerHook {
         rootTasks.set(i, filterTask);
       }
     }
-  }
+    HiveAuthzPrivileges stmtAuthObject = HiveAuthzPrivilegesMapV2.getHiveAuthzPrivileges(stmtOperation);
+    if (stmtOperation.equals(HiveOperation.CREATEFUNCTION)
+        || stmtOperation.equals(HiveOperation.DROPFUNCTION)
+        || stmtOperation.equals(HiveOperation.CREATETABLE)) {
+      try {
+        if (stmtAuthObject == null) {
+          // We don't handle authorizing this statement
+          return;
+        }
 
-  private HiveOperation getCurrentHiveStmtOp() {
-    SessionState sessState = SessionState.get();
-    if (sessState == null) {
-      LOG.warn("SessionState is null");
-      return null;
+        authorizeWithHiveBindings(context, stmtAuthObject, stmtOperation);
+      } catch (AuthorizationException e) {
+        executeOnFailureHooks(context, stmtOperation, e);
+        String permsRequired = "";
+        for (String perm : hiveAuthzBinding.getLastQueryPrivilegeErrors()) {
+          permsRequired += perm + ";";
+        }
+        SessionState.get().getConf().set(HiveAuthzConf.HIVE_SENTRY_AUTH_ERRORS, permsRequired);
+        String msgForLog =
+            HiveAuthzConf.HIVE_SENTRY_PRIVILEGE_ERROR_MESSAGE
+                + "\n Required privileges for this query: " + permsRequired;
+        String msgForConsole =
+            HiveAuthzConf.HIVE_SENTRY_PRIVILEGE_ERROR_MESSAGE + "\n " + e.getMessage();
+        // AuthorizationException is not a real exception, use the info level to record this.
+        LOG.info(msgForLog);
+        throw new SemanticException(msgForConsole, e);
+      } finally {
+        hiveAuthzBinding.close();
+      }
     }
-    return sessState.getHiveOperation();
   }
 
 }

http://git-wip-us.apache.org/repos/asf/sentry/blob/0702f54b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/HiveAuthzBindingSessionHookV2.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/HiveAuthzBindingSessionHookV2.java b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/HiveAuthzBindingSessionHookV2.java
index 3fbb626..b95bf60 100644
--- a/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/HiveAuthzBindingSessionHookV2.java
+++ b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/HiveAuthzBindingSessionHookV2.java
@@ -21,7 +21,7 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hive.service.cli.HiveSQLException;
 import org.apache.hive.service.cli.session.HiveSessionHookContext;
-import org.apache.sentry.binding.hive.HiveAuthzBindingHook;
+import org.apache.sentry.binding.hive.HiveAuthzBindingHookBase;
 import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
 
 import com.google.common.base.Joiner;
@@ -39,7 +39,7 @@ public class HiveAuthzBindingSessionHookV2 implements
       ConfVars.SCRATCHDIR.varname, ConfVars.LOCALSCRATCHDIR.varname,
       ConfVars.METASTOREURIS.varname, ConfVars.METASTORECONNECTURLKEY.varname,
       ConfVars.HADOOPBIN.varname, ConfVars.HIVESESSIONID.varname, ConfVars.HIVEAUXJARS.varname,
-      ConfVars.HIVESTATSDBCONNECTIONSTRING.varname, ConfVars.SCRATCHDIRPERMISSION.varname,
+      ConfVars.SCRATCHDIRPERMISSION.varname,
       ConfVars.HIVE_SECURITY_COMMAND_WHITELIST.varname,
       ConfVars.HIVE_AUTHORIZATION_TASK_FACTORY.varname,
       ConfVars.HIVE_CAPTURE_TRANSFORM_ENTITY.varname, HiveAuthzConf.HIVE_ACCESS_CONF_URL,
@@ -72,7 +72,7 @@ public class HiveAuthzBindingSessionHookV2 implements
     sessionConf.setBoolVar(ConfVars.HIVE_CAPTURE_TRANSFORM_ENTITY, true);
 
     // set security command list
-    HiveAuthzConf authzConf = HiveAuthzBindingHook.loadAuthzConf(sessionConf);
+    HiveAuthzConf authzConf = HiveAuthzBindingHookBase.loadAuthzConf(sessionConf);
     String commandWhitelist =
         authzConf.get(HiveAuthzConf.HIVE_SENTRY_SECURITY_COMMAND_WHITELIST,
             HiveAuthzConf.HIVE_SENTRY_SECURITY_COMMAND_WHITELIST_DEFAULT);

http://git-wip-us.apache.org/repos/asf/sentry/blob/0702f54b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/HiveAuthzPrivilegesMapV2.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/HiveAuthzPrivilegesMapV2.java b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/HiveAuthzPrivilegesMapV2.java
new file mode 100644
index 0000000..8993084
--- /dev/null
+++ b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/HiveAuthzPrivilegesMapV2.java
@@ -0,0 +1,303 @@
+/*
+ * 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.binding.hive.v2;
+
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.hive.ql.plan.HiveOperation;
+import org.apache.sentry.binding.hive.authz.HiveAuthzPrivileges;
+import org.apache.sentry.binding.hive.authz.HiveAuthzPrivileges.HiveOperationScope;
+import org.apache.sentry.binding.hive.authz.HiveAuthzPrivileges.HiveOperationType;
+import org.apache.sentry.core.model.db.DBModelAction;
+import org.apache.sentry.core.model.db.DBModelAuthorizable.AuthorizableType;
+
+public class HiveAuthzPrivilegesMapV2 {
+  private static final Map <HiveOperation, HiveAuthzPrivileges> hiveAuthzStmtPrivMap =
+    new HashMap<HiveOperation, HiveAuthzPrivileges>();
+  static {
+    HiveAuthzPrivileges createServerPrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
+        addInputObjectPriviledge(AuthorizableType.Server, EnumSet.of(DBModelAction.CREATE)).
+        setOperationScope(HiveOperationScope.SERVER).
+        setOperationType(HiveOperationType.DDL).
+        build();
+
+    HiveAuthzPrivileges tableCreatePrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
+        addOutputObjectPriviledge(AuthorizableType.Db, EnumSet.of(DBModelAction.CREATE)).
+        addInputObjectPriviledge(AuthorizableType.URI, EnumSet.of(DBModelAction.ALL)).//TODO: make it optional
+        addOutputObjectPriviledge(AuthorizableType.URI, EnumSet.of(DBModelAction.ALL)).
+        setOperationScope(HiveOperationScope.DATABASE).
+        setOperationType(HiveOperationType.DDL).
+        build();
+    HiveAuthzPrivileges dropDbPrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
+        addInputObjectPriviledge(AuthorizableType.Db, EnumSet.of(DBModelAction.DROP)).
+        setOperationScope(HiveOperationScope.DATABASE).
+        setOperationType(HiveOperationType.DDL).
+        build();
+    HiveAuthzPrivileges alterDbPrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
+        addOutputObjectPriviledge(AuthorizableType.Db, EnumSet.of(DBModelAction.ALTER)).
+        setOperationScope(HiveOperationScope.DATABASE).
+        setOperationType(HiveOperationType.DDL).
+        build();
+
+    HiveAuthzPrivileges alterTablePrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
+        addOutputObjectPriviledge(AuthorizableType.Table, EnumSet.of(DBModelAction.ALTER)).
+        setOperationScope(HiveOperationScope.TABLE).
+        setOperationType(HiveOperationType.DDL).
+        build();
+    HiveAuthzPrivileges dropTablePrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
+        addInputObjectPriviledge(AuthorizableType.Table, EnumSet.of(DBModelAction.DROP)).
+        setOperationScope(HiveOperationScope.TABLE).
+        setOperationType(HiveOperationType.DDL).
+        build();
+    HiveAuthzPrivileges indexTablePrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
+        addInputObjectPriviledge(AuthorizableType.Table, EnumSet.of(DBModelAction.INDEX)).
+        setOperationScope(HiveOperationScope.TABLE).
+        setOperationType(HiveOperationType.DDL).
+        build();
+
+    HiveAuthzPrivileges alterTableAndUriPrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
+        addOutputObjectPriviledge(AuthorizableType.Table, EnumSet.of(DBModelAction.ALTER)).
+        addOutputObjectPriviledge(AuthorizableType.URI, EnumSet.of(DBModelAction.ALL)).
+        setOperationScope(HiveOperationScope.TABLE).
+        setOperationType(HiveOperationType.DDL).
+        build();
+    HiveAuthzPrivileges addPartitionPrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
+        addOutputObjectPriviledge(AuthorizableType.Table, EnumSet.of(DBModelAction.ALTER)).
+        //TODO: Uncomment this if we want to make it more restrictive
+        //addInputObjectPriviledge(AuthorizableType.Table, EnumSet.of(DBModelAction.CREATE)).
+        addInputObjectPriviledge(AuthorizableType.URI, EnumSet.of(DBModelAction.SELECT)).//TODO: make it optional
+        addOutputObjectPriviledge(AuthorizableType.URI, EnumSet.of(DBModelAction.ALL)).
+        setOperationScope(HiveOperationScope.TABLE).
+        setOperationType(HiveOperationType.DDL).
+        build();
+    HiveAuthzPrivileges dropPartitionPrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
+        addInputObjectPriviledge(AuthorizableType.Table, EnumSet.of(DBModelAction.ALTER)).
+        addInputObjectPriviledge(AuthorizableType.Table, EnumSet.of(DBModelAction.DROP)).
+        setOperationScope(HiveOperationScope.TABLE).
+        setOperationType(HiveOperationType.DDL).
+        build();
+
+    HiveAuthzPrivileges alterTableRenamePrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
+        addInputObjectPriviledge(AuthorizableType.Table, EnumSet.of(DBModelAction.ALTER)).
+        addInputObjectPriviledge(AuthorizableType.Db, EnumSet.of(DBModelAction.CREATE)).
+        setOperationScope(HiveOperationScope.DATABASE).
+        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)).
+        addInputObjectPriviledge(AuthorizableType.URI, EnumSet.of(DBModelAction.ALL)).
+        addInputObjectPriviledge(AuthorizableType.Column, EnumSet.of(DBModelAction.SELECT)).
+        addOutputObjectPriviledge(AuthorizableType.Table, 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.ALL)).
+        addOutputObjectPriviledge(AuthorizableType.Table, EnumSet.of(DBModelAction.INSERT)).
+        setOperationScope(HiveOperationScope.TABLE).
+        setOperationType(HiveOperationType.DATA_LOAD).
+        build();
+
+    HiveAuthzPrivileges tableExportPrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
+        addInputObjectPriviledge(AuthorizableType.Table, EnumSet.of(DBModelAction.SELECT)).
+        addOutputObjectPriviledge(AuthorizableType.URI, EnumSet.of(DBModelAction.ALL)).
+        setOperationScope(HiveOperationScope.TABLE).
+        setOperationType(HiveOperationType.DATA_UNLOAD).
+        build();
+
+    HiveAuthzPrivileges tableMetaDataPrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
+        addInputObjectPriviledge(AuthorizableType.Table, EnumSet.of(DBModelAction.SELECT, DBModelAction.INSERT)).
+        setOperationScope(HiveOperationScope.TABLE).
+        setOperationType(HiveOperationType.INFO).
+        build();
+
+    // Metadata statements which only require column-level privileges.
+    HiveAuthzPrivileges columnMetaDataPrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
+        addInputObjectPriviledge(AuthorizableType.Column, EnumSet.of(DBModelAction.SELECT, DBModelAction.INSERT)).
+        setOperationScope(HiveOperationScope.COLUMN).
+        setOperationType(HiveOperationType.INFO).
+        build();
+
+    HiveAuthzPrivileges dbImportPrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
+        addOutputObjectPriviledge(AuthorizableType.Db, EnumSet.of(DBModelAction.CREATE)).
+        addInputObjectPriviledge(AuthorizableType.URI, EnumSet.of(DBModelAction.ALL)).
+        setOperationScope(HiveOperationScope.DATABASE).
+        setOperationType(HiveOperationType.DDL).
+        build();
+
+    HiveAuthzPrivileges createViewPrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
+    addOutputObjectPriviledge(AuthorizableType.Db, EnumSet.of(DBModelAction.CREATE)).
+    addInputObjectPriviledge(AuthorizableType.Table, 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, DBModelAction.INSERT)).
+      setOperationScope(HiveOperationScope.DATABASE).
+      setOperationType(HiveOperationType.INFO).
+      build();
+
+    HiveAuthzPrivileges tableLockPrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder()
+        .addInputObjectPriviledge(AuthorizableType.Table, EnumSet.of(DBModelAction.LOCK)).
+        setOperationScope(HiveOperationScope.TABLE).
+        setOperationType(HiveOperationType.DML).
+        build();
+
+    HiveAuthzPrivileges dbLockPrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder()
+        .addInputObjectPriviledge(AuthorizableType.Db, EnumSet.of(DBModelAction.LOCK))
+        .setOperationScope(HiveOperationScope.DATABASE).setOperationType(HiveOperationType.DML)
+        .build();
+
+    HiveAuthzPrivileges functionPrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
+        addInputObjectPriviledge(AuthorizableType.URI, EnumSet.of(DBModelAction.ALL)).
+        addOutputObjectPriviledge(AuthorizableType.URI, EnumSet.of(DBModelAction.ALL)).
+        setOperationScope(HiveOperationScope.FUNCTION).
+        setOperationType(HiveOperationType.DATA_LOAD).
+        build();
+
+    HiveAuthzPrivileges anyPrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
+        addInputObjectPriviledge(AuthorizableType.Column, EnumSet.of(DBModelAction.SELECT,
+            DBModelAction.INSERT, DBModelAction.ALTER, DBModelAction.CREATE, DBModelAction.DROP,
+            DBModelAction.INDEX, DBModelAction.LOCK)).
+        setOperationScope(HiveOperationScope.CONNECT).
+        setOperationType(HiveOperationType.QUERY).
+        build();
+
+    HiveAuthzPrivileges truncateTablePrivilege = new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
+        addOutputObjectPriviledge(AuthorizableType.Table, EnumSet.of(DBModelAction.DROP)).
+        setOperationScope(HiveOperationScope.TABLE).
+        setOperationType(HiveOperationType.DDL).
+        build();
+
+    hiveAuthzStmtPrivMap.put(HiveOperation.CREATEDATABASE, createServerPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.DROPDATABASE, dropDbPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.CREATETABLE, tableCreatePrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERDATABASE, alterDbPrivilege);
+
+    hiveAuthzStmtPrivMap.put(HiveOperation.DROPTABLE, dropTablePrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.CREATEVIEW, createViewPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.DROPVIEW, dropTablePrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.CREATEINDEX, indexTablePrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.DROPINDEX, indexTablePrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERINDEX_PROPS, indexTablePrivilege);//TODO: Needs test case
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERINDEX_REBUILD, indexTablePrivilege);
+
+
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_PROPERTIES, alterTablePrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_SERDEPROPERTIES, alterTablePrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_CLUSTER_SORT, alterTablePrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_FILEFORMAT, alterTablePrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_TOUCH, alterTablePrivilege);
+
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_RENAMECOL, alterTablePrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_ADDCOLS, alterTablePrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_REPLACECOLS, alterTablePrivilege);
+
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_RENAMEPART, alterTablePrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_ARCHIVE, alterTablePrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_UNARCHIVE, alterTablePrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERPARTITION_FILEFORMAT, alterTablePrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERPARTITION_SERDEPROPERTIES, alterTablePrivilege);
+
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_MERGEFILES, alterTablePrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_SKEWED, alterTablePrivilege);
+
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERPARTITION_SERIALIZER, alterTablePrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERPARTITION_MERGEFILES, alterTablePrivilege);
+
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERVIEW_PROPERTIES, alterTablePrivilege);
+
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_DROPPARTS, dropPartitionPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_ADDPARTS, addPartitionPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_RENAME, alterTableRenamePrivilege);
+
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_SERIALIZER, alterTableAndUriPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTABLE_LOCATION, alterTableAndUriPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERPARTITION_LOCATION, alterTableAndUriPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.ALTERTBLPART_SKEWED_LOCATION, alterTableAndUriPrivilege);//TODO: Needs test case
+
+    // MSCK REPAIR TABLE <table name> / ALTER TABLE RECOVER PARTITIONS <tableName>
+    hiveAuthzStmtPrivMap.put(HiveOperation.MSCK, alterTablePrivilege);
+
+    hiveAuthzStmtPrivMap.put(HiveOperation.ANALYZE_TABLE, tableQueryPrivilege);
+
+    // SWITCHDATABASE
+    hiveAuthzStmtPrivMap.put(HiveOperation.SWITCHDATABASE, anyPrivilege);
+
+    // CREATEFUNCTION
+    // DROPFUNCTION
+    hiveAuthzStmtPrivMap.put(HiveOperation.CREATEFUNCTION, functionPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.DROPFUNCTION, functionPrivilege);
+
+    // SHOWCOLUMNS
+    hiveAuthzStmtPrivMap.put(HiveOperation.SHOWCOLUMNS, columnMetaDataPrivilege);
+
+    // SHOWDATABASES
+    // SHOWTABLES
+    hiveAuthzStmtPrivMap.put(HiveOperation.SHOW_TABLESTATUS, tableMetaDataPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.SHOW_TBLPROPERTIES, tableMetaDataPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.SHOW_CREATETABLE, tableMetaDataPrivilege);
+    // SHOWFUNCTIONS
+    hiveAuthzStmtPrivMap.put(HiveOperation.SHOWINDEXES, tableMetaDataPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.SHOWPARTITIONS, tableMetaDataPrivilege);
+    // SHOWLOCKS
+    hiveAuthzStmtPrivMap.put(HiveOperation.EXPORT, tableExportPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.IMPORT, dbImportPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.LOAD, tableLoadPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.LOCKTABLE, tableLockPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.UNLOCKTABLE, tableLockPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.LOCKDB, dbLockPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.UNLOCKDB, dbLockPrivilege);
+    // CREATEROLE
+    // DROPROLE
+    // GRANT_PRIVILEGE
+    // REVOKE_PRIVILEGE
+    // SHOW_GRANT
+    // GRANT_ROLE
+    // REVOKE_ROLE
+    // SHOW_ROLE_GRANT
+    hiveAuthzStmtPrivMap.put(HiveOperation.CREATETABLE_AS_SELECT,
+        new HiveAuthzPrivileges.AuthzPrivilegeBuilder().
+        addInputObjectPriviledge(AuthorizableType.Table, EnumSet.of(DBModelAction.SELECT)).
+        addInputObjectPriviledge(AuthorizableType.Column, EnumSet.of(DBModelAction.SELECT)).
+        addInputObjectPriviledge(AuthorizableType.URI,EnumSet.of(DBModelAction.ALL)).
+        addOutputObjectPriviledge(AuthorizableType.Db, EnumSet.of(DBModelAction.CREATE)).
+        setOperationScope(HiveOperationScope.DATABASE).
+        setOperationType(HiveOperationType.DDL).
+        build());
+    hiveAuthzStmtPrivMap.put(HiveOperation.QUERY, tableQueryPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.DESCDATABASE, dbMetaDataPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.DESCTABLE, tableMetaDataPrivilege);
+    hiveAuthzStmtPrivMap.put(HiveOperation.TRUNCATETABLE, truncateTablePrivilege);
+  }
+
+  public static HiveAuthzPrivileges getHiveAuthzPrivileges(HiveOperation hiveStmtOp) {
+    return hiveAuthzStmtPrivMap.get(hiveStmtOp);
+  }
+}

http://git-wip-us.apache.org/repos/asf/sentry/blob/0702f54b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/SentryAuthorizerFactory.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/SentryAuthorizerFactory.java b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/SentryAuthorizerFactory.java
index 4a5cbcf..485ac43 100644
--- a/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/SentryAuthorizerFactory.java
+++ b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/SentryAuthorizerFactory.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzPluginEx
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzSessionContext;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzSessionContext.CLIENT_TYPE;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveMetastoreClientFactory;
-import org.apache.sentry.binding.hive.HiveAuthzBindingHook;
+import org.apache.sentry.binding.hive.HiveAuthzBindingHookBase;
 import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
 import org.apache.sentry.binding.hive.v2.authorizer.DefaultSentryAccessController;
 import org.apache.sentry.binding.hive.v2.authorizer.DefaultSentryValidator;
@@ -49,7 +49,7 @@ public class SentryAuthorizerFactory implements HiveAuthorizerFactory {
           throws HiveAuthzPluginException {
     HiveAuthzSessionContext sessionContext;
     try {
-      this.authzConf = HiveAuthzBindingHook.loadAuthzConf(conf);
+      this.authzConf = HiveAuthzBindingHookBase.loadAuthzConf(conf);
       sessionContext = applyTestSettings(ctx, conf);
       assertHiveCliAuthDisabled(conf, sessionContext);
     } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/sentry/blob/0702f54b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/authorizer/DefaultSentryValidator.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/authorizer/DefaultSentryValidator.java b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/authorizer/DefaultSentryValidator.java
index 70e0720..c9da3ab 100644
--- a/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/authorizer/DefaultSentryValidator.java
+++ b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/authorizer/DefaultSentryValidator.java
@@ -39,8 +39,8 @@ import org.apache.sentry.binding.hive.authz.HiveAuthzBinding;
 import org.apache.sentry.binding.hive.authz.HiveAuthzBinding.HiveHook;
 import org.apache.sentry.binding.hive.authz.HiveAuthzPrivileges;
 import org.apache.sentry.binding.hive.authz.HiveAuthzPrivileges.HiveOperationScope;
-import org.apache.sentry.binding.hive.authz.HiveAuthzPrivilegesMap;
 import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
+import org.apache.sentry.binding.hive.v2.HiveAuthzPrivilegesMapV2;
 import org.apache.sentry.binding.hive.v2.util.SentryAuthorizerUtil;
 import org.apache.sentry.binding.hive.v2.util.SimpleSemanticAnalyzer;
 import org.apache.sentry.core.common.Subject;
@@ -111,7 +111,6 @@ public class DefaultSentryValidator extends SentryHiveAuthorizationValidator {
       HiveOperation.SHOWINDEXES, HiveOperation.ALTERTABLE_PROPERTIES,
       HiveOperation.ALTERTABLE_SERDEPROPERTIES, HiveOperation.ALTERTABLE_CLUSTER_SORT,
       HiveOperation.ALTERTABLE_FILEFORMAT, HiveOperation.ALTERTABLE_TOUCH,
-      HiveOperation.ALTERTABLE_PROTECTMODE, HiveOperation.ALTERTABLE_RENAMECOL,
       HiveOperation.ALTERTABLE_ADDCOLS, HiveOperation.ALTERTABLE_REPLACECOLS,
       HiveOperation.ALTERTABLE_RENAMEPART, HiveOperation.ALTERTABLE_ARCHIVE,
       HiveOperation.ALTERTABLE_UNARCHIVE, HiveOperation.ALTERTABLE_SERIALIZER,
@@ -119,11 +118,11 @@ public class DefaultSentryValidator extends SentryHiveAuthorizationValidator {
       HiveOperation.ALTERTABLE_DROPPARTS, HiveOperation.ALTERTABLE_ADDPARTS,
       HiveOperation.ALTERTABLE_RENAME, HiveOperation.ALTERTABLE_LOCATION,
       HiveOperation.ALTERVIEW_PROPERTIES, HiveOperation.ALTERPARTITION_FILEFORMAT,
-      HiveOperation.ALTERPARTITION_PROTECTMODE, HiveOperation.ALTERPARTITION_SERDEPROPERTIES,
       HiveOperation.ALTERPARTITION_SERIALIZER, HiveOperation.ALTERPARTITION_MERGEFILES,
       HiveOperation.ALTERPARTITION_LOCATION, HiveOperation.ALTERTBLPART_SKEWED_LOCATION,
       HiveOperation.MSCK, HiveOperation.ALTERINDEX_REBUILD, HiveOperation.LOCKTABLE,
-      HiveOperation.UNLOCKTABLE, HiveOperation.SHOWCOLUMNS, HiveOperation.SHOW_TABLESTATUS, HiveOperation.LOAD);
+      HiveOperation.UNLOCKTABLE, HiveOperation.SHOWCOLUMNS, HiveOperation.SHOW_TABLESTATUS,
+      HiveOperation.LOAD, HiveOperation.TRUNCATETABLE);
   // input operations need to extend at Table scope
   private static final Set<HiveOperation> EX_TB_INPUT = Sets.newHashSet(HiveOperation.DROPTABLE,
       HiveOperation.DROPVIEW, HiveOperation.SHOW_TBLPROPERTIES, HiveOperation.SHOWINDEXES,
@@ -158,9 +157,9 @@ public class DefaultSentryValidator extends SentryHiveAuthorizationValidator {
     HiveAuthzPrivileges stmtAuthPrivileges = null;
     if (HiveOperation.DESCTABLE.equals(hiveOp) &&
         !(context.getCommandString().contains("EXTENDED") || context.getCommandString().contains("FORMATTED")) ) {
-      stmtAuthPrivileges = HiveAuthzPrivilegesMap.getHiveAuthzPrivileges(HiveOperation.SHOWCOLUMNS);
+      stmtAuthPrivileges = HiveAuthzPrivilegesMapV2.getHiveAuthzPrivileges(HiveOperation.SHOWCOLUMNS);
     } else {
-      stmtAuthPrivileges = HiveAuthzPrivilegesMap.getHiveAuthzPrivileges(hiveOp);
+      stmtAuthPrivileges = HiveAuthzPrivilegesMapV2.getHiveAuthzPrivileges(hiveOp);
     }
 
     HiveAuthzBinding hiveAuthzBinding = null;

http://git-wip-us.apache.org/repos/asf/sentry/blob/0702f54b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/metastore/AuthorizingObjectStoreV2.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/metastore/AuthorizingObjectStoreV2.java b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/metastore/AuthorizingObjectStoreV2.java
index 726f5ad..913bd00 100644
--- a/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/metastore/AuthorizingObjectStoreV2.java
+++ b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/metastore/AuthorizingObjectStoreV2.java
@@ -40,7 +40,7 @@ import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.HiveOperation;
 import org.apache.hadoop.hive.shims.Utils;
-import org.apache.sentry.binding.hive.HiveAuthzBindingHook;
+import org.apache.sentry.binding.hive.HiveAuthzBindingHookBase;
 import org.apache.sentry.binding.hive.authz.HiveAuthzBinding;
 import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
 import org.apache.sentry.binding.hive.conf.HiveAuthzConf.AuthzConfVars;
@@ -285,7 +285,7 @@ public class AuthorizingObjectStoreV2 extends ObjectStore {
       throws MetaException {
     if (needsAuthorization(getUserName())) {
       try {
-        return HiveAuthzBindingHook.filterShowDatabases(getHiveAuthzBinding(),
+        return HiveAuthzBindingHookBase.filterShowDatabases(getHiveAuthzBinding(),
             dbList, HiveOperation.SHOWDATABASES, getUserName());
       } catch (SemanticException e) {
         throw new MetaException("Error getting DB list " + e.getMessage());
@@ -306,7 +306,7 @@ public class AuthorizingObjectStoreV2 extends ObjectStore {
       throws MetaException {
     if (needsAuthorization(getUserName())) {
       try {
-        return HiveAuthzBindingHook.filterShowTables(getHiveAuthzBinding(),
+        return HiveAuthzBindingHookBase.filterShowTables(getHiveAuthzBinding(),
             tabList, HiveOperation.SHOWTABLES, getUserName(), dbName);
       } catch (SemanticException e) {
         throw new MetaException("Error getting Table list " + e.getMessage());

http://git-wip-us.apache.org/repos/asf/sentry/blob/0702f54b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/metastore/MetastoreAuthzBindingV2.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/metastore/MetastoreAuthzBindingV2.java b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/metastore/MetastoreAuthzBindingV2.java
index d937491..fe629fc 100644
--- a/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/metastore/MetastoreAuthzBindingV2.java
+++ b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/metastore/MetastoreAuthzBindingV2.java
@@ -17,12 +17,23 @@
  */
 package org.apache.sentry.binding.hive.v2.metastore;
 
+import java.io.IOException;
+import java.util.List;
+
+import javax.security.auth.login.LoginException;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.events.PreDropPartitionEvent;
+import org.apache.hadoop.hive.ql.metadata.AuthorizationException;
 import org.apache.hadoop.hive.ql.plan.HiveOperation;
-import org.apache.sentry.binding.metastore.MetastoreAuthzBinding;
+import org.apache.sentry.SentryUserException;
+import org.apache.sentry.binding.hive.authz.HiveAuthzBinding;
+import org.apache.sentry.binding.hive.v2.HiveAuthzPrivilegesMapV2;
+import org.apache.sentry.binding.metastore.MetastoreAuthzBindingBase;
+import org.apache.sentry.core.common.Subject;
+import org.apache.sentry.core.model.db.DBModelAuthorizable;
 
 /**
  * Sentry binding for Hive Metastore. The binding is integrated into Metastore
@@ -33,12 +44,13 @@ import org.apache.sentry.binding.metastore.MetastoreAuthzBinding;
  * passed down to the hive binding which handles the authorization. This ensures
  * that we follow the same privilege model and policies.
  */
-public class MetastoreAuthzBindingV2 extends MetastoreAuthzBinding {
+public class MetastoreAuthzBindingV2 extends MetastoreAuthzBindingBase {
 
   public MetastoreAuthzBindingV2(Configuration config) throws Exception {
     super(config);
   }
 
+  @Override
   protected void authorizeDropPartition(PreDropPartitionEvent context)
       throws InvalidOperationException, MetaException {
     authorizeMetastoreAccess(
@@ -51,4 +63,37 @@ public class MetastoreAuthzBindingV2 extends MetastoreAuthzBinding {
             context.getTable().getTableName()).build());
   }
 
+  /**
+   * Assemble the required privileges and requested privileges. Validate using
+   * Hive bind auth provider
+   * @param hiveOp
+   * @param inputHierarchy
+   * @param outputHierarchy
+   * @throws InvalidOperationException
+   */
+  @Override
+  protected void authorizeMetastoreAccess(HiveOperation hiveOp,
+      List<List<DBModelAuthorizable>> inputHierarchy,
+      List<List<DBModelAuthorizable>> outputHierarchy)
+      throws InvalidOperationException {
+    if (isSentryCacheOutOfSync()) {
+      throw invalidOperationException(new SentryUserException(
+          "Metastore/Sentry cache is out of sync"));
+    }
+    try {
+      HiveAuthzBinding hiveAuthzBinding = getHiveAuthzBinding();
+      hiveAuthzBinding.authorize(hiveOp, HiveAuthzPrivilegesMapV2
+          .getHiveAuthzPrivileges(hiveOp), new Subject(getUserName()),
+          inputHierarchy, outputHierarchy);
+    } catch (AuthorizationException e1) {
+      throw invalidOperationException(e1);
+    } catch (LoginException e1) {
+      throw invalidOperationException(e1);
+    } catch (IOException e1) {
+      throw invalidOperationException(e1);
+    } catch (Exception e) {
+      throw invalidOperationException(e);
+    }
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/sentry/blob/0702f54b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/metastore/SentryMetastorePostEventListenerV2.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/metastore/SentryMetastorePostEventListenerV2.java b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/metastore/SentryMetastorePostEventListenerV2.java
index 013d016..f1e981f 100644
--- a/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/metastore/SentryMetastorePostEventListenerV2.java
+++ b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/metastore/SentryMetastorePostEventListenerV2.java
@@ -24,10 +24,10 @@ import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.events.AddPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.DropPartitionEvent;
-import org.apache.sentry.binding.metastore.SentryMetastorePostEventListener;
+import org.apache.sentry.binding.metastore.SentryMetastorePostEventListenerBase;
 import org.apache.sentry.provider.db.SentryMetastoreListenerPlugin;
 
-public class SentryMetastorePostEventListenerV2 extends SentryMetastorePostEventListener {
+public class SentryMetastorePostEventListenerV2 extends SentryMetastorePostEventListenerBase {
 
   public SentryMetastorePostEventListenerV2(Configuration config) {
     super(config);

http://git-wip-us.apache.org/repos/asf/sentry/blob/0702f54b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/util/SimpleSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/util/SimpleSemanticAnalyzer.java b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/util/SimpleSemanticAnalyzer.java
index b50bbf4..4a3753b 100644
--- a/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/util/SimpleSemanticAnalyzer.java
+++ b/sentry-binding/sentry-binding-hive-v2/src/main/java/org/apache/sentry/binding/hive/v2/util/SimpleSemanticAnalyzer.java
@@ -144,6 +144,11 @@ public class SimpleSemanticAnalyzer {
    */
   private static final String UNLOCKTABLE = "^UNLOCK\\s+" + "TABLE\\s+" + "([A-Za-z0-9._]+)";
 
+  /**
+   * TRUNCATE TABLE tblname;
+   */
+  private static final String TRUNCATETABLE = "^TRUNCATE\\s+" + "TABLE\\s+" + "([A-Za-z0-9._]+)";
+
   private static Map<HiveOperation, String> OP_REGEX_MAP = new HashMap<HiveOperation, String>();
   static {
     // database metadata
@@ -161,7 +166,6 @@ public class SimpleSemanticAnalyzer {
     OP_REGEX_MAP.put(HiveOperation.ALTERTABLE_CLUSTER_SORT, ALTER_TABLE_REGEX);
     OP_REGEX_MAP.put(HiveOperation.ALTERTABLE_FILEFORMAT, ALTER_TABLE_REGEX);
     OP_REGEX_MAP.put(HiveOperation.ALTERTABLE_TOUCH, ALTER_TABLE_REGEX);
-    OP_REGEX_MAP.put(HiveOperation.ALTERTABLE_PROTECTMODE, ALTER_TABLE_REGEX);
     OP_REGEX_MAP.put(HiveOperation.ALTERTABLE_RENAMECOL, ALTER_TABLE_REGEX);
     OP_REGEX_MAP.put(HiveOperation.ALTERTABLE_ADDCOLS, ALTER_TABLE_REGEX);
     OP_REGEX_MAP.put(HiveOperation.ALTERTABLE_REPLACECOLS, ALTER_TABLE_REGEX);
@@ -176,7 +180,6 @@ public class SimpleSemanticAnalyzer {
     OP_REGEX_MAP.put(HiveOperation.ALTERTABLE_RENAME, ALTER_TABLE_REGEX);
     OP_REGEX_MAP.put(HiveOperation.ALTERTABLE_LOCATION, ALTER_TABLE_REGEX);
     OP_REGEX_MAP.put(HiveOperation.ALTERPARTITION_FILEFORMAT, ALTER_TABLE_REGEX);
-    OP_REGEX_MAP.put(HiveOperation.ALTERPARTITION_PROTECTMODE, ALTER_TABLE_REGEX);
     OP_REGEX_MAP.put(HiveOperation.ALTERPARTITION_SERDEPROPERTIES, ALTER_TABLE_REGEX);
     OP_REGEX_MAP.put(HiveOperation.ALTERPARTITION_SERIALIZER, ALTER_TABLE_REGEX);
     OP_REGEX_MAP.put(HiveOperation.ALTERPARTITION_MERGEFILES, ALTER_TABLE_REGEX);
@@ -192,6 +195,7 @@ public class SimpleSemanticAnalyzer {
     OP_REGEX_MAP.put(HiveOperation.UNLOCKTABLE, UNLOCKTABLE);
     OP_REGEX_MAP.put(HiveOperation.SHOWCOLUMNS, SHOWCOLUMNS);
     OP_REGEX_MAP.put(HiveOperation.SHOW_TABLESTATUS, SHOW_TABLESTATUS);
+    OP_REGEX_MAP.put(HiveOperation.TRUNCATETABLE, TRUNCATETABLE);
   }
 
   public SimpleSemanticAnalyzer(HiveOperation hiveOp, String cmd) throws HiveAuthzPluginException {
@@ -218,7 +222,6 @@ public class SimpleSemanticAnalyzer {
       case ALTERTABLE_CLUSTER_SORT:
       case ALTERTABLE_FILEFORMAT:
       case ALTERTABLE_TOUCH:
-      case ALTERTABLE_PROTECTMODE:
       case ALTERTABLE_RENAMECOL:
       case ALTERTABLE_ADDCOLS:
       case ALTERTABLE_REPLACECOLS:
@@ -236,7 +239,6 @@ public class SimpleSemanticAnalyzer {
       case ALTERVIEW_PROPERTIES:
         // alter partition
       case ALTERPARTITION_FILEFORMAT:
-      case ALTERPARTITION_PROTECTMODE:
       case ALTERPARTITION_SERDEPROPERTIES:
       case ALTERPARTITION_SERIALIZER:
       case ALTERPARTITION_MERGEFILES:
@@ -250,6 +252,7 @@ public class SimpleSemanticAnalyzer {
       case LOCKTABLE:
       case UNLOCKTABLE:
       case SHOWCOLUMNS:
+      case TRUNCATETABLE:
         parseTableMeta(cmd, OP_REGEX_MAP.get(hiveOp));
         break;
       case SHOWINDEXES: