You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sentry.apache.org by ak...@apache.org on 2017/03/13 03:37:00 UTC

sentry git commit: SENTRY-1360: Refactor grantPrivilege of Sentry Client (Dapeng Sun, reviewed by Colin Ma)

Repository: sentry
Updated Branches:
  refs/heads/sentry-ha-redesign 3a02a6282 -> f19a68cb6


SENTRY-1360: Refactor grantPrivilege of Sentry Client (Dapeng Sun, reviewed by Colin Ma)


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

Branch: refs/heads/sentry-ha-redesign
Commit: f19a68cb69e72a9df3dbd3cefeb3a51535c71360
Parents: 3a02a62
Author: Alexander Kolbasov <ak...@cloudera.com>
Authored: Fri Mar 10 22:03:46 2017 -0800
Committer: Alexander Kolbasov <ak...@cloudera.com>
Committed: Sun Mar 12 20:36:44 2017 -0700

----------------------------------------------------------------------
 .../thrift/SentryPolicyServiceClient.java       |  7 ++
 .../SentryPolicyServiceClientDefaultImpl.java   | 80 +++++++++++---------
 .../db/tools/command/hive/CommandUtil.java      |  4 +-
 .../command/hive/GrantPrivilegeToRoleCmd.java   | 22 +-----
 4 files changed, 56 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/sentry/blob/f19a68cb/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 8949667..c2b03e5 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
@@ -104,6 +104,13 @@ public interface SentryPolicyServiceClient {
       String server, String db, String table, List<String> columnNames, String action,
       Boolean grantOption) throws SentryUserException;
 
+  Set<TSentryPrivilege> grantPrivileges(String requestorUserName, String
+      roleName, Set<TSentryPrivilege> privileges) throws SentryUserException;
+
+  TSentryPrivilege grantPrivilege(String requestorUserName, String roleName,
+                                  TSentryPrivilege privilege) throws
+      SentryUserException;
+
   void revokeURIPrivilege(String requestorUserName, String roleName, String server,
       String uri) throws SentryUserException;
 

http://git-wip-us.apache.org/repos/asf/sentry/blob/f19a68cb/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyServiceClientDefaultImpl.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyServiceClientDefaultImpl.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyServiceClientDefaultImpl.java
index 9494b75..2cf748e 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyServiceClientDefaultImpl.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyServiceClientDefaultImpl.java
@@ -530,6 +530,45 @@ public class SentryPolicyServiceClientDefaultImpl implements SentryPolicyService
         null, db, table, columnNames, action, grantOption);
   }
 
