You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sentry.apache.org by br...@apache.org on 2014/03/11 23:29:53 UTC

[1/4] SENTRY-126 - Implement alter role grant/revoke privilege in sentry service and sentry store (Shreepadma via Brock)

Repository: incubator-sentry
Updated Branches:
  refs/heads/db_policy_store e18a902d2 -> a7df761dd


http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7df761d/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 a451f58..78e0a87 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
@@ -26,6 +26,7 @@ import java.util.Set;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.sentry.provider.db.service.persistent.CommitContext;
 import org.apache.sentry.provider.db.service.persistent.SentryAlreadyExistsException;
+import org.apache.sentry.provider.db.service.persistent.SentryInvalidInputException;
 import org.apache.sentry.provider.db.service.persistent.SentryNoSuchObjectException;
 import org.apache.sentry.provider.db.service.persistent.SentryStore;
 import org.apache.sentry.provider.db.service.thrift.PolicyStoreConstants.PolicyStoreServerConfig;
@@ -71,21 +72,21 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
 
   @VisibleForTesting
   static List<NotificationHandler> createHandlers(Configuration conf)
-      throws SentryConfigurationException {
+  throws SentryConfigurationException {
     List<NotificationHandler> handlers = Lists.newArrayList();
     Iterable<String> notificationHandlers = Splitter.onPattern("[\\s,]").trimResults()
-        .omitEmptyStrings().split(conf.get(PolicyStoreServerConfig.NOTIFICATION_HANDLERS, ""));
+                                            .omitEmptyStrings().split(conf.get(PolicyStoreServerConfig.NOTIFICATION_HANDLERS, ""));
     for (String notificationHandler : notificationHandlers) {
       Class<?> clazz = null;
       try {
         clazz = Class.forName(notificationHandler);
         if (!NotificationHandler.class.isAssignableFrom(clazz)) {
           throw new SentryConfigurationException("Class " + notificationHandler + " is not a " +
-              NotificationHandler.class.getName());
+                                                 NotificationHandler.class.getName());
         }
       } catch (ClassNotFoundException e) {
         throw new SentryConfigurationException("Value " + notificationHandler +
-           " is not a class", e);
+                                               " is not a class", e);
       }
       Preconditions.checkNotNull(clazz, "Error class cannot be null");
       try {
@@ -98,6 +99,50 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
     return handlers;
   }
 
+  //TODO:Validate privilege scope?
+  private String constructPrivilegeName(TSentryPrivilege privilege) throws SentryInvalidInputException {
+    StringBuilder privilegeName = new StringBuilder();
+    String serverName = privilege.getServerName();
+    String dbName = privilege.getDbName();
+    String tableName = privilege.getTableName();
+    String uri = privilege.getURI();
+    String action = privilege.getAction();
+
+    if (serverName == null) {
+      throw new SentryInvalidInputException("Server name is null");
+    }
+
+    if (action.equalsIgnoreCase("SELECT") || action.equalsIgnoreCase("INSERT")) {
+      if (tableName == null || tableName.equals("")) {
+        throw new SentryInvalidInputException("Table name can't be null for SELECT/INSERT privilege");
+      }
+    }
+
+    if (dbName == null || dbName.equals("")) {
+      if (tableName != null && !tableName.equals("")) {
+        throw new SentryInvalidInputException("Db name can't be null");
+      }
+    }
+
+    if (uri == null || uri.equals("")) {
+      privilegeName.append(serverName);
+      privilegeName.append("+");
+      privilegeName.append(dbName);
+
+      if (tableName != null && !tableName.equals("")) {
+        privilegeName.append("+");
+        privilegeName.append(tableName);
+      }
+      privilegeName.append("+");
+      privilegeName.append(action);
+    } else {
+      privilegeName.append(serverName);
+      privilegeName.append("+");
+      privilegeName.append(uri);
+    }
+    return privilegeName.toString();
+  }
+
   @Override
   public TCreateSentryRoleResponse create_sentry_role(
     TCreateSentryRoleRequest request) throws TException {
@@ -118,19 +163,57 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
     }
     return response;
   }
+
   @Override
