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

git commit: SENTRY-241: Sentry GrantRevokeTask should fire the sentry failure look (Prasad Mujumdar via Sravya Tirukkovalur)

Repository: incubator-sentry
Updated Branches:
  refs/heads/master 8525fe62e -> 12d6c965f


SENTRY-241: Sentry GrantRevokeTask should fire the sentry failure look (Prasad Mujumdar via Sravya Tirukkovalur)


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

Branch: refs/heads/master
Commit: 12d6c965feda9e3a134c5877c5cb83dacd574834
Parents: 8525fe6
Author: Sravya Tirukkovalur <sr...@clouera.com>
Authored: Mon Jun 2 18:20:38 2014 -0700
Committer: Sravya Tirukkovalur <sr...@clouera.com>
Committed: Mon Jun 2 18:20:38 2014 -0700

----------------------------------------------------------------------
 .../hive/ql/exec/SentryGrantRevokeTask.java     |  67 ++++---
 .../binding/hive/HiveAuthzBindingHook.java      |  38 +++-
 .../dbprovider/AbstractTestWithDbProvider.java  |  14 +-
 .../TestDbSentryOnFailureHookLoading.java       | 177 +++++++++++++++++++
 .../e2e/hive/DummySentryOnFailureHook.java      |  16 +-
 5 files changed, 278 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/12d6c965/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java
index 1f56de7..df9b0db 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.io.OutputStreamWriter;
 import java.io.Serializable;
 import java.util.ArrayList;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
@@ -33,6 +34,9 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.hadoop.hive.ql.DriverContext;
 import org.apache.hadoop.hive.ql.QueryPlan;
+import org.apache.hadoop.hive.ql.hooks.ReadEntity;
+import org.apache.hadoop.hive.ql.hooks.WriteEntity;
+import org.apache.hadoop.hive.ql.metadata.AuthorizationException;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.DDLWork;
 import org.apache.hadoop.hive.ql.plan.GrantDesc;
@@ -47,6 +51,9 @@ import org.apache.hadoop.hive.ql.plan.api.StageType;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.ql.session.SessionState.LogHelper;
 import org.apache.sentry.SentryUserException;
+import org.apache.sentry.binding.hive.HiveAuthzBindingHook;
+import org.apache.sentry.binding.hive.SentryOnFailureHookContext;
+import org.apache.sentry.binding.hive.SentryOnFailureHookContextImpl;
 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;
@@ -87,6 +94,7 @@ public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable
   private String server;
   private Subject subject;
   private Set<String> subjectGroups;