+  public synchronized Set<TSentryPrivilege> grantPrivileges(
+      String requestorUserName, String roleName,
+      Set<TSentryPrivilege> privileges) throws SentryUserException {
+    return grantPrivilegesCore(requestorUserName, roleName, privileges);
+  }
+
+  public synchronized TSentryPrivilege grantPrivilege(String requestorUserName, String roleName,
+                                                      TSentryPrivilege privilege) throws SentryUserException {
+    return grantPrivilegeCore(requestorUserName, roleName, privilege);
+  }
+
+  private TSentryPrivilege grantPrivilegeCore(String requestorUserName, String roleName,
+                                              TSentryPrivilege privilege) throws SentryUserException {
+    Set<TSentryPrivilege> results =
+        grantPrivilegesCore(requestorUserName, roleName, ImmutableSet.of(privilege));
+    if (results != null && results.size() > 0) {
+      return results.iterator().next();
+    } else {
+      return new TSentryPrivilege();
+    }
+  }
+
+  private Set<TSentryPrivilege> grantPrivilegesCore(String requestorUserName, String roleName,
+                                                    Set<TSentryPrivilege> privileges) throws SentryUserException {
+    TAlterSentryRoleGrantPrivilegeRequest request = new TAlterSentryRoleGrantPrivilegeRequest();
+    request.setProtocol_version(ThriftConstants.TSENTRY_SERVICE_VERSION_CURRENT);
+    request.setRequestorUserName(requestorUserName);
+    request.setRoleName(roleName);
+    request.setPrivileges(privileges);
+    try {
+      TAlterSentryRoleGrantPrivilegeResponse response =
+          client.alter_sentry_role_grant_privilege(request);
+      Status.throwIfNotOk(response.getStatus());
+      return response.getPrivileges();
+    } catch (TException e) {
+      throw new SentryUserException(THRIFT_EXCEPTION_MESSAGE, e);
+    }
+  }
+
   @VisibleForTesting
   public static TSentryAuthorizable setupSentryAuthorizable(
       List<? extends Authorizable> authorizable) {
@@ -568,25 +607,9 @@ public class SentryPolicyServiceClientDefaultImpl implements SentryPolicyService
       String roleName, PrivilegeScope scope, String serverName, String uri, String db, String table,
       String column, String action, Boolean grantOption)
   throws SentryUserException {
-    TAlterSentryRoleGrantPrivilegeRequest request = new TAlterSentryRoleGrantPrivilegeRequest();
-    request.setProtocol_version(ThriftConstants.TSENTRY_SERVICE_VERSION_CURRENT);
-    request.setRequestorUserName(requestorUserName);
-    request.setRoleName(roleName);
-    Set<TSentryPrivilege> privileges = convertColumnPrivilege(scope,
-        serverName, uri, db, table, column, action, grantOption);
-    request.setPrivileges(privileges);
-    try {
-      TAlterSentryRoleGrantPrivilegeResponse response = client.alter_sentry_role_grant_privilege(request);
-      Status.throwIfNotOk(response.getStatus());
-      if (response.isSetPrivileges()
-          && response.getPrivilegesSize()>0 ) {
-        return response.getPrivileges().iterator().next();
-      } else {
-        return new TSentryPrivilege();
-      }
-    } catch (TException e) {
-      throw new SentryUserException(THRIFT_EXCEPTION_MESSAGE, e);
-    }
+    TSentryPrivilege privilege =
+        convertToTSentryPrivilege(scope, serverName, uri, db, table, column, action, grantOption);
+    return grantPrivilegeCore(requestorUserName, roleName, privilege);
   }
 
   private Set<TSentryPrivilege> grantPrivileges(String requestorUserName,
@@ -601,20 +624,9 @@ public class SentryPolicyServiceClientDefaultImpl implements SentryPolicyService
       String roleName, PrivilegeScope scope, String serverName, String uri, String db, String table,
       List<String> columns, String action, Boolean grantOption)
   throws SentryUserException {
-    TAlterSentryRoleGrantPrivilegeRequest request = new TAlterSentryRoleGrantPrivilegeRequest();
-    request.setProtocol_version(ThriftConstants.TSENTRY_SERVICE_VERSION_CURRENT);
-    request.setRequestorUserName(requestorUserName);
-    request.setRoleName(roleName);
     Set<TSentryPrivilege> privileges = convertColumnPrivileges(scope,
         serverName, uri, db, table, columns, action, grantOption);
-    request.setPrivileges(privileges);
-    try {
-      TAlterSentryRoleGrantPrivilegeResponse response = client.alter_sentry_role_grant_privilege(request);
-      Status.throwIfNotOk(response.getStatus());
-      return response.getPrivileges();
-    } catch (TException e) {
-      throw new SentryUserException(THRIFT_EXCEPTION_MESSAGE, e);
-    }
+    return grantPrivilegesCore(requestorUserName, roleName, privileges);
   }
 
   public synchronized void revokePrivileges(String requestorUserName, String roleName, Set<TSentryPrivilege> privileges) throws  SentryUserException {
@@ -815,10 +827,9 @@ public class SentryPolicyServiceClientDefaultImpl implements SentryPolicyService
     return setBuilder.build();
   }
 
-  private Set<TSentryPrivilege> convertColumnPrivilege(
+  private TSentryPrivilege convertToTSentryPrivilege(
       PrivilegeScope scope, String serverName, String uri, String db, String table, String column,
       String action, Boolean grantOption) {
-    ImmutableSet.Builder<TSentryPrivilege> setBuilder = ImmutableSet.builder();
     TSentryPrivilege privilege = new TSentryPrivilege();
     privilege.setPrivilegeScope(scope.toString());
     privilege.setServerName(serverName);
@@ -829,8 +840,7 @@ public class SentryPolicyServiceClientDefaultImpl implements SentryPolicyService
     privilege.setAction(action);
     privilege.setCreateTime(System.currentTimeMillis());
     privilege.setGrantOption(convertTSentryGrantOption(grantOption));
-    setBuilder.add(privilege);
-    return setBuilder.build();
+    return privilege;
   }
 
   private TSentryGrantOption convertTSentryGrantOption(Boolean grantOption) {

http://git-wip-us.apache.org/repos/asf/sentry/blob/f19a68cb/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/CommandUtil.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/CommandUtil.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/CommandUtil.java
index 2d2dcb5..51ee9ef 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/CommandUtil.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/CommandUtil.java
@@ -18,9 +18,10 @@
 package org.apache.sentry.provider.db.tools.command.hive;
 
 import org.apache.commons.lang.StringUtils;
-import org.apache.sentry.core.common.utils.SentryConstants;
 import org.apache.sentry.core.common.utils.KeyValue;
 import org.apache.sentry.core.common.utils.PolicyFileConstants;
+import org.apache.sentry.core.common.utils.SentryConstants;
+import org.apache.sentry.core.model.db.AccessConstants;
 import org.apache.sentry.provider.db.service.thrift.TSentryGrantOption;
 import org.apache.sentry.provider.db.service.thrift.TSentryPrivilege;
 import org.apache.sentry.service.thrift.ServiceConstants;
@@ -51,6 +52,7 @@ public final class CommandUtil {
         tSentryPrivilege.setColumnName(value);
       } else if (PolicyFileConstants.PRIVILEGE_URI_NAME.equalsIgnoreCase(key)) {
         tSentryPrivilege.setURI(value);
+        tSentryPrivilege.setAction(AccessConstants.ALL);
       } else if (PolicyFileConstants.PRIVILEGE_ACTION_NAME.equalsIgnoreCase(key)) {
         tSentryPrivilege.setAction(value);
       } else if (PolicyFileConstants.PRIVILEGE_GRANT_OPTION_NAME.equalsIgnoreCase(key)) {

http://git-wip-us.apache.org/repos/asf/sentry/blob/f19a68cb/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/GrantPrivilegeToRoleCmd.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/GrantPrivilegeToRoleCmd.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/GrantPrivilegeToRoleCmd.java
index a1ef2f9..e3d06a9 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/GrantPrivilegeToRoleCmd.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/tools/command/hive/GrantPrivilegeToRoleCmd.java
@@ -18,9 +18,7 @@
 package org.apache.sentry.provider.db.tools.command.hive;
 
 import org.apache.sentry.provider.db.service.thrift.SentryPolicyServiceClient;
-import org.apache.sentry.provider.db.service.thrift.TSentryGrantOption;
 import org.apache.sentry.provider.db.service.thrift.TSentryPrivilege;
-import org.apache.sentry.service.thrift.ServiceConstants;
 
 /**
  * The class for admin command to grant privilege to role.
@@ -38,24 +36,6 @@ public class GrantPrivilegeToRoleCmd implements Command {
   @Override
   public void execute(SentryPolicyServiceClient client, String requestorName) throws Exception {
     TSentryPrivilege tSentryPrivilege = CommandUtil.convertToTSentryPrivilege(privilegeStr);
-    boolean grantOption = tSentryPrivilege.getGrantOption().equals(TSentryGrantOption.TRUE) ? true : false;
-    if (ServiceConstants.PrivilegeScope.SERVER.toString().equals(tSentryPrivilege.getPrivilegeScope())) {
-      client.grantServerPrivilege(requestorName, roleName, tSentryPrivilege.getServerName(),
-              tSentryPrivilege.getAction(), grantOption);
-    } else if (ServiceConstants.PrivilegeScope.DATABASE.toString().equals(tSentryPrivilege.getPrivilegeScope())) {
-      client.grantDatabasePrivilege(requestorName, roleName, tSentryPrivilege.getServerName(),
-              tSentryPrivilege.getDbName(), tSentryPrivilege.getAction(), grantOption);
-    } else if (ServiceConstants.PrivilegeScope.TABLE.toString().equals(tSentryPrivilege.getPrivilegeScope())) {
-      client.grantTablePrivilege(requestorName, roleName, tSentryPrivilege.getServerName(),
-              tSentryPrivilege.getDbName(), tSentryPrivilege.getTableName(),
-              tSentryPrivilege.getAction(), grantOption);
-    } else if (ServiceConstants.PrivilegeScope.COLUMN.toString().equals(tSentryPrivilege.getPrivilegeScope())) {
-      client.grantColumnPrivilege(requestorName, roleName, tSentryPrivilege.getServerName(),
-              tSentryPrivilege.getDbName(), tSentryPrivilege.getTableName(),
-              tSentryPrivilege.getColumnName(), tSentryPrivilege.getAction(), grantOption);
-    } else if (ServiceConstants.PrivilegeScope.URI.toString().equals(tSentryPrivilege.getPrivilegeScope())) {
-      client.grantURIPrivilege(requestorName, roleName, tSentryPrivilege.getServerName(),
-              tSentryPrivilege.getURI(), grantOption);
-    }
+    client.grantPrivilege(requestorName, roleName, tSentryPrivilege);
   }
 }