Repository: sentry
Updated Branches:
refs/heads/master b31210701 -> c7c5df286
SENTRY-2413: Provide a configuration option to permit specific DB privileges to be granted
explicitly (Sergio Pena, reviewed by Na Li, kalyan kumar kalvagadda)
Project: http://git-wip-us.apache.org/repos/asf/sentry/repo
Commit: http://git-wip-us.apache.org/repos/asf/sentry/commit/c7c5df28
Tree: http://git-wip-us.apache.org/repos/asf/sentry/tree/c7c5df28
Diff: http://git-wip-us.apache.org/repos/asf/sentry/diff/c7c5df28
Branch: refs/heads/master
Commit: c7c5df28628049932db7f75e1b970bffe75522ae
Parents: b312107
Author: Sergio Pena <sergio.pena@cloudera.com>
Authored: Tue Sep 25 14:29:39 2018 -0500
Committer: Sergio Pena <sergio.pena@cloudera.com>
Committed: Tue Sep 25 14:29:39 2018 -0500
----------------------------------------------------------------------
.../sentry/service/common/ServiceConstants.java | 6 ++
.../sentry/api/common/SentryServiceUtil.java | 65 +++++++++++++++
.../api/common/TestSentryServiceUtil.java | 85 ++++++++++++++++++++
.../thrift/SentryPolicyStoreProcessor.java | 4 +
.../thrift/TestSentryPolicyStoreProcessor.java | 52 ++++++++++++
5 files changed, 212 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/sentry/blob/c7c5df28/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/service/common/ServiceConstants.java
----------------------------------------------------------------------
diff --git a/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/service/common/ServiceConstants.java
b/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/service/common/ServiceConstants.java
index adc1947..e90fe2d 100644
--- a/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/service/common/ServiceConstants.java
+++ b/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/service/common/ServiceConstants.java
@@ -250,6 +250,12 @@ public class ServiceConstants {
public static final String SENTRY_DB_POLICY_STORE_OWNER_AS_PRIVILEGE = "sentry.db.policy.store.owner.as.privilege";
public static final SentryOwnerPrivilegeType SENTRY_DB_POLICY_STORE_OWNER_AS_PRIVILEGE_DEFAULT
= SentryOwnerPrivilegeType.NONE;
+
+ /**
+ * List of privileges that are permitted to be granted by Sentry DB clients
+ */
+ public static final String SENTRY_DB_EXPLICIT_GRANTS_PERMITTED = "sentry.db.explicit.grants.permitted";
+ public static final String SENTRY_DB_EXPLICIT_GRANTS_PERMITTED_DEFAULT = "";
}
public static final String SENTRY_ZK_JAAS_NAME = "Sentry";
http://git-wip-us.apache.org/repos/asf/sentry/blob/c7c5df28/sentry-service/sentry-service-api/src/main/java/org/apache/sentry/api/common/SentryServiceUtil.java
----------------------------------------------------------------------
diff --git a/sentry-service/sentry-service-api/src/main/java/org/apache/sentry/api/common/SentryServiceUtil.java
b/sentry-service/sentry-service-api/src/main/java/org/apache/sentry/api/common/SentryServiceUtil.java
index 8cdbde4..83393a9 100644
--- a/sentry-service/sentry-service-api/src/main/java/org/apache/sentry/api/common/SentryServiceUtil.java
+++ b/sentry-service/sentry-service-api/src/main/java/org/apache/sentry/api/common/SentryServiceUtil.java
@@ -21,10 +21,13 @@ package org.apache.sentry.api.common;
import java.text.SimpleDateFormat;
import java.util.Arrays;
+import java.util.Collections;
import java.util.Date;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.List;
import java.util.Map;
+import java.util.Set;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
@@ -35,9 +38,12 @@ import org.apache.hadoop.conf.Configuration;
import static org.apache.sentry.binding.hive.conf.HiveAuthzConf.AuthzConfVars.AUTHZ_SYNC_ALTER_WITH_POLICY_STORE;
import static org.apache.sentry.binding.hive.conf.HiveAuthzConf.AuthzConfVars.AUTHZ_SYNC_CREATE_WITH_POLICY_STORE;
import static org.apache.sentry.binding.hive.conf.HiveAuthzConf.AuthzConfVars.AUTHZ_SYNC_DROP_WITH_POLICY_STORE;
+import static org.apache.sentry.service.common.ServiceConstants.ServerConfig.SENTRY_DB_EXPLICIT_GRANTS_PERMITTED;
+import static org.apache.sentry.service.common.ServiceConstants.ServerConfig.SENTRY_DB_EXPLICIT_GRANTS_PERMITTED_DEFAULT;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.sentry.core.common.exception.SentryGrantDeniedException;
import org.apache.sentry.core.common.exception.SentryInvalidInputException;
import org.apache.sentry.core.common.utils.SentryConstants;
import org.apache.sentry.core.common.utils.KeyValue;
@@ -331,4 +337,63 @@ public final class SentryServiceUtil {
return String.format("%s: %s", date, logMessage);
}
+
+ /**
+ * Checks if a list of privileges are permitted to be explicitly granted by any of the
Sentry DB
+ * clients.
+ * <p/>
+ * The list of privileges are checked against the configuration 'sentry.db.explicit.grants.permitted'
+ * that should exist in the Configuration object. This configuration has a list of comma-separated
+ * privileges. If an empty value is set, then it allows any privilege to be granted.
+ *
+ * @param conf The Configuration object that has the key=value of the privileges that are
permitted.
+ * @param privileges A set of privileges that need to be verified if are permitted.
+ * @throws SentryGrantDeniedException If at least one of the privileges in the set is not
permitted.
+ */
+ public static void checkDbExplicitGrantsPermitted(Configuration conf, Set<TSentryPrivilege>
privileges)
+ throws SentryGrantDeniedException {
+ Set<String> permittedGrants = getDbGrantsPermittedFromConf(conf);
+ if (permittedGrants.isEmpty()) {
+ return;
+ }
+
+ Set<String> deniedGrants = new HashSet<>();
+ for (TSentryPrivilege privilege : privileges) {
+ String action = privilege.getAction();
+ if (action != null) {
+ action = action.trim().toUpperCase();
+
+ // Will collect all grants not permitted so the exception thrown displays which privileges
+ // cannot be granted
+ if (!permittedGrants.contains(action)) {
+ deniedGrants.add(action);
+ }
+ }
+ }
+
+ if (!deniedGrants.isEmpty()) {
+ throw new SentryGrantDeniedException(
+ String.format("GRANT privilege for %s not permitted.", deniedGrants));
+ }
+ }
+
+ // Returns the list of privileges found on the Configuration object that are permitted
to be
+ // granted.
+ // The returned Set has all privileges in upper case and spaces trimmed to avoid mistakes
+ // during comparison.
+ private static Set<String> getDbGrantsPermittedFromConf(Configuration conf) {
+ String grantsConfig = conf.get(SENTRY_DB_EXPLICIT_GRANTS_PERMITTED,
+ SENTRY_DB_EXPLICIT_GRANTS_PERMITTED_DEFAULT).trim();
+
+ if (grantsConfig.isEmpty()) {
+ return Collections.emptySet();
+ }
+
+ Set<String> permittedGrants = new HashSet<>();
+ for (String grant : grantsConfig.split(",")) {
+ permittedGrants.add(grant.trim().toUpperCase());
+ }
+
+ return permittedGrants;
+ }
}
http://git-wip-us.apache.org/repos/asf/sentry/blob/c7c5df28/sentry-service/sentry-service-api/src/test/java/org/apache/sentry/api/common/TestSentryServiceUtil.java
----------------------------------------------------------------------
diff --git a/sentry-service/sentry-service-api/src/test/java/org/apache/sentry/api/common/TestSentryServiceUtil.java
b/sentry-service/sentry-service-api/src/test/java/org/apache/sentry/api/common/TestSentryServiceUtil.java
new file mode 100644
index 0000000..2dc0975
--- /dev/null
+++ b/sentry-service/sentry-service-api/src/test/java/org/apache/sentry/api/common/TestSentryServiceUtil.java
@@ -0,0 +1,85 @@
+/*
+ *
+ * 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.api.common;
+
+import static junit.framework.Assert.assertFalse;
+import static org.apache.sentry.service.common.ServiceConstants.ServerConfig.SENTRY_DB_EXPLICIT_GRANTS_PERMITTED;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import com.google.common.collect.Sets;
+import java.util.Collections;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.sentry.api.service.thrift.TSentryPrivilege;
+import org.apache.sentry.core.common.exception.SentryGrantDeniedException;
+import org.junit.Test;
+
+public class TestSentryServiceUtil {
+ @Test
+ public void testCheckPermittedDbGrants() {
+ Configuration conf = new Configuration();
+
+ // An empty configuration of permitted privileges should not throw exceptions
+ try {
+ SentryServiceUtil.checkDbExplicitGrantsPermitted(conf, Collections.emptySet());
+ } catch (SentryGrantDeniedException e) {
+ fail("An empty permitted privileges configuration should not throw an exception");
+ }
+
+ // Only ALL, SELECT and INSERT privileges are permitted
+ conf.set(SENTRY_DB_EXPLICIT_GRANTS_PERMITTED, "all,*,select,insert");
+
+ try {
+ SentryServiceUtil.checkDbExplicitGrantsPermitted(conf, Sets.newHashSet(
+ newTSentryPrivilege("select"), newTSentryPrivilege("insert"), newTSentryPrivilege("all"),
+ newTSentryPrivilege("*")
+ ));
+ } catch (SentryGrantDeniedException e) {
+ fail("ALL, *, SELECT and INSERT privileges should be permitted");
+ }
+
+ try {
+ SentryServiceUtil.checkDbExplicitGrantsPermitted(conf, Sets.newHashSet(
+ newTSentryPrivilege("select"), newTSentryPrivilege("create"), newTSentryPrivilege("all")
+ ));
+ fail("CREATE privileges should not be permitted");
+ } catch (SentryGrantDeniedException e) {
+ assertTrue("CREATE privileges should not be permitted", e.getMessage().contains("CREATE"));
+ assertFalse("SELECT privileges should be permitted", e.getMessage().contains("SELECT"));
+ assertFalse("ALL privileges should be permitted", e.getMessage().contains("ALL"));
+ }
+
+ conf.set(SENTRY_DB_EXPLICIT_GRANTS_PERMITTED, "select,insert,create");
+
+ try {
+ SentryServiceUtil.checkDbExplicitGrantsPermitted(conf, Sets.newHashSet(
+ newTSentryPrivilege("alter"), newTSentryPrivilege("create"), newTSentryPrivilege("drop")
+ ));
+ fail("ALTER and DROP privileges should not be permitted");
+ } catch (SentryGrantDeniedException e) {
+ assertTrue("ALTER privileges should not be permitted", e.getMessage().contains("ALTER"));
+ assertTrue("DROP privileges should not be permitted", e.getMessage().contains("DROP"));
+ assertFalse("CREATE privileges should be permitted", e.getMessage().contains("CREATE"));
+ }
+ }
+
+ private TSentryPrivilege newTSentryPrivilege(String action) {
+ return new TSentryPrivilege("", "server1", action);
+ }
+}
http://git-wip-us.apache.org/repos/asf/sentry/blob/c7c5df28/sentry-service/sentry-service-server/src/main/java/org/apache/sentry/api/service/thrift/SentryPolicyStoreProcessor.java
----------------------------------------------------------------------
diff --git a/sentry-service/sentry-service-server/src/main/java/org/apache/sentry/api/service/thrift/SentryPolicyStoreProcessor.java
b/sentry-service/sentry-service-server/src/main/java/org/apache/sentry/api/service/thrift/SentryPolicyStoreProcessor.java
index 3a9623b..3087529 100644
--- a/sentry-service/sentry-service-server/src/main/java/org/apache/sentry/api/service/thrift/SentryPolicyStoreProcessor.java
+++ b/sentry-service/sentry-service-server/src/main/java/org/apache/sentry/api/service/thrift/SentryPolicyStoreProcessor.java
@@ -260,6 +260,10 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface
{
if (request.isSetPrivilege()) {
request.setPrivileges(Sets.newHashSet(request.getPrivilege()));
}
+
+ // Throw an exception if one of the grants is not permitted.
+ SentryServiceUtil.checkDbExplicitGrantsPermitted(conf, request.getPrivileges());
+
// TODO: now only has SentryPlugin. Once add more SentryPolicyStorePlugins,
// TODO: need to differentiate the updates for different Plugins.
Preconditions.checkState(sentryPlugins.size() <= 1);
http://git-wip-us.apache.org/repos/asf/sentry/blob/c7c5df28/sentry-service/sentry-service-server/src/test/java/org/apache/sentry/api/service/thrift/TestSentryPolicyStoreProcessor.java
----------------------------------------------------------------------
diff --git a/sentry-service/sentry-service-server/src/test/java/org/apache/sentry/api/service/thrift/TestSentryPolicyStoreProcessor.java
b/sentry-service/sentry-service-server/src/test/java/org/apache/sentry/api/service/thrift/TestSentryPolicyStoreProcessor.java
index c23b385..059621a 100644
--- a/sentry-service/sentry-service-server/src/test/java/org/apache/sentry/api/service/thrift/TestSentryPolicyStoreProcessor.java
+++ b/sentry-service/sentry-service-server/src/test/java/org/apache/sentry/api/service/thrift/TestSentryPolicyStoreProcessor.java
@@ -42,6 +42,7 @@ import org.apache.sentry.service.common.ServiceConstants.SentryPrincipalType;
import org.apache.sentry.service.common.ServiceConstants.ServerConfig;
import org.apache.sentry.service.thrift.FullUpdateInitializerState;
import org.apache.sentry.service.thrift.SentryStateBank;
+import org.apache.thrift.TException;
import org.junit.After;
import org.junit.Assert;
@@ -54,6 +55,7 @@ import org.mockito.Mockito;
public class TestSentryPolicyStoreProcessor {
+ private static final String SERVERNAME = "server1";
private static final String DBNAME = "db1";
private static final String TABLENAME = "table1";
private static final String OWNER = "owner1";
@@ -331,6 +333,7 @@ public class TestSentryPolicyStoreProcessor {
private static TSentryPrivilege newSentryPrivilege(String scope, String dbname, String
tablename, String action) {
TSentryPrivilege privilege = new TSentryPrivilege();
privilege.setPrivilegeScope(scope);
+ privilege.setServerName(SERVERNAME);
privilege.setDbName(dbname);
privilege.setTableName(tablename);
privilege.setAction(action);
@@ -608,4 +611,53 @@ public class TestSentryPolicyStoreProcessor {
Assert.assertEquals(1, returnedResp.getPrivilegesMap().get("user2").size());
Assert.assertEquals(0, returnedResp.getPrivilegesMap().get("user3").size());
}
+
+ @Test
+ public void testGrantNotPermittedPrivilegesThrowsException() throws TException {
+ MockGroupMappingService.addUserGroupMapping("admin", Sets.newHashSet("admin"));
+
+ Configuration conf = new Configuration();
+ conf.set(ServerConfig.SENTRY_STORE_GROUP_MAPPING, MockGroupMappingService.class.getName());
+ conf.set(ServerConfig.ADMIN_GROUPS, "admin");
+ conf.set(ServerConfig.SENTRY_DB_EXPLICIT_GRANTS_PERMITTED, "ALL,SELECT,INSERT,CREATE");
+
+ // Initialize the SentryPolicyStoreProcessor with the permitted grants
+ SentryPolicyStoreProcessor policyStoreProcessor = null;
+ try {
+ policyStoreProcessor = new SentryPolicyStoreProcessor(ApiConstants.SentryPolicyServiceConstants.SENTRY_POLICY_SERVICE_NAME,
+ conf, sentryStore);
+ } catch (Exception e) {
+ Assert.fail("SentryPolicyStoreProcessor constructor should not throw an exception.");
+ }
+
+ TAlterSentryRoleGrantPrivilegeResponse response = null;
+ TAlterSentryRoleGrantPrivilegeRequest request =
+ new TAlterSentryRoleGrantPrivilegeRequest(ThriftConstants.TSENTRY_SERVICE_VERSION_CURRENT,
"admin", "role1");
+
+ // Attempt to grant the ALTER privilege
+ request.setPrivileges(Sets.newHashSet(newSentryPrivilege("SERVER", "", "", "ALTER")));
+ response = policyStoreProcessor.alter_sentry_role_grant_privilege(request);
+ Assert.assertEquals("Grant ALTER should not be permitted.",
+ Status.ACCESS_DENIED.getCode(), response.getStatus().getValue());
+
+ // Attempt to grant the SELECT privilege
+ request.setPrivileges(Sets.newHashSet(newSentryPrivilege("SERVER", "", "", "SELECT")));
+ response = policyStoreProcessor.alter_sentry_role_grant_privilege(request);
+ Assert.assertEquals("Grant SELECT should be permitted.",
+ Status.OK.getCode(), response.getStatus().getValue());
+
+ // Attempt to grant the ALTER,SELECT privilege
+ request.setPrivileges(Sets.newHashSet(
+ newSentryPrivilege("SERVER", "", "", "ALTER"),
+ newSentryPrivilege("SERVER", "", "", "SELECT")
+ ));
+
+ response = policyStoreProcessor.alter_sentry_role_grant_privilege(request);
+ Assert.assertEquals("Grant ALTER should not be permitted.",
+ Status.ACCESS_DENIED.getCode(), response.getStatus().getValue());
+ assertTrue("ALTER privileges should not be permitted",
+ response.getStatus().getMessage().contains("ALTER"));
+ Assert.assertFalse("SELECT privileges should be permitted",
+ response.getStatus().getMessage().contains("SELECT"));
+ }
}
|