-  public TCreateSentryPrivilegeResponse create_sentry_privilege(
-    TCreateSentryPrivilegeRequest request) throws TException {
-    TCreateSentryPrivilegeResponse response = new TCreateSentryPrivilegeResponse();
+  public TAlterSentryRoleGrantPrivilegeResponse alter_sentry_role_grant_privilege
+  (TAlterSentryRoleGrantPrivilegeRequest request) throws TException {
+
+    TAlterSentryRoleGrantPrivilegeResponse response = new TAlterSentryRoleGrantPrivilegeResponse();
     try {
-      CommitContext commitContext = sentryStore.createSentryPrivilege(request.getPrivilege());
+      String privilegeName = constructPrivilegeName(request.getPrivilege());
+      request.getPrivilege().setPrivilegeName(privilegeName);
+      CommitContext commitContext = sentryStore.alterSentryRoleGrantPrivilege(request.getRoleName(),
+                                    request.getPrivilege());
       response.setStatus(Status.OK());
-      notificationHandlerInvoker.create_sentry_privilege(commitContext,
+      notificationHandlerInvoker.alter_sentry_role_grant_privilege(commitContext,
           request, response);
-    } catch (SentryAlreadyExistsException e) {
-      String msg = "Privilege: " + request + " already exists.";
+    } catch (SentryNoSuchObjectException e) {
+      String msg = "Role: " + request.getRoleName() + " doesn't exist.";
       LOGGER.error(msg, e);
-      response.setStatus(Status.AlreadyExists(msg, e));
+      response.setStatus(Status.NoSuchObject(msg, e));
+    } catch (SentryInvalidInputException e) {
+      String msg = "Invalid input privilege object";
+      LOGGER.error(msg, e);
+      response.setStatus(Status.InvalidInput(msg, e));
+    } catch (Exception e) {
+      String msg = "Unknown error for request: " + request + ", message: " + e.getMessage();
+      LOGGER.error(msg, e);
+      response.setStatus(Status.RuntimeError(msg, e));
+    }
+
+    return response;
+  }
+
+  @Override
+  public TAlterSentryRoleRevokePrivilegeResponse alter_sentry_role_revoke_privilege
+  (TAlterSentryRoleRevokePrivilegeRequest request) throws TException {
+    TAlterSentryRoleRevokePrivilegeResponse response = new TAlterSentryRoleRevokePrivilegeResponse();
+    try {
+      String privilegeName = constructPrivilegeName(request.getPrivilege());
+      request.getPrivilege().setPrivilegeName(privilegeName);
+      CommitContext commitContext = sentryStore.alterSentryRoleRevokePrivilege(request.getRoleName(),
+                                    request.getPrivilege().getPrivilegeName());
+      response.setStatus(Status.OK());
+      notificationHandlerInvoker.alter_sentry_role_revoke_privilege(commitContext,
+          request, response);
+    } catch (SentryNoSuchObjectException e) {
+      String msg = "Privilege: " + request.getPrivilege().getPrivilegeName() + " doesn't exist.";
+      LOGGER.error(msg, e);
+      response.setStatus(Status.NoSuchObject(msg, e));
+    } catch (SentryInvalidInputException e) {
+      String msg = "Invalid input privilege object";
+      LOGGER.error(msg, e);
+      response.setStatus(Status.InvalidInput(msg, e));
     } catch (Exception e) {
       String msg = "Unknown error for request: " + request + ", message: " + e.getMessage();
       LOGGER.error(msg, e);
@@ -139,6 +222,7 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
     return response;
   }
 
+  @Override
   public TDropSentryRoleResponse drop_sentry_role(
     TDropSentryRoleRequest request)  throws TException {
     TDropSentryRoleResponse response = new TDropSentryRoleResponse();
@@ -180,9 +264,10 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
     }
     return response;
   }
+
   @Override
   public TAlterSentryRoleDeleteGroupsResponse alter_sentry_role_delete_groups(
-      TAlterSentryRoleDeleteGroupsRequest request) throws TException {
+    TAlterSentryRoleDeleteGroupsRequest request) throws TException {
     // TODO implement
     TAlterSentryRoleDeleteGroupsResponse response = new TAlterSentryRoleDeleteGroupsResponse();
     try {
@@ -252,4 +337,4 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
     }
     return response;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7df761d/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 0b2daf3..1686780 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
@@ -32,6 +32,7 @@ public enum Status {
   ALREADY_EXISTS(ThriftConstants.TSENTRY_STATUS_ALREADY_EXISTS),
   NO_SUCH_OBJECT(ThriftConstants.TSENTRY_STATUS_NO_SUCH_OBJECT),
   RUNTIME_ERROR(ThriftConstants.TSENTRY_STATUS_RUNTIME_ERROR),
+  INVALID_INPUT(ThriftConstants.TSENTRY_STATUS_INVALID_INPUT),
   UNKNOWN(-1)
   ;
   private int code;
@@ -64,6 +65,9 @@ public enum Status {
   public static TSentryResponseStatus Create(Status value, String message) {
     return Create(value, message, null);
   }
+  public static TSentryResponseStatus InvalidInput(String message, Throwable t) {
+    return Create(Status.INVALID_INPUT, message, t);
+  }
   public static TSentryResponseStatus Create(Status value, String message, @Nullable Throwable t) {
     TSentryResponseStatus status = new TSentryResponseStatus();
     status.setValue(value.getCode());

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7df761d/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 ed0ebc5..7a545be 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
@@ -30,6 +30,7 @@ const i32 TSENTRY_STATUS_OK = 0;
 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;
 
 struct TSentryResponseStatus {
 1: required i32 value,

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7df761d/sentry-provider/sentry-provider-db/src/main/resources/sentry_policy_service.thrift
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/resources/sentry_policy_service.thrift b/sentry-provider/sentry-provider-db/src/main/resources/sentry_policy_service.thrift
index 40f8a5f..d6e05b7 100644
--- a/sentry-provider/sentry-provider-db/src/main/resources/sentry_policy_service.thrift
+++ b/sentry-provider/sentry-provider-db/src/main/resources/sentry_policy_service.thrift
@@ -31,14 +31,14 @@ namespace cpp Apache.Sentry.Provider.Db.Service.Thrift
 
 struct TSentryPrivilege {
 1: required string privilegeScope,
-2: required string privilegeName,
+2: optional string privilegeName,
 3: required string serverName,
 4: optional string dbName,
 5: optional string tableName,
 6: optional string URI,
 7: required string action,
 8: required i64 createTime,
-9: required string grantorPrincipal
+9: optional string grantorPrincipal
 }
 
 struct TSentryRole {
@@ -47,6 +47,7 @@ struct TSentryRole {
 3: required i64 createTime,
 4: required string grantorPrincipal
 }
+
 // TODO fill out
 struct TSentryGroup {
 1: required string groupName
@@ -61,21 +62,23 @@ struct TCreateSentryRoleResponse {
 1: required sentry_common_service.TSentryResponseStatus status
 }
 
-struct TCreateSentryPrivilegeRequest {
+struct TListSentryRolesRequest {
 1: required i32 protocol_version = sentry_common_service.TSENTRY_SERVICE_V1,
-2: required string userName,
-3: required TSentryPrivilege privilege
+2: optional string userName,
+3: optional string groupName,
+4: optional string roleName
 }
-struct TCreateSentryPrivilegeResponse {
+struct TListSentryRolesResponse {
 1: required sentry_common_service.TSentryResponseStatus status
+2: required set<TSentryRole> roles
 }
 
-struct TCreateSentryPrivilegeRequest {
+struct TDropSentryRoleRequest {
 1: required i32 protocol_version = sentry_common_service.TSENTRY_SERVICE_V1,
-2: required string userName,
-3: required TSentryPrivilege privilege
+2: optional string userName,
+3: optional string roleName
 }
-struct TCreateSentryPrivilegeResponse {
+struct TDropSentryRoleResponse {
 1: required sentry_common_service.TSentryResponseStatus status
 }
 
@@ -85,6 +88,7 @@ struct TAlterSentryRoleAddGroupsRequest {
 3: required string roleName,
 4: required set<TSentryGroup> groups
 }
+
 struct TAlterSentryRoleAddGroupsResponse {
 1: required sentry_common_service.TSentryResponseStatus status
 }
@@ -97,25 +101,25 @@ struct TAlterSentryRoleDeleteGroupsResponse {
 1: required sentry_common_service.TSentryResponseStatus status
 }
 
-struct TListSentryRolesRequest {
+struct TAlterSentryRoleGrantPrivilegeRequest {
 1: required i32 protocol_version = sentry_common_service.TSENTRY_SERVICE_V1,
-2: optional string userName,
-3: optional string groupName,
-4: optional string roleName
+2: required string userName,
+3: required string roleName,
+4: required TSentryPrivilege privilege
 }
 
-struct TListSentryRolesResponse {
+struct TAlterSentryRoleGrantPrivilegeResponse {
 1: required sentry_common_service.TSentryResponseStatus status
-2: required set<TSentryRole> roles
 }
 
-struct TDropSentryRoleRequest {
+struct TAlterSentryRoleRevokePrivilegeRequest {
 1: required i32 protocol_version = sentry_common_service.TSENTRY_SERVICE_V1,
-2: optional string userName,
-3: optional string roleName
+2: required string userName,
+3: required string roleName,
+4: required TSentryPrivilege privilege
 }
 
-struct TDropSentryRoleResponse {
+struct TAlterSentryRoleRevokePrivilegeResponse {
 1: required sentry_common_service.TSentryResponseStatus status
 }
 
@@ -123,10 +127,10 @@ service SentryPolicyService
 {
   TCreateSentryRoleResponse create_sentry_role(1:TCreateSentryRoleRequest request)
   TDropSentryRoleResponse drop_sentry_role(1:TDropSentryRoleRequest request)
-
-  TCreateSentryPrivilegeResponse create_sentry_privilege(1:TCreateSentryPrivilegeRequest request)
-  //TDropSentryPrivilegeResponse drop_sentry_privilege(1:TDropSentryPrivilegeRequest request)
-
+  
+  TAlterSentryRoleGrantPrivilegeResponse alter_sentry_role_grant_privilege(1:TAlterSentryRoleGrantPrivilegeRequest request)
+  TAlterSentryRoleRevokePrivilegeResponse alter_sentry_role_revoke_privilege(1:TAlterSentryRoleRevokePrivilegeRequest request)
+  
   TAlterSentryRoleAddGroupsResponse alter_sentry_role_add_groups(1:TAlterSentryRoleAddGroupsRequest request)
   TAlterSentryRoleDeleteGroupsResponse alter_sentry_role_delete_groups(1:TAlterSentryRoleDeleteGroupsRequest request)
 

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7df761d/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestNotificationHandlerInvoker.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestNotificationHandlerInvoker.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestNotificationHandlerInvoker.java
index dab26e1..6a2f48f 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestNotificationHandlerInvoker.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/thrift/TestNotificationHandlerInvoker.java
@@ -62,14 +62,7 @@ public class TestNotificationHandlerInvoker {
         request, response);
   }
 
-  @Test
-  public void testCreateSentryPrivilege() throws Exception {
-    TCreateSentryPrivilegeRequest request = new TCreateSentryPrivilegeRequest();
-    TCreateSentryPrivilegeResponse response = new TCreateSentryPrivilegeResponse();
-    invoker.create_sentry_privilege(commitContext, request, response);
-    Mockito.verify(handler).create_sentry_privilege(commitContext,
-        request, response);
-  }
+
 
   @Test
   public void testAlterSentryRoleAddGroups() throws Exception {
@@ -95,18 +88,12 @@ public class TestNotificationHandlerInvoker {
     }
     @Override
     public void create_sentry_role(CommitContext args,
-        TCreateSentryRoleRequest request, TCreateSentryRoleResponse response) {
+                                   TCreateSentryRoleRequest request, TCreateSentryRoleResponse response) {
       throw new RuntimeException();
     }
     public void drop_sentry_role(CommitContext context,
-        TDropSentryRoleRequest request,
-        TDropSentryRoleResponse response) {
-      throw new RuntimeException();
-    }
-    @Override
-    public void create_sentry_privilege(CommitContext args,
-        TCreateSentryPrivilegeRequest request,
-        TCreateSentryPrivilegeResponse response) {
+                                 TDropSentryRoleRequest request,
+                                 TDropSentryRoleResponse response) {
       throw new RuntimeException();
     }
     @Override
@@ -117,8 +104,8 @@ public class TestNotificationHandlerInvoker {
     }
     @Override
     public void alter_sentry_role_delete_groups(
-        CommitContext args, TAlterSentryRoleDeleteGroupsRequest request,
-        TAlterSentryRoleDeleteGroupsResponse response) {
+      CommitContext args, TAlterSentryRoleDeleteGroupsRequest request,
+      TAlterSentryRoleDeleteGroupsResponse response) {
       throw new RuntimeException();
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7df761d/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 8e1be52..dae7674 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
@@ -70,4 +70,69 @@ public class TestSentryServiceIntegration extends SentryServiceIntegrationBase {
     assertOK(dropResp.getStatus());
     LOGGER.info("Successfully dropped role: admin_r");
   }
-}
\ No newline at end of file
+
+  @Test
+  public void testGrantRevokePrivilege() throws Exception {
+    TDropSentryRoleRequest dropReq = new TDropSentryRoleRequest();
+    dropReq.setProtocol_version(ThriftConstants.TSENTRY_SERVICE_VERSION_CURRENT);
+    dropReq.setRoleName("admin_testdb");
+    dropReq.setUserName("server_admin");
+    TDropSentryRoleResponse dropResp = client.dropRole(dropReq);
+    assertStatus(Status.NO_SUCH_OBJECT, dropResp.getStatus());
+    LOGGER.info("Successfully dropped role: admin_testdb");
+
+    TCreateSentryRoleRequest createReq = new TCreateSentryRoleRequest();
+    createReq.setProtocol_version(ThriftConstants.TSENTRY_SERVICE_VERSION_CURRENT);
+    createReq.setUserName("server_admin");
+    TSentryRole role = new TSentryRole();
+    role.setRoleName("admin_testdb");
+    role.setCreateTime(System.currentTimeMillis());
+    role.setGrantorPrincipal("server_admin");
+    role.setPrivileges(new HashSet<TSentryPrivilege>());
+    createReq.setRole(role);
+    TCreateSentryRoleResponse createResp = client.createRole(createReq);
+    assertOK(createResp.getStatus());
+    LOGGER.info("Successfully create role: admin_testdb");
+
+    TListSentryRolesRequest listReq = new TListSentryRolesRequest();
+    listReq.setProtocol_version(ThriftConstants.TSENTRY_SERVICE_VERSION_CURRENT);
+    listReq.setRoleName("admin_testdb");
+    listReq.setUserName("server_admin");
+    TListSentryRolesResponse listResp = client.listRoleByName(listReq);
+    Set<TSentryRole> roles = listResp.getRoles();
+    Preconditions.checkArgument(roles.size() == 1, "Incorrect number of roles");
+
+    TAlterSentryRoleGrantPrivilegeRequest grantReq = new TAlterSentryRoleGrantPrivilegeRequest();
+    grantReq.setProtocol_version(ThriftConstants.TSENTRY_SERVICE_VERSION_CURRENT);
+    grantReq.setRoleName("admin_testdb");
+    grantReq.setUserName("server_admin");
+    TSentryPrivilege privilege = new TSentryPrivilege();
+    privilege.setPrivilegeScope("DB");
+    privilege.setServerName("server1");
+    privilege.setDbName("testDB");
+    privilege.setAction("ALL");
+    privilege.setGrantorPrincipal("server_admin");
+    privilege.setCreateTime(System.currentTimeMillis());
+    grantReq.setPrivilege(privilege);
+    TAlterSentryRoleGrantPrivilegeResponse grantResp = client.grantPrivilege(grantReq);
+    assertOK(grantResp.getStatus());
+    LOGGER.info("Successfully granted privilege: " + privilege.toString());
+
+    TAlterSentryRoleRevokePrivilegeRequest revokeReq = new TAlterSentryRoleRevokePrivilegeRequest();
+    revokeReq.setProtocol_version(ThriftConstants.TSENTRY_SERVICE_VERSION_CURRENT);
+    revokeReq.setRoleName("admin_testdb");
+    revokeReq.setUserName("server_admin");
+    revokeReq.setPrivilege(privilege);
+    TAlterSentryRoleRevokePrivilegeResponse revokeResp = client.revokePrivilege(revokeReq);
+    assertOK(revokeResp.getStatus());
+    LOGGER.info("Successfully revoked privilege: " + privilege.toString());
+
+    dropReq.setProtocol_version(ThriftConstants.TSENTRY_SERVICE_VERSION_CURRENT);
+    dropReq.setRoleName("admin_testdb");
+    dropReq.setUserName("server_admin");
+    dropResp = client.dropRole(dropReq);
+    assertOK(dropResp.getStatus());
+    LOGGER.info("Successfully dropped role: admin_testdb");
+  }
+
+}


[2/4] SENTRY-126 - Implement alter role grant/revoke privilege in sentry service and sentry store (Shreepadma via Brock)

Posted by br...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7df761d/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TCreateSentryPrivilegeRequest.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TCreateSentryPrivilegeRequest.java b/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TCreateSentryPrivilegeRequest.java
deleted file mode 100644
index 34689fc..0000000
--- a/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TCreateSentryPrivilegeRequest.java
+++ /dev/null
@@ -1,592 +0,0 @@
-/**
- * Autogenerated by Thrift Compiler (0.9.0)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-package org.apache.sentry.provider.db.service.thrift;
-
-import org.apache.commons.lang.builder.HashCodeBuilder;
-import org.apache.thrift.scheme.IScheme;
-import org.apache.thrift.scheme.SchemeFactory;
-import org.apache.thrift.scheme.StandardScheme;
-
-import org.apache.thrift.scheme.TupleScheme;
-import org.apache.thrift.protocol.TTupleProtocol;
-import org.apache.thrift.protocol.TProtocolException;
-import org.apache.thrift.EncodingUtils;
-import org.apache.thrift.TException;
-import java.util.List;
-import java.util.ArrayList;
-import java.util.Map;
-import java.util.HashMap;
-import java.util.EnumMap;
-import java.util.Set;
-import java.util.HashSet;
-import java.util.EnumSet;
-import java.util.Collections;
-import java.util.BitSet;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class TCreateSentryPrivilegeRequest implements org.apache.thrift.TBase<TCreateSentryPrivilegeRequest, TCreateSentryPrivilegeRequest._Fields>, java.io.Serializable, Cloneable {
-  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCreateSentryPrivilegeRequest");
-
-  private static final org.apache.thrift.protocol.TField PROTOCOL_VERSION_FIELD_DESC = new org.apache.thrift.protocol.TField("protocol_version", org.apache.thrift.protocol.TType.I32, (short)1);
-  private static final org.apache.thrift.protocol.TField USER_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("userName", org.apache.thrift.protocol.TType.STRING, (short)2);
-  private static final org.apache.thrift.protocol.TField PRIVILEGE_FIELD_DESC = new org.apache.thrift.protocol.TField("privilege", org.apache.thrift.protocol.TType.STRUCT, (short)3);
-
-  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
-  static {
-    schemes.put(StandardScheme.class, new TCreateSentryPrivilegeRequestStandardSchemeFactory());
-    schemes.put(TupleScheme.class, new TCreateSentryPrivilegeRequestTupleSchemeFactory());
-  }
-
-  private int protocol_version; // required
-  private String userName; // required
-  private TSentryPrivilege privilege; // required
-
-  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
-  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-    PROTOCOL_VERSION((short)1, "protocol_version"),
-    USER_NAME((short)2, "userName"),
-    PRIVILEGE((short)3, "privilege");
-
-    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
-
-    static {
-      for (_Fields field : EnumSet.allOf(_Fields.class)) {
-        byName.put(field.getFieldName(), field);
-      }
-    }
-
-    /**
-     * Find the _Fields constant that matches fieldId, or null if its not found.
-     */
-    public static _Fields findByThriftId(int fieldId) {
-      switch(fieldId) {
-        case 1: // PROTOCOL_VERSION
-          return PROTOCOL_VERSION;
-        case 2: // USER_NAME
-          return USER_NAME;
-        case 3: // PRIVILEGE
-          return PRIVILEGE;
-        default:
-          return null;
-      }
-    }
-
-    /**
-     * Find the _Fields constant that matches fieldId, throwing an exception
-     * if it is not found.
-     */
-    public static _Fields findByThriftIdOrThrow(int fieldId) {
-      _Fields fields = findByThriftId(fieldId);
-      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
-      return fields;
-    }
-
-    /**
-     * Find the _Fields constant that matches name, or null if its not found.
-     */
-    public static _Fields findByName(String name) {
-      return byName.get(name);
-    }
-
-    private final short _thriftId;
-    private final String _fieldName;
-
-    _Fields(short thriftId, String fieldName) {
-      _thriftId = thriftId;
-      _fieldName = fieldName;
-    }
-
-    public short getThriftFieldId() {
-      return _thriftId;
-    }
-
-    public String getFieldName() {
-      return _fieldName;
-    }
-  }
-
-  // isset id assignments
-  private static final int __PROTOCOL_VERSION_ISSET_ID = 0;
-  private byte __isset_bitfield = 0;
-  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
-  static {
-    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-    tmpMap.put(_Fields.PROTOCOL_VERSION, new org.apache.thrift.meta_data.FieldMetaData("protocol_version", org.apache.thrift.TFieldRequirementType.REQUIRED, 
-        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
-    tmpMap.put(_Fields.USER_NAME, new org.apache.thrift.meta_data.FieldMetaData("userName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
-        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-    tmpMap.put(_Fields.PRIVILEGE, new org.apache.thrift.meta_data.FieldMetaData("privilege", org.apache.thrift.TFieldRequirementType.REQUIRED, 
-        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSentryPrivilege.class)));
-    metaDataMap = Collections.unmodifiableMap(tmpMap);
-    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCreateSentryPrivilegeRequest.class, metaDataMap);
-  }
-
-  public TCreateSentryPrivilegeRequest() {
-    this.protocol_version = 1;
-
-  }
-
-  public TCreateSentryPrivilegeRequest(
-    int protocol_version,
-    String userName,
-    TSentryPrivilege privilege)
-  {
-    this();
-    this.protocol_version = protocol_version;
-    setProtocol_versionIsSet(true);
-    this.userName = userName;
-    this.privilege = privilege;
-  }
-
-  /**
-   * Performs a deep copy on <i>other</i>.
-   */
-  public TCreateSentryPrivilegeRequest(TCreateSentryPrivilegeRequest other) {
-    __isset_bitfield = other.__isset_bitfield;
-    this.protocol_version = other.protocol_version;
-    if (other.isSetUserName()) {
-      this.userName = other.userName;
-    }
-    if (other.isSetPrivilege()) {
-      this.privilege = new TSentryPrivilege(other.privilege);
-    }
-  }
-
-  public TCreateSentryPrivilegeRequest deepCopy() {
-    return new TCreateSentryPrivilegeRequest(this);
-  }
-
-  @Override
-  public void clear() {
-    this.protocol_version = 1;
-
-    this.userName = null;
-    this.privilege = null;
-  }
-
-  public int getProtocol_version() {
-    return this.protocol_version;
-  }
-
-  public void setProtocol_version(int protocol_version) {
-    this.protocol_version = protocol_version;
-    setProtocol_versionIsSet(true);
-  }
-
-  public void unsetProtocol_version() {
-    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __PROTOCOL_VERSION_ISSET_ID);
-  }
-
-  /** Returns true if field protocol_version is set (has been assigned a value) and false otherwise */
-  public boolean isSetProtocol_version() {
-    return EncodingUtils.testBit(__isset_bitfield, __PROTOCOL_VERSION_ISSET_ID);
-  }
-
-  public void setProtocol_versionIsSet(boolean value) {
-    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __PROTOCOL_VERSION_ISSET_ID, value);
-  }
-
-  public String getUserName() {
-    return this.userName;
-  }
-
-  public void setUserName(String userName) {
-    this.userName = userName;
-  }
-
-  public void unsetUserName() {
-    this.userName = null;
-  }
-
-  /** Returns true if field userName is set (has been assigned a value) and false otherwise */
-  public boolean isSetUserName() {
-    return this.userName != null;
-  }
-
-  public void setUserNameIsSet(boolean value) {
-    if (!value) {
-      this.userName = null;
-    }
-  }
-
-  public TSentryPrivilege getPrivilege() {
-    return this.privilege;
-  }
-
-  public void setPrivilege(TSentryPrivilege privilege) {
-    this.privilege = privilege;
-  }
-
-  public void unsetPrivilege() {
-    this.privilege = null;
-  }
-
-  /** Returns true if field privilege is set (has been assigned a value) and false otherwise */
-  public boolean isSetPrivilege() {
-    return this.privilege != null;
-  }
-
-  public void setPrivilegeIsSet(boolean value) {
-    if (!value) {
-      this.privilege = null;
-    }
-  }
-
-  public void setFieldValue(_Fields field, Object value) {
-    switch (field) {
-    case PROTOCOL_VERSION:
-      if (value == null) {
-        unsetProtocol_version();
-      } else {
-        setProtocol_version((Integer)value);
-      }
-      break;
-
-    case USER_NAME:
-      if (value == null) {
-        unsetUserName();
-      } else {
-        setUserName((String)value);
-      }
-      break;
-
-    case PRIVILEGE:
-      if (value == null) {
-        unsetPrivilege();
-      } else {
-        setPrivilege((TSentryPrivilege)value);
-      }
-      break;
-
-    }
-  }
-
-  public Object getFieldValue(_Fields field) {
-    switch (field) {
-    case PROTOCOL_VERSION:
-      return Integer.valueOf(getProtocol_version());
-
-    case USER_NAME:
-      return getUserName();
-
-    case PRIVILEGE:
-      return getPrivilege();
-
-    }
-    throw new IllegalStateException();
-  }
-
-  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
-  public boolean isSet(_Fields field) {
-    if (field == null) {
-      throw new IllegalArgumentException();
-    }
-
-    switch (field) {
-    case PROTOCOL_VERSION:
-      return isSetProtocol_version();
-    case USER_NAME:
-      return isSetUserName();
-    case PRIVILEGE:
-      return isSetPrivilege();
-    }
-    throw new IllegalStateException();
-  }
-
-  @Override
-  public boolean equals(Object that) {
-    if (that == null)
-      return false;
-    if (that instanceof TCreateSentryPrivilegeRequest)
-      return this.equals((TCreateSentryPrivilegeRequest)that);
-    return false;
-  }
-
-  public boolean equals(TCreateSentryPrivilegeRequest that) {
-    if (that == null)
-      return false;
-
-    boolean this_present_protocol_version = true;
-    boolean that_present_protocol_version = true;
-    if (this_present_protocol_version || that_present_protocol_version) {
-      if (!(this_present_protocol_version && that_present_protocol_version))
-        return false;
-      if (this.protocol_version != that.protocol_version)
-        return false;
-    }
-
-    boolean this_present_userName = true && this.isSetUserName();
-    boolean that_present_userName = true && that.isSetUserName();
-    if (this_present_userName || that_present_userName) {
-      if (!(this_present_userName && that_present_userName))
-        return false;
-      if (!this.userName.equals(that.userName))
-        return false;
-    }
-
-    boolean this_present_privilege = true && this.isSetPrivilege();
-    boolean that_present_privilege = true && that.isSetPrivilege();
-    if (this_present_privilege || that_present_privilege) {
-      if (!(this_present_privilege && that_present_privilege))
-        return false;
-      if (!this.privilege.equals(that.privilege))
-        return false;
-    }
-
-    return true;
-  }
-
-  @Override
-  public int hashCode() {
-    HashCodeBuilder builder = new HashCodeBuilder();
-
-    boolean present_protocol_version = true;
-    builder.append(present_protocol_version);
-    if (present_protocol_version)
-      builder.append(protocol_version);
-
-    boolean present_userName = true && (isSetUserName());
-    builder.append(present_userName);
-    if (present_userName)
-      builder.append(userName);
-
-    boolean present_privilege = true && (isSetPrivilege());
-    builder.append(present_privilege);
-    if (present_privilege)
-      builder.append(privilege);
-
-    return builder.toHashCode();
-  }
-
-  public int compareTo(TCreateSentryPrivilegeRequest other) {
-    if (!getClass().equals(other.getClass())) {
-      return getClass().getName().compareTo(other.getClass().getName());
-    }
-
-    int lastComparison = 0;
-    TCreateSentryPrivilegeRequest typedOther = (TCreateSentryPrivilegeRequest)other;
-
-    lastComparison = Boolean.valueOf(isSetProtocol_version()).compareTo(typedOther.isSetProtocol_version());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetProtocol_version()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.protocol_version, typedOther.protocol_version);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
-    lastComparison = Boolean.valueOf(isSetUserName()).compareTo(typedOther.isSetUserName());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetUserName()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.userName, typedOther.userName);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
-    lastComparison = Boolean.valueOf(isSetPrivilege()).compareTo(typedOther.isSetPrivilege());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetPrivilege()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.privilege, typedOther.privilege);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
-    return 0;
-  }
-
-  public _Fields fieldForId(int fieldId) {
-    return _Fields.findByThriftId(fieldId);
-  }
-
-  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
-    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
-  }
-
-  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
-    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder("TCreateSentryPrivilegeRequest(");
-    boolean first = true;
-
-    sb.append("protocol_version:");
-    sb.append(this.protocol_version);
-    first = false;
-    if (!first) sb.append(", ");
-    sb.append("userName:");
-    if (this.userName == null) {
-      sb.append("null");
-    } else {
-      sb.append(this.userName);
-    }
-    first = false;
-    if (!first) sb.append(", ");
-    sb.append("privilege:");
-    if (this.privilege == null) {
-      sb.append("null");
-    } else {
-      sb.append(this.privilege);
-    }
-    first = false;
-    sb.append(")");
-    return sb.toString();
-  }
-
-  public void validate() throws org.apache.thrift.TException {
-    // check for required fields
-    if (!isSetProtocol_version()) {
-      throw new org.apache.thrift.protocol.TProtocolException("Required field 'protocol_version' is unset! Struct:" + toString());
-    }
-
-    if (!isSetUserName()) {
-      throw new org.apache.thrift.protocol.TProtocolException("Required field 'userName' is unset! Struct:" + toString());
-    }
-
-    if (!isSetPrivilege()) {
-      throw new org.apache.thrift.protocol.TProtocolException("Required field 'privilege' is unset! Struct:" + toString());
-    }
-
-    // check for sub-struct validity
-    if (privilege != null) {
-      privilege.validate();
-    }
-  }
-
-  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
-    try {
-      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
-    } catch (org.apache.thrift.TException te) {
-      throw new java.io.IOException(te);
-    }
-  }
-
-  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
-    try {
-      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
-      __isset_bitfield = 0;
-      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
-    } catch (org.apache.thrift.TException te) {
-      throw new java.io.IOException(te);
-    }
-  }
-
-  private static class TCreateSentryPrivilegeRequestStandardSchemeFactory implements SchemeFactory {
-    public TCreateSentryPrivilegeRequestStandardScheme getScheme() {
-      return new TCreateSentryPrivilegeRequestStandardScheme();
-    }
-  }
-
-  private static class TCreateSentryPrivilegeRequestStandardScheme extends StandardScheme<TCreateSentryPrivilegeRequest> {
-
-    public void read(org.apache.thrift.protocol.TProtocol iprot, TCreateSentryPrivilegeRequest struct) throws org.apache.thrift.TException {
-      org.apache.thrift.protocol.TField schemeField;
-      iprot.readStructBegin();
-      while (true)
-      {
-        schemeField = iprot.readFieldBegin();
-        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-          break;
-        }
-        switch (schemeField.id) {
-          case 1: // PROTOCOL_VERSION
-            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
-              struct.protocol_version = iprot.readI32();
-              struct.setProtocol_versionIsSet(true);
-            } else { 
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-            }
-            break;
-          case 2: // USER_NAME
-            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-              struct.userName = iprot.readString();
-              struct.setUserNameIsSet(true);
-            } else { 
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-            }
-            break;
-          case 3: // PRIVILEGE
-            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-              struct.privilege = new TSentryPrivilege();
-              struct.privilege.read(iprot);
-              struct.setPrivilegeIsSet(true);
-            } else { 
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-            }
-            break;
-          default:
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-        }
-        iprot.readFieldEnd();
-      }
-      iprot.readStructEnd();
-      struct.validate();
-    }
-
-    public void write(org.apache.thrift.protocol.TProtocol oprot, TCreateSentryPrivilegeRequest struct) throws org.apache.thrift.TException {
-      struct.validate();
-
-      oprot.writeStructBegin(STRUCT_DESC);
-      oprot.writeFieldBegin(PROTOCOL_VERSION_FIELD_DESC);
-      oprot.writeI32(struct.protocol_version);
-      oprot.writeFieldEnd();
-      if (struct.userName != null) {
-        oprot.writeFieldBegin(USER_NAME_FIELD_DESC);
-        oprot.writeString(struct.userName);
-        oprot.writeFieldEnd();
-      }
-      if (struct.privilege != null) {
-        oprot.writeFieldBegin(PRIVILEGE_FIELD_DESC);
-        struct.privilege.write(oprot);
-        oprot.writeFieldEnd();
-      }
-      oprot.writeFieldStop();
-      oprot.writeStructEnd();
-    }
-
-  }
-
-  private static class TCreateSentryPrivilegeRequestTupleSchemeFactory implements SchemeFactory {
-    public TCreateSentryPrivilegeRequestTupleScheme getScheme() {
-      return new TCreateSentryPrivilegeRequestTupleScheme();
-    }
-  }
-
-  private static class TCreateSentryPrivilegeRequestTupleScheme extends TupleScheme<TCreateSentryPrivilegeRequest> {
-
-    @Override
-    public void write(org.apache.thrift.protocol.TProtocol prot, TCreateSentryPrivilegeRequest struct) throws org.apache.thrift.TException {
-      TTupleProtocol oprot = (TTupleProtocol) prot;
-      oprot.writeI32(struct.protocol_version);
-      oprot.writeString(struct.userName);
-      struct.privilege.write(oprot);
-    }
-
-    @Override
-    public void read(org.apache.thrift.protocol.TProtocol prot, TCreateSentryPrivilegeRequest struct) throws org.apache.thrift.TException {
-      TTupleProtocol iprot = (TTupleProtocol) prot;
-      struct.protocol_version = iprot.readI32();
-      struct.setProtocol_versionIsSet(true);
-      struct.userName = iprot.readString();
-      struct.setUserNameIsSet(true);
-      struct.privilege = new TSentryPrivilege();
-      struct.privilege.read(iprot);
-      struct.setPrivilegeIsSet(true);
-    }
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7df761d/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TCreateSentryPrivilegeResponse.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TCreateSentryPrivilegeResponse.java b/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TCreateSentryPrivilegeResponse.java
deleted file mode 100644
index a92698e..0000000
--- a/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TCreateSentryPrivilegeResponse.java
+++ /dev/null
@@ -1,390 +0,0 @@
-/**
- * Autogenerated by Thrift Compiler (0.9.0)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-package org.apache.sentry.provider.db.service.thrift;
-
-import org.apache.commons.lang.builder.HashCodeBuilder;
-import org.apache.thrift.scheme.IScheme;
-import org.apache.thrift.scheme.SchemeFactory;
-import org.apache.thrift.scheme.StandardScheme;
-
-import org.apache.thrift.scheme.TupleScheme;
-import org.apache.thrift.protocol.TTupleProtocol;
-import org.apache.thrift.protocol.TProtocolException;
-import org.apache.thrift.EncodingUtils;
-import org.apache.thrift.TException;
-import java.util.List;
-import java.util.ArrayList;
-import java.util.Map;
-import java.util.HashMap;
-import java.util.EnumMap;
-import java.util.Set;
-import java.util.HashSet;
-import java.util.EnumSet;
-import java.util.Collections;
-import java.util.BitSet;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class TCreateSentryPrivilegeResponse implements org.apache.thrift.TBase<TCreateSentryPrivilegeResponse, TCreateSentryPrivilegeResponse._Fields>, java.io.Serializable, Cloneable {
-  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCreateSentryPrivilegeResponse");
-
-  private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1);
-
-  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
-  static {
-    schemes.put(StandardScheme.class, new TCreateSentryPrivilegeResponseStandardSchemeFactory());
-    schemes.put(TupleScheme.class, new TCreateSentryPrivilegeResponseTupleSchemeFactory());
-  }
-
-  private org.apache.sentry.service.thrift.TSentryResponseStatus status; // required
-
-  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
-  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-    STATUS((short)1, "status");
-
-    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
-
-    static {
-      for (_Fields field : EnumSet.allOf(_Fields.class)) {
-        byName.put(field.getFieldName(), field);
-      }
-    }
-
-    /**
-     * Find the _Fields constant that matches fieldId, or null if its not found.
-     */
-    public static _Fields findByThriftId(int fieldId) {
-      switch(fieldId) {
-        case 1: // STATUS
-          return STATUS;
-        default:
-          return null;
-      }
-    }
-
-    /**
-     * Find the _Fields constant that matches fieldId, throwing an exception
-     * if it is not found.
-     */
-    public static _Fields findByThriftIdOrThrow(int fieldId) {
-      _Fields fields = findByThriftId(fieldId);
-      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
-      return fields;
-    }
-
-    /**
-     * Find the _Fields constant that matches name, or null if its not found.
-     */
-    public static _Fields findByName(String name) {
-      return byName.get(name);
-    }
-
-    private final short _thriftId;
-    private final String _fieldName;
-
-    _Fields(short thriftId, String fieldName) {
-      _thriftId = thriftId;
-      _fieldName = fieldName;
-    }
-
-    public short getThriftFieldId() {
-      return _thriftId;
-    }
-
-    public String getFieldName() {
-      return _fieldName;
-    }
-  }
-
-  // isset id assignments
-  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
-  static {
-    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-    tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, 
-        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.sentry.service.thrift.TSentryResponseStatus.class)));
-    metaDataMap = Collections.unmodifiableMap(tmpMap);
-    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCreateSentryPrivilegeResponse.class, metaDataMap);
-  }
-
-  public TCreateSentryPrivilegeResponse() {
-  }
-
-  public TCreateSentryPrivilegeResponse(
-    org.apache.sentry.service.thrift.TSentryResponseStatus status)
-  {
-    this();
-    this.status = status;
-  }
-
-  /**
-   * Performs a deep copy on <i>other</i>.
-   */
-  public TCreateSentryPrivilegeResponse(TCreateSentryPrivilegeResponse other) {
-    if (other.isSetStatus()) {
-      this.status = new org.apache.sentry.service.thrift.TSentryResponseStatus(other.status);
-    }
-  }
-
-  public TCreateSentryPrivilegeResponse deepCopy() {
-    return new TCreateSentryPrivilegeResponse(this);
-  }
-
-  @Override
-  public void clear() {
-    this.status = null;
-  }
-
-  public org.apache.sentry.service.thrift.TSentryResponseStatus getStatus() {
-    return this.status;
-  }
-
-  public void setStatus(org.apache.sentry.service.thrift.TSentryResponseStatus status) {
-    this.status = status;
-  }
-
-  public void unsetStatus() {
-    this.status = null;
-  }
-
-  /** Returns true if field status is set (has been assigned a value) and false otherwise */
-  public boolean isSetStatus() {
-    return this.status != null;
-  }
-
-  public void setStatusIsSet(boolean value) {
-    if (!value) {
-      this.status = null;
-    }
-  }
-
-  public void setFieldValue(_Fields field, Object value) {
-    switch (field) {
-    case STATUS:
-      if (value == null) {
-        unsetStatus();
-      } else {
-        setStatus((org.apache.sentry.service.thrift.TSentryResponseStatus)value);
-      }
-      break;
-
-    }
-  }
-
-  public Object getFieldValue(_Fields field) {
-    switch (field) {
-    case STATUS:
-      return getStatus();
-
-    }
-    throw new IllegalStateException();
-  }
-
-  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
-  public boolean isSet(_Fields field) {
-    if (field == null) {
-      throw new IllegalArgumentException();
-    }
-
-    switch (field) {
-    case STATUS:
-      return isSetStatus();
-    }
-    throw new IllegalStateException();
-  }
-
-  @Override
-  public boolean equals(Object that) {
-    if (that == null)
-      return false;
-    if (that instanceof TCreateSentryPrivilegeResponse)
-      return this.equals((TCreateSentryPrivilegeResponse)that);
-    return false;
-  }
-
-  public boolean equals(TCreateSentryPrivilegeResponse that) {
-    if (that == null)
-      return false;
-
-    boolean this_present_status = true && this.isSetStatus();
-    boolean that_present_status = true && that.isSetStatus();
-    if (this_present_status || that_present_status) {
-      if (!(this_present_status && that_present_status))
-        return false;
-      if (!this.status.equals(that.status))
-        return false;
-    }
-
-    return true;
-  }
-
-  @Override
-  public int hashCode() {
-    HashCodeBuilder builder = new HashCodeBuilder();
-
-    boolean present_status = true && (isSetStatus());
-    builder.append(present_status);
-    if (present_status)
-      builder.append(status);
-
-    return builder.toHashCode();
-  }
-
-  public int compareTo(TCreateSentryPrivilegeResponse other) {
-    if (!getClass().equals(other.getClass())) {
-      return getClass().getName().compareTo(other.getClass().getName());
-    }
-
-    int lastComparison = 0;
-    TCreateSentryPrivilegeResponse typedOther = (TCreateSentryPrivilegeResponse)other;
-
-    lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetStatus()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
-    return 0;
-  }
-
-  public _Fields fieldForId(int fieldId) {
-    return _Fields.findByThriftId(fieldId);
-  }
-
-  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
-    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
-  }
-
-  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
-    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder("TCreateSentryPrivilegeResponse(");
-    boolean first = true;
-
-    sb.append("status:");
-    if (this.status == null) {
-      sb.append("null");
-    } else {
-      sb.append(this.status);
-    }
-    first = false;
-    sb.append(")");
-    return sb.toString();
-  }
-
-  public void validate() throws org.apache.thrift.TException {
-    // check for required fields
-    if (!isSetStatus()) {
-      throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString());
-    }
-
-    // check for sub-struct validity
-    if (status != null) {
-      status.validate();
-    }
-  }
-
-  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
-    try {
-      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
-    } catch (org.apache.thrift.TException te) {
-      throw new java.io.IOException(te);
-    }
-  }
-
-  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
-    try {
-      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
-    } catch (org.apache.thrift.TException te) {
-      throw new java.io.IOException(te);
-    }
-  }
-
-  private static class TCreateSentryPrivilegeResponseStandardSchemeFactory implements SchemeFactory {
-    public TCreateSentryPrivilegeResponseStandardScheme getScheme() {
-      return new TCreateSentryPrivilegeResponseStandardScheme();
-    }
-  }
-
-  private static class TCreateSentryPrivilegeResponseStandardScheme extends StandardScheme<TCreateSentryPrivilegeResponse> {
-
-    public void read(org.apache.thrift.protocol.TProtocol iprot, TCreateSentryPrivilegeResponse struct) throws org.apache.thrift.TException {
-      org.apache.thrift.protocol.TField schemeField;
-      iprot.readStructBegin();
-      while (true)
-      {
-        schemeField = iprot.readFieldBegin();
-        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-          break;
-        }
-        switch (schemeField.id) {
-          case 1: // STATUS
-            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-              struct.status = new org.apache.sentry.service.thrift.TSentryResponseStatus();
-              struct.status.read(iprot);
-              struct.setStatusIsSet(true);
-            } else { 
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-            }
-            break;
-          default:
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-        }
-        iprot.readFieldEnd();
-      }
-      iprot.readStructEnd();
-      struct.validate();
-    }
-
-    public void write(org.apache.thrift.protocol.TProtocol oprot, TCreateSentryPrivilegeResponse struct) throws org.apache.thrift.TException {
-      struct.validate();
-
-      oprot.writeStructBegin(STRUCT_DESC);
-      if (struct.status != null) {
-        oprot.writeFieldBegin(STATUS_FIELD_DESC);
-        struct.status.write(oprot);
-        oprot.writeFieldEnd();
-      }
-      oprot.writeFieldStop();
-      oprot.writeStructEnd();
-    }
-
-  }
-
-  private static class TCreateSentryPrivilegeResponseTupleSchemeFactory implements SchemeFactory {
-    public TCreateSentryPrivilegeResponseTupleScheme getScheme() {
-      return new TCreateSentryPrivilegeResponseTupleScheme();
-    }
-  }
-
-  private static class TCreateSentryPrivilegeResponseTupleScheme extends TupleScheme<TCreateSentryPrivilegeResponse> {
-
-    @Override
-    public void write(org.apache.thrift.protocol.TProtocol prot, TCreateSentryPrivilegeResponse struct) throws org.apache.thrift.TException {
-      TTupleProtocol oprot = (TTupleProtocol) prot;
-      struct.status.write(oprot);
-    }
-
-    @Override
-    public void read(org.apache.thrift.protocol.TProtocol prot, TCreateSentryPrivilegeResponse struct) throws org.apache.thrift.TException {
-      TTupleProtocol iprot = (TTupleProtocol) prot;
-      struct.status = new org.apache.sentry.service.thrift.TSentryResponseStatus();
-      struct.status.read(iprot);
-      struct.setStatusIsSet(true);
-    }
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7df761d/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TListSentryRolesResponse.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TListSentryRolesResponse.java b/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TListSentryRolesResponse.java
index 32370ba..b035b12 100644
--- a/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TListSentryRolesResponse.java
+++ b/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TListSentryRolesResponse.java
@@ -447,14 +447,14 @@ public class TListSentryRolesResponse implements org.apache.thrift.TBase<TListSe
           case 2: // ROLES
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set16 = iprot.readSetBegin();
-                struct.roles = new HashSet<TSentryRole>(2*_set16.size);
-                for (int _i17 = 0; _i17 < _set16.size; ++_i17)
+                org.apache.thrift.protocol.TSet _set8 = iprot.readSetBegin();
+                struct.roles = new HashSet<TSentryRole>(2*_set8.size);
+                for (int _i9 = 0; _i9 < _set8.size; ++_i9)
                 {
-                  TSentryRole _elem18; // required
-                  _elem18 = new TSentryRole();
-                  _elem18.read(iprot);
-                  struct.roles.add(_elem18);
+                  TSentryRole _elem10; // required
+                  _elem10 = new TSentryRole();
+                  _elem10.read(iprot);
+                  struct.roles.add(_elem10);
                 }
                 iprot.readSetEnd();
               }
@@ -485,9 +485,9 @@ public class TListSentryRolesResponse implements org.apache.thrift.TBase<TListSe
         oprot.writeFieldBegin(ROLES_FIELD_DESC);
         {
           oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRUCT, struct.roles.size()));
-          for (TSentryRole _iter19 : struct.roles)
+          for (TSentryRole _iter11 : struct.roles)
           {
-            _iter19.write(oprot);
+            _iter11.write(oprot);
           }
           oprot.writeSetEnd();
         }
@@ -513,9 +513,9 @@ public class TListSentryRolesResponse implements org.apache.thrift.TBase<TListSe
       struct.status.write(oprot);
       {
         oprot.writeI32(struct.roles.size());
-        for (TSentryRole _iter20 : struct.roles)
+        for (TSentryRole _iter12 : struct.roles)
         {
-          _iter20.write(oprot);
+          _iter12.write(oprot);
         }
       }
     }
@@ -527,14 +527,14 @@ public class TListSentryRolesResponse implements org.apache.thrift.TBase<TListSe
       struct.status.read(iprot);
       struct.setStatusIsSet(true);
       {
-        org.apache.thrift.protocol.TSet _set21 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.roles = new HashSet<TSentryRole>(2*_set21.size);
-        for (int _i22 = 0; _i22 < _set21.size; ++_i22)
+        org.apache.thrift.protocol.TSet _set13 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.roles = new HashSet<TSentryRole>(2*_set13.size);
+        for (int _i14 = 0; _i14 < _set13.size; ++_i14)
         {
-          TSentryRole _elem23; // required
-          _elem23 = new TSentryRole();
-          _elem23.read(iprot);
-          struct.roles.add(_elem23);
+          TSentryRole _elem15; // required
+          _elem15 = new TSentryRole();
+          _elem15.read(iprot);
+          struct.roles.add(_elem15);
         }
       }
       struct.setRolesIsSet(true);

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7df761d/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TSentryPrivilege.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TSentryPrivilege.java b/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TSentryPrivilege.java
index 37f9fb7..9796562 100644
--- a/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TSentryPrivilege.java
+++ b/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TSentryPrivilege.java
@@ -51,14 +51,14 @@ public class TSentryPrivilege implements org.apache.thrift.TBase<TSentryPrivileg
   }
 
   private String privilegeScope; // required
-  private String privilegeName; // required
+  private String privilegeName; // optional
   private String serverName; // required
   private String dbName; // optional
   private String tableName; // optional
   private String URI; // optional
   private String action; // required
   private long createTime; // required
-  private String grantorPrincipal; // required
+  private String grantorPrincipal; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -145,13 +145,13 @@ public class TSentryPrivilege implements org.apache.thrift.TBase<TSentryPrivileg
   // isset id assignments
   private static final int __CREATETIME_ISSET_ID = 0;
   private byte __isset_bitfield = 0;
