HDFS-9708. FSNamesystem.initAuditLoggers() doesn't trim classnames (Mingliang Liu via stevel)
This commit is contained in:
parent
8f2622b6a0
commit
af2dccbca5
|
@ -2661,6 +2661,9 @@ Release 2.8.0 - UNRELEASED
|
|||
HDFS-9566. Remove expensive 'BlocksMap#getStorages(Block b, final
|
||||
DatanodeStorage.State state)' method (Daryn Sharp via vinayakumarb)
|
||||
|
||||
HDFS-9708. FSNamesystem.initAuditLoggers() doesn't trim classnames
|
||||
(Mingliang Liu via stevel)
|
||||
|
||||
Release 2.7.3 - UNRELEASED
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
|
|
@ -889,7 +889,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|||
|
||||
private List<AuditLogger> initAuditLoggers(Configuration conf) {
|
||||
// Initialize the custom access loggers if configured.
|
||||
Collection<String> alClasses = conf.getStringCollection(DFS_NAMENODE_AUDIT_LOGGERS_KEY);
|
||||
Collection<String> alClasses =
|
||||
conf.getTrimmedStringCollection(DFS_NAMENODE_AUDIT_LOGGERS_KEY);
|
||||
List<AuditLogger> auditLoggers = Lists.newArrayList();
|
||||
if (alClasses != null && !alClasses.isEmpty()) {
|
||||
for (String className : alClasses) {
|
||||
|
|
|
@ -20,14 +20,18 @@ package org.apache.hadoop.hdfs.server.namenode;
|
|||
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY;
|
||||
import static org.hamcrest.CoreMatchers.either;
|
||||
import static org.hamcrest.CoreMatchers.instanceOf;
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.net.InetAddress;
|
||||
import java.net.URI;
|
||||
import java.util.Collection;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileUtil;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.DFSTestUtil;
|
||||
|
@ -38,6 +42,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
|
|||
import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
|
||||
import org.apache.hadoop.hdfs.server.namenode.ha.HAState;
|
||||
import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
|
||||
import org.apache.hadoop.hdfs.server.namenode.top.TopAuditLogger;
|
||||
import org.apache.hadoop.test.GenericTestUtils;
|
||||
import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
|
||||
import org.apache.log4j.Level;
|
||||
|
@ -47,6 +52,7 @@ import org.junit.Test;
|
|||
import org.mockito.Mockito;
|
||||
import org.mockito.internal.util.reflection.Whitebox;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
|
@ -338,4 +344,76 @@ public class TestFSNamesystem {
|
|||
"safeReplication");
|
||||
assertEquals(2, safeReplication);
|
||||
}
|
||||
|
||||
@Test(timeout = 30000)
|
||||
public void testInitAuditLoggers() throws IOException {
|
||||
Configuration conf = new Configuration();
|
||||
FSImage fsImage = Mockito.mock(FSImage.class);
|
||||
FSEditLog fsEditLog = Mockito.mock(FSEditLog.class);
|
||||
Mockito.when(fsImage.getEditLog()).thenReturn(fsEditLog);
|
||||
FSNamesystem fsn;
|
||||
List<AuditLogger> auditLoggers;
|
||||
|
||||
// Not to specify any audit loggers in config
|
||||
conf.set(DFSConfigKeys.DFS_NAMENODE_AUDIT_LOGGERS_KEY, "");
|
||||
// Disable top logger
|
||||
conf.setBoolean(DFSConfigKeys.NNTOP_ENABLED_KEY, false);
|
||||
fsn = new FSNamesystem(conf, fsImage);
|
||||
auditLoggers = fsn.getAuditLoggers();
|
||||
assertTrue(auditLoggers.size() == 1);
|
||||
assertTrue(auditLoggers.get(0) instanceof FSNamesystem.DefaultAuditLogger);
|
||||
|
||||
// Not to specify any audit loggers in config
|
||||
conf.set(DFSConfigKeys.DFS_NAMENODE_AUDIT_LOGGERS_KEY, "");
|
||||
// Enable top logger
|
||||
conf.setBoolean(DFSConfigKeys.NNTOP_ENABLED_KEY, true);
|
||||
fsn = new FSNamesystem(conf, fsImage);
|
||||
auditLoggers = fsn.getAuditLoggers();
|
||||
assertTrue(auditLoggers.size() == 2);
|
||||
// the audit loggers order is not defined
|
||||
for (AuditLogger auditLogger : auditLoggers) {
|
||||
assertThat(auditLogger,
|
||||
either(instanceOf(FSNamesystem.DefaultAuditLogger.class))
|
||||
.or(instanceOf(TopAuditLogger.class)));
|
||||
}
|
||||
|
||||
// Configure default audit loggers in config
|
||||
conf.set(DFSConfigKeys.DFS_NAMENODE_AUDIT_LOGGERS_KEY, "default");
|
||||
// Enable top logger
|
||||
conf.setBoolean(DFSConfigKeys.NNTOP_ENABLED_KEY, true);
|
||||
fsn = new FSNamesystem(conf, fsImage);
|
||||
auditLoggers = fsn.getAuditLoggers();
|
||||
assertTrue(auditLoggers.size() == 2);
|
||||
for (AuditLogger auditLogger : auditLoggers) {
|
||||
assertThat(auditLogger,
|
||||
either(instanceOf(FSNamesystem.DefaultAuditLogger.class))
|
||||
.or(instanceOf(TopAuditLogger.class)));
|
||||
}
|
||||
|
||||
// Configure default and customized audit loggers in config with whitespaces
|
||||
conf.set(DFSConfigKeys.DFS_NAMENODE_AUDIT_LOGGERS_KEY,
|
||||
" default, org.apache.hadoop.hdfs.server.namenode.TestFSNamesystem$DummyAuditLogger ");
|
||||
// Enable top logger
|
||||
conf.setBoolean(DFSConfigKeys.NNTOP_ENABLED_KEY, true);
|
||||
fsn = new FSNamesystem(conf, fsImage);
|
||||
auditLoggers = fsn.getAuditLoggers();
|
||||
assertTrue(auditLoggers.size() == 3);
|
||||
for (AuditLogger auditLogger : auditLoggers) {
|
||||
assertThat(auditLogger,
|
||||
either(instanceOf(FSNamesystem.DefaultAuditLogger.class))
|
||||
.or(instanceOf(TopAuditLogger.class))
|
||||
.or(instanceOf(DummyAuditLogger.class)));
|
||||
}
|
||||
}
|
||||
|
||||
static class DummyAuditLogger implements AuditLogger {
|
||||
@Override
|
||||
public void initialize(Configuration conf) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void logAuditEvent(boolean succeeded, String userName,
|
||||
InetAddress addr, String cmd, String src, String dst, FileStatus stat) {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue