|
@@ -17,12 +17,18 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.crypto.key.kms.server;
|
|
|
|
|
|
+import static org.apache.hadoop.crypto.key.kms.server.KMSAuditLogger.AuditEvent;
|
|
|
+import static org.apache.hadoop.crypto.key.kms.server.KMSAuditLogger.OpStatus;
|
|
|
+
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
+import com.google.common.base.Preconditions;
|
|
|
+import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
+import org.apache.hadoop.util.ReflectionUtils;
|
|
|
+import org.apache.hadoop.util.Time;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
-import com.google.common.base.Joiner;
|
|
|
import com.google.common.base.Strings;
|
|
|
import com.google.common.cache.Cache;
|
|
|
import com.google.common.cache.CacheBuilder;
|
|
@@ -31,67 +37,24 @@ import com.google.common.cache.RemovalNotification;
|
|
|
import com.google.common.collect.Sets;
|
|
|
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
|
|
|
|
|
-import java.util.LinkedList;
|
|
|
-import java.util.List;
|
|
|
-import java.util.Set;
|
|
|
+import java.util.HashSet;
|
|
|
import java.util.concurrent.Callable;
|
|
|
import java.util.concurrent.ExecutionException;
|
|
|
import java.util.concurrent.Executors;
|
|
|
import java.util.concurrent.ScheduledExecutorService;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
-import java.util.concurrent.atomic.AtomicLong;
|
|
|
+import java.util.Collection;
|
|
|
+import java.util.LinkedList;
|
|
|
+import java.util.List;
|
|
|
+import java.util.Set;
|
|
|
|
|
|
/**
|
|
|
- * Provides convenience methods for audit logging consistently the different
|
|
|
+ * Provides convenience methods for audit logging consisting different
|
|
|
* types of events.
|
|
|
*/
|
|
|
public class KMSAudit {
|
|
|
-
|
|
|
- private static class AuditEvent {
|
|
|
- private final AtomicLong accessCount = new AtomicLong(-1);
|
|
|
- private final String keyName;
|
|
|
- private final String user;
|
|
|
- private final KMS.KMSOp op;
|
|
|
- private final String extraMsg;
|
|
|
- private final long startTime = System.currentTimeMillis();
|
|
|
-
|
|
|
- private AuditEvent(String keyName, String user, KMS.KMSOp op, String msg) {
|
|
|
- this.keyName = keyName;
|
|
|
- this.user = user;
|
|
|
- this.op = op;
|
|
|
- this.extraMsg = msg;
|
|
|
- }
|
|
|
-
|
|
|
- public String getExtraMsg() {
|
|
|
- return extraMsg;
|
|
|
- }
|
|
|
-
|
|
|
- public AtomicLong getAccessCount() {
|
|
|
- return accessCount;
|
|
|
- }
|
|
|
-
|
|
|
- public String getKeyName() {
|
|
|
- return keyName;
|
|
|
- }
|
|
|
-
|
|
|
- public String getUser() {
|
|
|
- return user;
|
|
|
- }
|
|
|
-
|
|
|
- public KMS.KMSOp getOp() {
|
|
|
- return op;
|
|
|
- }
|
|
|
-
|
|
|
- public long getStartTime() {
|
|
|
- return startTime;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- public static enum OpStatus {
|
|
|
- OK, UNAUTHORIZED, UNAUTHENTICATED, ERROR;
|
|
|
- }
|
|
|
-
|
|
|
- private static Set<KMS.KMSOp> AGGREGATE_OPS_WHITELIST = Sets.newHashSet(
|
|
|
+ @VisibleForTesting
|
|
|
+ static final Set<KMS.KMSOp> AGGREGATE_OPS_WHITELIST = Sets.newHashSet(
|
|
|
KMS.KMSOp.GET_KEY_VERSION, KMS.KMSOp.GET_CURRENT_KEY,
|
|
|
KMS.KMSOp.DECRYPT_EEK, KMS.KMSOp.GENERATE_EEK
|
|
|
);
|
|
@@ -102,17 +65,21 @@ public class KMSAudit {
|
|
|
|
|
|
public static final String KMS_LOGGER_NAME = "kms-audit";
|
|
|
|
|
|
- private static Logger AUDIT_LOG = LoggerFactory.getLogger(KMS_LOGGER_NAME);
|
|
|
+ private final static Logger LOG = LoggerFactory.getLogger(KMSAudit.class);
|
|
|
+ private final List<KMSAuditLogger> auditLoggers = new LinkedList<>();
|
|
|
|
|
|
/**
|
|
|
* Create a new KMSAudit.
|
|
|
*
|
|
|
- * @param windowMs Duplicate events within the aggregation window are quashed
|
|
|
- * to reduce log traffic. A single message for aggregated
|
|
|
- * events is printed at the end of the window, along with a
|
|
|
- * count of the number of aggregated events.
|
|
|
+ * @param conf The configuration object.
|
|
|
*/
|
|
|
- KMSAudit(long windowMs) {
|
|
|
+ KMSAudit(Configuration conf) {
|
|
|
+ // Duplicate events within the aggregation window are quashed
|
|
|
+ // to reduce log traffic. A single message for aggregated
|
|
|
+ // events is printed at the end of the window, along with a
|
|
|
+ // count of the number of aggregated events.
|
|
|
+ long windowMs = conf.getLong(KMSConfiguration.KMS_AUDIT_AGGREGATION_WINDOW,
|
|
|
+ KMSConfiguration.KMS_AUDIT_AGGREGATION_WINDOW_DEFAULT);
|
|
|
cache = CacheBuilder.newBuilder()
|
|
|
.expireAfterWrite(windowMs, TimeUnit.MILLISECONDS)
|
|
|
.removalListener(
|
|
@@ -122,7 +89,7 @@ public class KMSAudit {
|
|
|
RemovalNotification<String, AuditEvent> entry) {
|
|
|
AuditEvent event = entry.getValue();
|
|
|
if (event.getAccessCount().get() > 0) {
|
|
|
- KMSAudit.this.logEvent(event);
|
|
|
+ KMSAudit.this.logEvent(OpStatus.OK, event);
|
|
|
event.getAccessCount().set(0);
|
|
|
KMSAudit.this.cache.put(entry.getKey(), event);
|
|
|
}
|
|
@@ -136,87 +103,127 @@ public class KMSAudit {
|
|
|
cache.cleanUp();
|
|
|
}
|
|
|
}, windowMs / 10, windowMs / 10, TimeUnit.MILLISECONDS);
|
|
|
+ initializeAuditLoggers(conf);
|
|
|
}
|
|
|
|
|
|
- private void logEvent(AuditEvent event) {
|
|
|
- AUDIT_LOG.info(
|
|
|
- "OK[op={}, key={}, user={}, accessCount={}, interval={}ms] {}",
|
|
|
- event.getOp(), event.getKeyName(), event.getUser(),
|
|
|
- event.getAccessCount().get(),
|
|
|
- (System.currentTimeMillis() - event.getStartTime()),
|
|
|
- event.getExtraMsg());
|
|
|
+ /**
|
|
|
+ * Read the KMSAuditLogger classes from configuration. If any loggers fail to
|
|
|
+ * load, a RumTimeException will be thrown.
|
|
|
+ *
|
|
|
+ * @param conf The configuration.
|
|
|
+ * @return Collection of KMSAudigLogger classes.
|
|
|
+ */
|
|
|
+ private Set<Class<? extends KMSAuditLogger>> getAuditLoggerClasses(
|
|
|
+ final Configuration conf) {
|
|
|
+ Set<Class<? extends KMSAuditLogger>> result = new HashSet<>();
|
|
|
+ // getTrimmedStringCollection will remove duplicates.
|
|
|
+ Collection<String> classes =
|
|
|
+ conf.getTrimmedStringCollection(KMSConfiguration.KMS_AUDIT_LOGGER_KEY);
|
|
|
+ if (classes.isEmpty()) {
|
|
|
+ LOG.info("No audit logger configured, using default.");
|
|
|
+ result.add(SimpleKMSAuditLogger.class);
|
|
|
+ return result;
|
|
|
+ }
|
|
|
+
|
|
|
+ for (String c : classes) {
|
|
|
+ try {
|
|
|
+ Class<?> cls = conf.getClassByName(c);
|
|
|
+ result.add(cls.asSubclass(KMSAuditLogger.class));
|
|
|
+ } catch (ClassNotFoundException cnfe) {
|
|
|
+ throw new RuntimeException("Failed to load " + c + ", please check "
|
|
|
+ + "configuration " + KMSConfiguration.KMS_AUDIT_LOGGER_KEY, cnfe);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return result;
|
|
|
}
|
|
|
|
|
|
- private void op(OpStatus opStatus, final KMS.KMSOp op, final String user,
|
|
|
- final String key, final String extraMsg) {
|
|
|
+ /**
|
|
|
+ * Create a collection of KMSAuditLoggers from configuration, and initialize
|
|
|
+ * them. If any logger failed to be created or initialized, a RunTimeException
|
|
|
+ * is thrown.
|
|
|
+ */
|
|
|
+ private void initializeAuditLoggers(Configuration conf) {
|
|
|
+ Set<Class<? extends KMSAuditLogger>> classes = getAuditLoggerClasses(conf);
|
|
|
+ Preconditions
|
|
|
+ .checkState(!classes.isEmpty(), "Should have at least 1 audit logger.");
|
|
|
+ for (Class<? extends KMSAuditLogger> c : classes) {
|
|
|
+ final KMSAuditLogger logger = ReflectionUtils.newInstance(c, conf);
|
|
|
+ auditLoggers.add(logger);
|
|
|
+ }
|
|
|
+ for (KMSAuditLogger logger: auditLoggers) {
|
|
|
+ try {
|
|
|
+ LOG.info("Initializing audit logger {}", logger.getClass());
|
|
|
+ logger.initialize(conf);
|
|
|
+ } catch (Exception ex) {
|
|
|
+ throw new RuntimeException(
|
|
|
+ "Failed to initialize " + logger.getClass().getName(), ex);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private void logEvent(final OpStatus status, AuditEvent event) {
|
|
|
+ event.setEndTime(Time.now());
|
|
|
+ for (KMSAuditLogger logger: auditLoggers) {
|
|
|
+ logger.logAuditEvent(status, event);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private void op(final OpStatus opStatus, final KMS.KMSOp op,
|
|
|
+ final UserGroupInformation ugi, final String key, final String remoteHost,
|
|
|
+ final String extraMsg) {
|
|
|
+ final String user = ugi == null ? null: ugi.getShortUserName();
|
|
|
if (!Strings.isNullOrEmpty(user) && !Strings.isNullOrEmpty(key)
|
|
|
&& (op != null)
|
|
|
&& AGGREGATE_OPS_WHITELIST.contains(op)) {
|
|
|
String cacheKey = createCacheKey(user, key, op);
|
|
|
if (opStatus == OpStatus.UNAUTHORIZED) {
|
|
|
cache.invalidate(cacheKey);
|
|
|
- AUDIT_LOG.info("UNAUTHORIZED[op={}, key={}, user={}] {}", op, key, user,
|
|
|
- extraMsg);
|
|
|
+ logEvent(opStatus, new AuditEvent(op, ugi, key, remoteHost, extraMsg));
|
|
|
} else {
|
|
|
try {
|
|
|
AuditEvent event = cache.get(cacheKey, new Callable<AuditEvent>() {
|
|
|
@Override
|
|
|
public AuditEvent call() throws Exception {
|
|
|
- return new AuditEvent(key, user, op, extraMsg);
|
|
|
+ return new AuditEvent(op, ugi, key, remoteHost, extraMsg);
|
|
|
}
|
|
|
});
|
|
|
// Log first access (initialized as -1 so
|
|
|
// incrementAndGet() == 0 implies first access)
|
|
|
if (event.getAccessCount().incrementAndGet() == 0) {
|
|
|
event.getAccessCount().incrementAndGet();
|
|
|
- logEvent(event);
|
|
|
+ logEvent(opStatus, event);
|
|
|
}
|
|
|
} catch (ExecutionException ex) {
|
|
|
throw new RuntimeException(ex);
|
|
|
}
|
|
|
}
|
|
|
} else {
|
|
|
- List<String> kvs = new LinkedList<String>();
|
|
|
- if (op != null) {
|
|
|
- kvs.add("op=" + op);
|
|
|
- }
|
|
|
- if (!Strings.isNullOrEmpty(key)) {
|
|
|
- kvs.add("key=" + key);
|
|
|
- }
|
|
|
- if (!Strings.isNullOrEmpty(user)) {
|
|
|
- kvs.add("user=" + user);
|
|
|
- }
|
|
|
- if (kvs.size() == 0) {
|
|
|
- AUDIT_LOG.info("{} {}", opStatus.toString(), extraMsg);
|
|
|
- } else {
|
|
|
- String join = Joiner.on(", ").join(kvs);
|
|
|
- AUDIT_LOG.info("{}[{}] {}", opStatus.toString(), join, extraMsg);
|
|
|
- }
|
|
|
+ logEvent(opStatus, new AuditEvent(op, ugi, key, remoteHost, extraMsg));
|
|
|
}
|
|
|
}
|
|
|
|
|
|
public void ok(UserGroupInformation user, KMS.KMSOp op, String key,
|
|
|
String extraMsg) {
|
|
|
- op(OpStatus.OK, op, user.getShortUserName(), key, extraMsg);
|
|
|
+ op(OpStatus.OK, op, user, key, "Unknown", extraMsg);
|
|
|
}
|
|
|
|
|
|
public void ok(UserGroupInformation user, KMS.KMSOp op, String extraMsg) {
|
|
|
- op(OpStatus.OK, op, user.getShortUserName(), null, extraMsg);
|
|
|
+ op(OpStatus.OK, op, user, null, "Unknown", extraMsg);
|
|
|
}
|
|
|
|
|
|
public void unauthorized(UserGroupInformation user, KMS.KMSOp op, String key) {
|
|
|
- op(OpStatus.UNAUTHORIZED, op, user.getShortUserName(), key, "");
|
|
|
+ op(OpStatus.UNAUTHORIZED, op, user, key, "Unknown", "");
|
|
|
}
|
|
|
|
|
|
public void error(UserGroupInformation user, String method, String url,
|
|
|
String extraMsg) {
|
|
|
- op(OpStatus.ERROR, null, user.getShortUserName(), null, "Method:'" + method
|
|
|
+ op(OpStatus.ERROR, null, user, null, "Unknown", "Method:'" + method
|
|
|
+ "' Exception:'" + extraMsg + "'");
|
|
|
}
|
|
|
|
|
|
public void unauthenticated(String remoteHost, String method,
|
|
|
String url, String extraMsg) {
|
|
|
- op(OpStatus.UNAUTHENTICATED, null, null, null, "RemoteHost:"
|
|
|
+ op(OpStatus.UNAUTHENTICATED, null, null, null, remoteHost, "RemoteHost:"
|
|
|
+ remoteHost + " Method:" + method
|
|
|
+ " URL:" + url + " ErrorMsg:'" + extraMsg + "'");
|
|
|
}
|
|
@@ -227,6 +234,13 @@ public class KMSAudit {
|
|
|
|
|
|
public void shutdown() {
|
|
|
executor.shutdownNow();
|
|
|
+ for (KMSAuditLogger logger : auditLoggers) {
|
|
|
+ try {
|
|
|
+ logger.cleanup();
|
|
|
+ } catch (Exception ex) {
|
|
|
+ LOG.error("Failed to cleanup logger {}", logger.getClass(), ex);
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|