-  private _Fields optionals[] = {_Fields.DB_NAME,_Fields.TABLE_NAME,_Fields.URI};
+  private _Fields optionals[] = {_Fields.PRIVILEGE_NAME,_Fields.DB_NAME,_Fields.TABLE_NAME,_Fields.URI,_Fields.GRANTOR_PRINCIPAL};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
     tmpMap.put(_Fields.PRIVILEGE_SCOPE, new org.apache.thrift.meta_data.FieldMetaData("privilegeScope", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-    tmpMap.put(_Fields.PRIVILEGE_NAME, new org.apache.thrift.meta_data.FieldMetaData("privilegeName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+    tmpMap.put(_Fields.PRIVILEGE_NAME, new org.apache.thrift.meta_data.FieldMetaData("privilegeName", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.SERVER_NAME, new org.apache.thrift.meta_data.FieldMetaData("serverName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
@@ -165,7 +165,7 @@ public class TSentryPrivilege implements org.apache.thrift.TBase<TSentryPrivileg
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.CREATE_TIME, new org.apache.thrift.meta_data.FieldMetaData("createTime", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
-    tmpMap.put(_Fields.GRANTOR_PRINCIPAL, new org.apache.thrift.meta_data.FieldMetaData("grantorPrincipal", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+    tmpMap.put(_Fields.GRANTOR_PRINCIPAL, new org.apache.thrift.meta_data.FieldMetaData("grantorPrincipal", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TSentryPrivilege.class, metaDataMap);
@@ -176,20 +176,16 @@ public class TSentryPrivilege implements org.apache.thrift.TBase<TSentryPrivileg
 
   public TSentryPrivilege(
     String privilegeScope,
-    String privilegeName,
     String serverName,
     String action,
-    long createTime,
-    String grantorPrincipal)
+    long createTime)
   {
     this();
     this.privilegeScope = privilegeScope;
-    this.privilegeName = privilegeName;
     this.serverName = serverName;
     this.action = action;
     this.createTime = createTime;
     setCreateTimeIsSet(true);
-    this.grantorPrincipal = grantorPrincipal;
   }
 
   /**
@@ -861,14 +857,16 @@ public class TSentryPrivilege implements org.apache.thrift.TBase<TSentryPrivileg
       sb.append(this.privilegeScope);
     }
     first = false;
-    if (!first) sb.append(", ");
-    sb.append("privilegeName:");
-    if (this.privilegeName == null) {
-      sb.append("null");
-    } else {
-      sb.append(this.privilegeName);
+    if (isSetPrivilegeName()) {
+      if (!first) sb.append(", ");
+      sb.append("privilegeName:");
+      if (this.privilegeName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.privilegeName);
+      }
+      first = false;
     }
-    first = false;
     if (!first) sb.append(", ");
     sb.append("serverName:");
     if (this.serverName == null) {
@@ -919,14 +917,16 @@ public class TSentryPrivilege implements org.apache.thrift.TBase<TSentryPrivileg
     sb.append("createTime:");
     sb.append(this.createTime);
     first = false;
-    if (!first) sb.append(", ");
-    sb.append("grantorPrincipal:");
-    if (this.grantorPrincipal == null) {
-      sb.append("null");
-    } else {
-      sb.append(this.grantorPrincipal);
+    if (isSetGrantorPrincipal()) {
+      if (!first) sb.append(", ");
+      sb.append("grantorPrincipal:");
+      if (this.grantorPrincipal == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.grantorPrincipal);
+      }
+      first = false;
     }
-    first = false;
     sb.append(")");
     return sb.toString();
   }
@@ -937,10 +937,6 @@ public class TSentryPrivilege implements org.apache.thrift.TBase<TSentryPrivileg
       throw new org.apache.thrift.protocol.TProtocolException("Required field 'privilegeScope' is unset! Struct:" + toString());
     }
 
-    if (!isSetPrivilegeName()) {
-      throw new org.apache.thrift.protocol.TProtocolException("Required field 'privilegeName' is unset! Struct:" + toString());
-    }
-
     if (!isSetServerName()) {
       throw new org.apache.thrift.protocol.TProtocolException("Required field 'serverName' is unset! Struct:" + toString());
     }
@@ -953,10 +949,6 @@ public class TSentryPrivilege implements org.apache.thrift.TBase<TSentryPrivileg
       throw new org.apache.thrift.protocol.TProtocolException("Required field 'createTime' is unset! Struct:" + toString());
     }
 
-    if (!isSetGrantorPrincipal()) {
-      throw new org.apache.thrift.protocol.TProtocolException("Required field 'grantorPrincipal' is unset! Struct:" + toString());
-    }
-
     // check for sub-struct validity
   }
 
@@ -1087,9 +1079,11 @@ public class TSentryPrivilege implements org.apache.thrift.TBase<TSentryPrivileg
         oprot.writeFieldEnd();
       }
       if (struct.privilegeName != null) {
-        oprot.writeFieldBegin(PRIVILEGE_NAME_FIELD_DESC);
-        oprot.writeString(struct.privilegeName);
-        oprot.writeFieldEnd();
+        if (struct.isSetPrivilegeName()) {
+          oprot.writeFieldBegin(PRIVILEGE_NAME_FIELD_DESC);
+          oprot.writeString(struct.privilegeName);
+          oprot.writeFieldEnd();
+        }
       }
       if (struct.serverName != null) {
         oprot.writeFieldBegin(SERVER_NAME_FIELD_DESC);
@@ -1126,9 +1120,11 @@ public class TSentryPrivilege implements org.apache.thrift.TBase<TSentryPrivileg
       oprot.writeI64(struct.createTime);
       oprot.writeFieldEnd();
       if (struct.grantorPrincipal != null) {
-        oprot.writeFieldBegin(GRANTOR_PRINCIPAL_FIELD_DESC);
-        oprot.writeString(struct.grantorPrincipal);
-        oprot.writeFieldEnd();
+        if (struct.isSetGrantorPrincipal()) {
+          oprot.writeFieldBegin(GRANTOR_PRINCIPAL_FIELD_DESC);
+          oprot.writeString(struct.grantorPrincipal);
+          oprot.writeFieldEnd();
+        }
       }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
@@ -1148,22 +1144,29 @@ public class TSentryPrivilege implements org.apache.thrift.TBase<TSentryPrivileg
     public void write(org.apache.thrift.protocol.TProtocol prot, TSentryPrivilege struct) throws org.apache.thrift.TException {
       TTupleProtocol oprot = (TTupleProtocol) prot;
       oprot.writeString(struct.privilegeScope);
-      oprot.writeString(struct.privilegeName);
       oprot.writeString(struct.serverName);
       oprot.writeString(struct.action);
       oprot.writeI64(struct.createTime);
-      oprot.writeString(struct.grantorPrincipal);
       BitSet optionals = new BitSet();
-      if (struct.isSetDbName()) {
+      if (struct.isSetPrivilegeName()) {
         optionals.set(0);
       }
-      if (struct.isSetTableName()) {
+      if (struct.isSetDbName()) {
         optionals.set(1);
       }
-      if (struct.isSetURI()) {
+      if (struct.isSetTableName()) {
         optionals.set(2);
       }
-      oprot.writeBitSet(optionals, 3);
+      if (struct.isSetURI()) {
+        optionals.set(3);
+      }
+      if (struct.isSetGrantorPrincipal()) {
+        optionals.set(4);
+      }
+      oprot.writeBitSet(optionals, 5);
+      if (struct.isSetPrivilegeName()) {
+        oprot.writeString(struct.privilegeName);
+      }
       if (struct.isSetDbName()) {
         oprot.writeString(struct.dbName);
       }
@@ -1173,6 +1176,9 @@ public class TSentryPrivilege implements org.apache.thrift.TBase<TSentryPrivileg
       if (struct.isSetURI()) {
         oprot.writeString(struct.URI);
       }
+      if (struct.isSetGrantorPrincipal()) {
+        oprot.writeString(struct.grantorPrincipal);
+      }
     }
 
     @Override
@@ -1180,29 +1186,33 @@ public class TSentryPrivilege implements org.apache.thrift.TBase<TSentryPrivileg
       TTupleProtocol iprot = (TTupleProtocol) prot;
       struct.privilegeScope = iprot.readString();
       struct.setPrivilegeScopeIsSet(true);
-      struct.privilegeName = iprot.readString();
-      struct.setPrivilegeNameIsSet(true);
       struct.serverName = iprot.readString();
       struct.setServerNameIsSet(true);
       struct.action = iprot.readString();
       struct.setActionIsSet(true);
       struct.createTime = iprot.readI64();
       struct.setCreateTimeIsSet(true);
-      struct.grantorPrincipal = iprot.readString();
-      struct.setGrantorPrincipalIsSet(true);
-      BitSet incoming = iprot.readBitSet(3);
+      BitSet incoming = iprot.readBitSet(5);
       if (incoming.get(0)) {
+        struct.privilegeName = iprot.readString();
+        struct.setPrivilegeNameIsSet(true);
+      }
+      if (incoming.get(1)) {
         struct.dbName = iprot.readString();
         struct.setDbNameIsSet(true);
       }
-      if (incoming.get(1)) {
+      if (incoming.get(2)) {
         struct.tableName = iprot.readString();
         struct.setTableNameIsSet(true);
       }
-      if (incoming.get(2)) {
+      if (incoming.get(3)) {
         struct.URI = iprot.readString();
         struct.setURIIsSet(true);
       }
+      if (incoming.get(4)) {
+        struct.grantorPrincipal = iprot.readString();
+        struct.setGrantorPrincipalIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7df761d/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 8174fe2..c465737 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
@@ -43,4 +43,6 @@ public class sentry_common_serviceConstants {
 
   public static final int TSENTRY_STATUS_RUNTIME_ERROR = 3;
 
+  public static final int TSENTRY_STATUS_INVALID_INPUT = 4;
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7df761d/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/MSentryPrivilege.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/MSentryPrivilege.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/MSentryPrivilege.java
index 4c3e6ea..9642689 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/MSentryPrivilege.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/MSentryPrivilege.java
@@ -19,6 +19,7 @@
 package org.apache.sentry.provider.db.service.model;
 
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.Set;
 
 import javax.jdo.annotations.PersistenceCapable;
@@ -132,6 +133,19 @@ public class MSentryPrivilege {
   }
 
   public void appendRole(MSentryRole role) {
-    this.roles.add(role);
+    if (!roles.contains(role)) {
+      roles.add(role);
+      role.appendPrivilege(this);
+    }
+  }
+
+  public void removeRole(MSentryRole role) {
+    for (Iterator<MSentryRole> iter = roles.iterator(); iter.hasNext();) {
+      if (iter.next().getRoleName().equalsIgnoreCase(role.getRoleName())) {
+        iter.remove();
+        role.removePrivilege(this);
+        return;
+      }
+    }
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7df761d/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/MSentryRole.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/MSentryRole.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/MSentryRole.java
index 28416e0..9559c57 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/MSentryRole.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/MSentryRole.java
@@ -19,20 +19,23 @@
 package org.apache.sentry.provider.db.service.model;
 
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.Set;
 
 import javax.jdo.annotations.PersistenceCapable;
 
+import org.apache.sentry.provider.db.service.persistent.SentryNoSuchObjectException;
+
 @PersistenceCapable
 public class MSentryRole {
 
-  String roleName;
+  private String roleName;
   // set of privileges granted to this role
-  Set<MSentryPrivilege> privileges;
+  private Set<MSentryPrivilege> privileges;
   // set of groups this role belongs to
-  Set<MSentryGroup> groups;
-  long createTime;
-  String grantorPrincipal;
+  private Set<MSentryGroup> groups;
+  private long createTime;
+  private String grantorPrincipal;
 
   public MSentryRole() {
     privileges = new HashSet<MSentryPrivilege>();
@@ -87,12 +90,25 @@ public class MSentryRole {
     return groups;
   }
 
+  public void removePrivilege(MSentryPrivilege privilege) {
+    for (Iterator<MSentryPrivilege> iter = privileges.iterator(); iter.hasNext();) {
+      if (iter.next().getPrivilegeName().equalsIgnoreCase(privilege.getPrivilegeName())) {
+        iter.remove();
+        privilege.removeRole(this);
+        return;
+      }
+    }
+  }
+
   public void appendPrivileges(Set<MSentryPrivilege> privileges) {
     this.privileges.addAll(privileges);
   }
 
   public void appendPrivilege(MSentryPrivilege privilege) {
-    this.privileges.add(privilege);
+    if (!privileges.contains(privilege)) {
+      privileges.add(privilege);
+      privilege.appendRole(this);
+    }
   }
 
   public void appendGroups(Set<MSentryGroup> groups) {

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7df761d/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/SentryInvalidInputException.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/SentryInvalidInputException.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/SentryInvalidInputException.java
new file mode 100644
index 0000000..6ac9942
--- /dev/null
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/SentryInvalidInputException.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.service.persistent;
+
+import org.apache.sentry.SentryUserException;
+
+public class SentryInvalidInputException extends SentryUserException {
+  private static final long serialVersionUID = 2962080655835L;
+  public SentryInvalidInputException(String msg) {
+    super(msg);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7df761d/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/SentryStore.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/SentryStore.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/SentryStore.java
index 7419a0d..5df6657 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/SentryStore.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/SentryStore.java
@@ -61,108 +61,6 @@ public class SentryStore {
     }
   }
 
-  public CommitContext createSentryRole(TSentryRole role)
-      throws SentryAlreadyExistsException {
-    boolean rollbackTransaction = true;
-    PersistenceManager pm = null;
-    try {
-      pm = openTransaction();
-      Query query = pm.newQuery(MSentryRole.class);
-      query.setFilter("roleName == t");
-      query.declareParameters("java.lang.String t");
-      query.setUnique(true);
-      MSentryRole sentryRole = (MSentryRole) query.execute(role.getRoleName());
-      if (sentryRole == null) {
-        MSentryRole mRole = convertToMSentryRole(role);
-        pm.makePersistent(mRole);
-        CommitContext commit = commitUpdateTransaction(pm);
-        rollbackTransaction = false;
-        return commit;
-      } else {
-        throw new SentryAlreadyExistsException("Role: " + role.getRoleName());
-      }
-    } finally {
-      if (rollbackTransaction) {
-        rollbackTransaction(pm);
-      }
-    }
-  }
-
-  public CommitContext createSentryPrivilege(TSentryPrivilege privilege)
-      throws SentryAlreadyExistsException {
-    // TODO implement
-    throw new RuntimeException("TODO");
-  }
-
-  public CommitContext alterSentryRoleAddGroups()
-      throws SentryNoSuchObjectException {
-    // TODO implement
-    throw new RuntimeException("TODO");
-  }
-
-  public CommitContext alterSentryRoleDeleteGroups()
-      throws SentryNoSuchObjectException {
-    // TODO implement
-    throw new RuntimeException("TODO");
-  }
-
-
-  public CommitContext dropSentryRole(String roleName)
-      throws SentryNoSuchObjectException {
-    boolean rollbackTransaction = true;
-    PersistenceManager pm = null;
-    roleName = roleName.trim();
-    try {
-      pm = openTransaction();
-      Query query = pm.newQuery(MSentryRole.class);
-      query.setFilter("roleName == t");
-      query.declareParameters("java.lang.String t");
-      query.setUnique(true);
-      MSentryRole sentryRole = (MSentryRole) query.execute(roleName);
-      if (sentryRole == null) {
-        throw new SentryNoSuchObjectException("Role " + roleName);
-      } else {
-        pm.retrieve(sentryRole);
-        sentryRole.removePrivileges();
-        pm.deletePersistent(sentryRole);
-      }
-      CommitContext commit = commitUpdateTransaction(pm);
-      rollbackTransaction = false;
-      return commit;
-    } finally {
-      if (rollbackTransaction) {
-        rollbackTransaction(pm);
-      }
-    }
-  }
-
-  public TSentryRole getSentryRoleByName(String roleName)
-      throws SentryNoSuchObjectException {
-    boolean rollbackTransaction = true;
-    PersistenceManager pm = null;
-    roleName = roleName.trim();
-    try {
-      pm = openTransaction();
-      Query query = pm.newQuery(MSentryRole.class);
-      query.setFilter("roleName == t");
-      query.declareParameters("java.lang.String t");
-      query.setUnique(true);
-      MSentryRole sentryRole = (MSentryRole) query.execute(roleName);
-      if (sentryRole == null) {
-        throw new SentryNoSuchObjectException("Role " + roleName);
-      } else {
-        pm.retrieve(sentryRole);
-      }
-      rollbackTransaction = false;
-      commitTransaction(pm);
-      return convertToSentryRole(sentryRole);
-    } finally {
-      if (rollbackTransaction) {
-        rollbackTransaction(pm);
-      }
-    }
-  }
-
   private Properties getDataSourceProperties() {
     Properties prop = new Properties();
     // FIXME: Read from configuration, override the default
@@ -181,9 +79,9 @@ public class SentryStore {
     prop.setProperty("datanucleus.rdbms.useLegacyNativeValueStrategy", "true");
     prop.setProperty("datanucleus.plugin.pluginRegistryBundleCheck", "LOG");
     prop.setProperty("javax.jdo.option.ConnectionDriverName",
-        "org.apache.derby.jdbc.EmbeddedDriver");
+                     "org.apache.derby.jdbc.EmbeddedDriver");
     prop.setProperty("javax.jdo.PersistenceManagerFactoryClass",
-        "org.datanucleus.api.jdo.JDOPersistenceManagerFactory");
+                     "org.datanucleus.api.jdo.JDOPersistenceManagerFactory");
     prop.setProperty("javax.jdo.option.DetachAllOnCommit", "true");
     prop.setProperty("javax.jdo.option.NonTransactionalRead", "false");
     prop.setProperty("javax.jdo.option.NonTransactionalWrite", "false");
@@ -191,7 +89,7 @@ public class SentryStore {
     prop.setProperty("javax.jdo.option.ConnectionPassword", "Sentry");
     prop.setProperty("javax.jdo.option.Multithreaded", "true");
     prop.setProperty("javax.jdo.option.ConnectionURL",
-        "jdbc:derby:;databaseName=sentry_policy_db;create=true");
+                     "jdbc:derby:;databaseName=sentry_policy_db;create=true");
     return prop;
   }
 
@@ -205,6 +103,8 @@ public class SentryStore {
    *
    * Note that there's only one instance of PersistenceManagerFactory object
    * for the service.
+   *
+   * Synchronized because we obtain persistence manager
    */
   private synchronized PersistenceManager openTransaction() {
     PersistenceManager pm = pmf.getPersistenceManager();
@@ -230,7 +130,7 @@ public class SentryStore {
   private synchronized long incrementGetSequenceId() {
     return ++commitSequenceId;
   }
-  
+
   private void commitTransaction(PersistenceManager pm) {
     Transaction currentTransaction = pm.currentTransaction();
     try {
@@ -255,6 +155,170 @@ public class SentryStore {
     }
   }
 
+  public CommitContext createSentryRole(TSentryRole role)
+  throws SentryAlreadyExistsException {
+    boolean rollbackTransaction = true;
+    PersistenceManager pm = null;
+    try {
+      pm = openTransaction();
+      Query query = pm.newQuery(MSentryRole.class);
+      query.setFilter("this.roleName == t");
+      query.declareParameters("java.lang.String t");
+      query.setUnique(true);
+      MSentryRole sentryRole = (MSentryRole) query.execute(role.getRoleName());
+      if (sentryRole == null) {
+        MSentryRole mRole = convertToMSentryRole(role);
+        pm.makePersistent(mRole);
+        CommitContext commit = commitUpdateTransaction(pm);
+        rollbackTransaction = false;
+        return commit;
+      } else {
+        throw new SentryAlreadyExistsException("Role: " + role.getRoleName());
+      }
+    } finally {
+      if (rollbackTransaction) {
+        rollbackTransaction(pm);
+      }
+    }
+  }
+
+  //TODO: handle case where a) privilege already exists, b) role to privilege mapping already exists
+  public CommitContext alterSentryRoleGrantPrivilege(String roleName,
+      TSentryPrivilege privilege) throws SentryNoSuchObjectException {
+    boolean rollbackTransaction = true;
+    PersistenceManager pm = null;
+    try {
+      pm = openTransaction();
+      Query query = pm.newQuery(MSentryRole.class);
+      query.setFilter("this.roleName == t");
+      query.declareParameters("java.lang.String t");
+      query.setUnique(true);
+      MSentryRole mRole = (MSentryRole) query.execute(roleName);
+      if (mRole == null) {
+        throw new SentryNoSuchObjectException("Role: " + roleName);
+      } else {
+        MSentryPrivilege mPrivilege = convertToMSentryPrivilege(privilege);
+        // add privilege and role objects to each other. needed by datanucleus to model
+        // m:n relationships correctly through a join table.
+        mRole.appendPrivilege(mPrivilege);
+        pm.makePersistent(mRole);
+        pm.makePersistent(mPrivilege);
+        CommitContext commit = commitUpdateTransaction(pm);
+        rollbackTransaction = false;
+        return commit;
+      }
+    } finally {
+      if (rollbackTransaction) {
+        rollbackTransaction(pm);
+      }
+    }
+  }
+
+  public CommitContext alterSentryRoleRevokePrivilege(String roleName,
+      String privilegeName) throws SentryNoSuchObjectException {
+    boolean rollbackTransaction = true;
+    PersistenceManager pm = null;
+    try {
+      pm = openTransaction();
+      Query query = pm.newQuery(MSentryRole.class);
+      query.setFilter("this.roleName == t");
+      query.declareParameters("java.lang.String t");
+      query.setUnique(true);
+      MSentryRole mRole = (MSentryRole) query.execute(roleName);
+      if (mRole == null) {
+        throw new SentryNoSuchObjectException("Role: " + roleName);
+      } else {
+        query = pm.newQuery(MSentryPrivilege.class);
+        query.setFilter("this.privilegeName == t");
+        query.declareParameters("java.lang.String t");
+        query.setUnique(true);
+        MSentryPrivilege mPrivilege = (MSentryPrivilege) query.execute(privilegeName);
+        if (mPrivilege == null) {
+          throw new SentryNoSuchObjectException("Privilege: " + privilegeName);
+        } else {
+          // remove privilege and role objects from each other's set. needed by datanucleus to model
+          // m:n relationships correctly through a join table.
+          mRole.removePrivilege(mPrivilege);
+          CommitContext commit = commitUpdateTransaction(pm);
+          rollbackTransaction = false;
+          return commit;
+        }
+      }
+    } finally {
+      if (rollbackTransaction) {
+        rollbackTransaction(pm);
+      }
+    }
+  }
+
+  public CommitContext dropSentryRole(String roleName)
+  throws SentryNoSuchObjectException {
+    boolean rollbackTransaction = true;
+    PersistenceManager pm = null;
+    roleName = roleName.trim();
+    try {
+      pm = openTransaction();
+      Query query = pm.newQuery(MSentryRole.class);
+      query.setFilter("this.roleName == t");
+      query.declareParameters("java.lang.String t");
+      query.setUnique(true);
+      MSentryRole sentryRole = (MSentryRole) query.execute(roleName);
+      if (sentryRole == null) {
+        throw new SentryNoSuchObjectException("Role " + roleName);
+      } else {
+        pm.retrieve(sentryRole);
+        sentryRole.removePrivileges();
+        pm.deletePersistent(sentryRole);
+      }
+      CommitContext commit = commitUpdateTransaction(pm);
+      rollbackTransaction = false;
+      return commit;
+    } finally {
+      if (rollbackTransaction) {
+        rollbackTransaction(pm);
+      }
+    }
+  }
+
+  public CommitContext alterSentryRoleAddGroups()
+  throws SentryNoSuchObjectException {
+    // TODO implement
+    throw new RuntimeException("TODO");
+  }
+
+  public CommitContext alterSentryRoleDeleteGroups()
+  throws SentryNoSuchObjectException {
+    // TODO implement
+    throw new RuntimeException("TODO");
+  }
+
+  public TSentryRole getSentryRoleByName(String roleName)
+  throws SentryNoSuchObjectException {
+    boolean rollbackTransaction = true;
+    PersistenceManager pm = null;
+    roleName = roleName.trim();
+    try {
+      pm = openTransaction();
+      Query query = pm.newQuery(MSentryRole.class);
+      query.setFilter("this.roleName == t");
+      query.declareParameters("java.lang.String t");
+      query.setUnique(true);
+      MSentryRole sentryRole = (MSentryRole) query.execute(roleName);
+      if (sentryRole == null) {
+        throw new SentryNoSuchObjectException("Role " + roleName);
+      } else {
+        pm.retrieve(sentryRole);
+      }
+      rollbackTransaction = false;
+      commitTransaction(pm);
+      return convertToSentryRole(sentryRole);
+    } finally {
+      if (rollbackTransaction) {
+        rollbackTransaction(pm);
+      }
+    }
+  }
+
   private MSentryRole convertToMSentryRole(TSentryRole role) {
     MSentryRole mRole = new MSentryRole();
     mRole.setCreateTime(role.getCreateTime());
@@ -290,11 +354,9 @@ public class SentryStore {
     privilege.setTableName(mSentryPrivilege.getTableName());
     privilege.setURI(mSentryPrivilege.getURI());
     privilege.setGrantorPrincipal(mSentryPrivilege.getGrantorPrincipal());
-
     return privilege;
   }
 
-  @SuppressWarnings("unused")
   private MSentryPrivilege convertToMSentryPrivilege(TSentryPrivilege privilege) {
     MSentryPrivilege mSentryPrivilege = new MSentryPrivilege();
     mSentryPrivilege.setServerName(privilege.getServerName());
@@ -306,7 +368,6 @@ public class SentryStore {
     mSentryPrivilege.setGrantorPrincipal(privilege.getGrantorPrincipal());
     mSentryPrivilege.setURI(privilege.getURI());
     mSentryPrivilege.setPrivilegeName(privilege.getPrivilegeName());
-    //MSentryRole mSentryRole = convertToMSentryRole(role);
     return mSentryPrivilege;
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7df761d/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/NotificationHandler.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/NotificationHandler.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/NotificationHandler.java
index dbd8dae..506d433 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/NotificationHandler.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/NotificationHandler.java
@@ -44,16 +44,19 @@ public abstract class NotificationHandler {
   }
 
   public void create_sentry_role(CommitContext context,
-      TCreateSentryRoleRequest request, TCreateSentryRoleResponse response) {
+                                 TCreateSentryRoleRequest request, TCreateSentryRoleResponse response) {
   }
 
   public void drop_sentry_role(CommitContext context, TDropSentryRoleRequest request,
-      TDropSentryRoleResponse response) {
+                               TDropSentryRoleResponse response) {
   }
 
-  public void create_sentry_privilege(CommitContext context,
-      TCreateSentryPrivilegeRequest request,
-      TCreateSentryPrivilegeResponse response) {
+  public void alter_sentry_role_grant_privilege(CommitContext context, TAlterSentryRoleGrantPrivilegeRequest request,
+      TAlterSentryRoleGrantPrivilegeResponse response) {
+  }
+
+  public void alter_sentry_role_revoke_privilege(CommitContext context, TAlterSentryRoleRevokePrivilegeRequest request,
+      TAlterSentryRoleRevokePrivilegeResponse response) {
   }
 
   public void alter_sentry_role_add_groups(CommitContext context,
@@ -62,7 +65,7 @@ public abstract class NotificationHandler {
   }
 
   public void alter_sentry_role_delete_groups(
-      CommitContext context, TAlterSentryRoleDeleteGroupsRequest request,
-      TAlterSentryRoleDeleteGroupsResponse response) {
+    CommitContext context, TAlterSentryRoleDeleteGroupsRequest request,
+    TAlterSentryRoleDeleteGroupsResponse response) {
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7df761d/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/NotificationHandlerInvoker.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/NotificationHandlerInvoker.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/NotificationHandlerInvoker.java
index 2425ac3..9d9e867 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/NotificationHandlerInvoker.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/NotificationHandlerInvoker.java
@@ -38,58 +38,76 @@ public class NotificationHandlerInvoker extends NotificationHandler {
   private final ImmutableList<NotificationHandler> handlers;
 
   public NotificationHandlerInvoker(Configuration conf, NotificationHandler handler)
-      throws Exception {
+  throws Exception {
     this(conf, ImmutableList.of(handler));
   }
 
   public NotificationHandlerInvoker(Configuration conf, List<NotificationHandler> handlers)
-      throws Exception {
+  throws Exception {
     super(conf);
     this.handlers = ImmutableList.copyOf(handlers);
   }
 
   @Override
   public void create_sentry_role(CommitContext context,
-      TCreateSentryRoleRequest request, TCreateSentryRoleResponse response) {
+                                 TCreateSentryRoleRequest request, TCreateSentryRoleResponse response) {
     for (NotificationHandler handler : handlers) {
       try {
         LOGGER.debug("Calling " + handler);
         handler.create_sentry_role(context,  new TCreateSentryRoleRequest(request),
-            new TCreateSentryRoleResponse(response));
+                                   new TCreateSentryRoleResponse(response));
       } catch (Exception ex) {
         LOGGER.error("Unexpected error in " + handler + ". Request: "
-            + request + ", Response: " + response, ex);
+                     + request + ", Response: " + response, ex);
       }
     }
   }
 
   @Override
   public void drop_sentry_role(CommitContext context, TDropSentryRoleRequest request,
-      TDropSentryRoleResponse response) {
+                               TDropSentryRoleResponse response) {
     for (NotificationHandler handler : handlers) {
       try {
         LOGGER.debug("Calling " + handler);
         handler.drop_sentry_role(context,  new TDropSentryRoleRequest(request),
-            new TDropSentryRoleResponse(response));
+                                 new TDropSentryRoleResponse(response));
       } catch (Exception ex) {
         LOGGER.error("Unexpected error in " + handler + ". Request: "
-            + request + ", Response: " + response, ex);
+                     + request + ", Response: " + response, ex);
       }
     }
   }
 
   @Override
-  public void create_sentry_privilege(CommitContext context,
-      TCreateSentryPrivilegeRequest request,
-      TCreateSentryPrivilegeResponse response) {
+  public void alter_sentry_role_grant_privilege(CommitContext context,
+      TAlterSentryRoleGrantPrivilegeRequest request,
+      TAlterSentryRoleGrantPrivilegeResponse response) {
     for (NotificationHandler handler : handlers) {
       try {
         LOGGER.debug("Calling " + handler);
-        handler.create_sentry_privilege(context, new TCreateSentryPrivilegeRequest(request),
-            new TCreateSentryPrivilegeResponse(response));
+        handler.alter_sentry_role_grant_privilege(context,
+            new TAlterSentryRoleGrantPrivilegeRequest(request),
+            new TAlterSentryRoleGrantPrivilegeResponse(response));
       } catch (Exception ex) {
         LOGGER.error("Unexpected error in " + handler + ". Request: "
-            + request + ", Response: " + response, ex);
+                     + request + ", Response: " + response, ex);
+      }
+    }
+  }
+
+  @Override
+  public void alter_sentry_role_revoke_privilege(CommitContext context,
+      TAlterSentryRoleRevokePrivilegeRequest request,
+      TAlterSentryRoleRevokePrivilegeResponse response) {
+    for (NotificationHandler handler : handlers) {
+      try {
+        LOGGER.debug("Calling " + handler);
+        handler.alter_sentry_role_revoke_privilege(context,
+            new TAlterSentryRoleRevokePrivilegeRequest(request),
+            new TAlterSentryRoleRevokePrivilegeResponse(response));
+      } catch (Exception ex) {
+        LOGGER.error("Unexpected error in " + handler + ". Request: "
+                     + request + ", Response: " + response, ex);
       }
     }
   }
@@ -102,26 +120,26 @@ public class NotificationHandlerInvoker extends NotificationHandler {
       try {
         LOGGER.debug("Calling " + handler);
         handler.alter_sentry_role_add_groups(context, new TAlterSentryRoleAddGroupsRequest(request),
-            new TAlterSentryRoleAddGroupsResponse(response));
+                                             new TAlterSentryRoleAddGroupsResponse(response));
       } catch (Exception ex) {
         LOGGER.error("Unexpected error in " + handler + ". Request: "
-            + request + ", Response: " + response, ex);
+                     + request + ", Response: " + response, ex);
       }
     }
   }
 
   @Override
   public void alter_sentry_role_delete_groups(
-      CommitContext context, TAlterSentryRoleDeleteGroupsRequest request,
-      TAlterSentryRoleDeleteGroupsResponse response) {
+    CommitContext context, TAlterSentryRoleDeleteGroupsRequest request,
+    TAlterSentryRoleDeleteGroupsResponse response) {
     for (NotificationHandler handler : handlers) {
       try {
         LOGGER.debug("Calling " + handler);
         handler.alter_sentry_role_delete_groups(context, new TAlterSentryRoleDeleteGroupsRequest(request),
-            new TAlterSentryRoleDeleteGroupsResponse(response));
+                                                new TAlterSentryRoleDeleteGroupsResponse(response));
       } catch (Exception ex) {
         LOGGER.error("Unexpected error in " + handler + ". Request: "
-            + request + ", Response: " + response, ex);
+                     + request + ", Response: " + response, ex);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7df761d/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 aa83ffd..a4487ee 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
@@ -47,51 +47,61 @@ public class SentryPolicyServiceClient {
   private TTransport transport;
   private int connectionTimeout;
   private static final Logger LOGGER = LoggerFactory
-      .getLogger(SentryPolicyServiceClient.class);
+                                       .getLogger(SentryPolicyServiceClient.class);
 
   public SentryPolicyServiceClient(Configuration conf) throws Exception {
     this.conf = conf;
     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));
+                           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);
+                                         ClientConfig.SERVER_RPC_CONN_TIMEOUT_DEFAULT);
     String serverPrincipal = Preconditions.checkNotNull(
-        conf.get(ServerConfig.PRINCIPAL), ServerConfig.PRINCIPAL
-            + " is required");
+                               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 principal should have 3 parts: " + serverPrincipal);
     transport = new TSocket(serverAddress.getHostString(),
-        serverAddress.getPort(), connectionTimeout);
+                            serverAddress.getPort(), connectionTimeout);
     TTransport saslTransport = new TSaslClientTransport(
-        AuthMethod.KERBEROS.getMechanismName(), null, serverPrincipalParts[0],
-        serverPrincipalParts[1], ClientConfig.SASL_PROPERTIES, null, transport);
+      AuthMethod.KERBEROS.getMechanismName(), null, serverPrincipalParts[0],
+      serverPrincipalParts[1], ClientConfig.SASL_PROPERTIES, null, transport);
     saslTransport.open();
     LOGGER.info("Successfully opened transport");
     TMultiplexedProtocol protocol = new TMultiplexedProtocol(
-        new TBinaryProtocol(saslTransport),
-        SentryPolicyStoreProcessor.SENTRY_POLICY_SERVICE_NAME);
+      new TBinaryProtocol(saslTransport),
+      SentryPolicyStoreProcessor.SENTRY_POLICY_SERVICE_NAME);
     client = new SentryPolicyService.Client(protocol);
     LOGGER.info("Successfully created client");
   }
 
   public TCreateSentryRoleResponse createRole(TCreateSentryRoleRequest req)
-      throws TException {
+  throws TException {
     return client.create_sentry_role(req);
   }
 
   public TListSentryRolesResponse listRoleByName(TListSentryRolesRequest req)
-      throws TException {
+  throws TException {
     return client.list_sentry_roles_by_role_name(req);
   }
 
   public TDropSentryRoleResponse dropRole(TDropSentryRoleRequest req)
-      throws TException {
+  throws TException {
     return client.drop_sentry_role(req);
   }
 
+  public TAlterSentryRoleGrantPrivilegeResponse grantPrivilege(TAlterSentryRoleGrantPrivilegeRequest req)
+  throws TException {
+    return client.alter_sentry_role_grant_privilege(req);
+  }
+
+  public TAlterSentryRoleRevokePrivilegeResponse revokePrivilege(TAlterSentryRoleRevokePrivilegeRequest req)
+  throws TException {
+    return client.alter_sentry_role_revoke_privilege(req);
+  }
+
   public void close() {
     if (transport != null) {
       transport.close();


[3/4] SENTRY-126 - Implement alter role grant/revoke privilege in sentry service and sentry store (Shreepadma via Brock)

Posted by br...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7df761d/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TAlterSentryRoleGrantPrivilegeRequest.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TAlterSentryRoleGrantPrivilegeRequest.java b/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TAlterSentryRoleGrantPrivilegeRequest.java
new file mode 100644
index 0000000..237f55d
--- /dev/null
+++ b/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TAlterSentryRoleGrantPrivilegeRequest.java
@@ -0,0 +1,693 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.sentry.provider.db.service.thrift;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TAlterSentryRoleGrantPrivilegeRequest implements org.apache.thrift.TBase<TAlterSentryRoleGrantPrivilegeRequest, TAlterSentryRoleGrantPrivilegeRequest._Fields>, java.io.Serializable, Cloneable {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TAlterSentryRoleGrantPrivilegeRequest");
+
+  private static final org.apache.thrift.protocol.TField PROTOCOL_VERSION_FIELD_DESC = new org.apache.thrift.protocol.TField("protocol_version", org.apache.thrift.protocol.TType.I32, (short)1);
+  private static final org.apache.thrift.protocol.TField USER_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("userName", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField ROLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("roleName", org.apache.thrift.protocol.TType.STRING, (short)3);
+  private static final org.apache.thrift.protocol.TField PRIVILEGE_FIELD_DESC = new org.apache.thrift.protocol.TField("privilege", org.apache.thrift.protocol.TType.STRUCT, (short)4);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new TAlterSentryRoleGrantPrivilegeRequestStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new TAlterSentryRoleGrantPrivilegeRequestTupleSchemeFactory());
+  }
+
+  private int protocol_version; // required
+  private String userName; // required
+  private String roleName; // required
+  private TSentryPrivilege privilege; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    PROTOCOL_VERSION((short)1, "protocol_version"),
+    USER_NAME((short)2, "userName"),
+    ROLE_NAME((short)3, "roleName"),
+    PRIVILEGE((short)4, "privilege");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // PROTOCOL_VERSION
+          return PROTOCOL_VERSION;
+        case 2: // USER_NAME
+          return USER_NAME;
+        case 3: // ROLE_NAME
+          return ROLE_NAME;
+        case 4: // PRIVILEGE
+          return PRIVILEGE;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __PROTOCOL_VERSION_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.PROTOCOL_VERSION, new org.apache.thrift.meta_data.FieldMetaData("protocol_version", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.USER_NAME, new org.apache.thrift.meta_data.FieldMetaData("userName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.ROLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("roleName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.PRIVILEGE, new org.apache.thrift.meta_data.FieldMetaData("privilege", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSentryPrivilege.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TAlterSentryRoleGrantPrivilegeRequest.class, metaDataMap);
+  }
+
+  public TAlterSentryRoleGrantPrivilegeRequest() {
+    this.protocol_version = 1;
+
+  }
+
+  public TAlterSentryRoleGrantPrivilegeRequest(
+    int protocol_version,
+    String userName,
+    String roleName,
+    TSentryPrivilege privilege)
+  {
+    this();
+    this.protocol_version = protocol_version;
+    setProtocol_versionIsSet(true);
+    this.userName = userName;
+    this.roleName = roleName;
+    this.privilege = privilege;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public TAlterSentryRoleGrantPrivilegeRequest(TAlterSentryRoleGrantPrivilegeRequest other) {
+    __isset_bitfield = other.__isset_bitfield;
+    this.protocol_version = other.protocol_version;
+    if (other.isSetUserName()) {
+      this.userName = other.userName;
+    }
+    if (other.isSetRoleName()) {
+      this.roleName = other.roleName;
+    }
+    if (other.isSetPrivilege()) {
+      this.privilege = new TSentryPrivilege(other.privilege);
+    }
+  }
+
+  public TAlterSentryRoleGrantPrivilegeRequest deepCopy() {
+    return new TAlterSentryRoleGrantPrivilegeRequest(this);
+  }
+
+  @Override
+  public void clear() {
+    this.protocol_version = 1;
+
+    this.userName = null;
+    this.roleName = null;
+    this.privilege = null;
+  }
+
+  public int getProtocol_version() {
+    return this.protocol_version;
+  }
+
+  public void setProtocol_version(int protocol_version) {
+    this.protocol_version = protocol_version;
+    setProtocol_versionIsSet(true);
+  }
+
+  public void unsetProtocol_version() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __PROTOCOL_VERSION_ISSET_ID);
+  }
+
+  /** Returns true if field protocol_version is set (has been assigned a value) and false otherwise */
+  public boolean isSetProtocol_version() {
+    return EncodingUtils.testBit(__isset_bitfield, __PROTOCOL_VERSION_ISSET_ID);
+  }
+
+  public void setProtocol_versionIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __PROTOCOL_VERSION_ISSET_ID, value);
+  }
+
+  public String getUserName() {
+    return this.userName;
+  }
+
+  public void setUserName(String userName) {
+    this.userName = userName;
+  }
+
+  public void unsetUserName() {
+    this.userName = null;
+  }
+
+  /** Returns true if field userName is set (has been assigned a value) and false otherwise */
+  public boolean isSetUserName() {
+    return this.userName != null;
+  }
+
+  public void setUserNameIsSet(boolean value) {
+    if (!value) {
+      this.userName = null;
+    }
+  }
+
+  public String getRoleName() {
+    return this.roleName;
+  }
+
+  public void setRoleName(String roleName) {
+    this.roleName = roleName;
+  }
+
+  public void unsetRoleName() {
+    this.roleName = null;
+  }
+
+  /** Returns true if field roleName is set (has been assigned a value) and false otherwise */
+  public boolean isSetRoleName() {
+    return this.roleName != null;
+  }
+
+  public void setRoleNameIsSet(boolean value) {
+    if (!value) {
+      this.roleName = null;
+    }
+  }
+
+  public TSentryPrivilege getPrivilege() {
+    return this.privilege;
+  }
+
+  public void setPrivilege(TSentryPrivilege privilege) {
+    this.privilege = privilege;
+  }
+
+  public void unsetPrivilege() {
+    this.privilege = null;
+  }
+
+  /** Returns true if field privilege is set (has been assigned a value) and false otherwise */
+  public boolean isSetPrivilege() {
+    return this.privilege != null;
+  }
+
+  public void setPrivilegeIsSet(boolean value) {
+    if (!value) {
+      this.privilege = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case PROTOCOL_VERSION:
+      if (value == null) {
+        unsetProtocol_version();
+      } else {
+        setProtocol_version((Integer)value);
+      }
+      break;
+
+    case USER_NAME:
+      if (value == null) {
+        unsetUserName();
+      } else {
+        setUserName((String)value);
+      }
+      break;
+
+    case ROLE_NAME:
+      if (value == null) {
+        unsetRoleName();
+      } else {
+        setRoleName((String)value);
+      }
+      break;
+
+    case PRIVILEGE:
+      if (value == null) {
+        unsetPrivilege();
+      } else {
+        setPrivilege((TSentryPrivilege)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case PROTOCOL_VERSION:
+      return Integer.valueOf(getProtocol_version());
+
+    case USER_NAME:
+      return getUserName();
+
+    case ROLE_NAME:
+      return getRoleName();
+
+    case PRIVILEGE:
+      return getPrivilege();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case PROTOCOL_VERSION:
+      return isSetProtocol_version();
+    case USER_NAME:
+      return isSetUserName();
+    case ROLE_NAME:
+      return isSetRoleName();
+    case PRIVILEGE:
+      return isSetPrivilege();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof TAlterSentryRoleGrantPrivilegeRequest)
+      return this.equals((TAlterSentryRoleGrantPrivilegeRequest)that);
+    return false;
+  }
+
+  public boolean equals(TAlterSentryRoleGrantPrivilegeRequest that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_protocol_version = true;
+    boolean that_present_protocol_version = true;
+    if (this_present_protocol_version || that_present_protocol_version) {
+      if (!(this_present_protocol_version && that_present_protocol_version))
+        return false;
+      if (this.protocol_version != that.protocol_version)
+        return false;
+    }
+
+    boolean this_present_userName = true && this.isSetUserName();
+    boolean that_present_userName = true && that.isSetUserName();
+    if (this_present_userName || that_present_userName) {
+      if (!(this_present_userName && that_present_userName))
+        return false;
+      if (!this.userName.equals(that.userName))
+        return false;
+    }
+
+    boolean this_present_roleName = true && this.isSetRoleName();
+    boolean that_present_roleName = true && that.isSetRoleName();
+    if (this_present_roleName || that_present_roleName) {
+      if (!(this_present_roleName && that_present_roleName))
+        return false;
+      if (!this.roleName.equals(that.roleName))
+        return false;
+    }
+
+    boolean this_present_privilege = true && this.isSetPrivilege();
+    boolean that_present_privilege = true && that.isSetPrivilege();
+    if (this_present_privilege || that_present_privilege) {
+      if (!(this_present_privilege && that_present_privilege))
+        return false;
+      if (!this.privilege.equals(that.privilege))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder builder = new HashCodeBuilder();
+
+    boolean present_protocol_version = true;
+    builder.append(present_protocol_version);
+    if (present_protocol_version)
+      builder.append(protocol_version);
+
+    boolean present_userName = true && (isSetUserName());
+    builder.append(present_userName);
+    if (present_userName)
+      builder.append(userName);
+
+    boolean present_roleName = true && (isSetRoleName());
+    builder.append(present_roleName);
+    if (present_roleName)
+      builder.append(roleName);
+
+    boolean present_privilege = true && (isSetPrivilege());
+    builder.append(present_privilege);
+    if (present_privilege)
+      builder.append(privilege);
+
+    return builder.toHashCode();
+  }
+
+  public int compareTo(TAlterSentryRoleGrantPrivilegeRequest other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    TAlterSentryRoleGrantPrivilegeRequest typedOther = (TAlterSentryRoleGrantPrivilegeRequest)other;
+
+    lastComparison = Boolean.valueOf(isSetProtocol_version()).compareTo(typedOther.isSetProtocol_version());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetProtocol_version()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.protocol_version, typedOther.protocol_version);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetUserName()).compareTo(typedOther.isSetUserName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetUserName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.userName, typedOther.userName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetRoleName()).compareTo(typedOther.isSetRoleName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetRoleName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.roleName, typedOther.roleName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetPrivilege()).compareTo(typedOther.isSetPrivilege());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetPrivilege()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.privilege, typedOther.privilege);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("TAlterSentryRoleGrantPrivilegeRequest(");
+    boolean first = true;
+
+    sb.append("protocol_version:");
+    sb.append(this.protocol_version);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("userName:");
+    if (this.userName == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.userName);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("roleName:");
+    if (this.roleName == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.roleName);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("privilege:");
+    if (this.privilege == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.privilege);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetProtocol_version()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'protocol_version' is unset! Struct:" + toString());
+    }
+
+    if (!isSetUserName()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'userName' is unset! Struct:" + toString());
+    }
+
+    if (!isSetRoleName()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'roleName' is unset! Struct:" + toString());
+    }
+
+    if (!isSetPrivilege()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'privilege' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+    if (privilege != null) {
+      privilege.validate();
+    }
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class TAlterSentryRoleGrantPrivilegeRequestStandardSchemeFactory implements SchemeFactory {
+    public TAlterSentryRoleGrantPrivilegeRequestStandardScheme getScheme() {
+      return new TAlterSentryRoleGrantPrivilegeRequestStandardScheme();
+    }
+  }
+
+  private static class TAlterSentryRoleGrantPrivilegeRequestStandardScheme extends StandardScheme<TAlterSentryRoleGrantPrivilegeRequest> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, TAlterSentryRoleGrantPrivilegeRequest struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // PROTOCOL_VERSION
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.protocol_version = iprot.readI32();
+              struct.setProtocol_versionIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // USER_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.userName = iprot.readString();
+              struct.setUserNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // ROLE_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.roleName = iprot.readString();
+              struct.setRoleNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // PRIVILEGE
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.privilege = new TSentryPrivilege();
+              struct.privilege.read(iprot);
+              struct.setPrivilegeIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, TAlterSentryRoleGrantPrivilegeRequest struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      oprot.writeFieldBegin(PROTOCOL_VERSION_FIELD_DESC);
+      oprot.writeI32(struct.protocol_version);
+      oprot.writeFieldEnd();
+      if (struct.userName != null) {
+        oprot.writeFieldBegin(USER_NAME_FIELD_DESC);
+        oprot.writeString(struct.userName);
+        oprot.writeFieldEnd();
+      }
+      if (struct.roleName != null) {
+        oprot.writeFieldBegin(ROLE_NAME_FIELD_DESC);
+        oprot.writeString(struct.roleName);
+        oprot.writeFieldEnd();
+      }
+      if (struct.privilege != null) {
+        oprot.writeFieldBegin(PRIVILEGE_FIELD_DESC);
+        struct.privilege.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class TAlterSentryRoleGrantPrivilegeRequestTupleSchemeFactory implements SchemeFactory {
+    public TAlterSentryRoleGrantPrivilegeRequestTupleScheme getScheme() {
+      return new TAlterSentryRoleGrantPrivilegeRequestTupleScheme();
+    }
+  }
+
+  private static class TAlterSentryRoleGrantPrivilegeRequestTupleScheme extends TupleScheme<TAlterSentryRoleGrantPrivilegeRequest> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, TAlterSentryRoleGrantPrivilegeRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeI32(struct.protocol_version);
+      oprot.writeString(struct.userName);
+      oprot.writeString(struct.roleName);
+      struct.privilege.write(oprot);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, TAlterSentryRoleGrantPrivilegeRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.protocol_version = iprot.readI32();
+      struct.setProtocol_versionIsSet(true);
+      struct.userName = iprot.readString();
+      struct.setUserNameIsSet(true);
+      struct.roleName = iprot.readString();
+      struct.setRoleNameIsSet(true);
+      struct.privilege = new TSentryPrivilege();
+      struct.privilege.read(iprot);
+      struct.setPrivilegeIsSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7df761d/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TAlterSentryRoleGrantPrivilegeResponse.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TAlterSentryRoleGrantPrivilegeResponse.java b/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TAlterSentryRoleGrantPrivilegeResponse.java
new file mode 100644
index 0000000..05e9f95
--- /dev/null
+++ b/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TAlterSentryRoleGrantPrivilegeResponse.java
@@ -0,0 +1,390 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.sentry.provider.db.service.thrift;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TAlterSentryRoleGrantPrivilegeResponse implements org.apache.thrift.TBase<TAlterSentryRoleGrantPrivilegeResponse, TAlterSentryRoleGrantPrivilegeResponse._Fields>, java.io.Serializable, Cloneable {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TAlterSentryRoleGrantPrivilegeResponse");
+
+  private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new TAlterSentryRoleGrantPrivilegeResponseStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new TAlterSentryRoleGrantPrivilegeResponseTupleSchemeFactory());
+  }
+
+  private org.apache.sentry.service.thrift.TSentryResponseStatus status; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    STATUS((short)1, "status");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // STATUS
+          return STATUS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.sentry.service.thrift.TSentryResponseStatus.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TAlterSentryRoleGrantPrivilegeResponse.class, metaDataMap);
+  }
+
+  public TAlterSentryRoleGrantPrivilegeResponse() {
+  }
+
+  public TAlterSentryRoleGrantPrivilegeResponse(
+    org.apache.sentry.service.thrift.TSentryResponseStatus status)
+  {
+    this();
+    this.status = status;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public TAlterSentryRoleGrantPrivilegeResponse(TAlterSentryRoleGrantPrivilegeResponse other) {
+    if (other.isSetStatus()) {
+      this.status = new org.apache.sentry.service.thrift.TSentryResponseStatus(other.status);
+    }
+  }
+
+  public TAlterSentryRoleGrantPrivilegeResponse deepCopy() {
+    return new TAlterSentryRoleGrantPrivilegeResponse(this);
+  }
+
+  @Override
+  public void clear() {
+    this.status = null;
+  }
+
+  public org.apache.sentry.service.thrift.TSentryResponseStatus getStatus() {
+    return this.status;
+  }
+
+  public void setStatus(org.apache.sentry.service.thrift.TSentryResponseStatus status) {
+    this.status = status;
+  }
+
+  public void unsetStatus() {
+    this.status = null;
+  }
+
+  /** Returns true if field status is set (has been assigned a value) and false otherwise */
+  public boolean isSetStatus() {
+    return this.status != null;
+  }
+
+  public void setStatusIsSet(boolean value) {
+    if (!value) {
+      this.status = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case STATUS:
+      if (value == null) {
+        unsetStatus();
+      } else {
+        setStatus((org.apache.sentry.service.thrift.TSentryResponseStatus)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case STATUS:
+      return getStatus();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case STATUS:
+      return isSetStatus();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof TAlterSentryRoleGrantPrivilegeResponse)
+      return this.equals((TAlterSentryRoleGrantPrivilegeResponse)that);
+    return false;
+  }
+
+  public boolean equals(TAlterSentryRoleGrantPrivilegeResponse that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_status = true && this.isSetStatus();
+    boolean that_present_status = true && that.isSetStatus();
+    if (this_present_status || that_present_status) {
+      if (!(this_present_status && that_present_status))
+        return false;
+      if (!this.status.equals(that.status))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder builder = new HashCodeBuilder();
+
+    boolean present_status = true && (isSetStatus());
+    builder.append(present_status);
+    if (present_status)
+      builder.append(status);
+
+    return builder.toHashCode();
+  }
+
+  public int compareTo(TAlterSentryRoleGrantPrivilegeResponse other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    TAlterSentryRoleGrantPrivilegeResponse typedOther = (TAlterSentryRoleGrantPrivilegeResponse)other;
+
+    lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetStatus()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("TAlterSentryRoleGrantPrivilegeResponse(");
+    boolean first = true;
+
+    sb.append("status:");
+    if (this.status == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.status);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetStatus()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+    if (status != null) {
+      status.validate();
+    }
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class TAlterSentryRoleGrantPrivilegeResponseStandardSchemeFactory implements SchemeFactory {
+    public TAlterSentryRoleGrantPrivilegeResponseStandardScheme getScheme() {
+      return new TAlterSentryRoleGrantPrivilegeResponseStandardScheme();
+    }
+  }
+
+  private static class TAlterSentryRoleGrantPrivilegeResponseStandardScheme extends StandardScheme<TAlterSentryRoleGrantPrivilegeResponse> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, TAlterSentryRoleGrantPrivilegeResponse struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // STATUS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.status = new org.apache.sentry.service.thrift.TSentryResponseStatus();
+              struct.status.read(iprot);
+              struct.setStatusIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, TAlterSentryRoleGrantPrivilegeResponse struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.status != null) {
+        oprot.writeFieldBegin(STATUS_FIELD_DESC);
+        struct.status.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class TAlterSentryRoleGrantPrivilegeResponseTupleSchemeFactory implements SchemeFactory {
+    public TAlterSentryRoleGrantPrivilegeResponseTupleScheme getScheme() {
+      return new TAlterSentryRoleGrantPrivilegeResponseTupleScheme();
+    }
+  }
+
+  private static class TAlterSentryRoleGrantPrivilegeResponseTupleScheme extends TupleScheme<TAlterSentryRoleGrantPrivilegeResponse> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, TAlterSentryRoleGrantPrivilegeResponse struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      struct.status.write(oprot);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, TAlterSentryRoleGrantPrivilegeResponse struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.status = new org.apache.sentry.service.thrift.TSentryResponseStatus();
+      struct.status.read(iprot);
+      struct.setStatusIsSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7df761d/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TAlterSentryRoleRevokePrivilegeRequest.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TAlterSentryRoleRevokePrivilegeRequest.java b/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TAlterSentryRoleRevokePrivilegeRequest.java
new file mode 100644
index 0000000..43a6055
--- /dev/null
+++ b/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TAlterSentryRoleRevokePrivilegeRequest.java
@@ -0,0 +1,693 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.sentry.provider.db.service.thrift;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TAlterSentryRoleRevokePrivilegeRequest implements org.apache.thrift.TBase<TAlterSentryRoleRevokePrivilegeRequest, TAlterSentryRoleRevokePrivilegeRequest._Fields>, java.io.Serializable, Cloneable {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TAlterSentryRoleRevokePrivilegeRequest");
+
+  private static final org.apache.thrift.protocol.TField PROTOCOL_VERSION_FIELD_DESC = new org.apache.thrift.protocol.TField("protocol_version", org.apache.thrift.protocol.TType.I32, (short)1);
+  private static final org.apache.thrift.protocol.TField USER_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("userName", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField ROLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("roleName", org.apache.thrift.protocol.TType.STRING, (short)3);
+  private static final org.apache.thrift.protocol.TField PRIVILEGE_FIELD_DESC = new org.apache.thrift.protocol.TField("privilege", org.apache.thrift.protocol.TType.STRUCT, (short)4);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new TAlterSentryRoleRevokePrivilegeRequestStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new TAlterSentryRoleRevokePrivilegeRequestTupleSchemeFactory());
+  }
+
+  private int protocol_version; // required
+  private String userName; // required
+  private String roleName; // required
+  private TSentryPrivilege privilege; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    PROTOCOL_VERSION((short)1, "protocol_version"),
+    USER_NAME((short)2, "userName"),
+    ROLE_NAME((short)3, "roleName"),
+    PRIVILEGE((short)4, "privilege");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // PROTOCOL_VERSION
+          return PROTOCOL_VERSION;
+        case 2: // USER_NAME
+          return USER_NAME;
+        case 3: // ROLE_NAME
+          return ROLE_NAME;
+        case 4: // PRIVILEGE
+          return PRIVILEGE;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __PROTOCOL_VERSION_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.PROTOCOL_VERSION, new org.apache.thrift.meta_data.FieldMetaData("protocol_version", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.USER_NAME, new org.apache.thrift.meta_data.FieldMetaData("userName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.ROLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("roleName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.PRIVILEGE, new org.apache.thrift.meta_data.FieldMetaData("privilege", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSentryPrivilege.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TAlterSentryRoleRevokePrivilegeRequest.class, metaDataMap);
+  }
+
+  public TAlterSentryRoleRevokePrivilegeRequest() {
+    this.protocol_version = 1;
+
+  }
+
+  public TAlterSentryRoleRevokePrivilegeRequest(
+    int protocol_version,
+    String userName,
+    String roleName,
+    TSentryPrivilege privilege)
+  {
+    this();
+    this.protocol_version = protocol_version;
+    setProtocol_versionIsSet(true);
+    this.userName = userName;
+    this.roleName = roleName;
+    this.privilege = privilege;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public TAlterSentryRoleRevokePrivilegeRequest(TAlterSentryRoleRevokePrivilegeRequest other) {
+    __isset_bitfield = other.__isset_bitfield;
+    this.protocol_version = other.protocol_version;
+    if (other.isSetUserName()) {
+      this.userName = other.userName;
+    }
+    if (other.isSetRoleName()) {
+      this.roleName = other.roleName;
+    }
+    if (other.isSetPrivilege()) {
+      this.privilege = new TSentryPrivilege(other.privilege);
+    }
+  }
+
+  public TAlterSentryRoleRevokePrivilegeRequest deepCopy() {
+    return new TAlterSentryRoleRevokePrivilegeRequest(this);
+  }
+
+  @Override
+  public void clear() {
+    this.protocol_version = 1;
+
+    this.userName = null;
+    this.roleName = null;
+    this.privilege = null;
+  }
+
+  public int getProtocol_version() {
+    return this.protocol_version;
+  }
+
+  public void setProtocol_version(int protocol_version) {
+    this.protocol_version = protocol_version;
+    setProtocol_versionIsSet(true);
+  }
+
+  public void unsetProtocol_version() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __PROTOCOL_VERSION_ISSET_ID);
+  }
+
+  /** Returns true if field protocol_version is set (has been assigned a value) and false otherwise */
+  public boolean isSetProtocol_version() {
+    return EncodingUtils.testBit(__isset_bitfield, __PROTOCOL_VERSION_ISSET_ID);
+  }
+
+  public void setProtocol_versionIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __PROTOCOL_VERSION_ISSET_ID, value);
+  }
+
+  public String getUserName() {
+    return this.userName;
+  }
+
+  public void setUserName(String userName) {
+    this.userName = userName;
+  }
+
+  public void unsetUserName() {
+    this.userName = null;
+  }
+
+  /** Returns true if field userName is set (has been assigned a value) and false otherwise */
+  public boolean isSetUserName() {
+    return this.userName != null;
+  }
+
+  public void setUserNameIsSet(boolean value) {
+    if (!value) {
+      this.userName = null;
+    }
+  }
+
+  public String getRoleName() {
+    return this.roleName;
+  }
+
+  public void setRoleName(String roleName) {
+    this.roleName = roleName;
+  }
+
+  public void unsetRoleName() {
+    this.roleName = null;
+  }
+
+  /** Returns true if field roleName is set (has been assigned a value) and false otherwise */
+  public boolean isSetRoleName() {
+    return this.roleName != null;
+  }
+
+  public void setRoleNameIsSet(boolean value) {
+    if (!value) {
+      this.roleName = null;
+    }
+  }
+
+  public TSentryPrivilege getPrivilege() {
+    return this.privilege;
+  }
+
+  public void setPrivilege(TSentryPrivilege privilege) {
+    this.privilege = privilege;
+  }
+
+  public void unsetPrivilege() {
+    this.privilege = null;
+  }
+
+  /** Returns true if field privilege is set (has been assigned a value) and false otherwise */
+  public boolean isSetPrivilege() {
+    return this.privilege != null;
+  }
+
+  public void setPrivilegeIsSet(boolean value) {
+    if (!value) {
+      this.privilege = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case PROTOCOL_VERSION:
+      if (value == null) {
+        unsetProtocol_version();
+      } else {
+        setProtocol_version((Integer)value);
+      }
+      break;
+
+    case USER_NAME:
+      if (value == null) {
+        unsetUserName();
+      } else {
+        setUserName((String)value);
+      }
+      break;
+
+    case ROLE_NAME:
+      if (value == null) {
+        unsetRoleName();
+      } else {
+        setRoleName((String)value);
+      }
+      break;
+
+    case PRIVILEGE:
+      if (value == null) {
+        unsetPrivilege();
+      } else {
+        setPrivilege((TSentryPrivilege)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case PROTOCOL_VERSION:
+      return Integer.valueOf(getProtocol_version());
+
+    case USER_NAME:
+      return getUserName();
+
+    case ROLE_NAME:
+      return getRoleName();
+
+    case PRIVILEGE:
+      return getPrivilege();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case PROTOCOL_VERSION:
+      return isSetProtocol_version();
+    case USER_NAME:
+      return isSetUserName();
+    case ROLE_NAME:
+      return isSetRoleName();
+    case PRIVILEGE:
+      return isSetPrivilege();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof TAlterSentryRoleRevokePrivilegeRequest)
+      return this.equals((TAlterSentryRoleRevokePrivilegeRequest)that);
+    return false;
+  }
+
+  public boolean equals(TAlterSentryRoleRevokePrivilegeRequest that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_protocol_version = true;
+    boolean that_present_protocol_version = true;
+    if (this_present_protocol_version || that_present_protocol_version) {
+      if (!(this_present_protocol_version && that_present_protocol_version))
+        return false;
+      if (this.protocol_version != that.protocol_version)
+        return false;
+    }
+
+    boolean this_present_userName = true && this.isSetUserName();
+    boolean that_present_userName = true && that.isSetUserName();
+    if (this_present_userName || that_present_userName) {
+      if (!(this_present_userName && that_present_userName))
+        return false;
+      if (!this.userName.equals(that.userName))
+        return false;
+    }
+
+    boolean this_present_roleName = true && this.isSetRoleName();
+    boolean that_present_roleName = true && that.isSetRoleName();
+    if (this_present_roleName || that_present_roleName) {
+      if (!(this_present_roleName && that_present_roleName))
+        return false;
+      if (!this.roleName.equals(that.roleName))
+        return false;
+    }
+
+    boolean this_present_privilege = true && this.isSetPrivilege();
+    boolean that_present_privilege = true && that.isSetPrivilege();
+    if (this_present_privilege || that_present_privilege) {
+      if (!(this_present_privilege && that_present_privilege))
+        return false;
+      if (!this.privilege.equals(that.privilege))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder builder = new HashCodeBuilder();
+
+    boolean present_protocol_version = true;
+    builder.append(present_protocol_version);
+    if (present_protocol_version)
+      builder.append(protocol_version);
+
+    boolean present_userName = true && (isSetUserName());
+    builder.append(present_userName);
+    if (present_userName)
+      builder.append(userName);
+
+    boolean present_roleName = true && (isSetRoleName());
+    builder.append(present_roleName);
+    if (present_roleName)
+      builder.append(roleName);
+
+    boolean present_privilege = true && (isSetPrivilege());
+    builder.append(present_privilege);
+    if (present_privilege)
+      builder.append(privilege);
+
+    return builder.toHashCode();
+  }
+
+  public int compareTo(TAlterSentryRoleRevokePrivilegeRequest other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    TAlterSentryRoleRevokePrivilegeRequest typedOther = (TAlterSentryRoleRevokePrivilegeRequest)other;
+
+    lastComparison = Boolean.valueOf(isSetProtocol_version()).compareTo(typedOther.isSetProtocol_version());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetProtocol_version()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.protocol_version, typedOther.protocol_version);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetUserName()).compareTo(typedOther.isSetUserName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetUserName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.userName, typedOther.userName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetRoleName()).compareTo(typedOther.isSetRoleName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetRoleName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.roleName, typedOther.roleName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetPrivilege()).compareTo(typedOther.isSetPrivilege());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetPrivilege()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.privilege, typedOther.privilege);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("TAlterSentryRoleRevokePrivilegeRequest(");
+    boolean first = true;
+
+    sb.append("protocol_version:");
+    sb.append(this.protocol_version);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("userName:");
+    if (this.userName == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.userName);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("roleName:");
+    if (this.roleName == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.roleName);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("privilege:");
+    if (this.privilege == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.privilege);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetProtocol_version()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'protocol_version' is unset! Struct:" + toString());
+    }
+
+    if (!isSetUserName()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'userName' is unset! Struct:" + toString());
+    }
+
+    if (!isSetRoleName()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'roleName' is unset! Struct:" + toString());
+    }
+
+    if (!isSetPrivilege()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'privilege' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+    if (privilege != null) {
+      privilege.validate();
+    }
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class TAlterSentryRoleRevokePrivilegeRequestStandardSchemeFactory implements SchemeFactory {
+    public TAlterSentryRoleRevokePrivilegeRequestStandardScheme getScheme() {
+      return new TAlterSentryRoleRevokePrivilegeRequestStandardScheme();
+    }
+  }
+
+  private static class TAlterSentryRoleRevokePrivilegeRequestStandardScheme extends StandardScheme<TAlterSentryRoleRevokePrivilegeRequest> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, TAlterSentryRoleRevokePrivilegeRequest struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // PROTOCOL_VERSION
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.protocol_version = iprot.readI32();
+              struct.setProtocol_versionIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // USER_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.userName = iprot.readString();
+              struct.setUserNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // ROLE_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.roleName = iprot.readString();
+              struct.setRoleNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // PRIVILEGE
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.privilege = new TSentryPrivilege();
+              struct.privilege.read(iprot);
+              struct.setPrivilegeIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, TAlterSentryRoleRevokePrivilegeRequest struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      oprot.writeFieldBegin(PROTOCOL_VERSION_FIELD_DESC);
+      oprot.writeI32(struct.protocol_version);
+      oprot.writeFieldEnd();
+      if (struct.userName != null) {
+        oprot.writeFieldBegin(USER_NAME_FIELD_DESC);
+        oprot.writeString(struct.userName);
+        oprot.writeFieldEnd();
+      }
+      if (struct.roleName != null) {
+        oprot.writeFieldBegin(ROLE_NAME_FIELD_DESC);
+        oprot.writeString(struct.roleName);
+        oprot.writeFieldEnd();
+      }
+      if (struct.privilege != null) {
+        oprot.writeFieldBegin(PRIVILEGE_FIELD_DESC);
+        struct.privilege.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class TAlterSentryRoleRevokePrivilegeRequestTupleSchemeFactory implements SchemeFactory {
+    public TAlterSentryRoleRevokePrivilegeRequestTupleScheme getScheme() {
+      return new TAlterSentryRoleRevokePrivilegeRequestTupleScheme();
+    }
+  }
+
+  private static class TAlterSentryRoleRevokePrivilegeRequestTupleScheme extends TupleScheme<TAlterSentryRoleRevokePrivilegeRequest> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, TAlterSentryRoleRevokePrivilegeRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeI32(struct.protocol_version);
+      oprot.writeString(struct.userName);
+      oprot.writeString(struct.roleName);
+      struct.privilege.write(oprot);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, TAlterSentryRoleRevokePrivilegeRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.protocol_version = iprot.readI32();
+      struct.setProtocol_versionIsSet(true);
+      struct.userName = iprot.readString();
+      struct.setUserNameIsSet(true);
+      struct.roleName = iprot.readString();
+      struct.setRoleNameIsSet(true);
+      struct.privilege = new TSentryPrivilege();
+      struct.privilege.read(iprot);
+      struct.setPrivilegeIsSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7df761d/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TAlterSentryRoleRevokePrivilegeResponse.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TAlterSentryRoleRevokePrivilegeResponse.java b/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TAlterSentryRoleRevokePrivilegeResponse.java
new file mode 100644
index 0000000..d431e37
--- /dev/null
+++ b/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TAlterSentryRoleRevokePrivilegeResponse.java
@@ -0,0 +1,390 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.sentry.provider.db.service.thrift;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TAlterSentryRoleRevokePrivilegeResponse implements org.apache.thrift.TBase<TAlterSentryRoleRevokePrivilegeResponse, TAlterSentryRoleRevokePrivilegeResponse._Fields>, java.io.Serializable, Cloneable {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TAlterSentryRoleRevokePrivilegeResponse");
+
+  private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new TAlterSentryRoleRevokePrivilegeResponseStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new TAlterSentryRoleRevokePrivilegeResponseTupleSchemeFactory());
+  }
+
+  private org.apache.sentry.service.thrift.TSentryResponseStatus status; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    STATUS((short)1, "status");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // STATUS
+          return STATUS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.sentry.service.thrift.TSentryResponseStatus.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TAlterSentryRoleRevokePrivilegeResponse.class, metaDataMap);
+  }
+
+  public TAlterSentryRoleRevokePrivilegeResponse() {
+  }
+
+  public TAlterSentryRoleRevokePrivilegeResponse(
+    org.apache.sentry.service.thrift.TSentryResponseStatus status)
+  {
+    this();
+    this.status = status;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public TAlterSentryRoleRevokePrivilegeResponse(TAlterSentryRoleRevokePrivilegeResponse other) {
+    if (other.isSetStatus()) {
+      this.status = new org.apache.sentry.service.thrift.TSentryResponseStatus(other.status);
+    }
+  }
+
+  public TAlterSentryRoleRevokePrivilegeResponse deepCopy() {
+    return new TAlterSentryRoleRevokePrivilegeResponse(this);
+  }
+
+  @Override
+  public void clear() {
+    this.status = null;
+  }
+
+  public org.apache.sentry.service.thrift.TSentryResponseStatus getStatus() {
+    return this.status;
+  }
+
+  public void setStatus(org.apache.sentry.service.thrift.TSentryResponseStatus status) {
+    this.status = status;
+  }
+
+  public void unsetStatus() {
+    this.status = null;
+  }
+
+  /** Returns true if field status is set (has been assigned a value) and false otherwise */
+  public boolean isSetStatus() {
+    return this.status != null;
+  }
+
+  public void setStatusIsSet(boolean value) {
+    if (!value) {
+      this.status = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case STATUS:
+      if (value == null) {
+        unsetStatus();
+      } else {
+        setStatus((org.apache.sentry.service.thrift.TSentryResponseStatus)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case STATUS:
+      return getStatus();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case STATUS:
+      return isSetStatus();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof TAlterSentryRoleRevokePrivilegeResponse)
+      return this.equals((TAlterSentryRoleRevokePrivilegeResponse)that);
+    return false;
+  }
+
+  public boolean equals(TAlterSentryRoleRevokePrivilegeResponse that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_status = true && this.isSetStatus();
+    boolean that_present_status = true && that.isSetStatus();
+    if (this_present_status || that_present_status) {
+      if (!(this_present_status && that_present_status))
+        return false;
+      if (!this.status.equals(that.status))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder builder = new HashCodeBuilder();
+
+    boolean present_status = true && (isSetStatus());
+    builder.append(present_status);
+    if (present_status)
+      builder.append(status);
+
+    return builder.toHashCode();
+  }
+
+  public int compareTo(TAlterSentryRoleRevokePrivilegeResponse other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    TAlterSentryRoleRevokePrivilegeResponse typedOther = (TAlterSentryRoleRevokePrivilegeResponse)other;
+
+    lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetStatus()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("TAlterSentryRoleRevokePrivilegeResponse(");
+    boolean first = true;
+
+    sb.append("status:");
+    if (this.status == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.status);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetStatus()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+    if (status != null) {
+      status.validate();
+    }
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class TAlterSentryRoleRevokePrivilegeResponseStandardSchemeFactory implements SchemeFactory {
+    public TAlterSentryRoleRevokePrivilegeResponseStandardScheme getScheme() {
+      return new TAlterSentryRoleRevokePrivilegeResponseStandardScheme();
+    }
+  }
+
+  private static class TAlterSentryRoleRevokePrivilegeResponseStandardScheme extends StandardScheme<TAlterSentryRoleRevokePrivilegeResponse> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, TAlterSentryRoleRevokePrivilegeResponse struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // STATUS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.status = new org.apache.sentry.service.thrift.TSentryResponseStatus();
+              struct.status.read(iprot);
+              struct.setStatusIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, TAlterSentryRoleRevokePrivilegeResponse struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.status != null) {
+        oprot.writeFieldBegin(STATUS_FIELD_DESC);
+        struct.status.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class TAlterSentryRoleRevokePrivilegeResponseTupleSchemeFactory implements SchemeFactory {
+    public TAlterSentryRoleRevokePrivilegeResponseTupleScheme getScheme() {
+      return new TAlterSentryRoleRevokePrivilegeResponseTupleScheme();
+    }
+  }
+
+  private static class TAlterSentryRoleRevokePrivilegeResponseTupleScheme extends TupleScheme<TAlterSentryRoleRevokePrivilegeResponse> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, TAlterSentryRoleRevokePrivilegeResponse struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      struct.status.write(oprot);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, TAlterSentryRoleRevokePrivilegeResponse struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.status = new org.apache.sentry.service.thrift.TSentryResponseStatus();
+      struct.status.read(iprot);
+      struct.setStatusIsSet(true);
+    }
+  }
+
+}
+


[4/4] git commit: SENTRY-126 - Implement alter role grant/revoke privilege in sentry service and sentry store (Shreepadma via Brock)

Posted by br...@apache.org.
SENTRY-126 - Implement alter role grant/revoke privilege in sentry service and sentry store (Shreepadma via Brock)


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

Branch: refs/heads/db_policy_store
Commit: a7df761ddf7bff8baae6b43c2b1c3502179dbb0f
Parents: e18a902
Author: Brock Noland <br...@apache.org>
Authored: Tue Mar 11 17:29:11 2014 -0500
Committer: Brock Noland <br...@apache.org>
Committed: Tue Mar 11 17:29:40 2014 -0500

----------------------------------------------------------------------
 .../db/service/thrift/SentryPolicyService.java  | 1014 ++++++++++++++++--
 .../TAlterSentryRoleAddGroupsRequest.java       |   36 +-
 .../TAlterSentryRoleGrantPrivilegeRequest.java  |  693 ++++++++++++
 .../TAlterSentryRoleGrantPrivilegeResponse.java |  390 +++++++
 .../TAlterSentryRoleRevokePrivilegeRequest.java |  693 ++++++++++++
 ...TAlterSentryRoleRevokePrivilegeResponse.java |  390 +++++++
 .../thrift/TCreateSentryPrivilegeRequest.java   |  592 ----------
 .../thrift/TCreateSentryPrivilegeResponse.java  |  390 -------
 .../thrift/TListSentryRolesResponse.java        |   36 +-
 .../db/service/thrift/TSentryPrivilege.java     |  112 +-
 .../thrift/sentry_common_serviceConstants.java  |    2 +
 .../db/service/model/MSentryPrivilege.java      |   16 +-
 .../provider/db/service/model/MSentryRole.java  |   28 +-
 .../persistent/SentryInvalidInputException.java |   27 +
 .../db/service/persistent/SentryStore.java      |  279 +++--
 .../db/service/thrift/NotificationHandler.java  |   17 +-
 .../thrift/NotificationHandlerInvoker.java      |   58 +-
 .../thrift/SentryPolicyServiceClient.java       |   42 +-
 .../thrift/SentryPolicyStoreProcessor.java      |  113 +-
 .../apache/sentry/service/thrift/Status.java    |    4 +
 .../main/resources/sentry_common_service.thrift |    1 +
 .../main/resources/sentry_policy_service.thrift |   52 +-
 .../thrift/TestNotificationHandlerInvoker.java  |   25 +-
 .../thrift/TestSentryServiceIntegration.java    |   67 +-
 24 files changed, 3687 insertions(+), 1390 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7df761d/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/SentryPolicyService.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/SentryPolicyService.java b/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/SentryPolicyService.java
index 04e6655..6f02595 100644
--- a/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/SentryPolicyService.java
+++ b/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/SentryPolicyService.java
@@ -39,7 +39,9 @@ public class SentryPolicyService {
 
     public TDropSentryRoleResponse drop_sentry_role(TDropSentryRoleRequest request) throws org.apache.thrift.TException;
 
-    public TCreateSentryPrivilegeResponse create_sentry_privilege(TCreateSentryPrivilegeRequest request) throws org.apache.thrift.TException;
+    public TAlterSentryRoleGrantPrivilegeResponse alter_sentry_role_grant_privilege(TAlterSentryRoleGrantPrivilegeRequest request) throws org.apache.thrift.TException;
+
+    public TAlterSentryRoleRevokePrivilegeResponse alter_sentry_role_revoke_privilege(TAlterSentryRoleRevokePrivilegeRequest request) throws org.apache.thrift.TException;
 
     public TAlterSentryRoleAddGroupsResponse alter_sentry_role_add_groups(TAlterSentryRoleAddGroupsRequest request) throws org.apache.thrift.TException;
 
@@ -57,7 +59,9 @@ public class SentryPolicyService {
 
     public void drop_sentry_role(TDropSentryRoleRequest request, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.drop_sentry_role_call> resultHandler) throws org.apache.thrift.TException;
 
-    public void create_sentry_privilege(TCreateSentryPrivilegeRequest request, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.create_sentry_privilege_call> resultHandler) throws org.apache.thrift.TException;
+    public void alter_sentry_role_grant_privilege(TAlterSentryRoleGrantPrivilegeRequest request, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.alter_sentry_role_grant_privilege_call> resultHandler) throws org.apache.thrift.TException;
+
+    public void alter_sentry_role_revoke_privilege(TAlterSentryRoleRevokePrivilegeRequest request, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.alter_sentry_role_revoke_privilege_call> resultHandler) throws org.apache.thrift.TException;
 
     public void alter_sentry_role_add_groups(TAlterSentryRoleAddGroupsRequest request, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.alter_sentry_role_add_groups_call> resultHandler) throws org.apache.thrift.TException;
 
@@ -135,27 +139,50 @@ public class SentryPolicyService {
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "drop_sentry_role failed: unknown result");
     }
 
-    public TCreateSentryPrivilegeResponse create_sentry_privilege(TCreateSentryPrivilegeRequest request) throws org.apache.thrift.TException
+    public TAlterSentryRoleGrantPrivilegeResponse alter_sentry_role_grant_privilege(TAlterSentryRoleGrantPrivilegeRequest request) throws org.apache.thrift.TException
+    {
+      send_alter_sentry_role_grant_privilege(request);
+      return recv_alter_sentry_role_grant_privilege();
+    }
+
+    public void send_alter_sentry_role_grant_privilege(TAlterSentryRoleGrantPrivilegeRequest request) throws org.apache.thrift.TException
+    {
+      alter_sentry_role_grant_privilege_args args = new alter_sentry_role_grant_privilege_args();
+      args.setRequest(request);
+      sendBase("alter_sentry_role_grant_privilege", args);
+    }
+
+    public TAlterSentryRoleGrantPrivilegeResponse recv_alter_sentry_role_grant_privilege() throws org.apache.thrift.TException
+    {
+      alter_sentry_role_grant_privilege_result result = new alter_sentry_role_grant_privilege_result();
+      receiveBase(result, "alter_sentry_role_grant_privilege");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "alter_sentry_role_grant_privilege failed: unknown result");
+    }
+
+    public TAlterSentryRoleRevokePrivilegeResponse alter_sentry_role_revoke_privilege(TAlterSentryRoleRevokePrivilegeRequest request) throws org.apache.thrift.TException
     {
-      send_create_sentry_privilege(request);
-      return recv_create_sentry_privilege();
+      send_alter_sentry_role_revoke_privilege(request);
+      return recv_alter_sentry_role_revoke_privilege();
     }
 
-    public void send_create_sentry_privilege(TCreateSentryPrivilegeRequest request) throws org.apache.thrift.TException
+    public void send_alter_sentry_role_revoke_privilege(TAlterSentryRoleRevokePrivilegeRequest request) throws org.apache.thrift.TException
     {
-      create_sentry_privilege_args args = new create_sentry_privilege_args();
+      alter_sentry_role_revoke_privilege_args args = new alter_sentry_role_revoke_privilege_args();
       args.setRequest(request);
-      sendBase("create_sentry_privilege", args);
+      sendBase("alter_sentry_role_revoke_privilege", args);
     }
 
-    public TCreateSentryPrivilegeResponse recv_create_sentry_privilege() throws org.apache.thrift.TException
+    public TAlterSentryRoleRevokePrivilegeResponse recv_alter_sentry_role_revoke_privilege() throws org.apache.thrift.TException
     {
-      create_sentry_privilege_result result = new create_sentry_privilege_result();
-      receiveBase(result, "create_sentry_privilege");
+      alter_sentry_role_revoke_privilege_result result = new alter_sentry_role_revoke_privilege_result();
+      receiveBase(result, "alter_sentry_role_revoke_privilege");
       if (result.isSetSuccess()) {
         return result.success;
       }
-      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "create_sentry_privilege failed: unknown result");
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "alter_sentry_role_revoke_privilege failed: unknown result");
     }
 
     public TAlterSentryRoleAddGroupsResponse alter_sentry_role_add_groups(TAlterSentryRoleAddGroupsRequest request) throws org.apache.thrift.TException
@@ -332,35 +359,67 @@ public class SentryPolicyService {
       }
     }
 
-    public void create_sentry_privilege(TCreateSentryPrivilegeRequest request, org.apache.thrift.async.AsyncMethodCallback<create_sentry_privilege_call> resultHandler) throws org.apache.thrift.TException {
+    public void alter_sentry_role_grant_privilege(TAlterSentryRoleGrantPrivilegeRequest request, org.apache.thrift.async.AsyncMethodCallback<alter_sentry_role_grant_privilege_call> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      alter_sentry_role_grant_privilege_call method_call = new alter_sentry_role_grant_privilege_call(request, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class alter_sentry_role_grant_privilege_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private TAlterSentryRoleGrantPrivilegeRequest request;
+      public alter_sentry_role_grant_privilege_call(TAlterSentryRoleGrantPrivilegeRequest request, org.apache.thrift.async.AsyncMethodCallback<alter_sentry_role_grant_privilege_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.request = request;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("alter_sentry_role_grant_privilege", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        alter_sentry_role_grant_privilege_args args = new alter_sentry_role_grant_privilege_args();
+        args.setRequest(request);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public TAlterSentryRoleGrantPrivilegeResponse getResult() throws org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_alter_sentry_role_grant_privilege();
+      }
+    }
+
+    public void alter_sentry_role_revoke_privilege(TAlterSentryRoleRevokePrivilegeRequest request, org.apache.thrift.async.AsyncMethodCallback<alter_sentry_role_revoke_privilege_call> resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      create_sentry_privilege_call method_call = new create_sentry_privilege_call(request, resultHandler, this, ___protocolFactory, ___transport);
+      alter_sentry_role_revoke_privilege_call method_call = new alter_sentry_role_revoke_privilege_call(request, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class create_sentry_privilege_call extends org.apache.thrift.async.TAsyncMethodCall {
-      private TCreateSentryPrivilegeRequest request;
-      public create_sentry_privilege_call(TCreateSentryPrivilegeRequest request, org.apache.thrift.async.AsyncMethodCallback<create_sentry_privilege_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+    public static class alter_sentry_role_revoke_privilege_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private TAlterSentryRoleRevokePrivilegeRequest request;
+      public alter_sentry_role_revoke_privilege_call(TAlterSentryRoleRevokePrivilegeRequest request, org.apache.thrift.async.AsyncMethodCallback<alter_sentry_role_revoke_privilege_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
         this.request = request;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("create_sentry_privilege", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        create_sentry_privilege_args args = new create_sentry_privilege_args();
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("alter_sentry_role_revoke_privilege", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        alter_sentry_role_revoke_privilege_args args = new alter_sentry_role_revoke_privilege_args();
         args.setRequest(request);
         args.write(prot);
         prot.writeMessageEnd();
       }
 
-      public TCreateSentryPrivilegeResponse getResult() throws org.apache.thrift.TException {
+      public TAlterSentryRoleRevokePrivilegeResponse getResult() throws org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new IllegalStateException("Method call not finished!");
         }
         org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
         org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_create_sentry_privilege();
+        return (new Client(prot)).recv_alter_sentry_role_revoke_privilege();
       }
     }
 
@@ -507,7 +566,8 @@ public class SentryPolicyService {
     private static <I extends Iface> Map<String,  org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> getProcessMap(Map<String,  org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> processMap) {
       processMap.put("create_sentry_role", new create_sentry_role());
       processMap.put("drop_sentry_role", new drop_sentry_role());
-      processMap.put("create_sentry_privilege", new create_sentry_privilege());
+      processMap.put("alter_sentry_role_grant_privilege", new alter_sentry_role_grant_privilege());
+      processMap.put("alter_sentry_role_revoke_privilege", new alter_sentry_role_revoke_privilege());
       processMap.put("alter_sentry_role_add_groups", new alter_sentry_role_add_groups());
       processMap.put("alter_sentry_role_delete_groups", new alter_sentry_role_delete_groups());
       processMap.put("list_sentry_roles_by_group", new list_sentry_roles_by_group());
@@ -555,22 +615,42 @@ public class SentryPolicyService {
       }
     }
 
-    public static class create_sentry_privilege<I extends Iface> extends org.apache.thrift.ProcessFunction<I, create_sentry_privilege_args> {
-      public create_sentry_privilege() {
-        super("create_sentry_privilege");
+    public static class alter_sentry_role_grant_privilege<I extends Iface> extends org.apache.thrift.ProcessFunction<I, alter_sentry_role_grant_privilege_args> {
+      public alter_sentry_role_grant_privilege() {
+        super("alter_sentry_role_grant_privilege");
+      }
+
+      public alter_sentry_role_grant_privilege_args getEmptyArgsInstance() {
+        return new alter_sentry_role_grant_privilege_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public alter_sentry_role_grant_privilege_result getResult(I iface, alter_sentry_role_grant_privilege_args args) throws org.apache.thrift.TException {
+        alter_sentry_role_grant_privilege_result result = new alter_sentry_role_grant_privilege_result();
+        result.success = iface.alter_sentry_role_grant_privilege(args.request);
+        return result;
+      }
+    }
+
+    public static class alter_sentry_role_revoke_privilege<I extends Iface> extends org.apache.thrift.ProcessFunction<I, alter_sentry_role_revoke_privilege_args> {
+      public alter_sentry_role_revoke_privilege() {
+        super("alter_sentry_role_revoke_privilege");
       }
 
-      public create_sentry_privilege_args getEmptyArgsInstance() {
-        return new create_sentry_privilege_args();
+      public alter_sentry_role_revoke_privilege_args getEmptyArgsInstance() {
+        return new alter_sentry_role_revoke_privilege_args();
       }
 
       protected boolean isOneway() {
         return false;
       }
 
-      public create_sentry_privilege_result getResult(I iface, create_sentry_privilege_args args) throws org.apache.thrift.TException {
-        create_sentry_privilege_result result = new create_sentry_privilege_result();
-        result.success = iface.create_sentry_privilege(args.request);
+      public alter_sentry_role_revoke_privilege_result getResult(I iface, alter_sentry_role_revoke_privilege_args args) throws org.apache.thrift.TException {
+        alter_sentry_role_revoke_privilege_result result = new alter_sentry_role_revoke_privilege_result();
+        result.success = iface.alter_sentry_role_revoke_privilege(args.request);
         return result;
       }
     }
@@ -2109,18 +2189,744 @@ public class SentryPolicyService {
 
   }
 
-  public static class create_sentry_privilege_args implements org.apache.thrift.TBase<create_sentry_privilege_args, create_sentry_privilege_args._Fields>, java.io.Serializable, Cloneable   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("create_sentry_privilege_args");
+  public static class alter_sentry_role_grant_privilege_args implements org.apache.thrift.TBase<alter_sentry_role_grant_privilege_args, alter_sentry_role_grant_privilege_args._Fields>, java.io.Serializable, Cloneable   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("alter_sentry_role_grant_privilege_args");
+
+    private static final org.apache.thrift.protocol.TField REQUEST_FIELD_DESC = new org.apache.thrift.protocol.TField("request", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new alter_sentry_role_grant_privilege_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new alter_sentry_role_grant_privilege_argsTupleSchemeFactory());
+    }
+
+    private TAlterSentryRoleGrantPrivilegeRequest request; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      REQUEST((short)1, "request");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // REQUEST
+            return REQUEST;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.REQUEST, new org.apache.thrift.meta_data.FieldMetaData("request", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TAlterSentryRoleGrantPrivilegeRequest.class)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(alter_sentry_role_grant_privilege_args.class, metaDataMap);
+    }
+
+    public alter_sentry_role_grant_privilege_args() {
+    }
+
+    public alter_sentry_role_grant_privilege_args(
+      TAlterSentryRoleGrantPrivilegeRequest request)
+    {
+      this();
+      this.request = request;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public alter_sentry_role_grant_privilege_args(alter_sentry_role_grant_privilege_args other) {
+      if (other.isSetRequest()) {
+        this.request = new TAlterSentryRoleGrantPrivilegeRequest(other.request);
+      }
+    }
+
+    public alter_sentry_role_grant_privilege_args deepCopy() {
+      return new alter_sentry_role_grant_privilege_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.request = null;
+    }
+
+    public TAlterSentryRoleGrantPrivilegeRequest getRequest() {
+      return this.request;
+    }
+
+    public void setRequest(TAlterSentryRoleGrantPrivilegeRequest request) {
+      this.request = request;
+    }
+
+    public void unsetRequest() {
+      this.request = null;
+    }
+
+    /** Returns true if field request is set (has been assigned a value) and false otherwise */
+    public boolean isSetRequest() {
+      return this.request != null;
+    }
+
+    public void setRequestIsSet(boolean value) {
+      if (!value) {
+        this.request = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case REQUEST:
+        if (value == null) {
+          unsetRequest();
+        } else {
+          setRequest((TAlterSentryRoleGrantPrivilegeRequest)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case REQUEST:
+        return getRequest();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case REQUEST:
+        return isSetRequest();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof alter_sentry_role_grant_privilege_args)
+        return this.equals((alter_sentry_role_grant_privilege_args)that);
+      return false;
+    }
+
+    public boolean equals(alter_sentry_role_grant_privilege_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_request = true && this.isSetRequest();
+      boolean that_present_request = true && that.isSetRequest();
+      if (this_present_request || that_present_request) {
+        if (!(this_present_request && that_present_request))
+          return false;
+        if (!this.request.equals(that.request))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      HashCodeBuilder builder = new HashCodeBuilder();
+
+      boolean present_request = true && (isSetRequest());
+      builder.append(present_request);
+      if (present_request)
+        builder.append(request);
+
+      return builder.toHashCode();
+    }
+
+    public int compareTo(alter_sentry_role_grant_privilege_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      alter_sentry_role_grant_privilege_args typedOther = (alter_sentry_role_grant_privilege_args)other;
+
+      lastComparison = Boolean.valueOf(isSetRequest()).compareTo(typedOther.isSetRequest());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetRequest()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.request, typedOther.request);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("alter_sentry_role_grant_privilege_args(");
+      boolean first = true;
+
+      sb.append("request:");
+      if (this.request == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.request);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (request != null) {
+        request.validate();
+      }
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class alter_sentry_role_grant_privilege_argsStandardSchemeFactory implements SchemeFactory {
+      public alter_sentry_role_grant_privilege_argsStandardScheme getScheme() {
+        return new alter_sentry_role_grant_privilege_argsStandardScheme();
+      }
+    }
+
+    private static class alter_sentry_role_grant_privilege_argsStandardScheme extends StandardScheme<alter_sentry_role_grant_privilege_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, alter_sentry_role_grant_privilege_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // REQUEST
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.request = new TAlterSentryRoleGrantPrivilegeRequest();
+                struct.request.read(iprot);
+                struct.setRequestIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, alter_sentry_role_grant_privilege_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.request != null) {
+          oprot.writeFieldBegin(REQUEST_FIELD_DESC);
+          struct.request.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class alter_sentry_role_grant_privilege_argsTupleSchemeFactory implements SchemeFactory {
+      public alter_sentry_role_grant_privilege_argsTupleScheme getScheme() {
+        return new alter_sentry_role_grant_privilege_argsTupleScheme();
+      }
+    }
+
+    private static class alter_sentry_role_grant_privilege_argsTupleScheme extends TupleScheme<alter_sentry_role_grant_privilege_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, alter_sentry_role_grant_privilege_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetRequest()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetRequest()) {
+          struct.request.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, alter_sentry_role_grant_privilege_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.request = new TAlterSentryRoleGrantPrivilegeRequest();
+          struct.request.read(iprot);
+          struct.setRequestIsSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class alter_sentry_role_grant_privilege_result implements org.apache.thrift.TBase<alter_sentry_role_grant_privilege_result, alter_sentry_role_grant_privilege_result._Fields>, java.io.Serializable, Cloneable   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("alter_sentry_role_grant_privilege_result");
+
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new alter_sentry_role_grant_privilege_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new alter_sentry_role_grant_privilege_resultTupleSchemeFactory());
+    }
+
+    private TAlterSentryRoleGrantPrivilegeResponse success; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      SUCCESS((short)0, "success");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TAlterSentryRoleGrantPrivilegeResponse.class)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(alter_sentry_role_grant_privilege_result.class, metaDataMap);
+    }
+
+    public alter_sentry_role_grant_privilege_result() {
+    }
+
+    public alter_sentry_role_grant_privilege_result(
+      TAlterSentryRoleGrantPrivilegeResponse success)
+    {
+      this();
+      this.success = success;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public alter_sentry_role_grant_privilege_result(alter_sentry_role_grant_privilege_result other) {
+      if (other.isSetSuccess()) {
+        this.success = new TAlterSentryRoleGrantPrivilegeResponse(other.success);
+      }
+    }
+
+    public alter_sentry_role_grant_privilege_result deepCopy() {
+      return new alter_sentry_role_grant_privilege_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+    }
+
+    public TAlterSentryRoleGrantPrivilegeResponse getSuccess() {
+      return this.success;
+    }
+
+    public void setSuccess(TAlterSentryRoleGrantPrivilegeResponse success) {
+      this.success = success;
+    }
+
+    public void unsetSuccess() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean isSetSuccess() {
+      return this.success != null;
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((TAlterSentryRoleGrantPrivilegeResponse)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return getSuccess();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof alter_sentry_role_grant_privilege_result)
+        return this.equals((alter_sentry_role_grant_privilege_result)that);
+      return false;
+    }
+
+    public boolean equals(alter_sentry_role_grant_privilege_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true && this.isSetSuccess();
+      boolean that_present_success = true && that.isSetSuccess();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      HashCodeBuilder builder = new HashCodeBuilder();
+
+      boolean present_success = true && (isSetSuccess());
+      builder.append(present_success);
+      if (present_success)
+        builder.append(success);
+
+      return builder.toHashCode();
+    }
+
+    public int compareTo(alter_sentry_role_grant_privilege_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      alter_sentry_role_grant_privilege_result typedOther = (alter_sentry_role_grant_privilege_result)other;
+
+      lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+      }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("alter_sentry_role_grant_privilege_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (success != null) {
+        success.validate();
+      }
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class alter_sentry_role_grant_privilege_resultStandardSchemeFactory implements SchemeFactory {
+      public alter_sentry_role_grant_privilege_resultStandardScheme getScheme() {
+        return new alter_sentry_role_grant_privilege_resultStandardScheme();
+      }
+    }
+
+    private static class alter_sentry_role_grant_privilege_resultStandardScheme extends StandardScheme<alter_sentry_role_grant_privilege_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, alter_sentry_role_grant_privilege_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.success = new TAlterSentryRoleGrantPrivilegeResponse();
+                struct.success.read(iprot);
+                struct.setSuccessIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, alter_sentry_role_grant_privilege_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.success != null) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          struct.success.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class alter_sentry_role_grant_privilege_resultTupleSchemeFactory implements SchemeFactory {
+      public alter_sentry_role_grant_privilege_resultTupleScheme getScheme() {
+        return new alter_sentry_role_grant_privilege_resultTupleScheme();
+      }
+    }
+
+    private static class alter_sentry_role_grant_privilege_resultTupleScheme extends TupleScheme<alter_sentry_role_grant_privilege_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, alter_sentry_role_grant_privilege_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetSuccess()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetSuccess()) {
+          struct.success.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, alter_sentry_role_grant_privilege_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.success = new TAlterSentryRoleGrantPrivilegeResponse();
+          struct.success.read(iprot);
+          struct.setSuccessIsSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class alter_sentry_role_revoke_privilege_args implements org.apache.thrift.TBase<alter_sentry_role_revoke_privilege_args, alter_sentry_role_revoke_privilege_args._Fields>, java.io.Serializable, Cloneable   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("alter_sentry_role_revoke_privilege_args");
 
     private static final org.apache.thrift.protocol.TField REQUEST_FIELD_DESC = new org.apache.thrift.protocol.TField("request", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new create_sentry_privilege_argsStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new create_sentry_privilege_argsTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new alter_sentry_role_revoke_privilege_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new alter_sentry_role_revoke_privilege_argsTupleSchemeFactory());
     }
 
-    private TCreateSentryPrivilegeRequest request; // required
+    private TAlterSentryRoleRevokePrivilegeRequest request; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -2185,16 +2991,16 @@ public class SentryPolicyService {
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
       tmpMap.put(_Fields.REQUEST, new org.apache.thrift.meta_data.FieldMetaData("request", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCreateSentryPrivilegeRequest.class)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TAlterSentryRoleRevokePrivilegeRequest.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(create_sentry_privilege_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(alter_sentry_role_revoke_privilege_args.class, metaDataMap);
     }
 
-    public create_sentry_privilege_args() {
+    public alter_sentry_role_revoke_privilege_args() {
     }
 
-    public create_sentry_privilege_args(
-      TCreateSentryPrivilegeRequest request)
+    public alter_sentry_role_revoke_privilege_args(
+      TAlterSentryRoleRevokePrivilegeRequest request)
     {
       this();
       this.request = request;
@@ -2203,14 +3009,14 @@ public class SentryPolicyService {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public create_sentry_privilege_args(create_sentry_privilege_args other) {
+    public alter_sentry_role_revoke_privilege_args(alter_sentry_role_revoke_privilege_args other) {
       if (other.isSetRequest()) {
-        this.request = new TCreateSentryPrivilegeRequest(other.request);
+        this.request = new TAlterSentryRoleRevokePrivilegeRequest(other.request);
       }
     }
 
-    public create_sentry_privilege_args deepCopy() {
-      return new create_sentry_privilege_args(this);
+    public alter_sentry_role_revoke_privilege_args deepCopy() {
+      return new alter_sentry_role_revoke_privilege_args(this);
     }
 
     @Override
@@ -2218,11 +3024,11 @@ public class SentryPolicyService {
       this.request = null;
     }
 
-    public TCreateSentryPrivilegeRequest getRequest() {
+    public TAlterSentryRoleRevokePrivilegeRequest getRequest() {
       return this.request;
     }
 
-    public void setRequest(TCreateSentryPrivilegeRequest request) {
+    public void setRequest(TAlterSentryRoleRevokePrivilegeRequest request) {
       this.request = request;
     }
 
@@ -2247,7 +3053,7 @@ public class SentryPolicyService {
         if (value == null) {
           unsetRequest();
         } else {
-          setRequest((TCreateSentryPrivilegeRequest)value);
+          setRequest((TAlterSentryRoleRevokePrivilegeRequest)value);
         }
         break;
 
@@ -2280,12 +3086,12 @@ public class SentryPolicyService {
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof create_sentry_privilege_args)
-        return this.equals((create_sentry_privilege_args)that);
+      if (that instanceof alter_sentry_role_revoke_privilege_args)
+        return this.equals((alter_sentry_role_revoke_privilege_args)that);
       return false;
     }
 
-    public boolean equals(create_sentry_privilege_args that) {
+    public boolean equals(alter_sentry_role_revoke_privilege_args that) {
       if (that == null)
         return false;
 
@@ -2313,13 +3119,13 @@ public class SentryPolicyService {
       return builder.toHashCode();
     }
 
-    public int compareTo(create_sentry_privilege_args other) {
+    public int compareTo(alter_sentry_role_revoke_privilege_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
 
       int lastComparison = 0;
-      create_sentry_privilege_args typedOther = (create_sentry_privilege_args)other;
+      alter_sentry_role_revoke_privilege_args typedOther = (alter_sentry_role_revoke_privilege_args)other;
 
       lastComparison = Boolean.valueOf(isSetRequest()).compareTo(typedOther.isSetRequest());
       if (lastComparison != 0) {
@@ -2348,7 +3154,7 @@ public class SentryPolicyService {
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("create_sentry_privilege_args(");
+      StringBuilder sb = new StringBuilder("alter_sentry_role_revoke_privilege_args(");
       boolean first = true;
 
       sb.append("request:");
@@ -2386,15 +3192,15 @@ public class SentryPolicyService {
       }
     }
 
-    private static class create_sentry_privilege_argsStandardSchemeFactory implements SchemeFactory {
-      public create_sentry_privilege_argsStandardScheme getScheme() {
-        return new create_sentry_privilege_argsStandardScheme();
+    private static class alter_sentry_role_revoke_privilege_argsStandardSchemeFactory implements SchemeFactory {
+      public alter_sentry_role_revoke_privilege_argsStandardScheme getScheme() {
+        return new alter_sentry_role_revoke_privilege_argsStandardScheme();
       }
     }
 
-    private static class create_sentry_privilege_argsStandardScheme extends StandardScheme<create_sentry_privilege_args> {
+    private static class alter_sentry_role_revoke_privilege_argsStandardScheme extends StandardScheme<alter_sentry_role_revoke_privilege_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, create_sentry_privilege_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, alter_sentry_role_revoke_privilege_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -2406,7 +3212,7 @@ public class SentryPolicyService {
           switch (schemeField.id) {
             case 1: // REQUEST
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.request = new TCreateSentryPrivilegeRequest();
+                struct.request = new TAlterSentryRoleRevokePrivilegeRequest();
                 struct.request.read(iprot);
                 struct.setRequestIsSet(true);
               } else { 
@@ -2422,7 +3228,7 @@ public class SentryPolicyService {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, create_sentry_privilege_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, alter_sentry_role_revoke_privilege_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -2437,16 +3243,16 @@ public class SentryPolicyService {
 
     }
 
-    private static class create_sentry_privilege_argsTupleSchemeFactory implements SchemeFactory {
-      public create_sentry_privilege_argsTupleScheme getScheme() {
-        return new create_sentry_privilege_argsTupleScheme();
+    private static class alter_sentry_role_revoke_privilege_argsTupleSchemeFactory implements SchemeFactory {
+      public alter_sentry_role_revoke_privilege_argsTupleScheme getScheme() {
+        return new alter_sentry_role_revoke_privilege_argsTupleScheme();
       }
     }
 
-    private static class create_sentry_privilege_argsTupleScheme extends TupleScheme<create_sentry_privilege_args> {
+    private static class alter_sentry_role_revoke_privilege_argsTupleScheme extends TupleScheme<alter_sentry_role_revoke_privilege_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, create_sentry_privilege_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, alter_sentry_role_revoke_privilege_args struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
         if (struct.isSetRequest()) {
@@ -2459,11 +3265,11 @@ public class SentryPolicyService {
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, create_sentry_privilege_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, alter_sentry_role_revoke_privilege_args struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
-          struct.request = new TCreateSentryPrivilegeRequest();
+          struct.request = new TAlterSentryRoleRevokePrivilegeRequest();
           struct.request.read(iprot);
           struct.setRequestIsSet(true);
         }
@@ -2472,18 +3278,18 @@ public class SentryPolicyService {
 
   }
 
-  public static class create_sentry_privilege_result implements org.apache.thrift.TBase<create_sentry_privilege_result, create_sentry_privilege_result._Fields>, java.io.Serializable, Cloneable   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("create_sentry_privilege_result");
+  public static class alter_sentry_role_revoke_privilege_result implements org.apache.thrift.TBase<alter_sentry_role_revoke_privilege_result, alter_sentry_role_revoke_privilege_result._Fields>, java.io.Serializable, Cloneable   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("alter_sentry_role_revoke_privilege_result");
 
     private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new create_sentry_privilege_resultStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new create_sentry_privilege_resultTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new alter_sentry_role_revoke_privilege_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new alter_sentry_role_revoke_privilege_resultTupleSchemeFactory());
     }
 
-    private TCreateSentryPrivilegeResponse success; // required
+    private TAlterSentryRoleRevokePrivilegeResponse success; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -2548,16 +3354,16 @@ public class SentryPolicyService {
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
       tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCreateSentryPrivilegeResponse.class)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TAlterSentryRoleRevokePrivilegeResponse.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(create_sentry_privilege_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(alter_sentry_role_revoke_privilege_result.class, metaDataMap);
     }
 
-    public create_sentry_privilege_result() {
+    public alter_sentry_role_revoke_privilege_result() {
     }
 
-    public create_sentry_privilege_result(
-      TCreateSentryPrivilegeResponse success)
+    public alter_sentry_role_revoke_privilege_result(
+      TAlterSentryRoleRevokePrivilegeResponse success)
     {
       this();
       this.success = success;
@@ -2566,14 +3372,14 @@ public class SentryPolicyService {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public create_sentry_privilege_result(create_sentry_privilege_result other) {
+    public alter_sentry_role_revoke_privilege_result(alter_sentry_role_revoke_privilege_result other) {
       if (other.isSetSuccess()) {
-        this.success = new TCreateSentryPrivilegeResponse(other.success);
+        this.success = new TAlterSentryRoleRevokePrivilegeResponse(other.success);
       }
     }
 
-    public create_sentry_privilege_result deepCopy() {
-      return new create_sentry_privilege_result(this);
+    public alter_sentry_role_revoke_privilege_result deepCopy() {
+      return new alter_sentry_role_revoke_privilege_result(this);
     }
 
     @Override
@@ -2581,11 +3387,11 @@ public class SentryPolicyService {
       this.success = null;
     }
 
-    public TCreateSentryPrivilegeResponse getSuccess() {
+    public TAlterSentryRoleRevokePrivilegeResponse getSuccess() {
       return this.success;
     }
 
-    public void setSuccess(TCreateSentryPrivilegeResponse success) {
+    public void setSuccess(TAlterSentryRoleRevokePrivilegeResponse success) {
       this.success = success;
     }
 
@@ -2610,7 +3416,7 @@ public class SentryPolicyService {
         if (value == null) {
           unsetSuccess();
         } else {
-          setSuccess((TCreateSentryPrivilegeResponse)value);
+          setSuccess((TAlterSentryRoleRevokePrivilegeResponse)value);
         }
         break;
 
@@ -2643,12 +3449,12 @@ public class SentryPolicyService {
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof create_sentry_privilege_result)
-        return this.equals((create_sentry_privilege_result)that);
+      if (that instanceof alter_sentry_role_revoke_privilege_result)
+        return this.equals((alter_sentry_role_revoke_privilege_result)that);
       return false;
     }
 
-    public boolean equals(create_sentry_privilege_result that) {
+    public boolean equals(alter_sentry_role_revoke_privilege_result that) {
       if (that == null)
         return false;
 
@@ -2676,13 +3482,13 @@ public class SentryPolicyService {
       return builder.toHashCode();
     }
 
-    public int compareTo(create_sentry_privilege_result other) {
+    public int compareTo(alter_sentry_role_revoke_privilege_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
 
       int lastComparison = 0;
-      create_sentry_privilege_result typedOther = (create_sentry_privilege_result)other;
+      alter_sentry_role_revoke_privilege_result typedOther = (alter_sentry_role_revoke_privilege_result)other;
 
       lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess());
       if (lastComparison != 0) {
@@ -2711,7 +3517,7 @@ public class SentryPolicyService {
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("create_sentry_privilege_result(");
+      StringBuilder sb = new StringBuilder("alter_sentry_role_revoke_privilege_result(");
       boolean first = true;
 
       sb.append("success:");
@@ -2749,15 +3555,15 @@ public class SentryPolicyService {
       }
     }
 
-    private static class create_sentry_privilege_resultStandardSchemeFactory implements SchemeFactory {
-      public create_sentry_privilege_resultStandardScheme getScheme() {
-        return new create_sentry_privilege_resultStandardScheme();
+    private static class alter_sentry_role_revoke_privilege_resultStandardSchemeFactory implements SchemeFactory {
+      public alter_sentry_role_revoke_privilege_resultStandardScheme getScheme() {
+        return new alter_sentry_role_revoke_privilege_resultStandardScheme();
       }
     }
 
-    private static class create_sentry_privilege_resultStandardScheme extends StandardScheme<create_sentry_privilege_result> {
+    private static class alter_sentry_role_revoke_privilege_resultStandardScheme extends StandardScheme<alter_sentry_role_revoke_privilege_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, create_sentry_privilege_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, alter_sentry_role_revoke_privilege_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -2769,7 +3575,7 @@ public class SentryPolicyService {
           switch (schemeField.id) {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.success = new TCreateSentryPrivilegeResponse();
+                struct.success = new TAlterSentryRoleRevokePrivilegeResponse();
                 struct.success.read(iprot);
                 struct.setSuccessIsSet(true);
               } else { 
@@ -2785,7 +3591,7 @@ public class SentryPolicyService {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, create_sentry_privilege_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, alter_sentry_role_revoke_privilege_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -2800,16 +3606,16 @@ public class SentryPolicyService {
 
     }
 
-    private static class create_sentry_privilege_resultTupleSchemeFactory implements SchemeFactory {
-      public create_sentry_privilege_resultTupleScheme getScheme() {
-        return new create_sentry_privilege_resultTupleScheme();
+    private static class alter_sentry_role_revoke_privilege_resultTupleSchemeFactory implements SchemeFactory {
+      public alter_sentry_role_revoke_privilege_resultTupleScheme getScheme() {
+        return new alter_sentry_role_revoke_privilege_resultTupleScheme();
       }
     }
 
-    private static class create_sentry_privilege_resultTupleScheme extends TupleScheme<create_sentry_privilege_result> {
+    private static class alter_sentry_role_revoke_privilege_resultTupleScheme extends TupleScheme<alter_sentry_role_revoke_privilege_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, create_sentry_privilege_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, alter_sentry_role_revoke_privilege_result struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
         if (struct.isSetSuccess()) {
@@ -2822,11 +3628,11 @@ public class SentryPolicyService {
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, create_sentry_privilege_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, alter_sentry_role_revoke_privilege_result struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
-          struct.success = new TCreateSentryPrivilegeResponse();
+          struct.success = new TAlterSentryRoleRevokePrivilegeResponse();
           struct.success.read(iprot);
           struct.setSuccessIsSet(true);
         }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7df761d/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TAlterSentryRoleAddGroupsRequest.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TAlterSentryRoleAddGroupsRequest.java b/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TAlterSentryRoleAddGroupsRequest.java
index 034bfef..86dcc54 100644
--- a/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TAlterSentryRoleAddGroupsRequest.java
+++ b/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TAlterSentryRoleAddGroupsRequest.java
@@ -631,14 +631,14 @@ public class TAlterSentryRoleAddGroupsRequest implements org.apache.thrift.TBase
           case 4: // GROUPS
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set8 = iprot.readSetBegin();
-                struct.groups = new HashSet<TSentryGroup>(2*_set8.size);
-                for (int _i9 = 0; _i9 < _set8.size; ++_i9)
+                org.apache.thrift.protocol.TSet _set16 = iprot.readSetBegin();
+                struct.groups = new HashSet<TSentryGroup>(2*_set16.size);
+                for (int _i17 = 0; _i17 < _set16.size; ++_i17)
                 {
-                  TSentryGroup _elem10; // required
-                  _elem10 = new TSentryGroup();
-                  _elem10.read(iprot);
-                  struct.groups.add(_elem10);
+                  TSentryGroup _elem18; // required
+                  _elem18 = new TSentryGroup();
+                  _elem18.read(iprot);
+                  struct.groups.add(_elem18);
                 }
                 iprot.readSetEnd();
               }
@@ -677,9 +677,9 @@ public class TAlterSentryRoleAddGroupsRequest implements org.apache.thrift.TBase
         oprot.writeFieldBegin(GROUPS_FIELD_DESC);
         {
           oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRUCT, struct.groups.size()));
-          for (TSentryGroup _iter11 : struct.groups)
+          for (TSentryGroup _iter19 : struct.groups)
           {
-            _iter11.write(oprot);
+            _iter19.write(oprot);
           }
           oprot.writeSetEnd();
         }
@@ -707,9 +707,9 @@ public class TAlterSentryRoleAddGroupsRequest implements org.apache.thrift.TBase
       oprot.writeString(struct.roleName);
       {
         oprot.writeI32(struct.groups.size());
-        for (TSentryGroup _iter12 : struct.groups)
+        for (TSentryGroup _iter20 : struct.groups)
         {
-          _iter12.write(oprot);
+          _iter20.write(oprot);
         }
       }
     }
@@ -724,14 +724,14 @@ public class TAlterSentryRoleAddGroupsRequest implements org.apache.thrift.TBase
       struct.roleName = iprot.readString();
       struct.setRoleNameIsSet(true);
       {
-        org.apache.thrift.protocol.TSet _set13 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.groups = new HashSet<TSentryGroup>(2*_set13.size);
-        for (int _i14 = 0; _i14 < _set13.size; ++_i14)
+        org.apache.thrift.protocol.TSet _set21 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.groups = new HashSet<TSentryGroup>(2*_set21.size);
+        for (int _i22 = 0; _i22 < _set21.size; ++_i22)
         {
-          TSentryGroup _elem15; // required
-          _elem15 = new TSentryGroup();
-          _elem15.read(iprot);
-          struct.groups.add(_elem15);
+          TSentryGroup _elem23; // required
+          _elem23 = new TSentryGroup();
+          _elem23.read(iprot);
+          struct.groups.add(_elem23);
         }
       }
       struct.setGroupsIsSet(true);