You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sentry.apache.org by sh...@apache.org on 2014/03/19 00:30:08 UTC

[1/2] SENTRY-129: Implement Hive Sentry Authz DDL Task Factory (Brock Noland via Shreepadma Venugopalan)

Repository: incubator-sentry
Updated Branches:
  refs/heads/master fbf042e66 -> 63c134f36


http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/GSSCallback.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/GSSCallback.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/GSSCallback.java
index c4a0fd4..22f31cd 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/GSSCallback.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/GSSCallback.java
@@ -26,6 +26,7 @@ import javax.security.sasl.AuthorizeCallback;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.SaslRpcServer;
+import org.apache.sentry.service.thrift.ServiceConstants.ServerConfig;
 
 public class GSSCallback extends SaslRpcServer.SaslGssCallbackHandler {
 
@@ -54,12 +55,12 @@ public class GSSCallback extends SaslRpcServer.SaslGssCallbackHandler {
   }
 
   boolean allowConnect(String principal) {
-    String allowedPrincipals = conf.get("sentry.service.allow.connect");
+    String allowedPrincipals = conf.get(ServerConfig.ALLOW_CONNECT);
     if (allowedPrincipals == null) {
       return false;
     }
     List<String> items = Arrays.asList(allowedPrincipals.split("\\s*,\\s*"));
-    for (String item:items) {
+    for (String item : items) {
       if(comparePrincipals(item, principal)) {
         return true;
       }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/SentryService.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/SentryService.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/SentryService.java
index fbb0eef..bebaf0d 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/SentryService.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/SentryService.java
@@ -52,6 +52,7 @@ import org.apache.thrift.server.TThreadPoolServer;
 import org.apache.thrift.transport.TSaslServerTransport;
 import org.apache.thrift.transport.TServerSocket;
 import org.apache.thrift.transport.TServerTransport;
+import org.apache.thrift.transport.TTransportFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -71,6 +72,7 @@ public class SentryService implements Runnable {
   private final InetSocketAddress address;
   private final int maxThreads;
   private final int minThreads;
+  private boolean kerberos;
   private final String principal;
   private final String[] principalParts;
   private final String keytab;
@@ -90,20 +92,28 @@ public class SentryService implements Runnable {
         conf.get(ServerConfig.RPC_ADDRESS, ServerConfig.RPC_ADDRESS_DEFAULT),
         port);
     LOGGER.info("Configured on address " + address);
+    kerberos = ServerConfig.SECURITY_MODE_KERBEROS.equalsIgnoreCase(
+        conf.get(ServerConfig.SECURITY_MODE, ServerConfig.SECURITY_MODE_KERBEROS).trim());
     maxThreads = conf.getInt(ServerConfig.RPC_MAX_THREADS,
         ServerConfig.RPC_MAX_THREADS_DEFAULT);
     minThreads = conf.getInt(ServerConfig.RPC_MIN_THREADS,
         ServerConfig.RPC_MIN_THREADS_DEFAULT);
-    principal = Preconditions.checkNotNull(conf.get(ServerConfig.PRINCIPAL),
-        ServerConfig.PRINCIPAL + " is required");
-    principalParts = SaslRpcServer.splitKerberosName(principal);
-    Preconditions.checkArgument(principalParts.length == 3,
-        "Kerberos principal should have 3 parts: " + principal);
-    keytab = Preconditions.checkNotNull(conf.get(ServerConfig.KEY_TAB),
-        ServerConfig.KEY_TAB + " is required");
-    File keytabFile = new File(keytab);
-    Preconditions.checkState(keytabFile.isFile() && keytabFile.canRead(),
-        "Keytab " + keytab + " does not exist or is not readable.");
+    if (kerberos) {
+      principal = Preconditions.checkNotNull(conf.get(ServerConfig.PRINCIPAL),
+          ServerConfig.PRINCIPAL + " is required");
+      principalParts = SaslRpcServer.splitKerberosName(principal);
+      Preconditions.checkArgument(principalParts.length == 3,
+          "Kerberos principal should have 3 parts: " + principal);
+      keytab = Preconditions.checkNotNull(conf.get(ServerConfig.KEY_TAB),
+          ServerConfig.KEY_TAB + " is required");
+      File keytabFile = new File(keytab);
+      Preconditions.checkState(keytabFile.isFile() && keytabFile.canRead(),
+          "Keytab " + keytab + " does not exist or is not readable.");
+    } else {
+      principal = null;
+      principalParts = null;
+      keytab = null;
+    }
     serviceExecutor = Executors.newSingleThreadExecutor(new ThreadFactory() {
       private int count = 0;
 
@@ -120,60 +130,24 @@ public class SentryService implements Runnable {
   public void run() {
     LoginContext loginContext = null;
     try {
-      Subject subject = new Subject(false,
-          Sets.newHashSet(new KerberosPrincipal(principal)),
-          new HashSet<Object>(), new HashSet<Object>());
-      loginContext = new LoginContext("", subject, null,
-          KerberosConfiguration.createClientConfig(principal, new File(keytab)));
-      loginContext.login();
-      subject = loginContext.getSubject();
-      Subject.doAs(subject, new PrivilegedExceptionAction<Void>() {
-        @Override
-        public Void run() throws Exception {
-          Iterable<String> processorFactories = ConfUtilties.CLASS_SPLITTER
-              .split(conf.get(ServerConfig.PROCESSOR_FACTORIES,
-                  ServerConfig.PROCESSOR_FACTORIES_DEFAULT).trim());
-          TMultiplexedProcessor processor = new TMultiplexedProcessor();
-          boolean registeredProcessor = false;
-          for (String processorFactory : processorFactories) {
-            Class<?> clazz = conf.getClassByName(processorFactory);
-            if (!ProcessorFactory.class.isAssignableFrom(clazz)) {
-              throw new IllegalArgumentException("Processor Factory "
-                  + processorFactory + " is not a "
-                  + ProcessorFactory.class.getName());
-            }
-            try {
-              Constructor<?> constructor = clazz
-                  .getConstructor(Configuration.class);
-              ProcessorFactory factory = (ProcessorFactory) constructor
-                  .newInstance(conf);
-              registeredProcessor = registeredProcessor
-                  || factory.register(processor);
-            } catch (Exception e) {
-              throw new IllegalStateException("Could not create "
-                  + processorFactory, e);
-            }
+      if (kerberos) {
+        Subject subject = new Subject(false,
+            Sets.newHashSet(new KerberosPrincipal(principal)),
+            new HashSet<Object>(), new HashSet<Object>());
+        loginContext = new LoginContext("", subject, null,
+            KerberosConfiguration.createClientConfig(principal, new File(keytab)));
+        loginContext.login();
+        subject = loginContext.getSubject();
+        Subject.doAs(subject, new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws Exception {
+            runServer();
+            return null;
           }
-          if (!registeredProcessor) {
-            throw new IllegalStateException(
-                "Failed to register any processors from " + processorFactories);
-          }
-          TServerTransport serverTransport = new TServerSocket(address);
-          TSaslServerTransport.Factory saslTransportFactory = new TSaslServerTransport.Factory();
-          saslTransportFactory.addServerDefinition(AuthMethod.KERBEROS
-              .getMechanismName(), principalParts[0], principalParts[1],
-              ServerConfig.SASL_PROPERTIES, new GSSCallback(conf));
-          TThreadPoolServer.Args args = new TThreadPoolServer.Args(
-              serverTransport).processor(processor)
-              .transportFactory(saslTransportFactory)
-              .protocolFactory(new TBinaryProtocol.Factory())
-              .minWorkerThreads(minThreads).maxWorkerThreads(maxThreads);
-          thriftServer = new TThreadPoolServer(args);
-          LOGGER.info("Serving on " + address);
-          thriftServer.serve();
-          return null;
-        }
-      });
+        });
+      } else {
+        runServer();
+      }
     } catch (Throwable t) {
       LOGGER.error("Error starting server", t);
     } finally {
@@ -188,6 +162,56 @@ public class SentryService implements Runnable {
     }
   }
 
+  private void runServer() throws Exception {
+    Iterable<String> processorFactories = ConfUtilties.CLASS_SPLITTER
+        .split(conf.get(ServerConfig.PROCESSOR_FACTORIES,
+            ServerConfig.PROCESSOR_FACTORIES_DEFAULT).trim());
+    TMultiplexedProcessor processor = new TMultiplexedProcessor();
+    boolean registeredProcessor = false;
+    for (String processorFactory : processorFactories) {
+      Class<?> clazz = conf.getClassByName(processorFactory);
+      if (!ProcessorFactory.class.isAssignableFrom(clazz)) {
+        throw new IllegalArgumentException("Processor Factory "
+            + processorFactory + " is not a "
+            + ProcessorFactory.class.getName());
+      }
+      try {
+        Constructor<?> constructor = clazz
+            .getConstructor(Configuration.class);
+        ProcessorFactory factory = (ProcessorFactory) constructor
+            .newInstance(conf);
+        registeredProcessor = registeredProcessor
+            || factory.register(processor);
+      } catch (Exception e) {
+        throw new IllegalStateException("Could not create "
+            + processorFactory, e);
+      }
+    }
+    if (!registeredProcessor) {
+      throw new IllegalStateException(
+          "Failed to register any processors from " + processorFactories);
+    }
+    TServerTransport serverTransport = new TServerSocket(address);
+    TTransportFactory transportFactory = null;
+    if (kerberos) {
+      TSaslServerTransport.Factory saslTransportFactory = new TSaslServerTransport.Factory();
+      saslTransportFactory.addServerDefinition(AuthMethod.KERBEROS
+          .getMechanismName(), principalParts[0], principalParts[1],
+          ServerConfig.SASL_PROPERTIES, new GSSCallback(conf));
+      transportFactory = saslTransportFactory;
+    } else {
+      transportFactory = new TTransportFactory();
+    }
+    TThreadPoolServer.Args args = new TThreadPoolServer.Args(
+        serverTransport).processor(processor)
+        .transportFactory(transportFactory)
+        .protocolFactory(new TBinaryProtocol.Factory())
+        .minWorkerThreads(minThreads).maxWorkerThreads(maxThreads);
+    thriftServer = new TThreadPoolServer(args);
+    LOGGER.info("Serving on " + address);
+    thriftServer.serve();
+  }
+
   public InetSocketAddress getAddress() {
     return address;
   }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/ServiceConstants.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/ServiceConstants.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/ServiceConstants.java
index 59cb456..1b36690 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/ServiceConstants.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/ServiceConstants.java
@@ -45,6 +45,13 @@ public class ServiceConstants {
   }
   public static class ServerConfig {
     public static final ImmutableMap<String, String> SASL_PROPERTIES = ServiceConstants.SASL_PROPERTIES;
+    /**
+     * This configuration parameter is only meant to be used for testing purposes.
+     */
+    public static final String SECURITY_MODE = "sentry.service.security.mode";
+    public static final String SECURITY_MODE_KERBEROS = "kerberos";
+    public static final String SECURITY_MODE_NONE = "none";
+    public static final String ADMIN_GROUPS = "sentry.service.admin.group";
     public static final String PRINCIPAL = "sentry.service.server.principal";
     public static final String KEY_TAB = "sentry.service.server.keytab";
     public static final String RPC_PORT = "sentry.service.server.rpc-port";

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/Status.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/Status.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/Status.java
index e1549ca..c167837 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/Status.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/Status.java
@@ -23,6 +23,7 @@ import java.io.StringWriter;
 import javax.annotation.Nullable;
 
 import org.apache.sentry.SentryUserException;
+import org.apache.sentry.provider.db.SentryAccessDeniedException;
 import org.apache.sentry.provider.db.SentryAlreadyExistsException;
 import org.apache.sentry.provider.db.SentryInvalidInputException;
 import org.apache.sentry.provider.db.SentryNoSuchObjectException;
@@ -37,6 +38,7 @@ public enum Status {
   NO_SUCH_OBJECT(ThriftConstants.TSENTRY_STATUS_NO_SUCH_OBJECT),
   RUNTIME_ERROR(ThriftConstants.TSENTRY_STATUS_RUNTIME_ERROR),
   INVALID_INPUT(ThriftConstants.TSENTRY_STATUS_INVALID_INPUT),
+  ACCESS_DENIED(ThriftConstants.TSENTRY_STATUS_ACCESS_DENIED),
   UNKNOWN(-1)
   ;
   private int code;
@@ -57,6 +59,9 @@ public enum Status {
   public static TSentryResponseStatus OK() {
     return Create(Status.OK, "");
   }
+  public static TSentryResponseStatus AccessDenied(String message, Throwable t) {
+    return Create(Status.ACCESS_DENIED, message, t);
+  }
   public static TSentryResponseStatus AlreadyExists(String message, Throwable t) {
     return Create(Status.ALREADY_EXISTS, message, t);
   }
@@ -99,6 +104,8 @@ public enum Status {
       throw new RuntimeException(serverErrorToString(thriftStatus));
     case INVALID_INPUT:
       throw new SentryInvalidInputException(serverErrorToString(thriftStatus));
+    case ACCESS_DENIED:
+      throw new SentryAccessDeniedException(serverErrorToString(thriftStatus));
     case UNKNOWN:
       throw new AssertionError(serverErrorToString(thriftStatus));
     default:

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/sentry-provider/sentry-provider-db/src/main/resources/sentry_common_service.thrift
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/resources/sentry_common_service.thrift b/sentry-provider/sentry-provider-db/src/main/resources/sentry_common_service.thrift
index 7a545be..9456274 100644
--- a/sentry-provider/sentry-provider-db/src/main/resources/sentry_common_service.thrift
+++ b/sentry-provider/sentry-provider-db/src/main/resources/sentry_common_service.thrift
@@ -31,6 +31,7 @@ const i32 TSENTRY_STATUS_ALREADY_EXISTS = 1;
 const i32 TSENTRY_STATUS_NO_SUCH_OBJECT = 2;
 const i32 TSENTRY_STATUS_RUNTIME_ERROR = 3;
 const i32 TSENTRY_STATUS_INVALID_INPUT = 4;
+const i32 TSENTRY_STATUS_ACCESS_DENIED = 5;
 
 struct TSentryResponseStatus {
 1: required i32 value,

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServerWithoutKerberos.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServerWithoutKerberos.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServerWithoutKerberos.java
new file mode 100644
index 0000000..81a9ea4
--- /dev/null
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServerWithoutKerberos.java
@@ -0,0 +1,45 @@
+/**
+ * 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 createRequired 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.provider.db.service.thrift;
+import java.util.Set;
+
+import org.apache.sentry.service.thrift.SentryServiceIntegrationBase;
+import org.junit.Test;
+
+import com.google.common.collect.Sets;
+
+
+public class TestSentryServerWithoutKerberos extends SentryServiceIntegrationBase {
+
+  @Override
+  public void beforeSetup() throws Exception {
+    this.kerberos = false;
+  }
+
+  @Test
+  public void testCreateRole() throws Exception {
+    String requestorUserName = ADMIN_USER;
+    Set<String> requestorUserGroupNames = Sets.newHashSet(ADMIN_GROUP);
+    String roleName = "admin_r";
+    client.dropRoleIfExists(requestorUserName, requestorUserGroupNames, roleName);
+    client.createRole(requestorUserName, requestorUserGroupNames, roleName);
+    client.dropRole(requestorUserName, requestorUserGroupNames, roleName);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceFailureCase.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceFailureCase.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceFailureCase.java
index a4643bf..b97db4b 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceFailureCase.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceFailureCase.java
@@ -25,11 +25,17 @@ import org.apache.sentry.service.thrift.ServiceConstants.ServerConfig;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-public class TestSentryServiceFailureCase extends SentryServiceIntegrationBase {
+import com.google.common.base.Strings;
 
+public class TestSentryServiceFailureCase extends SentryServiceIntegrationBase {
+  private static final Logger LOGGER = LoggerFactory.getLogger(TestSentryServiceFailureCase.class);
+  private static final String PEER_CALLBACK_FAILURE = "Peer indicated failure: Problem with callback handler";
   @Before @Override
   public void setup() throws Exception {
+    this.kerberos = true;
     beforeSetup();
     setupConf();
     conf.set(ServerConfig.ALLOW_CONNECT, "");
@@ -37,9 +43,21 @@ public class TestSentryServiceFailureCase extends SentryServiceIntegrationBase {
     afterSetup();
   }
 
-  @Test(expected = PrivilegedActionException.class)
+  @Test
   public void testClientServerConnectionFailure()  throws Exception {
-    connectToSentryService();
-    Assert.fail("Failed to receive Exception");
+    try {
+      connectToSentryService();
+      Assert.fail("Failed to receive Exception");
+    } catch(PrivilegedActionException e) {
+      LOGGER.info("Excepted exception", e);
+      Exception cause = e.getException();
+      if (cause == null) {
+        throw e;
+      }
+      String msg = "Exception message: " + cause.getMessage() + " to contain " +
+          PEER_CALLBACK_FAILURE;
+      Assert.assertTrue(msg, Strings.nullToEmpty(cause.getMessage())
+          .contains(PEER_CALLBACK_FAILURE));
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceIntegration.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceIntegration.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceIntegration.java
index aa1e860..dcaa246 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceIntegration.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestSentryServiceIntegration.java
@@ -36,8 +36,8 @@ public class TestSentryServiceIntegration extends SentryServiceIntegrationBase {
 
   @Test
   public void testCreateRole() throws Exception {
-    String requestorUserName = "user_1";
-    Set<String> requestorUserGroupNames = new HashSet<String>();
+    String requestorUserName = ADMIN_USER;
+    Set<String> requestorUserGroupNames = Sets.newHashSet(ADMIN_GROUP);
     String roleName = "admin_r";
 
     client.dropRoleIfExists(requestorUserName, requestorUserGroupNames, roleName);
@@ -58,8 +58,8 @@ public class TestSentryServiceIntegration extends SentryServiceIntegrationBase {
   @Test
   public void testGrantRevokePrivilege() throws Exception {
     String server = "server1";
-    String requestorUserName = "server_admin";
-    Set<String> requestorUserGroupNames = new HashSet<String>();
+    String requestorUserName = ADMIN_USER;
+    Set<String> requestorUserGroupNames = Sets.newHashSet(ADMIN_GROUP);
     String roleName = "admin_testdb";
     String db = "testDB";
     String group = "group1";

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/SentryServiceIntegrationBase.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/SentryServiceIntegrationBase.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/SentryServiceIntegrationBase.java
index ee5ca69..61bad23 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/SentryServiceIntegrationBase.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/SentryServiceIntegrationBase.java
@@ -58,6 +58,8 @@ public abstract class SentryServiceIntegrationBase extends KerberosSecurityTestc
   protected static final String SERVER_KERBEROS_NAME = "sentry/" + SERVER_HOST + "@" + REALM;
   protected static final String CLIENT_PRINCIPAL = "hive/" + SERVER_HOST;
   protected static final String CLIENT_KERBEROS_NAME = "hive/" + SERVER_HOST + "@" + REALM;
+  protected static final String ADMIN_USER = "admin_user";
+  protected static final String ADMIN_GROUP = "admin_group";
 
   protected SentryService server;
   protected SentryPolicyServiceClient client;
@@ -68,10 +70,12 @@ public abstract class SentryServiceIntegrationBase extends KerberosSecurityTestc
   protected File clientKeytab;
   protected Subject clientSubject;
   protected LoginContext clientLoginContext;
+  protected boolean kerberos;
   protected final Configuration conf = new Configuration(false);
 
   @Before
   public void setup() throws Exception {
+    this.kerberos = true;
     beforeSetup();
     setupConf();
     startSentryService();
@@ -91,18 +95,23 @@ public abstract class SentryServiceIntegrationBase extends KerberosSecurityTestc
   }
 
   public void setupConf() throws Exception {
-    kdc = getKdc();
-    kdcWorkDir = getWorkDir();
-    serverKeytab = new File(kdcWorkDir, "server.keytab");
-    clientKeytab = new File(kdcWorkDir, "client.keytab");
-    kdc.createPrincipal(serverKeytab, SERVER_PRINCIPAL);
-    kdc.createPrincipal(clientKeytab, CLIENT_PRINCIPAL);
-
-    conf.set(ServerConfig.PRINCIPAL, SERVER_KERBEROS_NAME);
-    conf.set(ServerConfig.KEY_TAB, serverKeytab.getPath());
+    if (kerberos) {
+      kdc = getKdc();
+      kdcWorkDir = getWorkDir();
+      serverKeytab = new File(kdcWorkDir, "server.keytab");
+      clientKeytab = new File(kdcWorkDir, "client.keytab");
+      kdc.createPrincipal(serverKeytab, SERVER_PRINCIPAL);
+      kdc.createPrincipal(clientKeytab, CLIENT_PRINCIPAL);
+      conf.set(ServerConfig.PRINCIPAL, SERVER_KERBEROS_NAME);
+      conf.set(ServerConfig.KEY_TAB, serverKeytab.getPath());
+      conf.set(ServerConfig.ALLOW_CONNECT, CLIENT_KERBEROS_NAME);
+    } else {
+      LOGGER.info("Stopped KDC");
+      conf.set(ServerConfig.SECURITY_MODE, ServerConfig.SECURITY_MODE_NONE);
+    }
+    conf.set(ServerConfig.ADMIN_GROUPS, ADMIN_GROUP);
     conf.set(ServerConfig.RPC_ADDRESS, SERVER_HOST);
     conf.set(ServerConfig.RPC_PORT, String.valueOf(0));
-    conf.set(ServerConfig.ALLOW_CONNECT, CLIENT_KERBEROS_NAME);
     dbDir = new File(Files.createTempDir(), "sentry_policy_db");
     conf.set(ServerConfig.SENTRY_STORE_JDBC_URL,
         "jdbc:derby:;databaseName=" + dbDir.getPath() + ";create=true");
@@ -114,19 +123,24 @@ public abstract class SentryServiceIntegrationBase extends KerberosSecurityTestc
   public void connectToSentryService() throws Exception {
     // The client should already be logged in when running in hive/impala/solr
     // therefore we must manually login in the integration tests
-    clientSubject = new Subject(false, Sets.newHashSet(
-                                  new KerberosPrincipal(CLIENT_KERBEROS_NAME)), new HashSet<Object>(),
-                                new HashSet<Object>());
-    clientLoginContext = new LoginContext("", clientSubject, null,
-                                          KerberosConfiguration.createClientConfig(CLIENT_KERBEROS_NAME, clientKeytab));
-    clientLoginContext.login();
-    clientSubject = clientLoginContext.getSubject();
-    client = Subject.doAs(clientSubject, new PrivilegedExceptionAction<SentryPolicyServiceClient>() {
-      @Override
-      public SentryPolicyServiceClient run() throws Exception {
-        return new SentryServiceClientFactory().create(conf);
-      }
-    });
+    final SentryServiceClientFactory factory = new SentryServiceClientFactory();
+    if (kerberos) {
+      clientSubject = new Subject(false, Sets.newHashSet(
+          new KerberosPrincipal(CLIENT_KERBEROS_NAME)), new HashSet<Object>(),
+        new HashSet<Object>());
+      clientLoginContext = new LoginContext("", clientSubject, null,
+          KerberosConfiguration.createClientConfig(CLIENT_KERBEROS_NAME, clientKeytab));
+      clientLoginContext.login();
+      clientSubject = clientLoginContext.getSubject();
+      client = Subject.doAs(clientSubject, new PrivilegedExceptionAction<SentryPolicyServiceClient>() {
+        @Override
+        public SentryPolicyServiceClient run() throws Exception {
+          return factory.create(conf);
+        }
+      });
+    } else {
+      client = factory.create(conf);
+    }
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/SimpleFileProviderBackend.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/SimpleFileProviderBackend.java b/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/SimpleFileProviderBackend.java
index 9fcebbb..e7f69ac 100644
--- a/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/SimpleFileProviderBackend.java
+++ b/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/SimpleFileProviderBackend.java
@@ -107,10 +107,6 @@ public class SimpleFileProviderBackend implements ProviderBackend {
   private boolean allowPerDatabaseSection;
   private volatile boolean initialized;
 
-  public SimpleFileProviderBackend(String resourcePath) throws IOException {
-    this(new Configuration(), new Path(resourcePath));
-  }
-
   public SimpleFileProviderBackend(Configuration conf, String resourcePath) throws IOException {
     this(conf, new Path(resourcePath));
   }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/sentry-provider/sentry-provider-file/src/test/java/org/apache/sentry/provider/file/TestSimpleFileProvderBackend.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-file/src/test/java/org/apache/sentry/provider/file/TestSimpleFileProvderBackend.java b/sentry-provider/sentry-provider-file/src/test/java/org/apache/sentry/provider/file/TestSimpleFileProvderBackend.java
index df5acdc..cd203cd 100644
--- a/sentry-provider/sentry-provider-file/src/test/java/org/apache/sentry/provider/file/TestSimpleFileProvderBackend.java
+++ b/sentry-provider/sentry-provider-file/src/test/java/org/apache/sentry/provider/file/TestSimpleFileProvderBackend.java
@@ -25,6 +25,7 @@ import java.io.IOException;
 import java.util.HashSet;
 
 import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.sentry.core.common.ActiveRoleSet;
 import org.apache.sentry.provider.common.ProviderBackendContext;
 import org.junit.After;
@@ -46,7 +47,8 @@ public class TestSimpleFileProvderBackend {
   public void setup() throws IOException {
     baseDir = Files.createTempDir();
     PolicyFiles.copyToDir(baseDir, resourcePath);
-    backend = new SimpleFileProviderBackend(new File(baseDir, resourcePath).toString());
+    backend = new SimpleFileProviderBackend(new Configuration(), new File(baseDir, resourcePath)
+      .toString());
     context = new ProviderBackendContext();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/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 2b00d16..7a2c6be 100644
--- a/sentry-tests/sentry-tests-hive/pom.xml
+++ b/sentry-tests/sentry-tests-hive/pom.xml
@@ -195,6 +195,11 @@ limitations under the License.
     </dependency>
     <dependency>
       <groupId>org.apache.sentry</groupId>
+      <artifactId>sentry-provider-db</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.sentry</groupId>
       <artifactId>sentry-provider-file</artifactId>
       <scope>test</scope>
     </dependency>

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/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 4f7dd2d..99ca16e 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
@@ -34,11 +34,13 @@ import java.util.Set;
 import junit.framework.Assert;
 
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.sentry.provider.db.SentryAccessDeniedException;
 import org.apache.sentry.tests.e2e.hive.hiveserver.HiveServer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Charsets;
+import com.google.common.base.Strings;
 import com.google.common.collect.Sets;
 import com.google.common.io.Files;
 
@@ -48,7 +50,7 @@ public class Context {
       .getLogger(Context.class);
 
   public static final String AUTHZ_EXCEPTION_SQL_STATE = "42000";
-  public static final String AUTHZ_EXEC_HOOK_EXCEPTION_SQL_STATE = "08S01";
+  public static final String AUTHZ_LINK_FAILURE_SQL_STATE = "08S01";
   public static final String AUTHZ_EXCEPTION_ERROR_MSG = "No valid privileges";
 
   private final HiveServer hiveServer;
@@ -169,6 +171,18 @@ public class Context {
     }
   }
 
+  public void assertSentryServiceAccessDenied(Statement statement, String query)
+      throws SQLException {
+    try {
+      statement.execute(query);
+      Assert.fail("Expected SQLException for '" + query + "'");
+    } catch (SQLException e) {
+      verifyAuthzExceptionForState(e, AUTHZ_LINK_FAILURE_SQL_STATE);
+      Assert.assertTrue("Expected SentryAccessDeniedException in " + e.getMessage(),
+          Strings.nullToEmpty(e.getMessage()).contains(SentryAccessDeniedException.class
+              .getSimpleName()));
+    }
+  }
 
   // verify that the sqlexception is due to authorization failure
   public void verifyAuthzException(SQLException sqlException) throws SQLException{
@@ -177,7 +191,7 @@ public class Context {
 
   // verify that the sqlexception is due to authorization failure due to exec hooks
   public void verifyAuthzExecHookException(SQLException sqlException) throws SQLException{
-    verifyAuthzExceptionForState(sqlException, AUTHZ_EXEC_HOOK_EXCEPTION_SQL_STATE);
+    verifyAuthzExceptionForState(sqlException, AUTHZ_LINK_FAILURE_SQL_STATE);
   }
 
   // verify that the sqlexception is due to authorization failure

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestDatabaseProvider.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestDatabaseProvider.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestDatabaseProvider.java
new file mode 100644
index 0000000..b8163b3
--- /dev/null
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestDatabaseProvider.java
@@ -0,0 +1,124 @@
+/*
+ * 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 java.io.File;
+import java.sql.Connection;
+import java.sql.Statement;
+import java.util.Map;
+import java.util.concurrent.TimeoutException;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.sentry.binding.hive.SentryHiveAuthorizationTaskFactoryImpl;
+import org.apache.sentry.provider.db.SimpleDBProviderBackend;
+import org.apache.sentry.provider.file.PolicyFile;
+import org.apache.sentry.service.thrift.SentryService;
+import org.apache.sentry.service.thrift.SentryServiceFactory;
+import org.apache.sentry.service.thrift.ServiceConstants.ClientConfig;
+import org.apache.sentry.service.thrift.ServiceConstants.ServerConfig;
+import org.apache.sentry.tests.e2e.hive.hiveserver.HiveServerFactory;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.common.collect.Maps;
+import com.google.common.io.Files;
+
+public class TestDatabaseProvider extends AbstractTestWithHiveServer {
+  protected static final String SERVER_HOST = "localhost";
+
+  private Context context;
+  private Map<String, String> properties;
+  private File dbDir;
+  private SentryService server;
+  private Configuration conf;
+  private PolicyFile policyFile;
+
+  @Before
+  public void setup() throws Exception {
+    properties = Maps.newHashMap();
+    conf = new Configuration(false);
+    policyFile = new PolicyFile();
+    properties.put(HiveServerFactory.AUTHZ_PROVIDER_BACKEND, SimpleDBProviderBackend.class.getName());
+    properties.put(ConfVars.HIVE_AUTHORIZATION_TASK_FACTORY.varname,
+        SentryHiveAuthorizationTaskFactoryImpl.class.getName());
+    properties.put(ServerConfig.SECURITY_MODE, ServerConfig.SECURITY_MODE_NONE);
+    properties.put(ServerConfig.ADMIN_GROUPS, ADMINGROUP);
+    properties.put(ServerConfig.RPC_ADDRESS, SERVER_HOST);
+    properties.put(ServerConfig.RPC_PORT, String.valueOf(0));
+    dbDir = new File(Files.createTempDir(), "sentry_policy_db");
+    properties.put(ServerConfig.SENTRY_STORE_JDBC_URL,
+        "jdbc:derby:;databaseName=" + dbDir.getPath() + ";create=true");
+    for (Map.Entry<String, String> entry : properties.entrySet()) {
+      conf.set(entry.getKey(), entry.getValue());
+    }
+    server = new SentryServiceFactory().create(conf);
+    properties.put(ClientConfig.SERVER_RPC_ADDRESS, server.getAddress().getHostString());
+    properties.put(ClientConfig.SERVER_RPC_PORT, String.valueOf(server.getAddress().getPort()));
+    startSentryService();
+    context = createContext(properties);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if(context != null) {
+      context.close();
+    }
+    if (dbDir != null) {
+      FileUtils.deleteQuietly(dbDir);
+    }
+  }
+
+  private void startSentryService() throws Exception {
+    server.start();
+    final long start = System.currentTimeMillis();
+    while(!server.isRunning()) {
+      Thread.sleep(1000);
+      if(System.currentTimeMillis() - start > 60000L) {
+        throw new TimeoutException("Server did not start after 60 seconds");
+      }
+    }
+  }
+
+  @Test
+  public void testBasic() throws Exception {
+    policyFile
+      .setUserGroupMapping(StaticUserGroup.getStaticMapping())
+      .write(context.getPolicyFile());
+    Connection connection = context.createConnection(ADMIN1);
+    Statement statement = context.createStatement(connection);
+    statement.execute("CREATE ROLE admin_role");
+    statement.execute("GRANT ALL ON DATABASE default TO ROLE admin_role");
+    statement.execute("GRANT ROLE admin_role TO GROUP " + ADMINGROUP);
+    statement.execute("CREATE TABLE t1 (c1 string)");
+    statement.execute("CREATE ROLE user_role");
+    statement.execute("GRANT SELECT ON TABLE t1 TO ROLE user_role");
+    statement.execute("GRANT ROLE user_role TO GROUP " + USERGROUP1);
+    statement.close();
+    connection.close();
+    connection = context.createConnection(USER1_1);
+    statement = context.createStatement(connection);
+    context.assertSentryServiceAccessDenied(statement, "CREATE ROLE r2");
+    statement.execute("SELECT * FROM t1");
+    statement.close();
+    connection.close();
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/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 8c145ca..416411c 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
@@ -44,7 +44,6 @@ import com.google.common.io.Resources;
 public class TestPrivilegesAtDatabaseScope extends AbstractTestWithStaticConfiguration {
 
   private Context context;
-  private File dataFile;
   private PolicyFile policyFile;
 
   Map <String, String >testProperties;

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/HiveServerFactory.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/HiveServerFactory.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/HiveServerFactory.java
index 8af3f45..f00efdb 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/HiveServerFactory.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/hiveserver/HiveServerFactory.java
@@ -49,6 +49,7 @@ public class HiveServerFactory {
   public static final String WAREHOUSE_DIR = HiveConf.ConfVars.METASTOREWAREHOUSE.varname;
   public static final String AUTHZ_PROVIDER = HiveAuthzConf.AuthzConfVars.AUTHZ_PROVIDER.getVar();
   public static final String AUTHZ_PROVIDER_RESOURCE = HiveAuthzConf.AuthzConfVars.AUTHZ_PROVIDER_RESOURCE.getVar();
+  public static final String AUTHZ_PROVIDER_BACKEND = HiveAuthzConf.AuthzConfVars.AUTHZ_PROVIDER_BACKEND.getVar();
   public static final String AUTHZ_PROVIDER_FILENAME = "sentry-provider.ini";
   public static final String AUTHZ_SERVER_NAME = HiveAuthzConf.AuthzConfVars.AUTHZ_SERVER_NAME.getVar();
   public static final String ACCESS_TESTING_MODE = HiveAuthzConf.AuthzConfVars.SENTRY_TESTING_MODE.getVar();


[2/2] git commit: SENTRY-129: Implement Hive Sentry Authz DDL Task Factory (Brock Noland via Shreepadma Venugopalan)

Posted by sh...@apache.org.
SENTRY-129: Implement Hive Sentry Authz DDL Task Factory (Brock Noland via Shreepadma Venugopalan)


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

Branch: refs/heads/master
Commit: 63c134f360760a17434b33c26a612a846271523a
Parents: fbf042e
Author: Shreepadma Venugopalan <sh...@apache.org>
Authored: Tue Mar 18 16:29:40 2014 -0700
Committer: Shreepadma Venugopalan <sh...@apache.org>
Committed: Tue Mar 18 16:29:40 2014 -0700

----------------------------------------------------------------------
 pom.xml                                         |  10 +-
 sentry-binding/sentry-binding-hive/pom.xml      |  10 +
 .../apache/hadoop/hive/SentryHiveConstants.java |  34 ++
 .../hive/ql/exec/SentryGrantRevokeTask.java     | 496 +++++++++++++++++++
 .../binding/hive/HiveAuthzBindingHook.java      |  15 +-
 .../hive/HiveAuthzBindingSessionHook.java       |  41 +-
 .../SentryHiveAuthorizationTaskFactoryImpl.java | 330 ++++++++++++
 .../binding/hive/authz/HiveAuthzBinding.java    |   9 +-
 .../TestSentryHiveAuthorizationTaskFactory.java | 374 ++++++++++++++
 .../sentry/policy/db/DBPolicyFileBackend.java   |   3 +-
 .../policy/search/SearchPolicyFileBackend.java  |   3 +-
 .../thrift/sentry_common_serviceConstants.java  |   2 +
 .../db/SentryAccessDeniedException.java         |  27 +
 .../provider/db/SimpleDBProviderBackend.java    |   4 -
 .../thrift/SentryPolicyServiceClient.java       |  51 +-
 .../thrift/SentryPolicyStoreProcessor.java      |  49 ++
 .../sentry/service/thrift/GSSCallback.java      |   5 +-
 .../sentry/service/thrift/SentryService.java    | 150 +++---
 .../sentry/service/thrift/ServiceConstants.java |   7 +
 .../apache/sentry/service/thrift/Status.java    |   7 +
 .../main/resources/sentry_common_service.thrift |   1 +
 .../thrift/TestSentryServerWithoutKerberos.java |  45 ++
 .../thrift/TestSentryServiceFailureCase.java    |  26 +-
 .../thrift/TestSentryServiceIntegration.java    |   8 +-
 .../thrift/SentryServiceIntegrationBase.java    |  60 ++-
 .../file/SimpleFileProviderBackend.java         |   4 -
 .../file/TestSimpleFileProvderBackend.java      |   4 +-
 sentry-tests/sentry-tests-hive/pom.xml          |   5 +
 .../apache/sentry/tests/e2e/hive/Context.java   |  18 +-
 .../tests/e2e/hive/TestDatabaseProvider.java    | 124 +++++
 .../e2e/hive/TestPrivilegesAtDatabaseScope.java |   1 -
 .../e2e/hive/hiveserver/HiveServerFactory.java  |   1 +
 32 files changed, 1773 insertions(+), 151 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index ddf75f4..5bda553 100644
--- a/pom.xml
+++ b/pom.xml
@@ -67,8 +67,8 @@ limitations under the License.
     <jdo-api.version>3.0.1</jdo-api.version>
     <derby.version>10.4.2.0</derby.version>
     <commons-cli.version>1.2</commons-cli.version>
-    <hive.version>0.12.0-cdh5.0.0-beta-2-SNAPSHOT</hive.version>
-    <hadoop.version>2.2.0-cdh5.0.0-beta-2-SNAPSHOT</hadoop.version>
+    <hive.version>0.12.0-cdh5.0.0-SNAPSHOT</hive.version>
+    <hadoop.version>2.2.0-cdh5.0.0-SNAPSHOT</hadoop.version>
     <fest.reflect.version>1.4.1</fest.reflect.version>
     <guava.version>11.0.2</guava.version>
     <junit.version>4.9</junit.version>
@@ -79,8 +79,8 @@ limitations under the License.
     <shiro.version>1.2.1</shiro.version>
     <slf4j.version>1.6.1</slf4j.version>
     <solr.version>4.7.0</solr.version>
-    <solr.sentry.handlers.version>4.4.0-cdh5.0.0-beta-2-SNAPSHOT</solr.sentry.handlers.version>
-    <zookeeper.version>3.4.5-cdh5.0.0-beta-2-SNAPSHOT</zookeeper.version>
+    <solr.sentry.handlers.version>4.4.0-cdh5.0.0-SNAPSHOT</solr.sentry.handlers.version>
+    <zookeeper.version>3.4.5-cdh5.0.0-SNAPSHOT</zookeeper.version>
   </properties>
 
   <dependencyManagement>
@@ -471,7 +471,7 @@ limitations under the License.
           <artifactId>build-helper-maven-plugin</artifactId>
           <version>${build.helper.maven.plugin.version}</version>
         </plugin>
-	<plugin>
+        <plugin>
           <groupId>org.datanucleus</groupId>
           <artifactId>datanucleus-maven-plugin</artifactId>
           <version>${datanucleus.maven.plugin.version}</version>

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/sentry-binding/sentry-binding-hive/pom.xml
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/pom.xml b/sentry-binding/sentry-binding-hive/pom.xml
index 132f4bc..2ce4d87 100644
--- a/sentry-binding/sentry-binding-hive/pom.xml
+++ b/sentry-binding/sentry-binding-hive/pom.xml
@@ -70,6 +70,11 @@ limitations under the License.
       <groupId>org.apache.sentry</groupId>
       <artifactId>sentry-provider-common</artifactId>
     </dependency>
+    <!-- required for SentryGrantRevokeTask -->
+    <dependency>
+      <groupId>org.apache.sentry</groupId>
+      <artifactId>sentry-provider-db</artifactId>
+    </dependency>
     <dependency>
       <groupId>org.apache.sentry</groupId>
       <artifactId>sentry-provider-file</artifactId>
@@ -89,6 +94,11 @@ limitations under the License.
       <artifactId>hadoop-client</artifactId>
       <version>${hadoop.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/SentryHiveConstants.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/SentryHiveConstants.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/SentryHiveConstants.java
new file mode 100644
index 0000000..db14b6c
--- /dev/null
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/SentryHiveConstants.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive;
+
+import java.util.EnumSet;
+
+import org.apache.hadoop.hive.ql.security.authorization.Privilege.PrivilegeType;
+
+public class SentryHiveConstants {
+  // TODO add INSERT
+  public static final EnumSet<PrivilegeType> ALLOWED_PRIVS = EnumSet.of(PrivilegeType.ALL, PrivilegeType.SELECT);
+
+  public static final String PRIVILEGE_NOT_SUPPORTED = "Sentry does not support privilege: ";
+  public static final String COLUMN_PRIVS_NOT_SUPPORTED = "Sentry users should use views to grant privileges on columns";
+  public static final String PARTITION_PRIVS_NOT_SUPPORTED = "Sentry does not support partition level authorization";
+  public static final String GRANT_REVOKE_NOT_SUPPORTED_ON_OBJECT = "Sentry does not allow grant/revoke on: ";
+  public static final String GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL = "Sentry does not allow privileges to be granted/revoked to/from: ";
+  public static final String GRANT_OPTION_NOT_SUPPORTED = "Sentry does not allow WITH GRANT OPTION";
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/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
new file mode 100644
index 0000000..70b05b6
--- /dev/null
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java
@@ -0,0 +1,496 @@
+package org.apache.hadoop.hive.ql.exec;
+/**
+ * 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.
+ */
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.SentryHiveConstants;
+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.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.DDLWork;
+import org.apache.hadoop.hive.ql.plan.GrantDesc;
+import org.apache.hadoop.hive.ql.plan.GrantRevokeRoleDDL;
+import org.apache.hadoop.hive.ql.plan.PrincipalDesc;
+import org.apache.hadoop.hive.ql.plan.PrivilegeDesc;
+import org.apache.hadoop.hive.ql.plan.PrivilegeObjectDesc;
+import org.apache.hadoop.hive.ql.plan.RevokeDesc;
+import org.apache.hadoop.hive.ql.plan.RoleDDLDesc;
+import org.apache.hadoop.hive.ql.plan.ShowGrantDesc;
+import org.apache.hadoop.hive.ql.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.conf.HiveAuthzConf;
+import org.apache.sentry.binding.hive.conf.HiveAuthzConf.AuthzConfVars;
+import org.apache.sentry.core.common.Subject;
+import org.apache.sentry.provider.db.service.thrift.SentryPolicyServiceClient;
+import org.apache.sentry.service.thrift.SentryServiceClientFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Splitter;
+import com.google.common.collect.Iterables;
+
+// TODO remove this suppress
+@SuppressWarnings("unused")
+public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(SentryGrantRevokeTask.class);
+  private static final int RETURN_CODE_SUCCESS = 0;
+  private static final int RETURN_CODE_FAILURE = 1;
+  private static final Splitter DB_TBL_SPLITTER = Splitter.on(".").omitEmptyStrings().trimResults();
+  private static final int separator = Utilities.tabCode;
+  private static final int terminator = Utilities.newLineCode;
+  private static final long serialVersionUID = -7625118066790571999L;
+
+  private SentryServiceClientFactory sentryClientFactory;
+  private SentryPolicyServiceClient sentryClient;
+  private HiveConf conf;
+  private HiveAuthzConf authzConf;
+  private String server;
+  private Subject subject;
+  private Set<String> subjectGroups;
+
+
+  public SentryGrantRevokeTask() {
+    this(new SentryServiceClientFactory());
+  }
+  public SentryGrantRevokeTask(SentryServiceClientFactory sentryClientFactory) {
+    super();
+    this.sentryClientFactory = sentryClientFactory;
+  }
+
+
+  @Override
+  public void initialize(HiveConf conf, QueryPlan queryPlan, DriverContext ctx) {
+    super.initialize(conf, queryPlan, driverContext);
+    this.conf = conf;
+  }
+
+  @Override
+  public int execute(DriverContext driverContext) {
+    try {
+      try {
+        this.sentryClient = sentryClientFactory.create(authzConf);
+      } catch (Exception e) {
+        String msg = "Error creating Sentry client: " + e.getMessage();
+        LOG.error(msg, e);
+        throw new RuntimeException(msg, e);
+      }
+      Preconditions.checkNotNull(authzConf, "HiveAuthConf cannot be null");
+      Preconditions.checkNotNull(subject, "Subject cannot be null");
+      Preconditions.checkNotNull(subjectGroups, "Subject Groups 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(), subjectGroups,
+            work.getRoleDDLDesc());
+      }
+      if (work.getGrantDesc() != null) {
+        return processGrantDDL(conf, console, sentryClient, subject.getName(), subjectGroups,
+            server, work.getGrantDesc());
+      }
+      if (work.getRevokeDesc() != null) {
+        return processRevokeDDL(conf, console, sentryClient, subject.getName(), subjectGroups,
+            server, work.getRevokeDesc());
+      }
+      if (work.getShowGrantDesc() != null) {
+        return processShowGrantDDL(conf, console, subject.getName(), subjectGroups,
+            work.getShowGrantDesc());
+      }
+      if (work.getGrantRevokeRoleDDL() != null) {
+        return processGrantRevokeRoleDDL(conf, console, sentryClient, subject.getName(), subjectGroups,
+            work.getGrantRevokeRoleDDL());
+      }
+      throw new AssertionError("Unknown command passed to Sentry Grant/Revoke Task");
+    } catch(Throwable throwable) {
+      setException(throwable);
+      String msg = "Error processing Sentry command: " + throwable.getMessage();
+      LOG.error(msg, throwable);
+      console.printError(msg);
+      return RETURN_CODE_FAILURE;
+    } finally {
+      if (sentryClient != null) {
+        sentryClient.close();
+      }
+    }
+  }
+
+  public void setAuthzConf(HiveAuthzConf authzConf) {
+    Preconditions.checkState(this.authzConf == null,
+        "setAuthzConf should only be called once: " + this.authzConf);
+    this.authzConf = authzConf;
+  }
+  public void setSubject(Subject subject) {
+    Preconditions.checkState(this.subject == null,
+        "setSubject should only be called once: " + this.subject);
+    this.subject = subject;
+  }
+  public void setSubjectGroups(Set<String> subjectGroups) {
+    Preconditions.checkState(this.subjectGroups == null,
+        "setSubjectGroups should only be called once: " + this.subjectGroups);
+    this.subjectGroups = subjectGroups;
+  }
+
+  @VisibleForTesting
+  static int processRoleDDL(HiveConf conf, LogHelper console,
+      SentryPolicyServiceClient sentryClient, String subject,
+      Set<String> subjectGroups, RoleDDLDesc desc) throws SentryUserException {
+    RoleDDLDesc.RoleOperation operation = desc.getOperation();
+    DataOutputStream outStream = null;
+    String name = desc.getName();
+    try {
+      if (operation.equals(RoleDDLDesc.RoleOperation.CREATE_ROLE)) {
+        SessionState.get().getAuthenticator();
+        sentryClient.createRole(subject, subjectGroups, name);
+        return RETURN_CODE_SUCCESS;
+      } else if (operation.equals(RoleDDLDesc.RoleOperation.DROP_ROLE)) {
+        sentryClient.dropRole(subject, subjectGroups, name);
+        return RETURN_CODE_SUCCESS;
+      } else if (operation.equals(RoleDDLDesc.RoleOperation.SHOW_ROLE_GRANT)) {
+        PrincipalType principalType = desc.getPrincipalType();
+        if (principalType != PrincipalType.GROUP) {
+          String msg = SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL + principalType;
+          throw new HiveException(msg);
+        }
+        throw new AssertionError("TODO");
+        // TODO once retrieval API is implemented this can be implemented
+//        List<String> roles = sentryClient.getRoles(name);
+//        if (!roles.isEmpty()) {
+//          Path resFile = new Path(desc.getResFile());
+//          FileSystem fs = resFile.getFileSystem(conf);
+//          outStream = fs.create(resFile);
+//          for (String role : roles) {
+//            outStream.writeBytes("role name:" + role);
+//            outStream.write(terminator);
+//          }
+//          outStream = close(outStream);
+//        }
+//        return RETURN_CODE_SUCCESS;
+      } else {
+        throw new HiveException("Unkown role operation "
+            + operation.getOperationName());
+      }
+    } catch (HiveException e) {
+      String msg = "Error in role operation "
+          + operation.getOperationName() + " on role name "
+          + name + ", error message " + e.getMessage();
+      LOG.warn(msg, e);
+      console.printError(msg);
+      return RETURN_CODE_FAILURE;
+//    } catch (IOException e) {
+//      String msg = "IO Error in role operation " + e.getMessage();
+//      LOG.info(msg, e);
+//      console.printError(msg);
+//      return RETURN_CODE_FAILURE;
+    } finally {
+      closeQuiet(outStream);
+    }
+  }
+
+  @VisibleForTesting
+  static int processGrantDDL(HiveConf conf, LogHelper console,
+      SentryPolicyServiceClient sentryClient, String subject,
+      Set<String> subjectGroups, String server, GrantDesc desc) throws SentryUserException {
+    return processGrantRevokeDDL(console, sentryClient, subject, subjectGroups,
+        server, true, desc.getPrincipals(), desc.getPrivileges(), desc.getPrivilegeSubjectDesc());
+  }
+
+  @VisibleForTesting
+  static int processRevokeDDL(HiveConf conf, LogHelper console,
+      SentryPolicyServiceClient sentryClient, String subject,
+      Set<String> subjectGroups, String server, RevokeDesc desc) throws SentryUserException {
+    return processGrantRevokeDDL(console, sentryClient, subject, subjectGroups,
+        server, false, desc.getPrincipals(), desc.getPrivileges(),
+        desc.getPrivilegeSubjectDesc());
+  }
+
+  @VisibleForTesting
+  static int processShowGrantDDL(HiveConf conf, LogHelper console, String subject,
+      Set<String> subjectGroups, ShowGrantDesc desc) {
+    DataOutputStream outStream = null;
+    try {
+      Path resFile = new Path(desc.getResFile());
+      FileSystem fs = resFile.getFileSystem(conf);
+      outStream = fs.create(resFile);
+      PrincipalDesc principalDesc = desc.getPrincipalDesc();
+      PrivilegeObjectDesc hiveObjectDesc = desc.getHiveObj();
+      String principalName = principalDesc.getName();
+      List<String> columns = desc.getColumns();
+      if (columns != null && !columns.isEmpty()) {
+        throw new HiveException(SentryHiveConstants.COLUMN_PRIVS_NOT_SUPPORTED);
+      }
+      if (hiveObjectDesc == null) {
+        // TDOD get users from somewhere?
+        List<String> users = Collections.emptyList();
+        if (users != null && users.size() > 0) {
+          boolean first = true;
+          Collections.sort(users);
+          for (String usr : users) {
+            if (!first) {
+              outStream.write(terminator);
+            } else {
+              first = false;
+            }
+            // TODO write grant info
+          }
+        }
+      } else {
+        if (hiveObjectDesc.getPartSpec() != null) {
+          throw new HiveException(SentryHiveConstants.PARTITION_PRIVS_NOT_SUPPORTED);
+        }
+        String obj = hiveObjectDesc.getObject();
+        String dbName = null;
+        String tableName = null;
+        if (hiveObjectDesc.getTable()) {
+          DatabaseTable dbTable = parseDBTable(obj);
+          dbName = dbTable.getDatabase();
+          tableName = dbTable.getTable();
+        } else {
+          dbName = hiveObjectDesc.getObject();
+        }
+        if (hiveObjectDesc.getTable()) {
+          // show table level privileges
+          // TODO
+          List<String> tbls = Collections.emptyList();
+          if (tbls != null && tbls.size() > 0) {
+            boolean first = true;
+            Collections.sort(tbls);
+            for (String tbl : tbls) {
+              if (!first) {
+                outStream.write(terminator);
+              } else {
+                first = false;
+              }
+              // TODO write grant info
+            }
+          }
+        } else {
+          // show database level privileges
+          // TODO
+          List<String> dbs = Collections.emptyList();
+          if (dbs != null && dbs.size() > 0) {
+            boolean first = true;
+            Collections.sort(dbs);
+            for (String db : dbs) {
+              if (!first) {
+                outStream.write(terminator);
+              } else {
+                first = false;
+              }
+              // TODO write grant info
+            }
+          }
+        }
+      }
+      outStream = close(outStream);
+    } catch (HiveException e) {
+      String msg = "Error in show grant operation " + e.getMessage();
+      LOG.warn(msg, e);
+      console.printError(msg);
+      return RETURN_CODE_FAILURE;
+    } catch (IOException e) {
+      String msg = "IO Error in show grant " + e.getMessage();
+      LOG.info(msg, e);
+      console.printError(msg);
+      return RETURN_CODE_FAILURE;
+    } finally {
+      closeQuiet(outStream);
+    }
+    return RETURN_CODE_SUCCESS;
+  }
+
+  @VisibleForTesting
+  static int processGrantRevokeRoleDDL(HiveConf conf, LogHelper console,
+      SentryPolicyServiceClient sentryClient, String subject, Set<String> subjectGroups,
+      GrantRevokeRoleDDL desc) throws SentryUserException {
+    try {
+      boolean grantRole = desc.getGrant();
+      List<PrincipalDesc> principals = desc.getPrincipalDesc();
+      List<String> roles = desc.getRoles();
+      for (PrincipalDesc principal : principals) {
+        if (principal.getType() != PrincipalType.GROUP) {
+          String msg = SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL +
+              principal.getType();
+          throw new HiveException(msg);
+        }
+        String groupName = principal.getName();
+        for (String roleName : roles) {
+          if (grantRole) {
+            sentryClient.grantRoleToGroup(subject, subjectGroups, groupName, roleName);
+          } else {
+            sentryClient.revokeRoleFromGroup(subject, subjectGroups, groupName, roleName);
+          }
+        }
+      }
+    } catch (HiveException e) {
+      String msg = "Error in grant/revoke operation, error message " + e.getMessage();
+      LOG.warn(msg, e);
+      console.printError(msg);
+      return RETURN_CODE_FAILURE;
+    }
+    return RETURN_CODE_SUCCESS;
+  }
+
+  private static int processGrantRevokeDDL(LogHelper console,
+      SentryPolicyServiceClient sentryClient, String subject,
+      Set<String> subjectGroups, String server,
+      boolean isGrant, List<PrincipalDesc> principals,
+      List<PrivilegeDesc> privileges, PrivilegeObjectDesc privSubjectDesc) throws SentryUserException {
+    if (privileges == null || privileges.size() == 0) {
+      console.printError("No privilege found.");
+      return RETURN_CODE_FAILURE;
+    }
+    String dbName = null;
+    String tableName = null;
+    try {
+      if (privSubjectDesc == null) {
+        throw new HiveException("Privilege subject cannot be null");
+      }
+      if (privSubjectDesc.getPartSpec() != null) {
+        throw new HiveException(SentryHiveConstants.PARTITION_PRIVS_NOT_SUPPORTED);
+      }
+      // TODO how to grant all on server
+      String obj = privSubjectDesc.getObject();
+      if (privSubjectDesc.getTable()) {
+        DatabaseTable dbTable = parseDBTable(obj);
+        dbName = dbTable.getDatabase();
+        tableName = dbTable.getTable();
+      } else {
+        dbName = privSubjectDesc.getObject();
+      }
+      for (PrivilegeDesc privDesc : privileges) {
+        List<String> columns = privDesc.getColumns();
+        if (columns != null && !columns.isEmpty()) {
+          throw new HiveException(SentryHiveConstants.COLUMN_PRIVS_NOT_SUPPORTED);
+        }
+        if (!SentryHiveConstants.ALLOWED_PRIVS.contains(privDesc.getPrivilege().getPriv())) {
+          String msg = SentryHiveConstants.PRIVILEGE_NOT_SUPPORTED + privDesc.getPrivilege().getPriv();
+          throw new HiveException(msg);
+        }
+      }
+      for (PrincipalDesc princ : principals) {
+        if (princ.getType() != PrincipalType.ROLE) {
+          String msg = SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL + princ.getType();
+          throw new HiveException(msg);
+        }
+        for (PrivilegeDesc privDesc : privileges) {
+          if (isGrant) {
+            if (tableName == null) {
+              sentryClient.grantDatabasePrivilege(subject, subjectGroups, princ.getName(), server, dbName);
+            } else {
+              sentryClient.grantTablePrivilege(subject, subjectGroups, princ.getName(), server, dbName,
+                  tableName, privDesc.getPrivilege().getPriv().name());
+            }
+          } else {
+            if (tableName == null) {
+              sentryClient.revokeDatabasePrivilege(subject, subjectGroups, princ.getName(), server, dbName);
+            } else {
+              sentryClient.revokeTablePrivilege(subject, subjectGroups, princ.getName(), server, dbName,
+                  tableName, privDesc.getPrivilege().getPriv().name());
+            }
+          }
+        }
+      }
+      return RETURN_CODE_SUCCESS;
+    } catch (HiveException e) {
+      String msg = "Error in grant/revoke operation, error message " + e.getMessage();
+      LOG.warn(msg, e);
+      console.printError(msg);
+      return RETURN_CODE_FAILURE;
+    }
+  }
+
+  private static DatabaseTable parseDBTable(String obj) throws HiveException {
+    String[] dbTab = Iterables.toArray(DB_TBL_SPLITTER.split(obj), String.class);
+    if (dbTab.length == 2) {
+      return new DatabaseTable(dbTab[0], dbTab[1]);
+    } else if (dbTab.length == 1){
+      return new DatabaseTable(SessionState.get().getCurrentDatabase(), obj);
+    } else {
+      String msg = "Malformed database.table '" + obj + "'";
+      throw new HiveException(msg);
+    }
+  }
+
+  private static class DatabaseTable {
+    private final String database;
+    private final String table;
+    public DatabaseTable(String database, String table) {
+      this.database = database;
+      this.table = table;
+    }
+    public String getDatabase() {
+      return database;
+    }
+    public String getTable() {
+      return table;
+    }
+  }
+
+  /**
+   * Close to be used in the try block of a try-catch-finally
+   * statement. Returns null so the close/set to null idiom can be
+   * completed in a single line.
+   */
+  private static DataOutputStream close(DataOutputStream out)
+      throws IOException {
+    if (out != null) {
+      out.close();
+    }
+    return null;
+  }
+  /**
+   * Close to be used in the finally block of a try-catch-finally
+   * statement.
+   */
+  private static void closeQuiet(DataOutputStream out) {
+    try {
+      close(out);
+    } catch (IOException e) {
+      LOG.warn("Error closing output stream", e);
+    }
+  }
+
+  @Override
+  public boolean requireLock() {
+    return false;
+  }
+
+  @Override
+  public StageType getType() {
+    return StageType.DDL;
+  }
+
+  @Override
+  public String getName() {
+    return "SENTRY";
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/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 5ff7764..0b83299 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
@@ -35,6 +35,7 @@ import org.apache.hadoop.hive.ql.HiveDriverFilterHook;
 import org.apache.hadoop.hive.ql.HiveDriverFilterHookContext;
 import org.apache.hadoop.hive.ql.HiveDriverFilterHookResult;
 import org.apache.hadoop.hive.ql.HiveDriverFilterHookResultImpl;
+import org.apache.hadoop.hive.ql.exec.SentryGrantRevokeTask;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.hooks.Entity;
 import org.apache.hadoop.hive.ql.hooks.Entity.Type;
@@ -265,11 +266,23 @@ implements HiveDriverFilterHook {
   @Override
   public void postAnalyze(HiveSemanticAnalyzerHookContext context,
       List<Task<? extends Serializable>> rootTasks) throws SemanticException {
-
     HiveOperation stmtOperation = getCurrentHiveStmtOp();
     HiveAuthzPrivileges stmtAuthObject =
         HiveAuthzPrivilegesMap.getHiveAuthzPrivileges(stmtOperation);
 
+    // must occur above the null check on stmtAuthObject
+    // since GRANT/REVOKE/etc are not authorized by binding layer at present
+    Subject subject = getCurrentSubject(context);
+    Set<String> subjectGroups = hiveAuthzBinding.getGroups(subject);
+    for (Task<? extends Serializable> task : rootTasks) {
+      if (task instanceof SentryGrantRevokeTask) {
+        SentryGrantRevokeTask sentryTask = (SentryGrantRevokeTask)task;
+        sentryTask.setAuthzConf(authzConf);
+        sentryTask.setSubject(subject);
+        sentryTask.setSubjectGroups(subjectGroups);
+      }
+    }
+
     if (stmtAuthObject == null) {
       // We don't handle authorizing this statement
       return;

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingSessionHook.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingSessionHook.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingSessionHook.java
index 1506d8a..2e53acb 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingSessionHook.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingSessionHook.java
@@ -22,6 +22,8 @@ import org.apache.hive.service.cli.HiveSQLException;
 import org.apache.hive.service.cli.session.HiveSessionHookContext;
 import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
 
+import com.google.common.base.Joiner;
+
 public class HiveAuthzBindingSessionHook
     implements org.apache.hive.service.cli.session.HiveSessionHook {
 
@@ -32,25 +34,26 @@ public class HiveAuthzBindingSessionHook
   public static final String FILTER_HOOK =
     "org.apache.sentry.binding.hive.HiveAuthzBindingHook";
   public static final String SCRATCH_DIR_PERMISSIONS = "700";
-  public static final String ACCESS_RESTRICT_LIST =
-    ConfVars.SEMANTIC_ANALYZER_HOOK.varname + "," +
-    ConfVars.PREEXECHOOKS.varname + "," +
-    ConfVars.HIVE_EXEC_FILTER_HOOK.varname + "," +
-    ConfVars.HIVE_EXTENDED_ENITITY_CAPTURE.varname + "," +
-    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.HIVE_SECURITY_COMMAND_WHITELIST.varname + "," +
-    HiveAuthzConf.HIVE_ACCESS_CONF_URL + "," +
-    HiveAuthzConf.HIVE_SENTRY_CONF_URL + "," +
-    HiveAuthzConf.HIVE_ACCESS_SUBJECT_NAME + "," +
-    HiveAuthzConf.HIVE_SENTRY_SUBJECT_NAME;
+  public static final String ACCESS_RESTRICT_LIST = Joiner.on(",").join(
+    ConfVars.SEMANTIC_ANALYZER_HOOK.varname,
+    ConfVars.PREEXECHOOKS.varname,
+    ConfVars.HIVE_EXEC_FILTER_HOOK.varname,
+    ConfVars.HIVE_EXTENDED_ENITITY_CAPTURE.varname,
+    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.HIVE_SECURITY_COMMAND_WHITELIST.varname,
+    ConfVars.HIVE_AUTHORIZATION_TASK_FACTORY.varname,
+    HiveAuthzConf.HIVE_ACCESS_CONF_URL,
+    HiveAuthzConf.HIVE_SENTRY_CONF_URL,
+    HiveAuthzConf.HIVE_ACCESS_SUBJECT_NAME,
+    HiveAuthzConf.HIVE_SENTRY_SUBJECT_NAME);
 
   /**
    * The session hook for sentry authorization that sets the required session level configuration

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryHiveAuthorizationTaskFactoryImpl.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryHiveAuthorizationTaskFactoryImpl.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryHiveAuthorizationTaskFactoryImpl.java
new file mode 100644
index 0000000..252d93b
--- /dev/null
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryHiveAuthorizationTaskFactoryImpl.java
@@ -0,0 +1,330 @@
+/*
+ * 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;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.SentryHiveConstants;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.PrincipalType;
+import org.apache.hadoop.hive.ql.exec.SentryGrantRevokeTask;
+import org.apache.hadoop.hive.ql.exec.Task;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.hooks.ReadEntity;
+import org.apache.hadoop.hive.ql.hooks.WriteEntity;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.parse.authorization.HiveAuthorizationTaskFactory;
+import org.apache.hadoop.hive.ql.plan.DDLWork;
+import org.apache.hadoop.hive.ql.plan.GrantDesc;
+import org.apache.hadoop.hive.ql.plan.GrantRevokeRoleDDL;
+import org.apache.hadoop.hive.ql.plan.PrincipalDesc;
+import org.apache.hadoop.hive.ql.plan.PrivilegeDesc;
+import org.apache.hadoop.hive.ql.plan.PrivilegeObjectDesc;
+import org.apache.hadoop.hive.ql.plan.RevokeDesc;
+import org.apache.hadoop.hive.ql.plan.RoleDDLDesc;
+import org.apache.hadoop.hive.ql.plan.ShowGrantDesc;
+import org.apache.hadoop.hive.ql.security.authorization.Privilege;
+import org.apache.hadoop.hive.ql.security.authorization.PrivilegeRegistry;
+import org.apache.hadoop.hive.ql.session.SessionState;
+
+public class SentryHiveAuthorizationTaskFactoryImpl implements HiveAuthorizationTaskFactory {
+
+
+  public SentryHiveAuthorizationTaskFactoryImpl(HiveConf conf, Hive db) {
+
+  }
+
+  @Override
+  public Task<? extends Serializable> createCreateRoleTask(ASTNode ast, HashSet<ReadEntity> inputs,
+      HashSet<WriteEntity> outputs) {
+    String roleName = BaseSemanticAnalyzer.unescapeIdentifier(ast.getChild(0).getText());
+    RoleDDLDesc roleDesc = new RoleDDLDesc(roleName, RoleDDLDesc.RoleOperation.CREATE_ROLE);
+    return createTask(new DDLWork(inputs, outputs, roleDesc));
+  }
+  @Override
+  public Task<? extends Serializable> createDropRoleTask(ASTNode ast, HashSet<ReadEntity> inputs,
+      HashSet<WriteEntity> outputs) {
+    String roleName = BaseSemanticAnalyzer.unescapeIdentifier(ast.getChild(0).getText());
+    RoleDDLDesc roleDesc = new RoleDDLDesc(roleName, RoleDDLDesc.RoleOperation.DROP_ROLE);
+    return createTask(new DDLWork(inputs, outputs, roleDesc));
+  }
+  @Override
+  public Task<? extends Serializable> createShowRoleGrantTask(ASTNode ast, Path resultFile,
+      HashSet<ReadEntity> inputs, HashSet<WriteEntity> outputs) throws SemanticException {
+    ASTNode child = (ASTNode) ast.getChild(0);
+    PrincipalType principalType = PrincipalType.USER;
+    switch (child.getType()) {
+    case HiveParser.TOK_USER:
+      principalType = PrincipalType.USER;
+      break;
+    case HiveParser.TOK_GROUP:
+      principalType = PrincipalType.GROUP;
+      break;
+    case HiveParser.TOK_ROLE:
+      principalType = PrincipalType.ROLE;
+      break;
+    }
+    if (principalType != PrincipalType.GROUP) {
+      String msg = SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL + principalType;
+      throw new SemanticException(msg);
+    }
+    String principalName = BaseSemanticAnalyzer.unescapeIdentifier(child.getChild(0).getText());
+    RoleDDLDesc roleDesc = new RoleDDLDesc(principalName, principalType,
+        RoleDDLDesc.RoleOperation.SHOW_ROLE_GRANT, null);
+    roleDesc.setResFile(resultFile.toString());
+    return createTask(new DDLWork(inputs, outputs,  roleDesc));
+  }
+
+  @Override
+  public Task<? extends Serializable> createGrantTask(ASTNode ast, HashSet<ReadEntity> inputs,
+      HashSet<WriteEntity> outputs) throws SemanticException {
+    List<PrivilegeDesc> privilegeDesc = analyzePrivilegeListDef(
+        (ASTNode) ast.getChild(0));
+    List<PrincipalDesc> principalDesc = analyzePrincipalListDef(
+        (ASTNode) ast.getChild(1));
+    PrivilegeObjectDesc privilegeObj = null;
+
+    if (ast.getChildCount() > 2) {
+      for (int i = 2; i < ast.getChildCount(); i++) {
+        ASTNode astChild = (ASTNode) ast.getChild(i);
+        if (astChild.getType() == HiveParser.TOK_GRANT_WITH_OPTION) {
+          throw new SemanticException(SentryHiveConstants.GRANT_OPTION_NOT_SUPPORTED);
+        } else if (astChild.getType() == HiveParser.TOK_PRIV_OBJECT) {
+          privilegeObj = analyzePrivilegeObject(astChild);
+        }
+      }
+    }
+    String userName = null;
+    if (SessionState.get() != null
+        && SessionState.get().getAuthenticator() != null) {
+      userName = SessionState.get().getAuthenticator().getUserName();
+    }
+    if (privilegeObj.getPartSpec() != null) {
+      throw new SemanticException(SentryHiveConstants.PARTITION_PRIVS_NOT_SUPPORTED);
+    }
+    for (PrivilegeDesc privDesc : privilegeDesc) {
+      List<String> columns = privDesc.getColumns();
+      if (columns != null && !columns.isEmpty()) {
+        throw new SemanticException(SentryHiveConstants.COLUMN_PRIVS_NOT_SUPPORTED);
+      }
+    }
+    for (PrincipalDesc princ : principalDesc) {
+      if (princ.getType() != PrincipalType.ROLE) {
+        String msg = SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL + princ.getType();
+        throw new SemanticException(msg);
+      }
+    }
+    GrantDesc grantDesc = new GrantDesc(privilegeObj, privilegeDesc,
+        principalDesc, userName, PrincipalType.USER, false);
+    return createTask(new DDLWork(inputs, outputs, grantDesc));
+  }
+  @Override
+  public Task<? extends Serializable> createRevokeTask(ASTNode ast, HashSet<ReadEntity> inputs,
+      HashSet<WriteEntity> outputs) throws SemanticException {
+    List<PrivilegeDesc> privilegeDesc = analyzePrivilegeListDef((ASTNode) ast.getChild(0));
+    List<PrincipalDesc> principalDesc = analyzePrincipalListDef((ASTNode) ast.getChild(1));
+    PrivilegeObjectDesc privilegeObj = null;
+    if (ast.getChildCount() > 2) {
+      ASTNode astChild = (ASTNode) ast.getChild(2);
+      privilegeObj = analyzePrivilegeObject(astChild);
+    }
+    if (privilegeObj.getPartSpec() != null) {
+      throw new SemanticException(SentryHiveConstants.PARTITION_PRIVS_NOT_SUPPORTED);
+    }
+    for (PrivilegeDesc privDesc : privilegeDesc) {
+      List<String> columns = privDesc.getColumns();
+      if (columns != null && !columns.isEmpty()) {
+        throw new SemanticException(SentryHiveConstants.COLUMN_PRIVS_NOT_SUPPORTED);
+      }
+    }
+    for (PrincipalDesc princ : principalDesc) {
+      if (princ.getType() != PrincipalType.ROLE) {
+        String msg = SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL + princ.getType();
+        throw new SemanticException(msg);
+      }
+    }
+    RevokeDesc revokeDesc = new RevokeDesc(privilegeDesc, principalDesc, privilegeObj);
+    return createTask(new DDLWork(inputs, outputs, revokeDesc));
+  }
+
+  @Override
+  public Task<? extends Serializable> createGrantRoleTask(ASTNode ast, HashSet<ReadEntity> inputs, HashSet<WriteEntity> outputs)
+      throws SemanticException {
+    return analyzeGrantRevokeRole(true, ast, inputs, outputs);
+  }
+
+  @Override
+  public Task<? extends Serializable> createShowGrantTask(ASTNode ast, Path resultFile, HashSet<ReadEntity> inputs,
+      HashSet<WriteEntity> outputs) throws SemanticException {
+    PrivilegeObjectDesc privHiveObj = null;
+
+    ASTNode principal = (ASTNode) ast.getChild(0);
+    PrincipalType type = PrincipalType.USER;
+    switch (principal.getType()) {
+    case HiveParser.TOK_USER:
+      type = PrincipalType.USER;
+      break;
+    case HiveParser.TOK_GROUP:
+      type = PrincipalType.GROUP;
+      break;
+    case HiveParser.TOK_ROLE:
+      type = PrincipalType.ROLE;
+      break;
+    }
+    if (type != PrincipalType.ROLE) {
+      String msg = SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL + type;
+      throw new SemanticException(msg);
+    }
+    String principalName = BaseSemanticAnalyzer.unescapeIdentifier(principal.getChild(0).getText());
+    PrincipalDesc principalDesc = new PrincipalDesc(principalName, type);
+    if (ast.getChildCount() > 1) {
+      ASTNode child = (ASTNode) ast.getChild(1);
+      if (child.getToken().getType() == HiveParser.TOK_PRIV_OBJECT_COL) {
+        privHiveObj = new PrivilegeObjectDesc();
+        privHiveObj.setObject(BaseSemanticAnalyzer.unescapeIdentifier(child.getChild(0).getText()));
+        if (child.getChildCount() > 1) {
+          for (int i = 1; i < child.getChildCount(); i++) {
+            ASTNode grandChild = (ASTNode) child.getChild(i);
+            if (grandChild.getToken().getType() == HiveParser.TOK_PARTSPEC) {
+              throw new SemanticException(SentryHiveConstants.PARTITION_PRIVS_NOT_SUPPORTED);
+            } else if (grandChild.getToken().getType() == HiveParser.TOK_TABCOLNAME) {
+              throw new SemanticException(SentryHiveConstants.COLUMN_PRIVS_NOT_SUPPORTED);
+            } else {
+              privHiveObj.setTable(child.getChild(i) != null);
+            }
+          }
+        }
+      }
+    }
+
+    if (privHiveObj == null) {
+      throw new SemanticException(SentryHiveConstants.COLUMN_PRIVS_NOT_SUPPORTED);
+    }
+
+    ShowGrantDesc showGrant = new ShowGrantDesc(resultFile.toString(),
+        principalDesc, privHiveObj, null);
+    return createTask(new DDLWork(inputs, outputs, showGrant));
+  }
+
+  @Override
+  public Task<? extends Serializable> createRevokeRoleTask(ASTNode ast, HashSet<ReadEntity> inputs,
+      HashSet<WriteEntity> outputs) throws SemanticException {
+    return analyzeGrantRevokeRole(false, ast, inputs, outputs);
+  }
+
+  private Task<? extends Serializable> analyzeGrantRevokeRole(boolean isGrant, ASTNode ast,
+      HashSet<ReadEntity> inputs, HashSet<WriteEntity> outputs) throws SemanticException {
+    List<PrincipalDesc> principalDesc = analyzePrincipalListDef(
+        (ASTNode) ast.getChild(0));
+    List<String> roles = new ArrayList<String>();
+    for (int i = 1; i < ast.getChildCount(); i++) {
+      roles.add(BaseSemanticAnalyzer.unescapeIdentifier(ast.getChild(i).getText()));
+    }
+    String roleOwnerName = "";
+    if (SessionState.get() != null
+        && SessionState.get().getAuthenticator() != null) {
+      roleOwnerName = SessionState.get().getAuthenticator().getUserName();
+    }
+    for (PrincipalDesc princ : principalDesc) {
+      if (princ.getType() != PrincipalType.GROUP) {
+        String msg = SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_ON_OBJECT + princ.getType();
+        throw new SemanticException(msg);
+      }
+    }
+    GrantRevokeRoleDDL grantRevokeRoleDDL = new GrantRevokeRoleDDL(isGrant,
+        roles, principalDesc, roleOwnerName, PrincipalType.USER, false);
+    return createTask(new DDLWork(inputs, outputs, grantRevokeRoleDDL));
+  }
+
+  private PrivilegeObjectDesc analyzePrivilegeObject(ASTNode ast)
+      throws SemanticException {
+    PrivilegeObjectDesc subject = new PrivilegeObjectDesc();
+    subject.setObject(BaseSemanticAnalyzer.unescapeIdentifier(ast.getChild(0).getText()));
+    if (ast.getChildCount() > 1) {
+      for (int i = 0; i < ast.getChildCount(); i++) {
+        ASTNode astChild = (ASTNode) ast.getChild(i);
+        if (astChild.getToken().getType() == HiveParser.TOK_PARTSPEC) {
+          throw new SemanticException(SentryHiveConstants.PARTITION_PRIVS_NOT_SUPPORTED);
+        } else {
+          subject.setTable(ast.getChild(0) != null);
+        }
+      }
+    }
+    return subject;
+  }
+
+  private List<PrincipalDesc> analyzePrincipalListDef(ASTNode node) {
+    List<PrincipalDesc> principalList = new ArrayList<PrincipalDesc>();
+    for (int i = 0; i < node.getChildCount(); i++) {
+      ASTNode child = (ASTNode) node.getChild(i);
+      PrincipalType type = null;
+      switch (child.getType()) {
+      case HiveParser.TOK_USER:
+        type = PrincipalType.USER;
+        break;
+      case HiveParser.TOK_GROUP:
+        type = PrincipalType.GROUP;
+        break;
+      case HiveParser.TOK_ROLE:
+        type = PrincipalType.ROLE;
+        break;
+      }
+      String principalName = BaseSemanticAnalyzer.unescapeIdentifier(child.getChild(0).getText());
+      PrincipalDesc principalDesc = new PrincipalDesc(principalName, type);
+      principalList.add(principalDesc);
+    }
+    return principalList;
+  }
+
+  private List<PrivilegeDesc> analyzePrivilegeListDef(ASTNode node)
+      throws SemanticException {
+    List<PrivilegeDesc> ret = new ArrayList<PrivilegeDesc>();
+    for (int i = 0; i < node.getChildCount(); i++) {
+      ASTNode privilegeDef = (ASTNode) node.getChild(i);
+      ASTNode privilegeType = (ASTNode) privilegeDef.getChild(0);
+      Privilege privObj = PrivilegeRegistry.getPrivilege(privilegeType.getType());
+      if (privObj == null) {
+        throw new SemanticException("undefined privilege " + privilegeType.getType());
+      }
+      if (!SentryHiveConstants.ALLOWED_PRIVS.contains(privObj.getPriv())) {
+        String msg = SentryHiveConstants.PRIVILEGE_NOT_SUPPORTED + privObj.getPriv();
+        throw new SemanticException(msg);
+      }
+      if (privilegeDef.getChildCount() > 1) {
+        throw new SemanticException(SentryHiveConstants.COLUMN_PRIVS_NOT_SUPPORTED);
+      }
+      PrivilegeDesc privilegeDesc = new PrivilegeDesc(privObj, null);
+      ret.add(privilegeDesc);
+    }
+    return ret;
+  }
+
+  private static Task<? extends Serializable> createTask(DDLWork work) {
+    SentryGrantRevokeTask task = new SentryGrantRevokeTask();
+    task.setId("Stage-" + Integer.toString(TaskFactory.getAndIncrementId()));
+    task.setWork(work);
+    return task;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/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 3be0d69..eddf3ae 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
@@ -20,6 +20,7 @@ import java.lang.reflect.Constructor;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -146,10 +147,10 @@ public class HiveAuthzBinding {
         + policyEngineName + ", provider backend " + providerBackendName);
       // load the provider backend class
       Constructor<?> providerBackendConstructor =
-        Class.forName(providerBackendName).getDeclaredConstructor(String.class);
+        Class.forName(providerBackendName).getDeclaredConstructor(Configuration.class, String.class);
       providerBackendConstructor.setAccessible(true);
     ProviderBackend providerBackend = (ProviderBackend) providerBackendConstructor.
-        newInstance(new Object[] {resourceName});
+        newInstance(new Object[] {authzConf, resourceName});
 
     // load the policy engine class
     Constructor<?> policyConstructor =
@@ -235,6 +236,10 @@ public class HiveAuthzBinding {
       }
   }
 
+  public Set<String> getGroups(Subject subject) {
+    return authProvider.getGroupMapping().getGroups(subject.getName());
+  }
+
   public Server getAuthServer() {
     if (!open) {
       throw new IllegalStateException("Binding has been closed");

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/TestSentryHiveAuthorizationTaskFactory.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/TestSentryHiveAuthorizationTaskFactory.java b/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/TestSentryHiveAuthorizationTaskFactory.java
new file mode 100644
index 0000000..817537d
--- /dev/null
+++ b/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/TestSentryHiveAuthorizationTaskFactory.java
@@ -0,0 +1,374 @@
+/**
+ * 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;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.List;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.hive.SentryHiveConstants;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.metastore.api.PrincipalType;
+import org.apache.hadoop.hive.ql.Context;
+import org.apache.hadoop.hive.ql.exec.Task;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.metadata.Partition;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.DDLSemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.ParseDriver;
+import org.apache.hadoop.hive.ql.parse.ParseUtils;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.plan.DDLWork;
+import org.apache.hadoop.hive.ql.plan.GrantDesc;
+import org.apache.hadoop.hive.ql.plan.GrantRevokeRoleDDL;
+import org.apache.hadoop.hive.ql.plan.PrincipalDesc;
+import org.apache.hadoop.hive.ql.plan.PrivilegeDesc;
+import org.apache.hadoop.hive.ql.plan.RevokeDesc;
+import org.apache.hadoop.hive.ql.plan.RoleDDLDesc;
+import org.apache.hadoop.hive.ql.plan.RoleDDLDesc.RoleOperation;
+import org.apache.hadoop.hive.ql.plan.ShowGrantDesc;
+import org.apache.hadoop.hive.ql.security.HadoopDefaultAuthenticator;
+import org.apache.hadoop.hive.ql.security.authorization.Privilege;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+public class TestSentryHiveAuthorizationTaskFactory {
+
+  private static final String SELECT = "SELECT";
+  private static final String DB = "default";
+  private static final String TABLE = "table1";
+  private static final String GROUP = "group1";
+  private static final String ROLE = "role1";
+  private static final String USER = "user1";
+
+  private ParseDriver parseDriver;
+  private DDLSemanticAnalyzer analyzer;
+  private HiveConf conf;
+  private Context context;
+  private String currentUser;
+  private Hive db;
+  private Table table;
+  private Partition partition;
+
+  @Before
+  public void setup() throws Exception {
+    conf = new HiveConf();
+    conf.setVar(ConfVars.HIVE_AUTHORIZATION_TASK_FACTORY,
+        SentryHiveAuthorizationTaskFactoryImpl.class.getName());
+    db = Mockito.mock(Hive.class);
+    table = new Table(DB, TABLE);
+    partition = new Partition(table);
+    context = new Context(conf);
+    parseDriver = new ParseDriver();
+    analyzer = new DDLSemanticAnalyzer(conf, db);
+    SessionState.start(conf);
+    Mockito.when(db.getTable(TABLE, false)).thenReturn(table);
+    Mockito.when(db.getPartition(table, new HashMap<String, String>(), false))
+    .thenReturn(partition);
+
+    HadoopDefaultAuthenticator auth = new HadoopDefaultAuthenticator();
+    auth.setConf(conf);
+    currentUser = auth.getUserName();
+
+  }
+
+  /**
+   * CREATE ROLE ...
+   */
+  @Test
+  public void testCreateRole() throws Exception {
+    DDLWork work = analyze(parse("CREATE ROLE " + ROLE));
+    RoleDDLDesc roleDesc = work.getRoleDDLDesc();
+    Assert.assertNotNull("Role should not be null", roleDesc);
+    Assert.assertEquals(RoleOperation.CREATE_ROLE, roleDesc.getOperation());
+    Assert.assertFalse("Did not expect a group", roleDesc.getGroup());
+    Assert.assertEquals(ROLE, roleDesc.getName());
+  }
+
+  /**
+   * DROP ROLE ...
+   */
+  @Test
+  public void testDropRole() throws Exception {
+    DDLWork work = analyze(parse("DROP ROLE " + ROLE));
+    RoleDDLDesc roleDesc = work.getRoleDDLDesc();
+    Assert.assertNotNull("Role should not be null", roleDesc);
+    Assert.assertEquals(RoleOperation.DROP_ROLE, roleDesc.getOperation());
+    Assert.assertFalse("Did not expect a group", roleDesc.getGroup());
+    Assert.assertEquals(ROLE, roleDesc.getName());
+  }
+
+  /**
+   * GRANT ... ON TABLE ... TO USER ...
+   */
+  @Test
+  public void testGrantUserTable() throws Exception {
+    expectSemanticException("GRANT " + SELECT + " ON TABLE " + TABLE + " TO USER " + USER,
+        SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL + "USER");
+  }
+
+  /**
+   * GRANT ... ON TABLE ... TO ROLE ...
+   */
+  @Test
+  public void testGrantRoleTable() throws Exception {
+    DDLWork work = analyze(parse("GRANT " + SELECT + " ON TABLE " + TABLE
+        + " TO ROLE " + ROLE));
+    GrantDesc grantDesc = work.getGrantDesc();
+    Assert.assertNotNull("Grant should not be null", grantDesc);
+    for (PrincipalDesc principal : assertSize(1, grantDesc.getPrincipals())) {
+      Assert.assertEquals(PrincipalType.ROLE, principal.getType());
+      Assert.assertEquals(ROLE, principal.getName());
+    }
+    for (PrivilegeDesc privilege : assertSize(1, grantDesc.getPrivileges())) {
+      Assert.assertEquals(Privilege.SELECT, privilege.getPrivilege());
+    }
+    Assert.assertTrue("Expected table", grantDesc.getPrivilegeSubjectDesc()
+        .getTable());
+    Assert.assertEquals(TABLE, grantDesc.getPrivilegeSubjectDesc().getObject());
+  }
+  /**
+   * GRANT ... ON TABLE ... TO ROLE ... WITH GRANT OPTION
+   */
+  @Test
+  public void testGrantRoleTableWithGrantOption() throws Exception {
+    expectSemanticException("GRANT " + SELECT + " ON TABLE " + TABLE + " TO ROLE " + ROLE +
+        " WITH GRANT OPTION", "Sentry does not allow WITH GRANT OPTION");
+  }
+
+  /**
+   * GRANT ... ON TABLE ... TO GROUP ...
+   */
+  @Test
+  public void testGrantGroupTable() throws Exception {
+    expectSemanticException("GRANT " + SELECT + " ON TABLE " + TABLE + " TO GROUP " + GROUP,
+        SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL + "GROUP");
+  }
+
+  /**
+   * REVOKE ... ON TABLE ... FROM USER ...
+   */
+  @Test
+  public void testRevokeUserTable() throws Exception {
+    expectSemanticException("REVOKE " + SELECT + " ON TABLE " + TABLE + " FROM USER " + USER,
+        SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL + "USER");
+  }
+
+  /**
+   * REVOKE ... ON TABLE ... FROM ROLE ...
+   */
+  @Test
+  public void testRevokeRoleTable() throws Exception {
+    DDLWork work = analyze(parse("REVOKE " + SELECT + " ON TABLE " + TABLE
+        + " FROM ROLE " + ROLE));
+    RevokeDesc grantDesc = work.getRevokeDesc();
+    Assert.assertNotNull("Revoke should not be null", grantDesc);
+    for (PrincipalDesc principal : assertSize(1, grantDesc.getPrincipals())) {
+      Assert.assertEquals(PrincipalType.ROLE, principal.getType());
+      Assert.assertEquals(ROLE, principal.getName());
+    }
+    for (PrivilegeDesc privilege : assertSize(1, grantDesc.getPrivileges())) {
+      Assert.assertEquals(Privilege.SELECT, privilege.getPrivilege());
+    }
+    Assert.assertTrue("Expected table", grantDesc.getPrivilegeSubjectDesc()
+        .getTable());
+    Assert.assertEquals(TABLE, grantDesc.getPrivilegeSubjectDesc().getObject());
+  }
+
+  /**
+   * REVOKE ... ON TABLE ... FROM GROUP ...
+   */
+  @Test
+  public void testRevokeGroupTable() throws Exception {
+    expectSemanticException("REVOKE " + SELECT + " ON TABLE " + TABLE + " FROM GROUP " + GROUP,
+        SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL + "GROUP");
+  }
+
+  /**
+   * GRANT ROLE ... TO USER ...
+   */
+  @Test
+  public void testGrantRoleUser() throws Exception {
+    expectSemanticException("GRANT ROLE " + ROLE + " TO USER " + USER,
+        SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_ON_OBJECT + "USER");
+  }
+
+  /**
+   * GRANT ROLE ... TO ROLE ...
+   */
+  @Test
+  public void testGrantRoleRole() throws Exception {
+    expectSemanticException("GRANT ROLE " + ROLE + " TO ROLE " + ROLE,
+        SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_ON_OBJECT + "ROLE");
+  }
+
+  /**
+   * GRANT ROLE ... TO GROUP ...
+   */
+  @Test
+  public void testGrantRoleGroup() throws Exception {
+    DDLWork work = analyze(parse("GRANT ROLE " + ROLE + " TO GROUP " + GROUP));
+    GrantRevokeRoleDDL grantDesc = work.getGrantRevokeRoleDDL();
+    Assert.assertNotNull("Grant should not be null", grantDesc);
+    Assert.assertTrue("Expected grant ", grantDesc.getGrant());
+    Assert
+    .assertFalse("Grant option should be false", grantDesc.isGrantOption());
+    Assert.assertEquals(currentUser, grantDesc.getGrantor());
+    Assert.assertEquals(PrincipalType.USER, grantDesc.getGrantorType());
+    for (String role : assertSize(1, grantDesc.getRoles())) {
+      Assert.assertEquals(ROLE, role);
+    }
+    for (PrincipalDesc principal : assertSize(1, grantDesc.getPrincipalDesc())) {
+      Assert.assertEquals(PrincipalType.GROUP, principal.getType());
+      Assert.assertEquals(GROUP, principal.getName());
+    }
+  }
+
+  /**
+   * REVOKE ROLE ... FROM USER ...
+   */
+  @Test
+  public void testRevokeRoleUser() throws Exception {
+    expectSemanticException("REVOKE ROLE " + ROLE + " FROM USER " + USER,
+        SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_ON_OBJECT + "USER");
+  }
+
+  /**
+   * REVOKE ROLE ... FROM ROLE ...
+   */
+  @Test
+  public void testRevokeRoleRole() throws Exception {
+    expectSemanticException("REVOKE ROLE " + ROLE + " FROM ROLE " + ROLE,
+        SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_ON_OBJECT + "ROLE");
+  }
+
+  /**
+   * REVOKE ROLE ... FROM GROUP ...
+   */
+  @Test
+  public void testRevokeRoleGroup() throws Exception {
+    DDLWork work = analyze(parse("REVOKE ROLE " + ROLE + " FROM GROUP " + GROUP));
+    GrantRevokeRoleDDL grantDesc = work.getGrantRevokeRoleDDL();
+    Assert.assertNotNull("Grant should not be null", grantDesc);
+    Assert.assertFalse("Did not expect grant ", grantDesc.getGrant());
+    Assert.assertFalse("Grant option is always true ", grantDesc.isGrantOption());
+    Assert.assertEquals(currentUser, grantDesc.getGrantor());
+    Assert.assertEquals(PrincipalType.USER, grantDesc.getGrantorType());
+    for (String role : assertSize(1, grantDesc.getRoles())) {
+      Assert.assertEquals(ROLE, role);
+    }
+    for (PrincipalDesc principal : assertSize(1, grantDesc.getPrincipalDesc())) {
+      Assert.assertEquals(PrincipalType.GROUP, principal.getType());
+      Assert.assertEquals(GROUP, principal.getName());
+    }
+  }
+
+  /**
+   * SHOW ROLE GRANT USER ...
+   */
+  @Test
+  public void testShowRoleGrantUser() throws Exception {
+    expectSemanticException("SHOW ROLE GRANT USER " + USER,
+        SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL + "USER");
+  }
+
+  /**
+   * SHOW ROLE GRANT ROLE ...
+   */
+  @Test
+  public void testShowRoleGrantRole() throws Exception {
+    expectSemanticException("SHOW ROLE GRANT ROLE " + ROLE,
+        SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL + "ROLE");
+  }
+
+  /**
+   * SHOW ROLE GRANT GROUP ...
+   */
+  @Test
+  public void testShowRoleGrantGroup() throws Exception {
+    DDLWork work = analyze(parse("SHOW ROLE GRANT GROUP " + GROUP));
+    RoleDDLDesc roleDesc = work.getRoleDDLDesc();
+    Assert.assertNotNull("Role should not be null", roleDesc);
+    Assert.assertEquals(RoleOperation.SHOW_ROLE_GRANT, roleDesc.getOperation());
+    Assert.assertEquals(PrincipalType.GROUP, roleDesc.getPrincipalType());
+    Assert.assertEquals(GROUP, roleDesc.getName());
+  }
+
+  /**
+   * SHOW GRANT USER ... ON TABLE ...
+   */
+  @Test
+  public void testShowGrantUserOnTable() throws Exception {
+    expectSemanticException("SHOW GRANT USER " + USER + " ON TABLE " + TABLE,
+        SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL + "USER");
+  }
+
+  /**
+   * SHOW GRANT ROLE ... ON TABLE ...
+   */
+  @Test
+  public void testShowGrantRoleOnTable() throws Exception {
+    DDLWork work = analyze(parse("SHOW GRANT ROLE " + ROLE + " ON TABLE " + TABLE));
+    ShowGrantDesc grantDesc = work.getShowGrantDesc();
+    Assert.assertNotNull("Show grant should not be null", grantDesc);
+    Assert.assertEquals(PrincipalType.ROLE, grantDesc.getPrincipalDesc().getType());
+    Assert.assertEquals(ROLE, grantDesc.getPrincipalDesc().getName());
+    Assert.assertTrue("Expected table", grantDesc.getHiveObj().getTable());
+    Assert.assertEquals(TABLE, grantDesc.getHiveObj().getObject());
+    Assert.assertTrue("Expected table", grantDesc.getHiveObj().getTable());
+  }
+
+  /**
+   * SHOW GRANT GROUP ... ON TABLE ...
+   */
+  @Test
+  public void testShowGrantGroupOnTable() throws Exception {
+    expectSemanticException("SHOW GRANT GROUP " + GROUP + " ON TABLE " + TABLE,
+        SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL + "GROUP");
+  }
+
+  private void expectSemanticException(String command, String msg) throws Exception {
+    try {
+      analyze(parse(command));
+      Assert.fail("Expected command '" + command + "' to fail with '" + msg + "'");
+    } catch (SemanticException e) {
+      Assert.assertEquals(msg, e.getMessage());
+    }
+  }
+
+  private ASTNode parse(String command) throws Exception {
+    return ParseUtils.findRootNonNullToken(parseDriver.parse(command));
+  }
+
+  private DDLWork analyze(ASTNode ast) throws Exception {
+    analyzer.analyze(ast, context);
+    List<Task<? extends Serializable>> rootTasks = analyzer.getRootTasks();
+    return (DDLWork) assertSize(1, rootTasks).get(0).getWork();
+  }
+
+  private static <L extends List<?>> L assertSize(int size, L list) {
+    Assert.assertEquals(list.toString(), size, list.size());
+    return list;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/DBPolicyFileBackend.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/DBPolicyFileBackend.java b/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/DBPolicyFileBackend.java
index 661deff..9dc63e4 100644
--- a/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/DBPolicyFileBackend.java
+++ b/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/DBPolicyFileBackend.java
@@ -18,10 +18,11 @@ package org.apache.sentry.policy.db;
 
 import java.io.IOException;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.sentry.provider.file.SimpleFileProviderBackend;
 
 public class DBPolicyFileBackend extends SimpleDBPolicyEngine {
   public DBPolicyFileBackend(String server, String resource) throws IOException{
-    super(server, new SimpleFileProviderBackend(resource));
+    super(server, new SimpleFileProviderBackend(new Configuration(), resource));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/SearchPolicyFileBackend.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/SearchPolicyFileBackend.java b/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/SearchPolicyFileBackend.java
index fd8af78..be23e15 100644
--- a/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/SearchPolicyFileBackend.java
+++ b/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/SearchPolicyFileBackend.java
@@ -18,10 +18,11 @@ package org.apache.sentry.policy.search;
 
 import java.io.IOException;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.sentry.provider.file.SimpleFileProviderBackend;
 
 public class SearchPolicyFileBackend extends SimpleSearchPolicyEngine {
   public SearchPolicyFileBackend(String resource) throws IOException{
-    super(new SimpleFileProviderBackend(resource));
+    super(new SimpleFileProviderBackend(new Configuration(), resource));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/service/thrift/sentry_common_serviceConstants.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/service/thrift/sentry_common_serviceConstants.java b/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/service/thrift/sentry_common_serviceConstants.java
index c465737..4fdeaeb 100644
--- a/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/service/thrift/sentry_common_serviceConstants.java
+++ b/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/service/thrift/sentry_common_serviceConstants.java
@@ -45,4 +45,6 @@ public class sentry_common_serviceConstants {
 
   public static final int TSENTRY_STATUS_INVALID_INPUT = 4;
 
+  public static final int TSENTRY_STATUS_ACCESS_DENIED = 5;
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/SentryAccessDeniedException.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/SentryAccessDeniedException.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/SentryAccessDeniedException.java
new file mode 100644
index 0000000..8f1fa2b
--- /dev/null
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/SentryAccessDeniedException.java
@@ -0,0 +1,27 @@
+/**
+ * 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.provider.db;
+
+import org.apache.sentry.SentryUserException;
+
+public class SentryAccessDeniedException extends SentryUserException {
+  private static final long serialVersionUID = 2962080655835L;
+  public SentryAccessDeniedException(String msg) {
+    super(msg);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/SimpleDBProviderBackend.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/SimpleDBProviderBackend.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/SimpleDBProviderBackend.java
index bc4d7b5..b068aca 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/SimpleDBProviderBackend.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/SimpleDBProviderBackend.java
@@ -42,10 +42,6 @@ public class SimpleDBProviderBackend implements ProviderBackend {
 
   private volatile boolean initialized;
 
-  public SimpleDBProviderBackend(String resourcePath) throws IOException {
-    this(new Configuration(), new Path(resourcePath));
-  }
-
   public SimpleDBProviderBackend(Configuration conf, String resourcePath) throws IOException {
     this(conf, new Path(resourcePath));
   }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyServiceClient.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyServiceClient.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyServiceClient.java
index 84d9d8d..464569c 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyServiceClient.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyServiceClient.java
@@ -48,9 +48,9 @@ import com.google.common.collect.Sets;
 
 public class SentryPolicyServiceClient {
 
-  @SuppressWarnings("unused")
   private final Configuration conf;
   private final InetSocketAddress serverAddress;
+  private final boolean kerberos;
   private final String[] serverPrincipalParts;
   private SentryPolicyService.Client client;
   private TTransport transport;
@@ -60,31 +60,38 @@ public class SentryPolicyServiceClient {
 
   public SentryPolicyServiceClient(Configuration conf) throws IOException {
     this.conf = conf;
+    Preconditions.checkNotNull(this.conf, "Configuration object cannot be null");
     this.serverAddress = NetUtils.createSocketAddr(Preconditions.checkNotNull(
                            conf.get(ClientConfig.SERVER_RPC_ADDRESS), "Config key "
                            + ClientConfig.SERVER_RPC_ADDRESS + " is required"), conf.getInt(
                            ClientConfig.SERVER_RPC_PORT, ClientConfig.SERVER_RPC_PORT_DEFAULT));
     this.connectionTimeout = conf.getInt(ClientConfig.SERVER_RPC_CONN_TIMEOUT,
                                          ClientConfig.SERVER_RPC_CONN_TIMEOUT_DEFAULT);
-    String serverPrincipal = Preconditions.checkNotNull(
-                               conf.get(ServerConfig.PRINCIPAL), ServerConfig.PRINCIPAL
-                               + " is required");
-    serverPrincipalParts = SaslRpcServer.splitKerberosName(serverPrincipal);
-    Preconditions.checkArgument(serverPrincipalParts.length == 3,
-                                "Kerberos principal should have 3 parts: " + serverPrincipal);
+    kerberos = ServerConfig.SECURITY_MODE_KERBEROS.equalsIgnoreCase(
+        conf.get(ServerConfig.SECURITY_MODE, ServerConfig.SECURITY_MODE_KERBEROS).trim());
     transport = new TSocket(serverAddress.getHostString(),
-                            serverAddress.getPort(), connectionTimeout);
-    TTransport saslTransport = new TSaslClientTransport(
-      AuthMethod.KERBEROS.getMechanismName(), null, serverPrincipalParts[0],
-      serverPrincipalParts[1], ClientConfig.SASL_PROPERTIES, null, transport);
+        serverAddress.getPort(), connectionTimeout);
+    if (kerberos) {
+      String serverPrincipal = Preconditions.checkNotNull(
+          conf.get(ServerConfig.PRINCIPAL), ServerConfig.PRINCIPAL
+          + " is required");
+      serverPrincipalParts = SaslRpcServer.splitKerberosName(serverPrincipal);
+      Preconditions.checkArgument(serverPrincipalParts.length == 3,
+           "Kerberos principal should have 3 parts: " + serverPrincipal);
+      transport = new TSaslClientTransport(
+          AuthMethod.KERBEROS.getMechanismName(), null, serverPrincipalParts[0],
+          serverPrincipalParts[1], ClientConfig.SASL_PROPERTIES, null, transport);
+    } else {
+      serverPrincipalParts = null;
+    }
     try {
-      saslTransport.open();
+      transport.open();
     } catch (TTransportException e) {
       throw new IOException("Transport exception while opening transport: " + e.getMessage(), e);
     }
-    LOGGER.info("Successfully opened transport");
+    LOGGER.info("Successfully opened transport: " + transport + " to " + serverAddress);
     TMultiplexedProtocol protocol = new TMultiplexedProtocol(
-      new TBinaryProtocol(saslTransport),
+      new TBinaryProtocol(transport),
       SentryPolicyStoreProcessor.SENTRY_POLICY_SERVICE_NAME);
     client = new SentryPolicyService.Client(protocol);
     LOGGER.info("Successfully created client");
@@ -185,6 +192,7 @@ public class SentryPolicyServiceClient {
     privilege.setServerName(serverName);
     privilege.setURI(uri);
     privilege.setDbName(db);
+    privilege.setTableName(table);
     privilege.setAction(action);
     privilege.setGrantorPrincipal(requestorUserName);
     privilege.setCreateTime(System.currentTimeMillis());
@@ -283,6 +291,21 @@ public class SentryPolicyServiceClient {
     }
   }
 
+  public void revokeRoleFromGroup(String requestorUserName, Set<String> requestorUserGroupName,
+      String groupName, String roleName)
+  throws SentryUserException {
+    TAlterSentryRoleDeleteGroupsRequest request = new TAlterSentryRoleDeleteGroupsRequest(ThriftConstants.
+        TSENTRY_SERVICE_VERSION_CURRENT, requestorUserName, requestorUserGroupName,
+        roleName, Sets.newHashSet(new TSentryGroup(groupName)));
+    try {
+      TAlterSentryRoleDeleteGroupsResponse response = client.alter_sentry_role_delete_groups(request);
+      Status.throwIfNotOk(response.getStatus());
+    } catch (TException e) {
+      String msg = "Thrift exception occured: " + e.getMessage();
+      throw new SentryUserException(msg, e);
+    }
+  }
+
   public void close() {
     if (transport != null) {
       transport.close();

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyStoreProcessor.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyStoreProcessor.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyStoreProcessor.java
index 722b490..9562783 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyStoreProcessor.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyStoreProcessor.java
@@ -24,6 +24,7 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.sentry.provider.db.SentryAccessDeniedException;
 import org.apache.sentry.provider.db.SentryAlreadyExistsException;
 import org.apache.sentry.provider.db.SentryInvalidInputException;
 import org.apache.sentry.provider.db.SentryNoSuchObjectException;
@@ -32,6 +33,7 @@ import org.apache.sentry.provider.db.service.persistent.SentryStore;
 import org.apache.sentry.provider.db.service.thrift.PolicyStoreConstants.PolicyStoreServerConfig;
 import org.apache.sentry.service.thrift.Status;
 import org.apache.sentry.service.thrift.TSentryResponseStatus;
+import org.apache.sentry.service.thrift.ServiceConstants.ServerConfig;
 import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -39,6 +41,7 @@ import org.slf4j.LoggerFactory;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Splitter;
+import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 
@@ -52,6 +55,7 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
   private final Configuration conf;
   private final SentryStore sentryStore;
   private final NotificationHandlerInvoker notificationHandlerInvoker;
+  private final ImmutableSet<String> adminGroups;
   private boolean isReady;
 
   public SentryPolicyStoreProcessor(String name, Configuration conf) throws Exception {
@@ -63,6 +67,8 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
     isReady = false;
     sentryStore = new SentryStore(conf);
     isReady = true;
+    adminGroups = ImmutableSet.copyOf(toTrimedLower(Sets.newHashSet(conf.getStrings(
+        ServerConfig.ADMIN_GROUPS, new String[]{}))));
   }
 
   public void stop() {
@@ -145,11 +151,31 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
     return privilegeName.toString();
   }
 
+  private static Set<String> toTrimedLower(Set<String> s) {
+    Set<String> result = Sets.newHashSet();
+    for (String v : s) {
+      result.add(v.trim().toLowerCase());
+    }
+    return result;
+  }
+
+  private void authorize(String requestorUser, Set<String> requestorGroups)
+  throws SentryAccessDeniedException {
+    requestorGroups = toTrimedLower(requestorGroups);
+    if (Sets.intersection(adminGroups, requestorGroups).isEmpty()) {
+      String msg = "User: " + requestorUser + " is part of " + requestorGroups +
+          " which does not, intersect admin groups " + adminGroups;
+      LOGGER.warn(msg);
+      throw new SentryAccessDeniedException("Access denied to " + requestorUser);
+    }
+  }
+
   @Override
   public TCreateSentryRoleResponse create_sentry_role(
     TCreateSentryRoleRequest request) throws TException {
     TCreateSentryRoleResponse response = new TCreateSentryRoleResponse();
     try {
+      authorize(request.getRequestorUserName(), request.getRequestorGroupNames());
       CommitContext commitContext = sentryStore.createSentryRole(request.getRoleName(),
           request.getRequestorUserName());
       response.setStatus(Status.OK());
@@ -159,6 +185,9 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
       String msg = "Role: " + request + " already exists.";
       LOGGER.error(msg, e);
       response.setStatus(Status.AlreadyExists(msg, e));
+    } catch (SentryAccessDeniedException e) {
+      LOGGER.error(e.getMessage(), e);
+      response.setStatus(Status.AccessDenied(e.getMessage(), e));
     } catch (Exception e) {
       String msg = "Unknown error for request: " + request + ", message: " + e.getMessage();
       LOGGER.error(msg, e);
@@ -173,6 +202,7 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
 
     TAlterSentryRoleGrantPrivilegeResponse response = new TAlterSentryRoleGrantPrivilegeResponse();
     try {
+      authorize(request.getRequestorUserName(), request.getRequestorGroupNames());
       String privilegeName = constructPrivilegeName(request.getPrivilege());
       request.getPrivilege().setPrivilegeName(privilegeName);
       CommitContext commitContext = sentryStore.alterSentryRoleGrantPrivilege(request.getRoleName(),
@@ -188,6 +218,9 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
       String msg = "Invalid input privilege object";
       LOGGER.error(msg, e);
       response.setStatus(Status.InvalidInput(msg, e));
+    } catch (SentryAccessDeniedException e) {
+      LOGGER.error(e.getMessage(), e);
+      response.setStatus(Status.AccessDenied(e.getMessage(), e));
     } catch (Exception e) {
       String msg = "Unknown error for request: " + request + ", message: " + e.getMessage();
       LOGGER.error(msg, e);
@@ -202,6 +235,7 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
   (TAlterSentryRoleRevokePrivilegeRequest request) throws TException {
     TAlterSentryRoleRevokePrivilegeResponse response = new TAlterSentryRoleRevokePrivilegeResponse();
     try {
+      authorize(request.getRequestorUserName(), request.getRequestorGroupNames());
       String privilegeName = constructPrivilegeName(request.getPrivilege());
       request.getPrivilege().setPrivilegeName(privilegeName);
       CommitContext commitContext = sentryStore.alterSentryRoleRevokePrivilege(request.getRoleName(),
@@ -217,6 +251,9 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
       String msg = "Invalid input privilege object";
       LOGGER.error(msg, e);
       response.setStatus(Status.InvalidInput(msg, e));
+    } catch (SentryAccessDeniedException e) {
+      LOGGER.error(e.getMessage(), e);
+      response.setStatus(Status.AccessDenied(e.getMessage(), e));
     } catch (Exception e) {
       String msg = "Unknown error for request: " + request + ", message: " + e.getMessage();
       LOGGER.error(msg, e);
@@ -232,6 +269,7 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
     TDropSentryRoleResponse response = new TDropSentryRoleResponse();
     TSentryResponseStatus status;
     try {
+      authorize(request.getRequestorUserName(), request.getRequestorGroupNames());
       CommitContext commitContext = sentryStore.dropSentryRole(request.getRoleName());
       response.setStatus(Status.OK());
       notificationHandlerInvoker.drop_sentry_role(commitContext,
@@ -240,6 +278,9 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
       String msg = "Role :" + request + " does not exist.";
       LOGGER.error(msg, e);
       response.setStatus(Status.NoSuchObject(msg, e));
+    } catch (SentryAccessDeniedException e) {
+      LOGGER.error(e.getMessage(), e);
+      response.setStatus(Status.AccessDenied(e.getMessage(), e));
     } catch (Exception e) {
       String msg = "Unknown error for request: " + request + ", message: " + e.getMessage();
       LOGGER.error(msg, e);
@@ -253,6 +294,7 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
     TAlterSentryRoleAddGroupsRequest request) throws TException {
     TAlterSentryRoleAddGroupsResponse response = new TAlterSentryRoleAddGroupsResponse();
     try {
+      authorize(request.getRequestorUserName(), request.getRequestorGroupNames());
       CommitContext commitContext = sentryStore.alterSentryRoleAddGroups(request.getRequestorUserName(),
                                     request.getRoleName(), request.getGroups());
       response.setStatus(Status.OK());
@@ -262,6 +304,9 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
       String msg = "Role: " + request + " does not exist.";
       LOGGER.error(msg, e);
       response.setStatus(Status.NoSuchObject(msg, e));
+    } catch (SentryAccessDeniedException e) {
+      LOGGER.error(e.getMessage(), e);
+      response.setStatus(Status.AccessDenied(e.getMessage(), e));
     } catch (Exception e) {
       String msg = "Unknown error for request: " + request + ", message: " + e.getMessage();
       LOGGER.error(msg, e);
@@ -275,6 +320,7 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
     TAlterSentryRoleDeleteGroupsRequest request) throws TException {
     TAlterSentryRoleDeleteGroupsResponse response = new TAlterSentryRoleDeleteGroupsResponse();
     try {
+      authorize(request.getRequestorUserName(), request.getRequestorGroupNames());
       CommitContext commitContext = sentryStore.alterSentryRoleDeleteGroups(request.getRoleName(),
           request.getGroups());
       response.setStatus(Status.OK());
@@ -284,6 +330,9 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
       String msg = "Role: " + request + " does not exist.";
       LOGGER.error(msg, e);
       response.setStatus(Status.NoSuchObject(msg, e));
+    } catch (SentryAccessDeniedException e) {
+      LOGGER.error(e.getMessage(), e);
+      response.setStatus(Status.AccessDenied(e.getMessage(), e));
     } catch (Exception e) {
       String msg = "Unknown error adding groups to role: " + request;
       LOGGER.error(msg, e);