diff --git a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/FileBasedKMSACLs.java b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/FileBasedKMSACLs.java
new file mode 100644
index 0000000000000..a0ff58157ff17
--- /dev/null
+++ b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/FileBasedKMSACLs.java
@@ -0,0 +1,274 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.crypto.key.kms.server;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.key.kms.server.KeyAuthorizationKeyProvider.KeyOpType;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.AccessControlList;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Provides access to the AccessControlLists used by KMS,
+ * hot-reloading them if the kms-acls.xml file where the ACLs
+ * are defined has been updated.
+ */
+@InterfaceAudience.Private
+public class FileBasedKMSACLs extends KMSACLs {
+ private static final Logger LOG = LoggerFactory.getLogger(KMSACLs.class);
+
+
+ private static final String ACL_DEFAULT =
+ AccessControlList.WILDCARD_ACL_VALUE;
+
+ private volatile Map acls;
+ private volatile Map blacklistedAcls;
+ @VisibleForTesting
+ volatile Map> keyAcls;
+ @VisibleForTesting
+ volatile Map defaultKeyAcls = new HashMap<>();
+ @VisibleForTesting
+ volatile Map whitelistKeyAcls = new HashMap<>();
+ private long lastReload;
+
+ FileBasedKMSACLs(Configuration conf) {
+ if (conf == null) {
+ conf = loadACLs();
+ }
+ setKMSACLs(conf);
+ setKeyACLs(conf);
+ }
+
+ public FileBasedKMSACLs() {
+ this(null);
+ }
+
+ private void setKMSACLs(Configuration conf) {
+ Map tempAcls = new HashMap();
+ Map tempBlacklist = new HashMap();
+ for (Type aclType : Type.values()) {
+ String aclStr = conf.get(aclType.getAclConfigKey(), ACL_DEFAULT);
+ tempAcls.put(aclType, new AccessControlList(aclStr));
+ String blacklistStr = conf.get(aclType.getBlacklistConfigKey());
+ if (blacklistStr != null) {
+ // Only add if blacklist is present
+ tempBlacklist.put(aclType, new AccessControlList(blacklistStr));
+ LOG.info("'{}' Blacklist '{}'", aclType, blacklistStr);
+ }
+ LOG.info("'{}' ACL '{}'", aclType, aclStr);
+ }
+ acls = tempAcls;
+ blacklistedAcls = tempBlacklist;
+ }
+
+ @VisibleForTesting
+ void setKeyACLs(Configuration conf) {
+ Map> tempKeyAcls =
+ new HashMap>();
+ Map allKeyACLS =
+ conf.getValByRegex(KMSConfiguration.KEY_ACL_PREFIX_REGEX);
+ for (Map.Entry keyAcl : allKeyACLS.entrySet()) {
+ String k = keyAcl.getKey();
+ // this should be of type "key.acl.."
+ int keyNameStarts = KMSConfiguration.KEY_ACL_PREFIX.length();
+ int keyNameEnds = k.lastIndexOf(".");
+ if (keyNameStarts >= keyNameEnds) {
+ LOG.warn("Invalid key name '{}'", k);
+ } else {
+ String aclStr = keyAcl.getValue();
+ String keyName = k.substring(keyNameStarts, keyNameEnds);
+ String keyOp = k.substring(keyNameEnds + 1);
+ KeyOpType aclType = null;
+ try {
+ aclType = KeyOpType.valueOf(keyOp);
+ } catch (IllegalArgumentException e) {
+ LOG.warn("Invalid key Operation '{}'", keyOp);
+ }
+ if (aclType != null) {
+ // On the assumption this will be single threaded.. else we need to
+ // ConcurrentHashMap
+ HashMap aclMap =
+ tempKeyAcls.get(keyName);
+ if (aclMap == null) {
+ aclMap = new HashMap();
+ tempKeyAcls.put(keyName, aclMap);
+ }
+ aclMap.put(aclType, new AccessControlList(aclStr));
+ LOG.info("KEY_NAME '{}' KEY_OP '{}' ACL '{}'",
+ keyName, aclType, aclStr);
+ }
+ }
+ }
+ keyAcls = tempKeyAcls;
+
+ final Map tempDefaults = new HashMap<>();
+ final Map tempWhitelists = new HashMap<>();
+ for (KeyOpType keyOp : KeyOpType.values()) {
+ parseAclsWithPrefix(conf, KMSConfiguration.DEFAULT_KEY_ACL_PREFIX,
+ keyOp, tempDefaults);
+ parseAclsWithPrefix(conf, KMSConfiguration.WHITELIST_KEY_ACL_PREFIX,
+ keyOp, tempWhitelists);
+ }
+ defaultKeyAcls = tempDefaults;
+ whitelistKeyAcls = tempWhitelists;
+ }
+
+ /**
+ * Parse the acls from configuration with the specified prefix. Currently
+ * only 2 possible prefixes: whitelist and default.
+ *
+ * @param conf The configuration.
+ * @param prefix The prefix.
+ * @param keyOp The key operation.
+ * @param results The collection of results to add to.
+ */
+ private void parseAclsWithPrefix(final Configuration conf,
+ final String prefix, final KeyOpType keyOp,
+ Map results) {
+ String confKey = prefix + keyOp;
+ String aclStr = conf.get(confKey);
+ if (aclStr != null) {
+ if (keyOp == KeyOpType.ALL) {
+ // Ignore All operation for default key and whitelist key acls
+ LOG.warn("Invalid KEY_OP '{}' for {}, ignoring", keyOp, prefix);
+ } else {
+ if (aclStr.equals("*")) {
+ LOG.info("{} for KEY_OP '{}' is set to '*'", prefix, keyOp);
+ }
+ results.put(keyOp, new AccessControlList(aclStr));
+ }
+ }
+ }
+
+ @Override
+ public void loadAcls(boolean forceReload) {
+ try {
+ if (forceReload || KMSConfiguration.isACLsFileNewer(lastReload)) {
+ setKMSACLs(loadACLs());
+ setKeyACLs(loadACLs());
+ }
+ } catch (Exception ex) {
+ LOG.warn(
+ String.format("Could not reload ACLs file: '%s'", ex.toString()), ex);
+ }
+ }
+
+ private Configuration loadACLs() {
+ LOG.debug("Loading ACLs file");
+ lastReload = System.currentTimeMillis();
+ Configuration conf = KMSConfiguration.getACLsConf();
+ // triggering the resource loading.
+ conf.get(Type.CREATE.getAclConfigKey());
+ return conf;
+ }
+
+ /**
+ * First Check if user is in ACL for the KMS operation, if yes, then
+ * return true if user is not present in any configured blacklist for
+ * the operation
+ * @param keyOperation KMS Operation
+ * @param ugi UserGroupInformation of user
+ * @return true is user has access
+ */
+ @Override
+ public boolean hasAccess(Type keyOperation, UserGroupInformation ugi) {
+ boolean access = acls.get(keyOperation).isUserAllowed(ugi);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Checking user [{}] for: {} {} ", ugi.getShortUserName(),
+ keyOperation.toString(), acls.get(keyOperation).getAclString());
+ }
+ if (access) {
+ AccessControlList blacklist = blacklistedAcls.get(keyOperation);
+ access = (blacklist == null) || !blacklist.isUserInList(ugi);
+ if (LOG.isDebugEnabled()) {
+ if (blacklist == null) {
+ LOG.debug("No blacklist for {}", keyOperation.toString());
+ } else if (access) {
+ LOG.debug("user is not in {}" , blacklist.getAclString());
+ } else {
+ LOG.debug("user is in {}" , blacklist.getAclString());
+ }
+ }
+ }
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("User: [{}], Type: {} Result: {}", ugi.getShortUserName(),
+ keyOperation.toString(), access);
+ }
+ return access;
+ }
+
+ @Override
+ public boolean hasAccessToKey(String keyName, UserGroupInformation ugi,
+ KeyOpType opType) {
+ boolean access = checkKeyAccess(keyName, ugi, opType)
+ || checkKeyAccess(whitelistKeyAcls, ugi, opType);
+ if (!access) {
+ KMSWebApp.getKMSAudit().unauthorized(ugi, opType, keyName);
+ }
+ return access;
+ }
+
+ private boolean checkKeyAccess(String keyName, UserGroupInformation ugi,
+ KeyOpType opType) {
+ Map keyAcl = keyAcls.get(keyName);
+ if (keyAcl == null) {
+ // If No key acl defined for this key, check to see if
+ // there are key defaults configured for this operation
+ LOG.debug("Key: {} has no ACLs defined, using defaults.", keyName);
+ keyAcl = defaultKeyAcls;
+ }
+ boolean access = checkKeyAccess(keyAcl, ugi, opType);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("User: [{}], OpType: {}, KeyName: {} Result: {}",
+ ugi.getShortUserName(), opType.toString(), keyName, access);
+ }
+ return access;
+ }
+
+ private boolean checkKeyAccess(Map keyAcl,
+ UserGroupInformation ugi, KeyOpType opType) {
+ AccessControlList acl = keyAcl.get(opType);
+ if (acl == null) {
+ // If no acl is specified for this operation,
+ // deny access
+ LOG.debug("No ACL available for key, denying access for {}", opType);
+ return false;
+ } else {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Checking user [{}] for: {}: {}", ugi.getShortUserName(),
+ opType.toString(), acl.getAclString());
+ }
+ return acl.isUserAllowed(ugi);
+ }
+ }
+
+
+ @Override
+ public boolean isACLPresent(String keyName, KeyOpType opType) {
+ return (keyAcls.containsKey(keyName)
+ || defaultKeyAcls.containsKey(opType)
+ || whitelistKeyAcls.containsKey(opType));
+ }
+}
diff --git a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSACLs.java b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSACLs.java
index ba0fe825b4eb1..4b7082f61117c 100644
--- a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSACLs.java
+++ b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSACLs.java
@@ -17,33 +17,26 @@
*/
package org.apache.hadoop.crypto.key.kms.server;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.crypto.key.kms.server.KMS.KMSOp;
import org.apache.hadoop.crypto.key.kms.server.KeyAuthorizationKeyProvider.KeyACLs;
import org.apache.hadoop.crypto.key.kms.server.KeyAuthorizationKeyProvider.KeyOpType;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.authorize.AccessControlList;
import org.apache.hadoop.security.authorize.AuthorizationException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-
-import com.google.common.annotations.VisibleForTesting;
-
/**
- * Provides access to the AccessControlLists used by KMS,
- * hot-reloading them if the kms-acls.xml file where the ACLs
- * are defined has been updated.
+ * Base skeleton class for the management of the Keys.
+ * Used by the KMSWebApp to communicate with the key backend storage.
*/
@InterfaceAudience.Private
-public class KMSACLs implements Runnable, KeyACLs {
+public abstract class KMSACLs implements Runnable, KeyACLs {
private static final Logger LOG = LoggerFactory.getLogger(KMSACLs.class);
private static final String UNAUTHORIZED_MSG_WITH_KEY =
@@ -52,6 +45,14 @@ public class KMSACLs implements Runnable, KeyACLs {
private static final String UNAUTHORIZED_MSG_WITHOUT_KEY =
"User:%s not allowed to do '%s'";
+ private static final int RELOADER_SLEEP_MILLIS = 1000;
+
+ private ScheduledExecutorService executorService;
+
+ /**
+ * Enumeration of all the different key operation type.
+ *
+ */
public enum Type {
CREATE, DELETE, ROLLOVER, GET, GET_KEYS, GET_METADATA,
SET_KEY_MATERIAL, GENERATE_EEK, DECRYPT_EEK;
@@ -65,205 +66,95 @@ public String getBlacklistConfigKey() {
}
}
- public static final String ACL_DEFAULT = AccessControlList.WILDCARD_ACL_VALUE;
-
- public static final int RELOADER_SLEEP_MILLIS = 1000;
-
- private volatile Map acls;
- private volatile Map blacklistedAcls;
- @VisibleForTesting
- volatile Map> keyAcls;
- @VisibleForTesting
- volatile Map defaultKeyAcls = new HashMap<>();
- @VisibleForTesting
- volatile Map whitelistKeyAcls = new HashMap<>();
- private ScheduledExecutorService executorService;
- private long lastReload;
-
- KMSACLs(Configuration conf) {
- if (conf == null) {
- conf = loadACLs();
- }
- setKMSACLs(conf);
- setKeyACLs(conf);
- }
-
- public KMSACLs() {
- this(null);
- }
-
- private void setKMSACLs(Configuration conf) {
- Map tempAcls = new HashMap();
- Map tempBlacklist = new HashMap();
- for (Type aclType : Type.values()) {
- String aclStr = conf.get(aclType.getAclConfigKey(), ACL_DEFAULT);
- tempAcls.put(aclType, new AccessControlList(aclStr));
- String blacklistStr = conf.get(aclType.getBlacklistConfigKey());
- if (blacklistStr != null) {
- // Only add if blacklist is present
- tempBlacklist.put(aclType, new AccessControlList(blacklistStr));
- LOG.info("'{}' Blacklist '{}'", aclType, blacklistStr);
- }
- LOG.info("'{}' ACL '{}'", aclType, aclStr);
- }
- acls = tempAcls;
- blacklistedAcls = tempBlacklist;
- }
-
- @VisibleForTesting
- void setKeyACLs(Configuration conf) {
- Map> tempKeyAcls =
- new HashMap>();
- Map allKeyACLS =
- conf.getValByRegex(KMSConfiguration.KEY_ACL_PREFIX_REGEX);
- for (Map.Entry keyAcl : allKeyACLS.entrySet()) {
- String k = keyAcl.getKey();
- // this should be of type "key.acl.."
- int keyNameStarts = KMSConfiguration.KEY_ACL_PREFIX.length();
- int keyNameEnds = k.lastIndexOf(".");
- if (keyNameStarts >= keyNameEnds) {
- LOG.warn("Invalid key name '{}'", k);
- } else {
- String aclStr = keyAcl.getValue();
- String keyName = k.substring(keyNameStarts, keyNameEnds);
- String keyOp = k.substring(keyNameEnds + 1);
- KeyOpType aclType = null;
- try {
- aclType = KeyOpType.valueOf(keyOp);
- } catch (IllegalArgumentException e) {
- LOG.warn("Invalid key Operation '{}'", keyOp);
- }
- if (aclType != null) {
- // On the assumption this will be single threaded.. else we need to
- // ConcurrentHashMap
- HashMap aclMap =
- tempKeyAcls.get(keyName);
- if (aclMap == null) {
- aclMap = new HashMap();
- tempKeyAcls.put(keyName, aclMap);
- }
- aclMap.put(aclType, new AccessControlList(aclStr));
- LOG.info("KEY_NAME '{}' KEY_OP '{}' ACL '{}'",
- keyName, aclType, aclStr);
- }
- }
- }
- keyAcls = tempKeyAcls;
+ /**
+ * First Check if user is in ACL for the KMS operation, if yes, then return
+ * true if user is not present in any configured blacklist for the operation.
+ *
+ * @param keyOperationType
+ * KMS Operation
+ * @param ugi
+ * UserGroupInformation of user
+ * @return true is user has access
+ */
+ public abstract boolean hasAccess(Type keyOperationType,
+ UserGroupInformation ugi);
- final Map tempDefaults = new HashMap<>();
- final Map tempWhitelists = new HashMap<>();
- for (KeyOpType keyOp : KeyOpType.values()) {
- parseAclsWithPrefix(conf, KMSConfiguration.DEFAULT_KEY_ACL_PREFIX,
- keyOp, tempDefaults);
- parseAclsWithPrefix(conf, KMSConfiguration.WHITELIST_KEY_ACL_PREFIX,
- keyOp, tempWhitelists);
- }
- defaultKeyAcls = tempDefaults;
- whitelistKeyAcls = tempWhitelists;
- }
+ /**
+ * This is called by the KeyProvider to check if the given user is
+ * authorized to perform the specified operation on the given acl name.
+ * @param aclName name of the key ACL
+ * @param ugi User's UserGroupInformation
+ * @param opType Operation Type
+ * @return true if user has access to the aclName and opType else false
+ */
+ @Override
+ public abstract boolean hasAccessToKey(String aclName,
+ UserGroupInformation ugi, KeyOpType opType);
/**
- * Parse the acls from configuration with the specified prefix. Currently
- * only 2 possible prefixes: whitelist and default.
*
- * @param conf The configuration.
- * @param prefix The prefix.
- * @param keyOp The key operation.
- * @param results The collection of results to add to.
+ * @param aclName ACL name
+ * @param opType Operation Type
+ * @return true if AclName exists else false
*/
- private void parseAclsWithPrefix(final Configuration conf,
- final String prefix, final KeyOpType keyOp,
- Map results) {
- String confKey = prefix + keyOp;
- String aclStr = conf.get(confKey);
- if (aclStr != null) {
- if (keyOp == KeyOpType.ALL) {
- // Ignore All operation for default key and whitelist key acls
- LOG.warn("Invalid KEY_OP '{}' for {}, ignoring", keyOp, prefix);
- } else {
- if (aclStr.equals("*")) {
- LOG.info("{} for KEY_OP '{}' is set to '*'", prefix, keyOp);
- }
- results.put(keyOp, new AccessControlList(aclStr));
- }
- }
- }
-
@Override
- public void run() {
- try {
- if (KMSConfiguration.isACLsFileNewer(lastReload)) {
- setKMSACLs(loadACLs());
- setKeyACLs(loadACLs());
- }
- } catch (Exception ex) {
- LOG.warn(
- String.format("Could not reload ACLs file: '%s'", ex.toString()), ex);
- }
- }
+ public abstract boolean isACLPresent(String aclName, KeyOpType opType);
+ /**
+ * Loads the ACLs from the persistent store.
+ *
+ * @param forceReload
+ * if true, the cache should be avoided.
+ */
+ public abstract void loadAcls(boolean forceReload);
+
+ /**
+ * Starts the reloader background process.
+ */
public synchronized void startReloader() {
if (executorService == null) {
+ LOG.debug("Starting background reloader for ACL list, implementation: "
+ + this.getClass().getName());
executorService = Executors.newScheduledThreadPool(1);
executorService.scheduleAtFixedRate(this, RELOADER_SLEEP_MILLIS,
RELOADER_SLEEP_MILLIS, TimeUnit.MILLISECONDS);
}
}
+ /**
+ * Stops the reloader background process.
+ */
public synchronized void stopReloader() {
if (executorService != null) {
+ LOG.debug("Stoping background reloader for ACL list");
executorService.shutdownNow();
executorService = null;
}
}
- private Configuration loadACLs() {
- LOG.debug("Loading ACLs file");
- lastReload = System.currentTimeMillis();
- Configuration conf = KMSConfiguration.getACLsConf();
- // triggering the resource loading.
- conf.get(Type.CREATE.getAclConfigKey());
- return conf;
+ @Override
+ public void run() {
+ loadAcls(false);
}
/**
- * First Check if user is in ACL for the KMS operation, if yes, then
- * return true if user is not present in any configured blacklist for
- * the operation
- * @param type KMS Operation
- * @param ugi UserGroupInformation of user
- * @return true is user has access
+ * Asserts, that the given user has the necessary rights to perform the given
+ * operation on the key.
+ *
+ * @param operationType
+ * the Key operation type.
+ * @param ugi
+ * UserGroupInformation of the user
+ * @param operation
+ * the KMS Operation
+ * @param key
+ * the key name
+ * @throws AccessControlException
+ * if the access is denied.
*/
- public boolean hasAccess(Type type, UserGroupInformation ugi) {
- boolean access = acls.get(type).isUserAllowed(ugi);
- if (LOG.isDebugEnabled()) {
- LOG.debug("Checking user [{}] for: {} {} ", ugi.getShortUserName(),
- type.toString(), acls.get(type).getAclString());
- }
- if (access) {
- AccessControlList blacklist = blacklistedAcls.get(type);
- access = (blacklist == null) || !blacklist.isUserInList(ugi);
- if (LOG.isDebugEnabled()) {
- if (blacklist == null) {
- LOG.debug("No blacklist for {}", type.toString());
- } else if (access) {
- LOG.debug("user is not in {}" , blacklist.getAclString());
- } else {
- LOG.debug("user is in {}" , blacklist.getAclString());
- }
- }
- }
- if (LOG.isDebugEnabled()) {
- LOG.debug("User: [{}], Type: {} Result: {}", ugi.getShortUserName(),
- type.toString(), access);
- }
- return access;
- }
-
- public void assertAccess(KMSACLs.Type aclType,
- UserGroupInformation ugi, KMSOp operation, String key)
- throws AccessControlException {
- if (!KMSWebApp.getACLs().hasAccess(aclType, ugi)) {
+ void assertAccess(Type operationType, UserGroupInformation ugi,
+ KMSOp operation, String key) throws AccessControlException {
+ if (!hasAccess(operationType, ugi)) {
KMSWebApp.getUnauthorizedCallsMeter().mark();
KMSWebApp.getKMSAudit().unauthorized(ugi, operation, key);
throw new AuthorizationException(String.format(
@@ -273,61 +164,4 @@ public void assertAccess(KMSACLs.Type aclType,
}
}
- @Override
- public boolean hasAccessToKey(String keyName, UserGroupInformation ugi,
- KeyOpType opType) {
- boolean access = checkKeyAccess(keyName, ugi, opType)
- || checkKeyAccess(whitelistKeyAcls, ugi, opType);
- if (!access) {
- KMSWebApp.getKMSAudit().unauthorized(ugi, opType, keyName);
- }
- return access;
- }
-
- private boolean checkKeyAccess(String keyName, UserGroupInformation ugi,
- KeyOpType opType) {
- Map keyAcl = keyAcls.get(keyName);
- if (keyAcl == null) {
- // If No key acl defined for this key, check to see if
- // there are key defaults configured for this operation
- LOG.debug("Key: {} has no ACLs defined, using defaults.", keyName);
- keyAcl = defaultKeyAcls;
- }
- boolean access = checkKeyAccess(keyAcl, ugi, opType);
- if (LOG.isDebugEnabled()) {
- LOG.debug("User: [{}], OpType: {}, KeyName: {} Result: {}",
- ugi.getShortUserName(), opType.toString(), keyName, access);
- }
- return access;
- }
-
- private boolean checkKeyAccess(Map keyAcl,
- UserGroupInformation ugi, KeyOpType opType) {
- AccessControlList acl = keyAcl.get(opType);
- if (acl == null) {
- // If no acl is specified for this operation,
- // deny access
- LOG.debug("No ACL available for key, denying access for {}", opType);
- return false;
- } else {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Checking user [{}] for: {}: {}", ugi.getShortUserName(),
- opType.toString(), acl.getAclString());
- }
- return acl.isUserAllowed(ugi);
- }
- }
-
-
- @Override
- public boolean isACLPresent(String keyName, KeyOpType opType) {
- return (keyAcls.containsKey(keyName)
- || defaultKeyAcls.containsKey(opType)
- || whitelistKeyAcls.containsKey(opType));
- }
-
- @VisibleForTesting
- void forceNextReloadForTesting() {
- lastReload = 0;
- }
}
diff --git a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAudit.java b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAudit.java
index 13a2d5c57a74e..7337d674dddfe 100644
--- a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAudit.java
+++ b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAudit.java
@@ -23,7 +23,6 @@
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.crypto.key.kms.server.KMSACLs.Type;
import org.apache.hadoop.crypto.key.kms.server.KeyAuthorizationKeyProvider.KeyOpType;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.ReflectionUtils;
diff --git a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAuditLogger.java b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAuditLogger.java
index 2e2ba1d6a1b8f..90a625fc227f2 100644
--- a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAuditLogger.java
+++ b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAuditLogger.java
@@ -23,7 +23,6 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.crypto.key.kms.server.KMSACLs.Type;
import org.apache.hadoop.security.UserGroupInformation;
/**
diff --git a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSConfiguration.java b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSConfiguration.java
index 35ffb429816f7..f36d8900e0c28 100644
--- a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSConfiguration.java
+++ b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSConfiguration.java
@@ -102,6 +102,10 @@ public class KMSConfiguration {
public static final String KEY_AUTHORIZATION_ENABLE = CONFIG_PREFIX +
"key.authorization.enable";
+ // Property to specify KeyManagementACLs implementation
+ public static final String KEY_MANAGEMENT_ACL_CLASS = CONFIG_PREFIX +
+ "key.management.acl.class";
+
public static final boolean KEY_AUTHORIZATION_ENABLE_DEFAULT = true;
private static final String LOG4J_PROPERTIES = "kms-log4j.properties";
diff --git a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebApp.java b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebApp.java
index 0640e25b76c4b..3240e74954bed 100644
--- a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebApp.java
+++ b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebApp.java
@@ -34,6 +34,7 @@
import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
import org.apache.hadoop.crypto.key.KeyProviderFactory;
import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.VersionInfo;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -92,7 +93,7 @@ public void contextInitialized(ServletContextEvent sce) {
LOG.info(" KMS Hadoop Version: " + VersionInfo.getVersion());
LOG.info("-------------------------------------------------------------");
- kmsAcls = new KMSACLs();
+ kmsAcls = createKMSACLs(kmsConf);
kmsAcls.startReloader();
metricRegistry = new MetricRegistry();
@@ -176,6 +177,13 @@ public void contextInitialized(ServletContextEvent sce) {
}
}
+ private KMSACLs createKMSACLs(Configuration conf) {
+ Class extends KMSACLs> aclClass = conf.getClass(
+ KMSConfiguration.KEY_MANAGEMENT_ACL_CLASS, FileBasedKMSACLs.class,
+ KMSACLs.class);
+ return ReflectionUtils.newInstance(aclClass, conf);
+ }
+
@Override
public void contextDestroyed(ServletContextEvent sce) {
try {
diff --git a/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java b/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
index e37f2753d1818..bcd9fe98e0295 100644
--- a/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
+++ b/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
@@ -33,6 +33,7 @@
import org.apache.hadoop.crypto.key.kms.KMSDelegationToken;
import org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider;
import org.apache.hadoop.crypto.key.kms.ValueQueue;
+import org.apache.hadoop.crypto.key.kms.server.KeyAuthorizationKeyProvider.KeyOpType;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.Text;
@@ -371,6 +372,37 @@ public AppConfigurationEntry[] getAppConfigurationEntry(String name) {
}
}
+ private static class TestKeyManagementACLs extends KMSACLs {
+
+ @Override
+ public boolean hasAccessToKey(String aclName, UserGroupInformation ugi,
+ KeyOpType opType) {
+ return false;
+ }
+
+ @Override
+ public boolean isACLPresent(String aclName, KeyOpType opType) {
+ return false;
+ }
+
+ @Override
+ public boolean hasAccess(Type type, UserGroupInformation ugi) {
+ return false;
+ }
+
+ @Override
+ public void loadAcls(boolean forceReload) {
+ }
+
+ @Override
+ public void startReloader() {
+ }
+
+ @Override
+ public void stopReloader() {
+ }
+ }
+
private static MiniKdc kdc;
private static File keytab;
@@ -1360,6 +1392,34 @@ public Void run() throws Exception {
});
}
+ @Test
+ public void testKMSAclConfigurable() throws Exception {
+ Configuration conf = new Configuration();
+ conf.set("hadoop.security.authentication", "kerberos");
+ final File testDir = getTestDir();
+ conf = createBaseKMSConf(testDir, conf);
+ conf.set("hadoop.kms.authentication.type", "kerberos");
+ conf.set("hadoop.kms.authentication.kerberos.keytab",
+ keytab.getAbsolutePath());
+ conf.set("hadoop.kms.authentication.kerberos.principal", "HTTP/localhost");
+ conf.set("hadoop.kms.authentication.kerberos.name.rules", "DEFAULT");
+ conf.set("hadoop.kms.key.management.acl.class",
+ TestKeyManagementACLs.class.getName());
+ writeConf(testDir, conf);
+
+ runServer(null, null, testDir, new KMSCallable() {
+
+ @Override
+ public Void call() throws Exception {
+ Assert.assertNotNull("KMSWebApp.getACLs() is not null",
+ KMSWebApp.getACLs());
+ Assert.assertEquals("Expected KeyManagementACLs type",
+ TestKeyManagementACLs.class, KMSWebApp.getACLs().getClass());
+ return null;
+ }
+ });
+ }
+
@Test
public void testKMSAuthFailureRetry() throws Exception {
Configuration conf = new Configuration();
@@ -1751,8 +1811,7 @@ public Void run() throws Exception {
conf.set(KMSACLs.Type.CREATE.getAclConfigKey(), "foo");
conf.set(KMSACLs.Type.GENERATE_EEK.getAclConfigKey(), "foo");
writeConf(testDir, conf);
- KMSWebApp.getACLs().forceNextReloadForTesting();
- KMSWebApp.getACLs().run(); // forcing a reload by hand.
+ KMSWebApp.getACLs().loadAcls(true); // forcing a reload by hand.
// should not be able to create a key now
doAs("CREATE", new PrivilegedExceptionAction() {
diff --git a/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMSACLs.java b/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMSACLs.java
index 4828fe1582732..43511644d8d67 100644
--- a/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMSACLs.java
+++ b/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMSACLs.java
@@ -39,7 +39,7 @@ public class TestKMSACLs {
@Test
public void testDefaults() {
- final KMSACLs acls = new KMSACLs(new Configuration(false));
+ final KMSACLs acls = new FileBasedKMSACLs(new Configuration(false));
for (KMSACLs.Type type : KMSACLs.Type.values()) {
Assert.assertTrue(acls.hasAccess(type,
UserGroupInformation.createRemoteUser("foo")));
@@ -52,7 +52,7 @@ public void testCustom() {
for (KMSACLs.Type type : KMSACLs.Type.values()) {
conf.set(type.getAclConfigKey(), type.toString() + " ");
}
- final KMSACLs acls = new KMSACLs(conf);
+ final KMSACLs acls = new FileBasedKMSACLs(conf);
for (KMSACLs.Type type : KMSACLs.Type.values()) {
Assert.assertTrue(acls.hasAccess(type,
UserGroupInformation.createRemoteUser(type.toString())));
@@ -71,7 +71,7 @@ public void testKeyAclConfigurationLoad() {
conf.set(WHITELIST_KEY_ACL_PREFIX + "MANAGEMENT", "DECRYPT_EEK");
conf.set(DEFAULT_KEY_ACL_PREFIX + "ALL", "invalid");
conf.set(WHITELIST_KEY_ACL_PREFIX + "ALL", "invalid");
- final KMSACLs acls = new KMSACLs(conf);
+ final FileBasedKMSACLs acls = new FileBasedKMSACLs(conf);
Assert.assertTrue("expected key ACL size is 2 but got "
+ acls.keyAcls.size(), acls.keyAcls.size() == 2);
Assert.assertTrue("expected whitelist ACL size is 1 but got "
@@ -97,7 +97,7 @@ public void testKeyAclDuplicateEntries() {
conf.set(DEFAULT_KEY_ACL_PREFIX + "DECRYPT_EEK", "");
conf.set(WHITELIST_KEY_ACL_PREFIX + "DECRYPT_EEK", "whitelist1");
conf.set(WHITELIST_KEY_ACL_PREFIX + "DECRYPT_EEK", "*");
- final KMSACLs acls = new KMSACLs(conf);
+ final FileBasedKMSACLs acls = new FileBasedKMSACLs(conf);
Assert.assertTrue("expected key ACL size is 2 but got "
+ acls.keyAcls.size(), acls.keyAcls.size() == 2);
assertKeyAcl("test_key_1", acls, KeyOpType.DECRYPT_EEK, "decrypt2");
@@ -121,7 +121,7 @@ public void testKeyAclReload() {
conf.set(WHITELIST_KEY_ACL_PREFIX + "MANAGEMENT", "");
conf.set(WHITELIST_KEY_ACL_PREFIX + "GENERATE_EEK", "*");
conf.set(WHITELIST_KEY_ACL_PREFIX + "DECRYPT_EEK", "admin_decrypt1");
- final KMSACLs acls = new KMSACLs(conf);
+ final FileBasedKMSACLs acls = new FileBasedKMSACLs(conf);
// update config and hot-reload.
conf.set(DEFAULT_KEY_ACL_PREFIX + "READ", "read2");
@@ -187,19 +187,19 @@ public void testKeyAclReload() {
+ acls.defaultKeyAcls, 1, acls.defaultKeyAcls.size());
}
- private void assertDefaultKeyAcl(final KMSACLs acls, final KeyOpType op,
- final String... names) {
+ private void assertDefaultKeyAcl(final FileBasedKMSACLs acls,
+ final KeyOpType op, final String... names) {
final AccessControlList acl = acls.defaultKeyAcls.get(op);
assertAcl(acl, op, names);
}
- private void assertWhitelistKeyAcl(final KMSACLs acls, final KeyOpType op,
- final String... names) {
+ private void assertWhitelistKeyAcl(final FileBasedKMSACLs acls,
+ final KeyOpType op, final String... names) {
final AccessControlList acl = acls.whitelistKeyAcls.get(op);
assertAcl(acl, op, names);
}
- private void assertKeyAcl(final String keyName, final KMSACLs acls,
+ private void assertKeyAcl(final String keyName, final FileBasedKMSACLs acls,
final KeyOpType op, final String... names) {
Assert.assertTrue(acls.keyAcls.containsKey(keyName));
final HashMap keyacl =
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryptionWithKMS.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryptionWithKMS.java
index 642d5e5370752..faf8243687f54 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryptionWithKMS.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryptionWithKMS.java
@@ -85,7 +85,7 @@ public void testReencryptionKMSACLs() throws Exception {
try (Writer writer = new FileWriter(kmsAcl)) {
acl.writeXml(writer);
}
- KMSWebApp.getACLs().run();
+ KMSWebApp.getACLs().loadAcls(true);
testReencryptionBasic();
}