+  private String ipAddress;
 
 
   public SentryGrantRevokeTask() {
@@ -95,6 +103,7 @@ public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable
   public SentryGrantRevokeTask(SentryServiceClientFactory sentryClientFactory) {
     super();
     this.sentryClientFactory = sentryClientFactory;
+
   }
 
 
@@ -119,27 +128,41 @@ public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable
       Preconditions.checkNotNull(subject, "Subject cannot be null");
       server = Preconditions.checkNotNull(authzConf.get(AuthzConfVars.AUTHZ_SERVER_NAME.getVar()),
           "Config " + AuthzConfVars.AUTHZ_SERVER_NAME.getVar() + " is required");
-      if (work.getRoleDDLDesc() != null) {
-        return processRoleDDL(conf, console, sentryClient, subject.getName(),
-            hiveAuthzBinding, work.getRoleDDLDesc());
-      }
-      if (work.getGrantDesc() != null) {
-        return processGrantDDL(conf, console, sentryClient, subject.getName(),
-            server, work.getGrantDesc());
-      }
-      if (work.getRevokeDesc() != null) {
-        return processRevokeDDL(conf, console, sentryClient, subject.getName(),
-            server, work.getRevokeDesc());
-      }
-      if (work.getShowGrantDesc() != null) {
-        return processShowGrantDDL(conf, console, sentryClient, subject.getName(), server,
-            work.getShowGrantDesc());
-      }
-      if (work.getGrantRevokeRoleDDL() != null) {
-        return processGrantRevokeRoleDDL(conf, console, sentryClient, subject.getName(),
-            work.getGrantRevokeRoleDDL());
+      try {
+        if (work.getRoleDDLDesc() != null) {
+          return processRoleDDL(conf, console, sentryClient, subject.getName(),
+              hiveAuthzBinding, work.getRoleDDLDesc());
+        }
+        if (work.getGrantDesc() != null) {
+          return processGrantDDL(conf, console, sentryClient,
+              subject.getName(), server, work.getGrantDesc());
+        }
+        if (work.getRevokeDesc() != null) {
+          return processRevokeDDL(conf, console, sentryClient,
+              subject.getName(), server, work.getRevokeDesc());
+        }
+        if (work.getShowGrantDesc() != null) {
+          return processShowGrantDDL(conf, console, sentryClient, subject.getName(), server,
+              work.getShowGrantDesc());
+        }
+        if (work.getGrantRevokeRoleDDL() != null) {
+          return processGrantRevokeRoleDDL(conf, console, sentryClient,
+              subject.getName(), work.getGrantRevokeRoleDDL());
+        }
+        throw new AssertionError(
+            "Unknown command passed to Sentry Grant/Revoke Task");
+      } catch (SentryUserException e) {
+        String csHooks = authzConf.get(
+            HiveAuthzConf.AuthzConfVars.AUTHZ_ONFAILURE_HOOKS.getVar(), "")
+            .trim();
+        SentryOnFailureHookContext hookContext = new SentryOnFailureHookContextImpl(
+            queryPlan.getQueryString(), new HashSet<ReadEntity>(),
+            new HashSet<WriteEntity>(), SessionState.get().getHiveOperation(),
+            null, null, null, null, subject.getName(), ipAddress,
+            new AuthorizationException(e), conf);
+        HiveAuthzBindingHook.runFailureHook(hookContext, csHooks);
+        throw e; // rethrow the exception for logging
       }
-      throw new AssertionError("Unknown command passed to Sentry Grant/Revoke Task");
     } catch(Throwable throwable) {
       setException(throwable);
       String msg = "Error processing Sentry command: " + throwable.getMessage();
@@ -174,6 +197,10 @@ public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable
     this.subjectGroups = subjectGroups;
   }
 
+  public void setIpAddress(String ipAddress) {
+    this.ipAddress = ipAddress;
+  }
+
   private int processRoleDDL(HiveConf conf, LogHelper console,
       SentryPolicyServiceClient sentryClient, String subject,
       HiveAuthzBinding hiveAuthzBinding, RoleDDLDesc desc)

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/12d6c965/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 f8c694c..a362363 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
@@ -85,7 +85,6 @@ implements HiveDriverFilterHook {
 
   public HiveAuthzBindingHook() throws Exception {
     SessionState session = SessionState.get();
-    boolean depreicatedConfigFile = false;
     if(session == null) {
       throw new IllegalStateException("Session has not been started");
     }
@@ -93,6 +92,13 @@ implements HiveDriverFilterHook {
     if(hiveConf == null) {
       throw new IllegalStateException("Session HiveConf is null");
     }
+    authzConf = loadAuthzConf(hiveConf);
+    hiveAuthzBinding = new HiveAuthzBinding(hiveConf, authzConf);
+  }
+
+  public static HiveAuthzConf loadAuthzConf(HiveConf hiveConf) {
+    boolean depreicatedConfigFile = false;
+    HiveAuthzConf newAuthzConf = null;
     String hiveAuthzConf = hiveConf.get(HiveAuthzConf.HIVE_SENTRY_CONF_URL);
     if(hiveAuthzConf == null || (hiveAuthzConf = hiveAuthzConf.trim()).isEmpty()) {
       hiveAuthzConf = hiveConf.get(HiveAuthzConf.HIVE_ACCESS_CONF_URL);
@@ -104,7 +110,7 @@ implements HiveDriverFilterHook {
           + " value '" + hiveAuthzConf + "' is invalid.");
     }
     try {
-      authzConf = new HiveAuthzConf(new URL(hiveAuthzConf));
+      newAuthzConf = new HiveAuthzConf(new URL(hiveAuthzConf));
     } catch (MalformedURLException e) {
       if (depreicatedConfigFile) {
         throw new IllegalArgumentException("Configuration key " + HiveAuthzConf.HIVE_ACCESS_CONF_URL
@@ -114,7 +120,7 @@ implements HiveDriverFilterHook {
             + " specifies a malformed URL '" + hiveAuthzConf + "'", e);
       }
     }
-    hiveAuthzBinding = new HiveAuthzBinding(hiveConf, authzConf);
+    return newAuthzConf;
   }
 
   /**
@@ -281,6 +287,7 @@ implements HiveDriverFilterHook {
         sentryTask.setAuthzConf(authzConf);
         sentryTask.setSubject(subject);
         sentryTask.setSubjectGroups(subjectGroups);
+        sentryTask.setIpAddress(context.getIpAddress());
       }
     }
 
@@ -313,8 +320,11 @@ implements HiveDriverFilterHook {
         context.getCommand(), context.getInputs(), context.getOutputs(),
         hiveOp, currDB, currTab, udfURI, partitionURI, context.getUserName(),
         context.getIpAddress(), e, context.getConf());
+    String csHooks = authzConf.get(
+        HiveAuthzConf.AuthzConfVars.AUTHZ_ONFAILURE_HOOKS.getVar(), "").trim();
+
     try {
-      for (Hook aofh : getHooks(HiveAuthzConf.AuthzConfVars.AUTHZ_ONFAILURE_HOOKS)) {
+      for (Hook aofh : getHooks(csHooks)) {
         ((SentryOnFailureHook)aofh).run(hookCtx);
       }
     } catch (Exception ex) {
@@ -322,6 +332,16 @@ implements HiveDriverFilterHook {
     }
   }
 
+  public static void runFailureHook(SentryOnFailureHookContext hookContext,
+      String csHooks) {
+    try {
+      for (Hook aofh : getHooks(csHooks)) {
+        ((SentryOnFailureHook) aofh).run(hookContext);
+      }
+    } catch (Exception ex) {
+      LOG.error("Error executing hook:", ex);
+    }
+  }
   /**
    * Convert the input/output entities into authorizables. generate
    * authorizables for cases like Database and metadata operations where the
@@ -703,8 +723,8 @@ implements HiveDriverFilterHook {
    * @return
    * @throws Exception
    */
-  private List<Hook> getHooks(HiveAuthzConf.AuthzConfVars hookConfVar) throws Exception {
-    return getHooks(hookConfVar, Hook.class);
+  private static List<Hook> getHooks(String csHooks) throws Exception {
+    return getHooks(csHooks, Hook.class);
   }
 
   /**
@@ -718,11 +738,11 @@ implements HiveDriverFilterHook {
    *                    they are listed in the value of hookConfVar
    * @throws Exception
    */
-  private <T extends Hook> List<T> getHooks(HiveAuthzConf.AuthzConfVars hookConfVar, Class<T> clazz)
+  private static <T extends Hook> List<T> getHooks(String csHooks,
+      Class<T> clazz)
       throws Exception {
 
     List<T> hooks = new ArrayList<T>();
-    String csHooks = authzConf.get(hookConfVar.getVar(), "").trim();
     if (csHooks.isEmpty()) {
       return hooks;
     }
@@ -733,7 +753,7 @@ implements HiveDriverFilterHook {
             (T) Class.forName(hookClass, true, JavaUtils.getClassLoader()).newInstance();
         hooks.add(hook);
       } catch (ClassNotFoundException e) {
-        LOG.error(hookConfVar.getVar() + " Class not found:" + e.getMessage());
+        LOG.error(hookClass + " Class not found:" + e.getMessage());
         throw e;
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/12d6c965/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/AbstractTestWithDbProvider.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/AbstractTestWithDbProvider.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/AbstractTestWithDbProvider.java
index c6f1ce2..09a14bf 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/AbstractTestWithDbProvider.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/AbstractTestWithDbProvider.java
@@ -50,7 +50,7 @@ public abstract class AbstractTestWithDbProvider extends AbstractTestWithHiveSer
 
   protected static final String SERVER_HOST = "localhost";
 
-  private Map<String, String> properties;
+  private Map<String, String> properties = Maps.newHashMap();
   private File dbDir;
   private SentryService server;
   private Configuration conf;
@@ -62,8 +62,13 @@ public abstract class AbstractTestWithDbProvider extends AbstractTestWithHiveSer
   public static void setupTest() throws Exception {
   }
 
-  public void createContext() throws Exception {
-    properties = Maps.newHashMap();
+  @Override
+  public Context createContext(Map<String, String> properties) throws Exception {
+    this.properties = properties;
+    return createContext();
+  }
+
+  public Context createContext() throws Exception {
     conf = new Configuration(false);
     policyFile = PolicyFile.setAdminOnServer1(ADMINGROUP);
     properties.put(HiveServerFactory.AUTHZ_PROVIDER_BACKEND, SimpleDBProviderBackend.class.getName());
@@ -92,12 +97,13 @@ public abstract class AbstractTestWithDbProvider extends AbstractTestWithHiveSer
     properties.put(ClientConfig.SERVER_RPC_PORT,
         String.valueOf(server.getAddress().getPort()));
 
-    context = createContext(properties);
+    context = super.createContext(properties);
     policyFile
         .setUserGroupMapping(StaticUserGroup.getStaticMapping())
         .write(context.getPolicyFile(), policyFilePath);
 
     startSentryService();
+    return context;
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/12d6c965/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbSentryOnFailureHookLoading.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbSentryOnFailureHookLoading.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbSentryOnFailureHookLoading.java
new file mode 100644
index 0000000..41a31e8
--- /dev/null
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/dbprovider/TestDbSentryOnFailureHookLoading.java
@@ -0,0 +1,177 @@
+/*
+ * 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 static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+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;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.hive.ql.plan.HiveOperation;
+import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
+import org.apache.sentry.provider.file.PolicyFile;
+import org.apache.sentry.tests.e2e.hive.DummySentryOnFailureHook;
+import org.apache.sentry.tests.e2e.hive.StaticUserGroup;
+import org.apache.sentry.tests.e2e.hive.hiveserver.HiveServerFactory;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.common.io.Resources;
+
+public class TestDbSentryOnFailureHookLoading extends AbstractTestWithDbProvider {
+
+  private PolicyFile policyFile;
+
+  Map<String, String > testProperties;
+  private static final String SINGLE_TYPE_DATA_FILE_NAME = "kv1.dat";
+
+  @Before
+  public void setup() throws Exception {
+    testProperties = new HashMap<String, String>();
+    testProperties.put(HiveAuthzConf.AuthzConfVars.AUTHZ_ONFAILURE_HOOKS.getVar(),
+        DummySentryOnFailureHook.class.getName());
+    policyFile = PolicyFile.setAdminOnServer1(ADMINGROUP);
+    createContext(testProperties);
+    DummySentryOnFailureHook.invoked = false;
+  }
+
+  /* Admin creates database DB_2
+   * user1 tries to drop DB_2, but it has permissions for DB_1.
+   */
+  @Test
+  public void testOnFailureHookLoading() throws Exception {
+
+    // Do not run this test if run with external HiveServer2
+    // This test checks for a static member, which will not
+    // be set if HiveServer2 and the test run in different JVMs
+    String hiveServer2Type = System.getProperty(
+        HiveServerFactory.HIVESERVER2_TYPE);
+    if (hiveServer2Type != null &&
+        HiveServerFactory.HiveServer2Type.valueOf(hiveServer2Type.trim()) !=
+        HiveServerFactory.HiveServer2Type.InternalHiveServer2) {
+      return;
+    }
+
+    File dataDir = context.getDataDir();
+    //copy data file to test dir
+    File 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();
+
+    policyFile
+.addRolesToGroup(USERGROUP1, "all_db1")
+        .addPermissionsToRole("all_db1", "server=server1->db=DB_1")
+        .setUserGroupMapping(StaticUserGroup.getStaticMapping())
+        .write(context.getPolicyFile());
+
+    // setup db objects needed by the test
+    Connection connection = context.createConnection(ADMIN1);
+    Statement statement = context.createStatement(connection);
+    statement.execute("CREATE ROLE admin_role");
+    statement.execute("GRANT ALL ON SERVER "
+        + HiveServerFactory.DEFAULT_AUTHZ_SERVER_NAME + " TO ROLE admin_role");
+    statement.execute("GRANT ROLE admin_role TO GROUP " + ADMINGROUP);
+
+    statement.execute("CREATE ROLE all_db1");
+    statement.execute("GRANT ALL ON DATABASE DB_1 TO ROLE all_db1");
+    statement.execute("GRANT ROLE all_db1 TO GROUP " + USERGROUP1);
+
+    statement.execute("DROP DATABASE IF EXISTS DB_1 CASCADE");
+    statement.execute("DROP DATABASE IF EXISTS DB_2 CASCADE");
+    statement.execute("CREATE DATABASE DB_1");
+    statement.execute("CREATE DATABASE DB_2");
+    statement.close();
+    connection.close();
+
+    // test execution
+    connection = context.createConnection(USER1_1);
+    statement = context.createStatement(connection);
+
+    // negative test case: user can't create table in other user's database
+    assertFalse(DummySentryOnFailureHook.invoked);
+    DummySentryOnFailureHook.setHiveOp(HiveOperation.CREATETABLE);
+      try {
+      statement.execute("CREATE TABLE DB2.TAB2(id INT)");
+      Assert.fail("Expected SQL exception");
+    } catch (SQLException e) {
+      assertTrue(DummySentryOnFailureHook.invoked);
+    }
+
+    statement.close();
+    connection.close();
+
+    //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();
+    context.close();
+  }
+
+  /*
+   * Admin creates database DB_2 user1 tries to drop DB_2, but it has
+   * permissions for DB_1.
+   */
+  @Test
+  public void testOnFailureHookForAuthDDL() throws Exception {
+
+    // Do not run this test if run with external HiveServer2
+    // This test checks for a static member, which will not
+    // be set if HiveServer2 and the test run in different JVMs
+    String hiveServer2Type = System
+        .getProperty(HiveServerFactory.HIVESERVER2_TYPE);
+    if (hiveServer2Type != null
+        && HiveServerFactory.HiveServer2Type.valueOf(hiveServer2Type.trim()) != HiveServerFactory.HiveServer2Type.InternalHiveServer2) {
+      return;
+    }
+
+    File dataDir = context.getDataDir();
+    policyFile.addRolesToGroup(USERGROUP1, "all_db1")
+        .addPermissionsToRole("all_db1", "server=server1->db=DB_1")
+        .setUserGroupMapping(StaticUserGroup.getStaticMapping())
+        .write(context.getPolicyFile());
+
+    Connection connection = context.createConnection(USER1_1);
+    Statement statement = context.createStatement(connection);
+
+    // negative test case: non admin user can't create role
+    assertFalse(DummySentryOnFailureHook.invoked);
+    DummySentryOnFailureHook.setHiveOp(HiveOperation.CREATEROLE);
+    try {
+      statement.execute("CREATE ROLE fooTest");
+      Assert.fail("Expected SQL exception");
+    } catch (SQLException e) {
+      assertTrue(DummySentryOnFailureHook.invoked);
+    }
+
+    statement.close();
+    connection.close();
+    context.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/12d6c965/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/DummySentryOnFailureHook.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/DummySentryOnFailureHook.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/DummySentryOnFailureHook.java
index e4055a7..079f273 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/DummySentryOnFailureHook.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/DummySentryOnFailureHook.java
@@ -17,16 +17,30 @@
 
 package org.apache.sentry.tests.e2e.hive;
 
+import junit.framework.Assert;
+
+import org.apache.hadoop.hive.ql.plan.HiveOperation;
 import org.apache.sentry.binding.hive.SentryOnFailureHook;
 import org.apache.sentry.binding.hive.SentryOnFailureHookContext;
 
 public class DummySentryOnFailureHook implements SentryOnFailureHook {
 
-  static boolean invoked = false;
+  public static boolean invoked = false;
+  public static boolean checkHiveOp = false;
+  public static HiveOperation hiveOp;
+
+  public static void setHiveOp(HiveOperation newHiveOp) {
+    checkHiveOp = true;
+    hiveOp = newHiveOp;
+  }
 
   @Override
   public void run(SentryOnFailureHookContext failureHookContext)
       throws Exception {
     invoked = true;
+    if (checkHiveOp) {
+      checkHiveOp = false;
+      Assert.assertTrue(hiveOp.equals(failureHookContext.getHiveOp()));
+    }
   }
 }