HADOOP-18207. Introduce hadoop-logging module (#5503)

Reviewed-by: Duo Zhang <zhangduo@apache.org>
This commit is contained in:
Viraj Jasani 2023-06-02 18:07:34 -07:00 committed by GitHub
parent 160b9fc3c9
commit 03a499821c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
120 changed files with 1165 additions and 1106 deletions

View File

@ -46,16 +46,6 @@
<artifactId>slf4j-api</artifactId> <artifactId>slf4j-api</artifactId>
<scope>compile</scope> <scope>compile</scope>
</dependency> </dependency>
<dependency>
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
<scope>runtime</scope>
</dependency>
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-log4j12</artifactId>
<scope>runtime</scope>
</dependency>
</dependencies> </dependencies>
<build> <build>

View File

@ -82,14 +82,14 @@
<scope>compile</scope> <scope>compile</scope>
</dependency> </dependency>
<dependency> <dependency>
<groupId>log4j</groupId> <groupId>org.apache.hadoop</groupId>
<artifactId>log4j</artifactId> <artifactId>hadoop-logging</artifactId>
<scope>runtime</scope>
</dependency> </dependency>
<dependency> <dependency>
<groupId>org.slf4j</groupId> <groupId>org.apache.hadoop</groupId>
<artifactId>slf4j-log4j12</artifactId> <artifactId>hadoop-logging</artifactId>
<scope>runtime</scope> <scope>test</scope>
<type>test-jar</type>
</dependency> </dependency>
<dependency> <dependency>
<groupId>org.apache.hadoop</groupId> <groupId>org.apache.hadoop</groupId>

View File

@ -15,8 +15,7 @@ package org.apache.hadoop.security.authentication.util;
import java.util.Random; import java.util.Random;
import org.apache.log4j.Level; import org.apache.hadoop.logging.HadoopLoggerUtils;
import org.apache.log4j.LogManager;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
@ -30,9 +29,8 @@ public class TestRandomSignerSecretProvider {
private final int timeout = 500; private final int timeout = 500;
private final long rolloverFrequency = timeout / 2; private final long rolloverFrequency = timeout / 2;
{ static {
LogManager.getLogger( HadoopLoggerUtils.setLogLevel(RolloverSignerSecretProvider.LOG.getName(), "DEBUG");
RolloverSignerSecretProvider.LOG.getName()).setLevel(Level.DEBUG);
} }
@Test @Test

View File

@ -19,8 +19,7 @@ import java.util.Random;
import javax.servlet.ServletContext; import javax.servlet.ServletContext;
import org.apache.curator.test.TestingServer; import org.apache.curator.test.TestingServer;
import org.apache.log4j.Level; import org.apache.hadoop.logging.HadoopLoggerUtils;
import org.apache.log4j.LogManager;
import org.junit.After; import org.junit.After;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
@ -39,9 +38,8 @@ public class TestZKSignerSecretProvider {
private final int timeout = 100; private final int timeout = 100;
private final long rolloverFrequency = timeout / 2; private final long rolloverFrequency = timeout / 2;
{ static {
LogManager.getLogger( HadoopLoggerUtils.setLogLevel(RolloverSignerSecretProvider.LOG.getName(), "DEBUG");
RolloverSignerSecretProvider.LOG.getName()).setLevel(Level.DEBUG);
} }
@Before @Before

View File

@ -411,6 +411,16 @@
<artifactId>lz4-java</artifactId> <artifactId>lz4-java</artifactId>
<scope>provided</scope> <scope>provided</scope>
</dependency> </dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-logging</artifactId>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-logging</artifactId>
<scope>test</scope>
<type>test-jar</type>
</dependency>
</dependencies> </dependencies>
<build> <build>

View File

@ -299,7 +299,7 @@ log4j.appender.NMAUDIT.MaxBackupIndex=${nm.audit.log.maxbackupindex}
yarn.ewma.cleanupInterval=300 yarn.ewma.cleanupInterval=300
yarn.ewma.messageAgeLimitSeconds=86400 yarn.ewma.messageAgeLimitSeconds=86400
yarn.ewma.maxUniqueMessages=250 yarn.ewma.maxUniqueMessages=250
log4j.appender.EWMA=org.apache.hadoop.yarn.util.Log4jWarningErrorMetricsAppender log4j.appender.EWMA=org.apache.hadoop.logging.appenders.Log4jWarningErrorMetricsAppender
log4j.appender.EWMA.cleanupInterval=${yarn.ewma.cleanupInterval} log4j.appender.EWMA.cleanupInterval=${yarn.ewma.cleanupInterval}
log4j.appender.EWMA.messageAgeLimitSeconds=${yarn.ewma.messageAgeLimitSeconds} log4j.appender.EWMA.messageAgeLimitSeconds=${yarn.ewma.messageAgeLimitSeconds}
log4j.appender.EWMA.maxUniqueMessages=${yarn.ewma.maxUniqueMessages} log4j.appender.EWMA.maxUniqueMessages=${yarn.ewma.maxUniqueMessages}

View File

@ -42,6 +42,7 @@ import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured; import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.http.HttpServer2; import org.apache.hadoop.http.HttpServer2;
import org.apache.hadoop.logging.HadoopLoggerUtils;
import org.apache.hadoop.security.authentication.client.AuthenticatedURL; import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
import org.apache.hadoop.security.authentication.client.KerberosAuthenticator; import org.apache.hadoop.security.authentication.client.KerberosAuthenticator;
import org.apache.hadoop.security.ssl.SSLFactory; import org.apache.hadoop.security.ssl.SSLFactory;
@ -50,8 +51,6 @@ import org.apache.hadoop.util.GenericsUtil;
import org.apache.hadoop.util.ServletUtil; import org.apache.hadoop.util.ServletUtil;
import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner; import org.apache.hadoop.util.ToolRunner;
import org.apache.log4j.Level;
import org.apache.log4j.Logger;
/** /**
* Change log level in runtime. * Change log level in runtime.
@ -349,7 +348,7 @@ public class LogLevel {
} }
if (GenericsUtil.isLog4jLogger(logName)) { if (GenericsUtil.isLog4jLogger(logName)) {
process(Logger.getLogger(logName), level, out); process(logName, level, out);
} else { } else {
out.println("Sorry, setting log level is only supported for log4j loggers.<br />"); out.println("Sorry, setting log level is only supported for log4j loggers.<br />");
} }
@ -368,19 +367,17 @@ public class LogLevel {
+ "<input type='submit' value='Set Log Level' />" + "<input type='submit' value='Set Log Level' />"
+ "</form>"; + "</form>";
private static void process(Logger log, String level, private static void process(String log, String level, PrintWriter out) {
PrintWriter out) throws IOException {
if (level != null) { if (level != null) {
if (!level.equalsIgnoreCase(Level.toLevel(level) try {
.toString())) { HadoopLoggerUtils.setLogLevel(log, level);
out.println(MARKER + "Bad Level : <b>" + level + "</b><br />");
} else {
log.setLevel(Level.toLevel(level));
out.println(MARKER + "Setting Level to " + level + " ...<br />"); out.println(MARKER + "Setting Level to " + level + " ...<br />");
} catch (IllegalArgumentException e) {
out.println(MARKER + "Bad Level : <b>" + level + "</b><br />");
} }
} }
out.println(MARKER out.println(MARKER + "Effective Level: <b>" + HadoopLoggerUtils.getEffectiveLevel(log)
+ "Effective Level: <b>" + log.getEffectiveLevel() + "</b><br />"); + "</b><br />");
} }
} }

View File

@ -40,8 +40,8 @@ import org.apache.commons.lang3.time.FastDateFormat;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.logging.HadoopLoggerUtils;
import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.NetUtils;
import org.apache.log4j.LogManager;
import org.apache.hadoop.thirdparty.com.google.common.net.InetAddresses; import org.apache.hadoop.thirdparty.com.google.common.net.InetAddresses;
@ -761,7 +761,7 @@ public class StringUtils {
public void run() { public void run() {
log.info(toStartupShutdownString("SHUTDOWN_MSG: ", new String[]{ log.info(toStartupShutdownString("SHUTDOWN_MSG: ", new String[]{
"Shutting down " + classname + " at " + hostname})); "Shutting down " + classname + " at " + hostname}));
LogManager.shutdown(); HadoopLoggerUtils.shutdownLogManager();
} }
}, SHUTDOWN_HOOK_PRIORITY); }, SHUTDOWN_HOOK_PRIORITY);

View File

@ -68,6 +68,7 @@ import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.conf.Configuration.IntegerRanges; import org.apache.hadoop.conf.Configuration.IntegerRanges;
import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.logging.LogCapturer;
import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.alias.CredentialProvider; import org.apache.hadoop.security.alias.CredentialProvider;
import org.apache.hadoop.security.alias.CredentialProviderFactory; import org.apache.hadoop.security.alias.CredentialProviderFactory;
@ -76,10 +77,8 @@ import org.apache.hadoop.test.GenericTestUtils;
import static org.apache.hadoop.util.PlatformName.IBM_JAVA; import static org.apache.hadoop.util.PlatformName.IBM_JAVA;
import org.apache.log4j.AppenderSkeleton;
import org.apache.log4j.Logger;
import org.apache.log4j.spi.LoggingEvent;
import org.mockito.Mockito; import org.mockito.Mockito;
import org.slf4j.LoggerFactory;
public class TestConfiguration { public class TestConfiguration {
@ -220,9 +219,7 @@ public class TestConfiguration {
InputStream in2 = new ByteArrayInputStream(bytes2); InputStream in2 = new ByteArrayInputStream(bytes2);
// Attach our own log appender so we can verify output // Attach our own log appender so we can verify output
TestAppender appender = new TestAppender(); LogCapturer logCapturer = LogCapturer.captureLogs(LoggerFactory.getLogger("root"));
final Logger logger = Logger.getRootLogger();
logger.addAppender(appender);
try { try {
// Add the 2 different resources - this should generate a warning // Add the 2 different resources - this should generate a warning
@ -230,17 +227,13 @@ public class TestConfiguration {
conf.addResource(in2); conf.addResource(in2);
assertEquals("should see the first value", "A", conf.get("prop")); assertEquals("should see the first value", "A", conf.get("prop"));
List<LoggingEvent> events = appender.getLog(); String renderedMessage = logCapturer.getOutput();
assertEquals("overriding a final parameter should cause logging", 1, assertTrue("did not see expected string inside message " + renderedMessage,
events.size()); renderedMessage.contains(
LoggingEvent loggingEvent = events.get(0); "an attempt to override final parameter: " + "prop; Ignoring."));
String renderedMessage = loggingEvent.getRenderedMessage();
assertTrue("did not see expected string inside message "+ renderedMessage,
renderedMessage.contains("an attempt to override final parameter: "
+ "prop; Ignoring."));
} finally { } finally {
// Make sure the appender is removed // Make sure the appender is removed
logger.removeAppender(appender); logCapturer.stopCapturing();
} }
} }
@ -258,9 +251,7 @@ public class TestConfiguration {
InputStream in2 = new ByteArrayInputStream(bytes); InputStream in2 = new ByteArrayInputStream(bytes);
// Attach our own log appender so we can verify output // Attach our own log appender so we can verify output
TestAppender appender = new TestAppender(); LogCapturer logCapturer = LogCapturer.captureLogs(LoggerFactory.getLogger("root"));
final Logger logger = Logger.getRootLogger();
logger.addAppender(appender);
try { try {
// Add the resource twice from a stream - should not generate warnings // Add the resource twice from a stream - should not generate warnings
@ -268,20 +259,15 @@ public class TestConfiguration {
conf.addResource(in2); conf.addResource(in2);
assertEquals("A", conf.get("prop")); assertEquals("A", conf.get("prop"));
List<LoggingEvent> events = appender.getLog(); String appenderOutput = logCapturer.getOutput();
for (LoggingEvent loggingEvent : events) {
System.out.println("Event = " + loggingEvent.getRenderedMessage());
}
assertTrue("adding same resource twice should not cause logging", assertTrue("adding same resource twice should not cause logging",
events.isEmpty()); appenderOutput.isEmpty());
} finally { } finally {
// Make sure the appender is removed // Make sure the appender is removed
logger.removeAppender(appender); logCapturer.stopCapturing();
} }
} }
@Test @Test
public void testFinalWarningsMultiple() throws Exception { public void testFinalWarningsMultiple() throws Exception {
// Make a configuration file with a repeated final property // Make a configuration file with a repeated final property
@ -295,24 +281,19 @@ public class TestConfiguration {
InputStream in1 = new ByteArrayInputStream(bytes); InputStream in1 = new ByteArrayInputStream(bytes);
// Attach our own log appender so we can verify output // Attach our own log appender so we can verify output
TestAppender appender = new TestAppender(); LogCapturer logCapturer = LogCapturer.captureLogs(LoggerFactory.getLogger("root"));
final Logger logger = Logger.getRootLogger();
logger.addAppender(appender);
try { try {
// Add the resource - this should not produce a warning // Add the resource - this should not produce a warning
conf.addResource(in1); conf.addResource(in1);
assertEquals("should see the value", "A", conf.get("prop")); assertEquals("should see the value", "A", conf.get("prop"));
List<LoggingEvent> events = appender.getLog(); String appenderOutput = logCapturer.getOutput();
for (LoggingEvent loggingEvent : events) {
System.out.println("Event = " + loggingEvent.getRenderedMessage());
}
assertTrue("adding same resource twice should not cause logging", assertTrue("adding same resource twice should not cause logging",
events.isEmpty()); appenderOutput.isEmpty());
} finally { } finally {
// Make sure the appender is removed // Make sure the appender is removed
logger.removeAppender(appender); logCapturer.stopCapturing();
} }
} }
@ -329,48 +310,20 @@ public class TestConfiguration {
InputStream in1 = new ByteArrayInputStream(bytes); InputStream in1 = new ByteArrayInputStream(bytes);
// Attach our own log appender so we can verify output // Attach our own log appender so we can verify output
TestAppender appender = new TestAppender(); LogCapturer logCapturer = LogCapturer.captureLogs(LoggerFactory.getLogger("root"));
final Logger logger = Logger.getRootLogger();
logger.addAppender(appender);
try { try {
// Add the resource - this should produce a warning // Add the resource - this should produce a warning
conf.addResource(in1); conf.addResource(in1);
assertEquals("should see the value", "A", conf.get("prop")); assertEquals("should see the value", "A", conf.get("prop"));
List<LoggingEvent> events = appender.getLog(); String renderedMessage = logCapturer.getOutput();
assertEquals("overriding a final parameter should cause logging", 1, assertTrue("did not see expected string inside message " + renderedMessage,
events.size()); renderedMessage.contains(
LoggingEvent loggingEvent = events.get(0); "an attempt to override final parameter: " + "prop; Ignoring."));
String renderedMessage = loggingEvent.getRenderedMessage();
assertTrue("did not see expected string inside message "+ renderedMessage,
renderedMessage.contains("an attempt to override final parameter: "
+ "prop; Ignoring."));
} finally { } finally {
// Make sure the appender is removed // Make sure the appender is removed
logger.removeAppender(appender); logCapturer.stopCapturing();
}
}
/**
* A simple appender for white box testing.
*/
private static class TestAppender extends AppenderSkeleton {
private final List<LoggingEvent> log = new ArrayList<>();
@Override public boolean requiresLayout() {
return false;
}
@Override protected void append(final LoggingEvent loggingEvent) {
log.add(loggingEvent);
}
@Override public void close() {
}
public List<LoggingEvent> getLog() {
return new ArrayList<>(log);
} }
} }

View File

@ -36,8 +36,9 @@ import org.apache.hadoop.io.compress.zlib.BuiltInZlibDeflater;
import org.apache.hadoop.io.compress.zlib.ZlibCompressor; import org.apache.hadoop.io.compress.zlib.ZlibCompressor;
import org.apache.hadoop.io.compress.zlib.ZlibFactory; import org.apache.hadoop.io.compress.zlib.ZlibFactory;
import org.apache.hadoop.util.NativeCodeLoader; import org.apache.hadoop.util.NativeCodeLoader;
import org.apache.log4j.Logger;
import org.junit.Assert; import org.junit.Assert;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.thirdparty.com.google.common.base.Joiner; import org.apache.hadoop.thirdparty.com.google.common.base.Joiner;
import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableList; import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableList;
@ -47,9 +48,6 @@ import static org.junit.Assert.*;
public class CompressDecompressTester<T extends Compressor, E extends Decompressor> { public class CompressDecompressTester<T extends Compressor, E extends Decompressor> {
private static final Logger logger = Logger
.getLogger(CompressDecompressTester.class);
private final byte[] originalRawData; private final byte[] originalRawData;
private ImmutableList<TesterPair<T, E>> pairs = ImmutableList.of(); private ImmutableList<TesterPair<T, E>> pairs = ImmutableList.of();
@ -488,12 +486,12 @@ public class CompressDecompressTester<T extends Compressor, E extends Decompress
return false; return false;
} }
abstract static class TesterCompressionStrategy { abstract static class TesterCompressionStrategy {
protected final Logger logger = Logger.getLogger(getClass()); protected final Logger logger = LoggerFactory.getLogger(getClass());
abstract void assertCompression(String name, Compressor compressor, abstract void assertCompression(String name, Compressor compressor, Decompressor decompressor,
Decompressor decompressor, byte[] originalRawData) throws Exception; byte[] originalRawData) throws Exception;
} }
} }

View File

@ -29,6 +29,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.http.HttpServer2; import org.apache.hadoop.http.HttpServer2;
import org.apache.hadoop.log.LogLevel.CLI; import org.apache.hadoop.log.LogLevel.CLI;
import org.apache.hadoop.logging.HadoopLoggerUtils;
import org.apache.hadoop.minikdc.KerberosSecurityTestcase; import org.apache.hadoop.minikdc.KerberosSecurityTestcase;
import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.AuthenticationFilterInitializer; import org.apache.hadoop.security.AuthenticationFilterInitializer;
@ -40,12 +41,11 @@ import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
import org.junit.Assert; import org.junit.Assert;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.log4j.Level;
import org.apache.log4j.Logger;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.Before; import org.junit.Before;
import org.junit.BeforeClass; import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import javax.net.ssl.SSLException; import javax.net.ssl.SSLException;
@ -67,7 +67,7 @@ public class TestLogLevel extends KerberosSecurityTestcase {
private final String logName = TestLogLevel.class.getName(); private final String logName = TestLogLevel.class.getName();
private String clientPrincipal; private String clientPrincipal;
private String serverPrincipal; private String serverPrincipal;
private final Logger log = Logger.getLogger(logName); private final Logger log = LoggerFactory.getLogger(logName);
private final static String PRINCIPAL = "loglevel.principal"; private final static String PRINCIPAL = "loglevel.principal";
private final static String KEYTAB = "loglevel.keytab"; private final static String KEYTAB = "loglevel.keytab";
private static final String PREFIX = "hadoop.http.authentication."; private static final String PREFIX = "hadoop.http.authentication.";
@ -76,7 +76,7 @@ public class TestLogLevel extends KerberosSecurityTestcase {
public static void setUp() throws Exception { public static void setUp() throws Exception {
org.slf4j.Logger logger = org.slf4j.Logger logger =
LoggerFactory.getLogger(KerberosAuthenticator.class); LoggerFactory.getLogger(KerberosAuthenticator.class);
GenericTestUtils.setLogLevel(logger, Level.DEBUG); HadoopLoggerUtils.setLogLevel(logger.getName(), "DEBUG");
FileUtil.fullyDelete(BASEDIR); FileUtil.fullyDelete(BASEDIR);
if (!BASEDIR.mkdirs()) { if (!BASEDIR.mkdirs()) {
throw new Exception("unable to create the base directory for testing"); throw new Exception("unable to create the base directory for testing");
@ -230,7 +230,7 @@ public class TestLogLevel extends KerberosSecurityTestcase {
final String connectProtocol, final boolean isSpnego) final String connectProtocol, final boolean isSpnego)
throws Exception { throws Exception {
testDynamicLogLevel(bindProtocol, connectProtocol, isSpnego, testDynamicLogLevel(bindProtocol, connectProtocol, isSpnego,
Level.DEBUG.toString()); "DEBUG");
} }
/** /**
@ -250,9 +250,8 @@ public class TestLogLevel extends KerberosSecurityTestcase {
if (!LogLevel.isValidProtocol(connectProtocol)) { if (!LogLevel.isValidProtocol(connectProtocol)) {
throw new Exception("Invalid client protocol " + connectProtocol); throw new Exception("Invalid client protocol " + connectProtocol);
} }
Level oldLevel = log.getEffectiveLevel(); String oldLevel = HadoopLoggerUtils.getEffectiveLevel(log.getName());
Assert.assertNotEquals("Get default Log Level which shouldn't be ERROR.", Assert.assertNotEquals("Get default Log Level which shouldn't be ERROR.", "ERROR", oldLevel);
Level.ERROR, oldLevel);
// configs needed for SPNEGO at server side // configs needed for SPNEGO at server side
if (isSpnego) { if (isSpnego) {
@ -288,7 +287,7 @@ public class TestLogLevel extends KerberosSecurityTestcase {
}); });
server.stop(); server.stop();
// restore log level // restore log level
GenericTestUtils.setLogLevel(log, oldLevel); HadoopLoggerUtils.setLogLevel(log.getName(), oldLevel.toString());
} }
/** /**
@ -322,7 +321,7 @@ public class TestLogLevel extends KerberosSecurityTestcase {
cli.run(setLevelArgs); cli.run(setLevelArgs);
assertEquals("new level not equal to expected: ", newLevel.toUpperCase(), assertEquals("new level not equal to expected: ", newLevel.toUpperCase(),
log.getEffectiveLevel().toString()); HadoopLoggerUtils.getEffectiveLevel(log.getName()));
} }
/** /**

View File

@ -42,8 +42,9 @@ import org.apache.hadoop.metrics2.lib.MutableCounterLong;
import static org.apache.hadoop.metrics2.lib.Interns.info; import static org.apache.hadoop.metrics2.lib.Interns.info;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import org.apache.log4j.Logger;
import org.junit.Test; import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.management.MBeanAttributeInfo; import javax.management.MBeanAttributeInfo;
import javax.management.MBeanInfo; import javax.management.MBeanInfo;
@ -241,7 +242,7 @@ public class TestMetricsSourceAdapter {
private MetricsSourceAdapter sa = null; private MetricsSourceAdapter sa = null;
private ScheduledFuture<?> future = null; private ScheduledFuture<?> future = null;
private AtomicBoolean hasError = null; private AtomicBoolean hasError = null;
private static final Logger LOG = Logger.getLogger(SourceUpdater.class); private static final Logger LOG = LoggerFactory.getLogger(SourceUpdater.class);
public SourceUpdater(MetricsSourceAdapter sourceAdapter, public SourceUpdater(MetricsSourceAdapter sourceAdapter,
AtomicBoolean err) { AtomicBoolean err) {
@ -263,7 +264,7 @@ public class TestMetricsSourceAdapter {
} catch (Exception e) { } catch (Exception e) {
// catch all errors // catch all errors
hasError.set(true); hasError.set(true);
LOG.error(e.getStackTrace()); LOG.error("Something went wrong.", e);
} finally { } finally {
if (hasError.get()) { if (hasError.get()) {
LOG.error("Hit error, stopping now"); LOG.error("Hit error, stopping now");
@ -284,7 +285,7 @@ public class TestMetricsSourceAdapter {
private int cnt = 0; private int cnt = 0;
private ScheduledFuture<?> future = null; private ScheduledFuture<?> future = null;
private AtomicBoolean hasError = null; private AtomicBoolean hasError = null;
private static final Logger LOG = Logger.getLogger(SourceReader.class); private static final Logger LOG = LoggerFactory.getLogger(SourceReader.class);
public SourceReader( public SourceReader(
TestMetricsSource source, MetricsSourceAdapter sourceAdapter, TestMetricsSource source, MetricsSourceAdapter sourceAdapter,
@ -318,7 +319,7 @@ public class TestMetricsSourceAdapter {
} catch (Exception e) { } catch (Exception e) {
// catch other errors // catch other errors
hasError.set(true); hasError.set(true);
LOG.error(e.getStackTrace()); LOG.error("Something went wrong.", e);
} finally { } finally {
if (hasError.get()) { if (hasError.get()) {
future.cancel(false); future.cancel(false);

View File

@ -22,7 +22,7 @@ import java.util.List;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.logging.LogCapturer;
import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.Shell; import org.apache.hadoop.util.Shell;
import org.apache.hadoop.util.Shell.ExitCodeException; import org.apache.hadoop.util.Shell.ExitCodeException;
@ -41,8 +41,8 @@ public class TestShellBasedUnixGroupsMapping {
private static final Logger TESTLOG = private static final Logger TESTLOG =
LoggerFactory.getLogger(TestShellBasedUnixGroupsMapping.class); LoggerFactory.getLogger(TestShellBasedUnixGroupsMapping.class);
private final GenericTestUtils.LogCapturer shellMappingLog = private final LogCapturer shellMappingLog =
GenericTestUtils.LogCapturer.captureLogs( LogCapturer.captureLogs(
ShellBasedUnixGroupsMapping.LOG); ShellBasedUnixGroupsMapping.LOG);
private class TestGroupUserNotExist private class TestGroupUserNotExist

View File

@ -19,6 +19,8 @@ package org.apache.hadoop.security.ssl;
import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.logging.LogCapturer;
import org.junit.BeforeClass; import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
@ -42,7 +44,7 @@ public class TestReloadingX509KeyManager {
private static final String BASEDIR = GenericTestUtils.getTempPath( private static final String BASEDIR = GenericTestUtils.getTempPath(
TestReloadingX509TrustManager.class.getSimpleName()); TestReloadingX509TrustManager.class.getSimpleName());
private final GenericTestUtils.LogCapturer reloaderLog = GenericTestUtils.LogCapturer.captureLogs( private final LogCapturer reloaderLog = LogCapturer.captureLogs(
FileMonitoringTimerTask.LOG); FileMonitoringTimerTask.LOG);
@BeforeClass @BeforeClass

View File

@ -19,7 +19,7 @@ package org.apache.hadoop.security.ssl;
import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.GenericTestUtils.LogCapturer; import org.apache.hadoop.logging.LogCapturer;
import java.util.function.Supplier; import java.util.function.Supplier;

View File

@ -18,7 +18,7 @@
package org.apache.hadoop.service; package org.apache.hadoop.service;
import org.apache.hadoop.test.GenericTestUtils.LogCapturer; import org.apache.hadoop.logging.LogCapturer;
import org.junit.Test; import org.junit.Test;
import org.junit.runner.RunWith; import org.junit.runner.RunWith;
import org.mockito.Mock; import org.mockito.Mock;
@ -29,7 +29,7 @@ import org.slf4j.LoggerFactory;
import java.io.PrintWriter; import java.io.PrintWriter;
import static org.apache.hadoop.test.GenericTestUtils.LogCapturer.captureLogs; import static org.apache.hadoop.logging.LogCapturer.captureLogs;
import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThat;
import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.doThrow;
import static org.mockito.Mockito.times; import static org.mockito.Mockito.times;

View File

@ -25,7 +25,6 @@ import java.io.IOException;
import java.io.InputStreamReader; import java.io.InputStreamReader;
import java.io.OutputStream; import java.io.OutputStream;
import java.io.PrintStream; import java.io.PrintStream;
import java.io.StringWriter;
import java.lang.management.ManagementFactory; import java.lang.management.ManagementFactory;
import java.lang.management.ThreadInfo; import java.lang.management.ThreadInfo;
import java.lang.management.ThreadMXBean; import java.lang.management.ThreadMXBean;
@ -38,7 +37,6 @@ import java.util.Locale;
import java.util.Objects; import java.util.Objects;
import java.util.Random; import java.util.Random;
import java.util.Set; import java.util.Set;
import java.util.Enumeration;
import java.util.TreeSet; import java.util.TreeSet;
import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CountDownLatch; import java.util.concurrent.CountDownLatch;
@ -53,17 +51,11 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.logging.HadoopLoggerUtils;
import org.apache.hadoop.util.BlockingThreadPoolExecutorService; import org.apache.hadoop.util.BlockingThreadPoolExecutorService;
import org.apache.hadoop.util.DurationInfo; import org.apache.hadoop.util.DurationInfo;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
import org.apache.log4j.Appender;
import org.apache.log4j.Layout;
import org.apache.log4j.Level;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.log4j.PatternLayout;
import org.apache.log4j.WriterAppender;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Assume; import org.junit.Assume;
import org.mockito.invocation.InvocationOnMock; import org.mockito.invocation.InvocationOnMock;
@ -115,51 +107,17 @@ public abstract class GenericTestUtils {
public static final String ERROR_INVALID_ARGUMENT = public static final String ERROR_INVALID_ARGUMENT =
"Total wait time should be greater than check interval time"; "Total wait time should be greater than check interval time";
@Deprecated
public static Logger toLog4j(org.slf4j.Logger logger) {
return LogManager.getLogger(logger.getName());
}
/**
* @deprecated use {@link #disableLog(org.slf4j.Logger)} instead
*/
@Deprecated
public static void disableLog(Logger logger) {
logger.setLevel(Level.OFF);
}
public static void disableLog(org.slf4j.Logger logger) { public static void disableLog(org.slf4j.Logger logger) {
disableLog(toLog4j(logger)); HadoopLoggerUtils.setLogLevel(logger.getName(), "OFF");
}
public static void setLogLevel(Logger logger, Level level) {
logger.setLevel(level);
}
/**
* @deprecated
* use {@link #setLogLevel(org.slf4j.Logger, org.slf4j.event.Level)} instead
*/
@Deprecated
public static void setLogLevel(org.slf4j.Logger logger, Level level) {
setLogLevel(toLog4j(logger), level);
} }
public static void setLogLevel(org.slf4j.Logger logger, public static void setLogLevel(org.slf4j.Logger logger,
org.slf4j.event.Level level) { org.slf4j.event.Level level) {
setLogLevel(toLog4j(logger), Level.toLevel(level.toString())); HadoopLoggerUtils.setLogLevel(logger.getName(), level.toString());
} }
public static void setRootLogLevel(org.slf4j.event.Level level) { public static void setRootLogLevel(org.slf4j.event.Level level) {
setLogLevel(LogManager.getRootLogger(), Level.toLevel(level.toString())); HadoopLoggerUtils.setLogLevel("root", level.toString());
}
public static void setCurrentLoggersLogLevel(org.slf4j.event.Level level) {
for (Enumeration<?> loggers = LogManager.getCurrentLoggers();
loggers.hasMoreElements();) {
Logger logger = (Logger) loggers.nextElement();
logger.setLevel(Level.toLevel(level.toString()));
}
} }
public static org.slf4j.event.Level toLevel(String level) { public static org.slf4j.event.Level toLevel(String level) {
@ -471,47 +429,6 @@ public abstract class GenericTestUtils {
} }
} }
public static class LogCapturer {
private StringWriter sw = new StringWriter();
private WriterAppender appender;
private Logger logger;
public static LogCapturer captureLogs(org.slf4j.Logger logger) {
if (logger.getName().equals("root")) {
return new LogCapturer(org.apache.log4j.Logger.getRootLogger());
}
return new LogCapturer(toLog4j(logger));
}
public static LogCapturer captureLogs(Logger logger) {
return new LogCapturer(logger);
}
private LogCapturer(Logger logger) {
this.logger = logger;
Appender defaultAppender = Logger.getRootLogger().getAppender("stdout");
if (defaultAppender == null) {
defaultAppender = Logger.getRootLogger().getAppender("console");
}
final Layout layout = (defaultAppender == null) ? new PatternLayout() :
defaultAppender.getLayout();
this.appender = new WriterAppender(layout, sw);
logger.addAppender(this.appender);
}
public String getOutput() {
return sw.toString();
}
public void stopCapturing() {
logger.removeAppender(appender);
}
public void clearOutput() {
sw.getBuffer().setLength(0);
}
}
/** /**
* Mockito answer helper that triggers one latch as soon as the * Mockito answer helper that triggers one latch as soon as the
* method is called, then waits on another before continuing. * method is called, then waits on another before continuing.

View File

@ -26,6 +26,8 @@ import org.slf4j.LoggerFactory;
import java.util.function.Supplier; import java.util.function.Supplier;
import org.slf4j.event.Level; import org.slf4j.event.Level;
import org.apache.hadoop.logging.LogCapturer;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail; import static org.junit.Assert.fail;

View File

@ -22,8 +22,8 @@ import java.io.File;
import org.junit.Assert; import org.junit.Assert;
import org.apache.log4j.Logger;
import org.junit.Test; import org.junit.Test;
import org.slf4j.Logger;
public class TestClassUtil { public class TestClassUtil {
@Test(timeout=10000) @Test(timeout=10000)
@ -35,6 +35,6 @@ public class TestClassUtil {
Assert.assertTrue("Containing jar does not exist on file system ", Assert.assertTrue("Containing jar does not exist on file system ",
jarFile.exists()); jarFile.exists());
Assert.assertTrue("Incorrect jar file " + containingJar, Assert.assertTrue("Incorrect jar file " + containingJar,
jarFile.getName().matches("log4j.*[.]jar")); jarFile.getName().matches("slf4j-api.*[.]jar"));
} }
} }

View File

@ -28,7 +28,7 @@ import java.util.List;
import static org.junit.Assert.*; import static org.junit.Assert.*;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.GenericTestUtils.LogCapturer; import org.apache.hadoop.logging.LogCapturer;
import org.assertj.core.api.Assertions; import org.assertj.core.api.Assertions;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;

View File

@ -28,10 +28,12 @@ import java.util.Iterator;
import java.util.Random; import java.util.Random;
import org.junit.Assert; import org.junit.Assert;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.io.DataInputBuffer; import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.util.hash.Hash; import org.apache.hadoop.util.hash.Hash;
import org.apache.log4j.Logger;
import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableList; import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableList;
import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet; import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet;
@ -113,7 +115,7 @@ public class BloomFilterCommonTester<T extends Filter> {
} }
interface FilterTesterStrategy { interface FilterTesterStrategy {
final Logger logger = Logger.getLogger(FilterTesterStrategy.class); Logger logger = LoggerFactory.getLogger(FilterTesterStrategy.class);
void assertWhat(Filter filter, int numInsertions, int hashId, void assertWhat(Filter filter, int numInsertions, int hashId,
ImmutableSet<Integer> falsePositives); ImmutableSet<Integer> falsePositives);

View File

@ -53,6 +53,12 @@
<artifactId>hadoop-auth</artifactId> <artifactId>hadoop-auth</artifactId>
<scope>compile</scope> <scope>compile</scope>
</dependency> </dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-logging</artifactId>
<scope>test</scope>
<type>test-jar</type>
</dependency>
<dependency> <dependency>
<groupId>org.apache.hadoop.thirdparty</groupId> <groupId>org.apache.hadoop.thirdparty</groupId>
<artifactId>hadoop-shaded-guava</artifactId> <artifactId>hadoop-shaded-guava</artifactId>

View File

@ -49,6 +49,7 @@ import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.security.token.delegation.web.DelegationTokenIdentifier; import org.apache.hadoop.security.token.delegation.web.DelegationTokenIdentifier;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.logging.LogCapturer;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
import org.apache.http.client.utils.URIBuilder; import org.apache.http.client.utils.URIBuilder;
import org.junit.After; import org.junit.After;
@ -583,8 +584,8 @@ public class TestKMS {
@Test @Test
public void testStartStopHttpPseudo() throws Exception { public void testStartStopHttpPseudo() throws Exception {
// Make sure bogus errors don't get emitted. // Make sure bogus errors don't get emitted.
GenericTestUtils.LogCapturer logs = LogCapturer logs =
GenericTestUtils.LogCapturer.captureLogs(LoggerFactory.getLogger( LogCapturer.captureLogs(LoggerFactory.getLogger(
"com.sun.jersey.server.wadl.generators.AbstractWadlGeneratorGrammarGenerator")); "com.sun.jersey.server.wadl.generators.AbstractWadlGeneratorGrammarGenerator"));
try { try {
testStartStop(false, false); testStartStop(false, false);

View File

@ -18,23 +18,24 @@
package org.apache.hadoop.crypto.key.kms.server; package org.apache.hadoop.crypto.key.kms.server;
import java.io.ByteArrayOutputStream; import java.io.ByteArrayOutputStream;
import java.io.File;
import java.io.FilterOutputStream; import java.io.FilterOutputStream;
import java.io.InputStream;
import java.io.IOException; import java.io.IOException;
import java.io.OutputStream; import java.io.OutputStream;
import java.io.PrintStream; import java.io.PrintStream;
import java.net.URISyntaxException;
import java.net.URL;
import java.nio.file.Paths;
import java.util.List; import java.util.List;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.commons.lang3.reflect.FieldUtils;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.crypto.key.kms.server.KMS.KMSOp; import org.apache.hadoop.crypto.key.kms.server.KMS.KMSOp;
import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.logging.HadoopLoggerUtils;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.ThreadUtil;
import org.apache.log4j.LogManager;
import org.apache.log4j.PropertyConfigurator;
import org.junit.After; import org.junit.After;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
@ -67,24 +68,23 @@ public class TestKMSAudit {
public final Timeout testTimeout = new Timeout(180000L, TimeUnit.MILLISECONDS); public final Timeout testTimeout = new Timeout(180000L, TimeUnit.MILLISECONDS);
@Before @Before
public void setUp() throws IOException { public void setUp() throws IOException, URISyntaxException {
originalOut = System.err; originalOut = System.err;
memOut = new ByteArrayOutputStream(); memOut = new ByteArrayOutputStream();
filterOut = new FilterOut(memOut); filterOut = new FilterOut(memOut);
capturedOut = new PrintStream(filterOut); capturedOut = new PrintStream(filterOut);
System.setErr(capturedOut); System.setErr(capturedOut);
InputStream is = URL url = getClass().getClassLoader().getResource("log4j-kmsaudit.properties");
ThreadUtil.getResourceAsStream("log4j-kmsaudit.properties"); File file = Paths.get(url.toURI()).toFile();
PropertyConfigurator.configure(is); HadoopLoggerUtils.updateLog4jConfiguration(KMSAudit.class, file.getAbsolutePath());
IOUtils.closeStream(is);
Configuration conf = new Configuration(); Configuration conf = new Configuration();
this.kmsAudit = new KMSAudit(conf); this.kmsAudit = new KMSAudit(conf);
} }
@After @After
public void cleanUp() { public void cleanUp() throws Exception {
System.setErr(originalOut); System.setErr(originalOut);
LogManager.resetConfiguration(); HadoopLoggerUtils.resetConfiguration();
kmsAudit.shutdown(); kmsAudit.shutdown();
} }

View File

@ -0,0 +1,23 @@
<FindBugsFilter>
<!--
conversionPattern is only set once and used to initiate PatternLayout object
only once. It is set by log4j framework if set as part of log4j properties and accessed
only during first append operation.
-->
<Match>
<Class name="org.apache.hadoop.logging.appenders.AsyncRFAAppender"/>
<Field name="conversionPattern"/>
<Bug pattern="IS2_INCONSISTENT_SYNC"/>
</Match>
<!-- Following fields are used in ErrorsAndWarningsBlock, which is not a part of analysis of findbugs -->
<Match>
<Class name="org.apache.hadoop.logging.appenders.Log4jWarningErrorMetricsAppender$Element"/>
<Or>
<Field name="count"/>
<Field name="timestampSeconds"/>
</Or>
<Bug pattern="URF_UNREAD_PUBLIC_OR_PROTECTED_FIELD"/>
</Match>
</FindBugsFilter>

View File

@ -0,0 +1,125 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
Licensed 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. See accompanying LICENSE file.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<parent>
<artifactId>hadoop-project</artifactId>
<groupId>org.apache.hadoop</groupId>
<version>3.4.0-SNAPSHOT</version>
<relativePath>../../hadoop-project</relativePath>
</parent>
<modelVersion>4.0.0</modelVersion>
<artifactId>hadoop-logging</artifactId>
<version>3.4.0-SNAPSHOT</version>
<packaging>jar</packaging>
<name>Apache Hadoop Logging</name>
<description>Logging Support for Apache Hadoop project</description>
<properties>
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
</properties>
<dependencies>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-annotations</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-lang3</artifactId>
</dependency>
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-api</artifactId>
</dependency>
<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-log4j12</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
<scope>provided</scope>
</dependency>
</dependencies>
<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-source-plugin</artifactId>
<executions>
<execution>
<phase>prepare-package</phase>
<goals>
<goal>jar</goal>
</goals>
</execution>
</executions>
<configuration>
<attach>true</attach>
</configuration>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-jar-plugin</artifactId>
<executions>
<execution>
<id>prepare-jar</id>
<phase>prepare-package</phase>
<goals>
<goal>jar</goal>
</goals>
</execution>
<execution>
<id>prepare-test-jar</id>
<phase>prepare-package</phase>
<goals>
<goal>test-jar</goal>
</goals>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.rat</groupId>
<artifactId>apache-rat-plugin</artifactId>
<configuration>
<excludes>
<exclude>dev-support/findbugsExcludeFile.xml</exclude>
</excludes>
</configuration>
</plugin>
<plugin>
<groupId>com.github.spotbugs</groupId>
<artifactId>spotbugs-maven-plugin</artifactId>
<configuration>
<excludeFilterFile>${basedir}/dev-support/findbugsExcludeFile.xml</excludeFilterFile>
</configuration>
</plugin>
</plugins>
</build>
</project>

View File

@ -0,0 +1,145 @@
/*
* 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.logging;
import java.io.FileInputStream;
import java.io.Flushable;
import java.io.IOException;
import java.io.InputStream;
import java.io.PrintWriter;
import java.io.StringWriter;
import java.util.Enumeration;
import java.util.Map;
import java.util.Properties;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.log4j.Appender;
import org.apache.log4j.Level;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.log4j.PropertyConfigurator;
/**
* Hadoop's internal class that access log4j APIs directly.
* <p/>
* This class will depend on log4j directly, so callers should not use this class directly to avoid
* introducing log4j dependencies to downstream users. Please call the methods in
* {@link HadoopLoggerUtils}, as they will call the methods here through reflection.
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
final class HadoopInternalLog4jUtils {
private HadoopInternalLog4jUtils() {
}
static void setLogLevel(String loggerName, String levelName) {
if (loggerName == null) {
throw new IllegalArgumentException("logger name cannot be null");
}
Logger logger = loggerName.equalsIgnoreCase("root") ?
LogManager.getRootLogger() :
LogManager.getLogger(loggerName);
Level level = Level.toLevel(levelName.toUpperCase());
if (!level.toString().equalsIgnoreCase(levelName)) {
throw new IllegalArgumentException("Unsupported log level " + levelName);
}
logger.setLevel(level);
}
static void shutdownLogManager() {
LogManager.shutdown();
}
static String getEffectiveLevel(String loggerName) {
Logger logger = loggerName.equalsIgnoreCase("root") ?
LogManager.getRootLogger() :
LogManager.getLogger(loggerName);
return logger.getEffectiveLevel().toString();
}
static void resetConfiguration() {
LogManager.resetConfiguration();
}
static void updateLog4jConfiguration(Class<?> targetClass, String log4jPath) throws Exception {
Properties customProperties = new Properties();
try (FileInputStream fs = new FileInputStream(log4jPath);
InputStream is = targetClass.getResourceAsStream("/log4j.properties")) {
customProperties.load(fs);
Properties originalProperties = new Properties();
originalProperties.load(is);
for (Map.Entry<Object, Object> entry : customProperties.entrySet()) {
originalProperties.setProperty(entry.getKey().toString(), entry.getValue().toString());
}
LogManager.resetConfiguration();
PropertyConfigurator.configure(originalProperties);
}
}
static boolean hasAppenders(String logger) {
return Logger.getLogger(logger)
.getAllAppenders()
.hasMoreElements();
}
@SuppressWarnings("unchecked")
static void syncLogs() {
// flush standard streams
//
System.out.flush();
System.err.flush();
// flush flushable appenders
//
final Logger rootLogger = Logger.getRootLogger();
flushAppenders(rootLogger);
final Enumeration<Logger> allLoggers = rootLogger.getLoggerRepository().
getCurrentLoggers();
while (allLoggers.hasMoreElements()) {
final Logger l = allLoggers.nextElement();
flushAppenders(l);
}
}
@SuppressWarnings("unchecked")
private static void flushAppenders(Logger l) {
final Enumeration<Appender> allAppenders = l.getAllAppenders();
while (allAppenders.hasMoreElements()) {
final Appender a = allAppenders.nextElement();
if (a instanceof Flushable) {
try {
((Flushable) a).flush();
} catch (IOException ioe) {
System.err.println(a + ": Failed to flush!"
+ stringifyException(ioe));
}
}
}
}
private static String stringifyException(Throwable e) {
StringWriter stringWriter = new StringWriter();
PrintWriter printWriter = new PrintWriter(stringWriter);
e.printStackTrace(printWriter);
printWriter.close();
return stringWriter.toString();
}
}

View File

@ -0,0 +1,142 @@
/*
* 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.logging;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* A bridge class for operating on logging framework, such as changing log4j log level, etc.
* Will call the methods in {@link HadoopInternalLog4jUtils} to perform operations on log4j level.
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
public final class HadoopLoggerUtils {
private static final String INTERNAL_UTILS_CLASS =
"org.apache.hadoop.logging.HadoopInternalLog4jUtils";
private HadoopLoggerUtils() {
}
private static Method getMethod(String methodName, Class<?>... args) {
try {
Class<?> clazz = Class.forName(INTERNAL_UTILS_CLASS);
return clazz.getDeclaredMethod(methodName, args);
} catch (ClassNotFoundException | NoSuchMethodException e) {
throw new AssertionError("should not happen", e);
}
}
private static void throwUnchecked(Throwable throwable) {
if (throwable instanceof RuntimeException) {
throw (RuntimeException) throwable;
}
if (throwable instanceof Error) {
throw (Error) throwable;
}
}
public static void shutdownLogManager() {
Method method = getMethod("shutdownLogManager");
try {
method.invoke(null);
} catch (IllegalAccessException e) {
throw new AssertionError("should not happen", e);
} catch (InvocationTargetException e) {
throwUnchecked(e.getCause());
throw new AssertionError("Failed to execute, should not happen", e.getCause());
}
}
public static void setLogLevel(String loggerName, String levelName) {
Method method = getMethod("setLogLevel", String.class, String.class);
try {
method.invoke(null, loggerName, levelName);
} catch (IllegalAccessException e) {
throw new AssertionError("should not happen", e);
} catch (InvocationTargetException e) {
throwUnchecked(e.getCause());
throw new AssertionError("Failed to execute, should not happen", e.getCause());
}
}
public static String getEffectiveLevel(String loggerName) {
Method method = getMethod("getEffectiveLevel", String.class);
try {
return (String) method.invoke(null, loggerName);
} catch (IllegalAccessException e) {
throw new AssertionError("should not happen", e);
} catch (InvocationTargetException e) {
throwUnchecked(e.getCause());
throw new AssertionError("Failed to execute, should not happen", e.getCause());
}
}
public static void resetConfiguration() {
Method method = getMethod("resetConfiguration");
try {
method.invoke(null);
} catch (IllegalAccessException e) {
throw new AssertionError("should not happen", e);
} catch (InvocationTargetException e) {
throwUnchecked(e.getCause());
throw new AssertionError("Failed to execute, should not happen", e.getCause());
}
}
public static void updateLog4jConfiguration(Class<?> targetClass, String log4jPath) {
Method method = getMethod("updateLog4jConfiguration", Class.class, String.class);
try {
method.invoke(null, targetClass, log4jPath);
} catch (IllegalAccessException e) {
throw new AssertionError("should not happen", e);
} catch (InvocationTargetException e) {
throwUnchecked(e.getCause());
throw new AssertionError("Failed to execute, should not happen", e.getCause());
}
}
public static boolean hasAppenders(String logger) {
Method method = getMethod("hasAppenders", String.class);
try {
return (Boolean) method.invoke(null, logger);
} catch (IllegalAccessException e) {
throw new AssertionError("should not happen", e);
} catch (InvocationTargetException e) {
throwUnchecked(e.getCause());
throw new AssertionError("Failed to execute, should not happen", e.getCause());
}
}
public synchronized static void syncLogs() {
Method method = getMethod("syncLogs");
try {
method.invoke(null);
} catch (IllegalAccessException e) {
throw new AssertionError("should not happen", e);
} catch (InvocationTargetException e) {
throwUnchecked(e.getCause());
throw new AssertionError("Failed to execute, should not happen", e.getCause());
}
}
}

View File

@ -16,7 +16,7 @@
* limitations under the License. * limitations under the License.
*/ */
package org.apache.hadoop.hdfs.util; package org.apache.hadoop.logging.appenders;
import java.io.IOException; import java.io.IOException;

View File

@ -16,12 +16,10 @@
* limitations under the License. * limitations under the License.
*/ */
package org.apache.hadoop.yarn.util; package org.apache.hadoop.logging.appenders;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Time;
import org.apache.log4j.AppenderSkeleton; import org.apache.log4j.AppenderSkeleton;
import org.apache.log4j.Level; import org.apache.log4j.Level;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
@ -113,16 +111,13 @@ public class Log4jWarningErrorMetricsAppender extends AppenderSkeleton {
/** /**
* Create an appender to keep track of the errors and warnings logged by the * Create an appender to keep track of the errors and warnings logged by the
* system. * system.
* *
* @param cleanupIntervalSeconds * @param cleanupIntervalSeconds the interval at which old messages are purged to prevent the
* the interval at which old messages are purged to prevent the * message stores from growing unbounded.
* message stores from growing unbounded * @param messageAgeLimitSeconds the maximum age of a message in seconds before it is purged from
* @param messageAgeLimitSeconds * the store.
* the maximum age of a message in seconds before it is purged from * @param maxUniqueMessages the maximum number of unique messages of each type we keep before
* the store * we start purging.
* @param maxUniqueMessages
* the maximum number of unique messages of each type we keep before
* we start purging
*/ */
public Log4jWarningErrorMetricsAppender(int cleanupIntervalSeconds, public Log4jWarningErrorMetricsAppender(int cleanupIntervalSeconds,
long messageAgeLimitSeconds, int maxUniqueMessages) { long messageAgeLimitSeconds, int maxUniqueMessages) {
@ -143,6 +138,20 @@ public class Log4jWarningErrorMetricsAppender extends AppenderSkeleton {
this.setThreshold(Level.WARN); this.setThreshold(Level.WARN);
} }
private static String join(CharSequence separator, String[] strings) {
StringBuilder sb = new StringBuilder();
boolean first = true;
for (String s : strings) {
if (first) {
first = false;
} else {
sb.append(separator);
}
sb.append(s);
}
return sb.toString();
}
/** /**
* {@inheritDoc} * {@inheritDoc}
*/ */
@ -151,7 +160,7 @@ public class Log4jWarningErrorMetricsAppender extends AppenderSkeleton {
String message = event.getRenderedMessage(); String message = event.getRenderedMessage();
String[] throwableStr = event.getThrowableStrRep(); String[] throwableStr = event.getThrowableStrRep();
if (throwableStr != null) { if (throwableStr != null) {
message = message + "\n" + StringUtils.join("\n", throwableStr); message = message + "\n" + join("\n", throwableStr);
message = message =
org.apache.commons.lang3.StringUtils.left(message, MAX_MESSAGE_SIZE); org.apache.commons.lang3.StringUtils.left(message, MAX_MESSAGE_SIZE);
} }
@ -232,7 +241,7 @@ public class Log4jWarningErrorMetricsAppender extends AppenderSkeleton {
* getErrorMessagesAndCounts since the message store is purged at regular * getErrorMessagesAndCounts since the message store is purged at regular
* intervals to prevent it from growing without bounds, while the store for * intervals to prevent it from growing without bounds, while the store for
* the counts is purged less frequently. * the counts is purged less frequently.
* *
* @param cutoffs * @param cutoffs
* list of timestamp cutoffs(in seconds) for which the counts are * list of timestamp cutoffs(in seconds) for which the counts are
* desired * desired
@ -248,7 +257,7 @@ public class Log4jWarningErrorMetricsAppender extends AppenderSkeleton {
* getWarningMessagesAndCounts since the message store is purged at regular * getWarningMessagesAndCounts since the message store is purged at regular
* intervals to prevent it from growing without bounds, while the store for * intervals to prevent it from growing without bounds, while the store for
* the counts is purged less frequently. * the counts is purged less frequently.
* *
* @param cutoffs * @param cutoffs
* list of timestamp cutoffs(in seconds) for which the counts are * list of timestamp cutoffs(in seconds) for which the counts are
* desired * desired
@ -285,7 +294,7 @@ public class Log4jWarningErrorMetricsAppender extends AppenderSkeleton {
* differ from the ones provided by getErrorCounts since the message store is * differ from the ones provided by getErrorCounts since the message store is
* purged at regular intervals to prevent it from growing without bounds, * purged at regular intervals to prevent it from growing without bounds,
* while the store for the counts is purged less frequently. * while the store for the counts is purged less frequently.
* *
* @param cutoffs * @param cutoffs
* list of timestamp cutoffs(in seconds) for which the counts are * list of timestamp cutoffs(in seconds) for which the counts are
* desired * desired
@ -304,7 +313,7 @@ public class Log4jWarningErrorMetricsAppender extends AppenderSkeleton {
* may differ from the ones provided by getWarningCounts since the message * may differ from the ones provided by getWarningCounts since the message
* store is purged at regular intervals to prevent it from growing without * store is purged at regular intervals to prevent it from growing without
* bounds, while the store for the counts is purged less frequently. * bounds, while the store for the counts is purged less frequently.
* *
* @param cutoffs * @param cutoffs
* list of timestamp cutoffs(in seconds) for which the counts are * list of timestamp cutoffs(in seconds) for which the counts are
* desired * desired
@ -322,7 +331,7 @@ public class Log4jWarningErrorMetricsAppender extends AppenderSkeleton {
SortedSet<PurgeElement> purgeInformation) { SortedSet<PurgeElement> purgeInformation) {
if (purgeInformation.size() > maxUniqueMessages) { if (purgeInformation.size() > maxUniqueMessages) {
ErrorAndWarningsCleanup cleanup = new ErrorAndWarningsCleanup(); ErrorAndWarningsCleanup cleanup = new ErrorAndWarningsCleanup();
long cutoff = Time.now() - (messageAgeLimitSeconds * 1000); long cutoff = System.currentTimeMillis() - (messageAgeLimitSeconds * 1000);
cutoff = (cutoff / 1000); cutoff = (cutoff / 1000);
cleanup.cleanupMessages(map, purgeInformation, cutoff, maxUniqueMessages); cleanup.cleanupMessages(map, purgeInformation, cutoff, maxUniqueMessages);
} }
@ -379,7 +388,7 @@ public class Log4jWarningErrorMetricsAppender extends AppenderSkeleton {
@Override @Override
public void run() { public void run() {
long cutoff = Time.now() - (messageAgeLimitSeconds * 1000); long cutoff = System.currentTimeMillis() - (messageAgeLimitSeconds * 1000);
cutoff = (cutoff / 1000); cutoff = (cutoff / 1000);
cleanupMessages(errors, errorsPurgeInformation, cutoff, maxUniqueMessages); cleanupMessages(errors, errorsPurgeInformation, cutoff, maxUniqueMessages);
cleanupMessages(warnings, warningsPurgeInformation, cutoff, cleanupMessages(warnings, warningsPurgeInformation, cutoff,

View File

@ -0,0 +1,65 @@
/*
* 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.logging;
import java.io.StringWriter;
import org.apache.log4j.Appender;
import org.apache.log4j.Layout;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.log4j.PatternLayout;
import org.apache.log4j.WriterAppender;
public class LogCapturer {
private final StringWriter sw = new StringWriter();
private final Appender appender;
private final Logger logger;
public static LogCapturer captureLogs(org.slf4j.Logger logger) {
if (logger.getName().equals("root")) {
return new LogCapturer(Logger.getRootLogger());
}
return new LogCapturer(LogManager.getLogger(logger.getName()));
}
private LogCapturer(Logger logger) {
this.logger = logger;
Appender defaultAppender = Logger.getRootLogger().getAppender("stdout");
if (defaultAppender == null) {
defaultAppender = Logger.getRootLogger().getAppender("console");
}
final Layout layout =
(defaultAppender == null) ? new PatternLayout() : defaultAppender.getLayout();
this.appender = new WriterAppender(layout, sw);
logger.addAppender(this.appender);
}
public String getOutput() {
return sw.toString();
}
public void stopCapturing() {
logger.removeAppender(appender);
}
public void clearOutput() {
sw.getBuffer().setLength(0);
}
}

View File

@ -0,0 +1,37 @@
/*
* 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.logging.test;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.logging.HadoopLoggerUtils;
public class TestSyncLogs {
private static final Logger LOG = LoggerFactory.getLogger(TestSyncLogs.class);
@Test
public void testSyncLogs() {
LOG.info("Testing log sync");
HadoopLoggerUtils.syncLogs();
}
}

View File

@ -0,0 +1,18 @@
# Licensed 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.
# log4j configuration used during build and unit tests
log4j.rootLogger=debug,stdout
log4j.threshold=ALL
log4j.appender.stdout=org.apache.log4j.ConsoleAppender
log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n

View File

@ -38,11 +38,6 @@
<groupId>org.apache.kerby</groupId> <groupId>org.apache.kerby</groupId>
<artifactId>kerb-simplekdc</artifactId> <artifactId>kerb-simplekdc</artifactId>
</dependency> </dependency>
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-log4j12</artifactId>
<scope>compile</scope>
</dependency>
<dependency> <dependency>
<groupId>junit</groupId> <groupId>junit</groupId>
<artifactId>junit</artifactId> <artifactId>junit</artifactId>

View File

@ -38,6 +38,7 @@
<module>hadoop-minikdc</module> <module>hadoop-minikdc</module>
<module>hadoop-kms</module> <module>hadoop-kms</module>
<module>hadoop-registry</module> <module>hadoop-registry</module>
<module>hadoop-logging</module>
</modules> </modules>
<build> <build>

View File

@ -86,6 +86,12 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
<artifactId>netty-all</artifactId> <artifactId>netty-all</artifactId>
<scope>test</scope> <scope>test</scope>
</dependency> </dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-logging</artifactId>
<scope>test</scope>
<type>test-jar</type>
</dependency>
<dependency> <dependency>
<groupId>org.mock-server</groupId> <groupId>org.mock-server</groupId>
<artifactId>mockserver-netty</artifactId> <artifactId>mockserver-netty</artifactId>

View File

@ -31,6 +31,7 @@ import static org.apache.hadoop.security.ssl.FileBasedKeyStoresFactory.SSL_MONIT
import org.apache.hadoop.security.ssl.KeyStoreTestUtil; import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
import org.apache.hadoop.security.ssl.SSLFactory; import org.apache.hadoop.security.ssl.SSLFactory;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.logging.LogCapturer;
import org.apache.hadoop.util.Lists; import org.apache.hadoop.util.Lists;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
@ -61,8 +62,8 @@ public final class TestURLConnectionFactory {
public void testSSLInitFailure() throws Exception { public void testSSLInitFailure() throws Exception {
Configuration conf = new Configuration(); Configuration conf = new Configuration();
conf.set(SSLFactory.SSL_HOSTNAME_VERIFIER_KEY, "foo"); conf.set(SSLFactory.SSL_HOSTNAME_VERIFIER_KEY, "foo");
GenericTestUtils.LogCapturer logs = LogCapturer logs =
GenericTestUtils.LogCapturer.captureLogs( LogCapturer.captureLogs(
LoggerFactory.getLogger(URLConnectionFactory.class)); LoggerFactory.getLogger(URLConnectionFactory.class));
URLConnectionFactory.newDefaultURLConnectionFactory(conf); URLConnectionFactory.newDefaultURLConnectionFactory(conf);
Assert.assertTrue("Expected log for ssl init failure not found!", Assert.assertTrue("Expected log for ssl init failure not found!",

View File

@ -182,6 +182,12 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
<artifactId>junit-jupiter-params</artifactId> <artifactId>junit-jupiter-params</artifactId>
<scope>test</scope> <scope>test</scope>
</dependency> </dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-logging</artifactId>
<scope>test</scope>
<type>test-jar</type>
</dependency>
</dependencies> </dependencies>
<build> <build>

View File

@ -40,6 +40,7 @@ import org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys;
import org.apache.hadoop.hdfs.server.federation.router.RemoteMethod; import org.apache.hadoop.hdfs.server.federation.router.RemoteMethod;
import org.apache.hadoop.hdfs.server.federation.router.RouterRpcClient; import org.apache.hadoop.hdfs.server.federation.router.RouterRpcClient;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.logging.LogCapturer;
import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_FAIR_HANDLER_COUNT_KEY_PREFIX; import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_FAIR_HANDLER_COUNT_KEY_PREFIX;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
@ -48,8 +49,8 @@ public class TestRouterRefreshFairnessPolicyController {
private static final Logger LOG = private static final Logger LOG =
LoggerFactory.getLogger(TestRouterRefreshFairnessPolicyController.class); LoggerFactory.getLogger(TestRouterRefreshFairnessPolicyController.class);
private final GenericTestUtils.LogCapturer controllerLog = private final LogCapturer controllerLog =
GenericTestUtils.LogCapturer.captureLogs(AbstractRouterRpcFairnessPolicyController.LOG); LogCapturer.captureLogs(AbstractRouterRpcFairnessPolicyController.LOG);
private StateStoreDFSCluster cluster; private StateStoreDFSCluster cluster;

View File

@ -22,7 +22,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.server.federation.router.FederationUtil; import org.apache.hadoop.hdfs.server.federation.router.FederationUtil;
import org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys; import org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.logging.LogCapturer;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
import org.junit.Test; import org.junit.Test;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -179,7 +179,7 @@ public class TestRouterRpcFairnessPolicyController {
private void verifyInstantiationError(Configuration conf, int handlerCount, private void verifyInstantiationError(Configuration conf, int handlerCount,
int totalDedicatedHandlers) { int totalDedicatedHandlers) {
GenericTestUtils.LogCapturer logs = GenericTestUtils.LogCapturer LogCapturer logs = LogCapturer
.captureLogs(LoggerFactory.getLogger( .captureLogs(LoggerFactory.getLogger(
StaticRouterRpcFairnessPolicyController.class)); StaticRouterRpcFairnessPolicyController.class));
try { try {

View File

@ -40,7 +40,6 @@ import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.LogVerificationAppender;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates; import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo.DatanodeInfoBuilder; import org.apache.hadoop.hdfs.protocol.DatanodeInfo.DatanodeInfoBuilder;
@ -55,6 +54,7 @@ import org.apache.hadoop.hdfs.server.federation.resolver.MountTableResolver;
import org.apache.hadoop.hdfs.server.federation.resolver.NamenodeStatusReport; import org.apache.hadoop.hdfs.server.federation.resolver.NamenodeStatusReport;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
import org.apache.hadoop.http.HttpConfig; import org.apache.hadoop.http.HttpConfig;
import org.apache.hadoop.logging.LogCapturer;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
@ -322,11 +322,7 @@ public class TestRouterNamenodeMonitoring {
int httpsRequests, int requestsPerService) { int httpsRequests, int requestsPerService) {
// Attach our own log appender so we can verify output // Attach our own log appender so we can verify output
final LogVerificationAppender appender = LogCapturer logCapturer = LogCapturer.captureLogs(LoggerFactory.getLogger("root"));
new LogVerificationAppender();
final org.apache.log4j.Logger logger =
org.apache.log4j.Logger.getRootLogger();
logger.addAppender(appender);
GenericTestUtils.setRootLogLevel(Level.DEBUG); GenericTestUtils.setRootLogLevel(Level.DEBUG);
// Setup and start the Router // Setup and start the Router
@ -347,8 +343,11 @@ public class TestRouterNamenodeMonitoring {
heartbeatService.getNamenodeStatusReport(); heartbeatService.getNamenodeStatusReport();
} }
} }
assertEquals(httpsRequests * 2, appender.countLinesWithMessage("JMX URL: https://")); assertEquals(2, org.apache.commons.lang3.StringUtils.countMatches(logCapturer.getOutput(),
assertEquals(httpRequests * 2, appender.countLinesWithMessage("JMX URL: http://")); "JMX URL: https://"));
assertEquals(2, org.apache.commons.lang3.StringUtils.countMatches(logCapturer.getOutput(),
"JMX URL: http://"));
logCapturer.stopCapturing();
} }
/** /**

View File

@ -135,6 +135,8 @@ import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.service.Service.STATE; import org.apache.hadoop.service.Service.STATE;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.LambdaTestUtils; import org.apache.hadoop.test.LambdaTestUtils;
import org.apache.hadoop.logging.LogCapturer;
import org.codehaus.jettison.json.JSONException; import org.codehaus.jettison.json.JSONException;
import org.codehaus.jettison.json.JSONObject; import org.codehaus.jettison.json.JSONObject;
import org.junit.AfterClass; import org.junit.AfterClass;
@ -2067,8 +2069,8 @@ public class TestRouterRpc {
@Test @Test
public void testMkdirsWithCallerContext() throws IOException { public void testMkdirsWithCallerContext() throws IOException {
GenericTestUtils.LogCapturer auditlog = LogCapturer auditlog =
GenericTestUtils.LogCapturer.captureLogs(FSNamesystem.AUDIT_LOG); LogCapturer.captureLogs(FSNamesystem.AUDIT_LOG);
// Current callerContext is null // Current callerContext is null
assertNull(CallerContext.getCurrent()); assertNull(CallerContext.getCurrent());
@ -2094,8 +2096,8 @@ public class TestRouterRpc {
@Test @Test
public void testRealUserPropagationInCallerContext() public void testRealUserPropagationInCallerContext()
throws IOException, InterruptedException { throws IOException, InterruptedException {
GenericTestUtils.LogCapturer auditlog = LogCapturer auditlog =
GenericTestUtils.LogCapturer.captureLogs(FSNamesystem.AUDIT_LOG); LogCapturer.captureLogs(FSNamesystem.AUDIT_LOG);
// Current callerContext is null // Current callerContext is null
assertNull(CallerContext.getCurrent()); assertNull(CallerContext.getCurrent());
@ -2139,8 +2141,8 @@ public class TestRouterRpc {
@Test @Test
public void testAddClientIpPortToCallerContext() throws IOException { public void testAddClientIpPortToCallerContext() throws IOException {
GenericTestUtils.LogCapturer auditLog = LogCapturer auditLog =
GenericTestUtils.LogCapturer.captureLogs(FSNamesystem.AUDIT_LOG); LogCapturer.captureLogs(FSNamesystem.AUDIT_LOG);
// 1. ClientIp and ClientPort are not set on the client. // 1. ClientIp and ClientPort are not set on the client.
// Set client context. // Set client context.
@ -2174,8 +2176,8 @@ public class TestRouterRpc {
@Test @Test
public void testAddClientIdAndCallIdToCallerContext() throws IOException { public void testAddClientIdAndCallIdToCallerContext() throws IOException {
GenericTestUtils.LogCapturer auditLog = LogCapturer auditLog =
GenericTestUtils.LogCapturer.captureLogs(FSNamesystem.AUDIT_LOG); LogCapturer.captureLogs(FSNamesystem.AUDIT_LOG);
// 1. ClientId and ClientCallId are not set on the client. // 1. ClientId and ClientCallId are not set on the client.
// Set client context. // Set client context.

View File

@ -72,6 +72,8 @@ import org.apache.hadoop.ipc.CallerContext;
import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.ipc.StandbyException; import org.apache.hadoop.ipc.StandbyException;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.logging.LogCapturer;
import org.junit.Test; import org.junit.Test;
import org.slf4j.event.Level; import org.slf4j.event.Level;
@ -276,12 +278,10 @@ public class TestRouterRpcMultiDestination extends TestRouterRpc {
@Test @Test
public void testPreviousBlockNotNull() public void testPreviousBlockNotNull()
throws IOException, URISyntaxException { throws IOException, URISyntaxException {
final GenericTestUtils.LogCapturer stateChangeLog = final LogCapturer stateChangeLog = LogCapturer.captureLogs(NameNode.stateChangeLog);
GenericTestUtils.LogCapturer.captureLogs(NameNode.stateChangeLog);
GenericTestUtils.setLogLevel(NameNode.stateChangeLog, Level.DEBUG); GenericTestUtils.setLogLevel(NameNode.stateChangeLog, Level.DEBUG);
final GenericTestUtils.LogCapturer nameNodeLog = final LogCapturer nameNodeLog = LogCapturer.captureLogs(NameNode.LOG);
GenericTestUtils.LogCapturer.captureLogs(NameNode.LOG);
GenericTestUtils.setLogLevel(NameNode.LOG, Level.DEBUG); GenericTestUtils.setLogLevel(NameNode.LOG, Level.DEBUG);
final FederationRPCMetrics metrics = getRouterContext(). final FederationRPCMetrics metrics = getRouterContext().
@ -454,8 +454,8 @@ public class TestRouterRpcMultiDestination extends TestRouterRpc {
@Test @Test
public void testCallerContextWithMultiDestinations() throws IOException { public void testCallerContextWithMultiDestinations() throws IOException {
GenericTestUtils.LogCapturer auditLog = LogCapturer auditLog =
GenericTestUtils.LogCapturer.captureLogs(FSNamesystem.AUDIT_LOG); LogCapturer.captureLogs(FSNamesystem.AUDIT_LOG);
// set client context // set client context
CallerContext.setCurrent( CallerContext.setCurrent(

View File

@ -310,14 +310,4 @@
<Method name="reconcile" /> <Method name="reconcile" />
<Bug pattern="SWL_SLEEP_WITH_LOCK_HELD" /> <Bug pattern="SWL_SLEEP_WITH_LOCK_HELD" />
</Match> </Match>
<!--
conversionPattern is only set once and used to initiate PatternLayout object
only once. It is set by log4j framework if set as part of log4j properties and accessed
only during first append operation.
-->
<Match>
<Class name="org.apache.hadoop.hdfs.util.AsyncRFAAppender"/>
<Field name="conversionPattern"/>
<Bug pattern="IS2_INCONSISTENT_SYNC"/>
</Match>
</FindBugsFilter> </FindBugsFilter>

View File

@ -164,6 +164,12 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
<artifactId>hadoop-minikdc</artifactId> <artifactId>hadoop-minikdc</artifactId>
<scope>test</scope> <scope>test</scope>
</dependency> </dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-logging</artifactId>
<scope>test</scope>
<type>test-jar</type>
</dependency>
<dependency> <dependency>
<groupId>org.mockito</groupId> <groupId>org.mockito</groupId>
<artifactId>mockito-core</artifactId> <artifactId>mockito-core</artifactId>

View File

@ -31,6 +31,8 @@ import javax.management.ObjectName;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import org.apache.hadoop.logging.HadoopLoggerUtils;
import org.apache.hadoop.metrics2.util.MBeans; import org.apache.hadoop.metrics2.util.MBeans;
/** /**
@ -111,11 +113,8 @@ public class MetricsLoggerTask implements Runnable {
.substring(0, maxLogLineLength) + "..."); .substring(0, maxLogLineLength) + "...");
} }
// TODO : hadoop-logging module to hide log4j implementation details, this method
// can directly call utility from hadoop-logging.
private static boolean hasAppenders(Logger logger) { private static boolean hasAppenders(Logger logger) {
return org.apache.log4j.Logger.getLogger(logger.getName()).getAllAppenders() return HadoopLoggerUtils.hasAppenders(logger.getName());
.hasMoreElements();
} }
/** /**

View File

@ -32,11 +32,11 @@ import org.apache.hadoop.hdfs.server.namenode.top.metrics.TopMetrics;
import org.apache.hadoop.hdfs.server.namenode.visitor.INodeCountVisitor; import org.apache.hadoop.hdfs.server.namenode.visitor.INodeCountVisitor;
import org.apache.hadoop.hdfs.server.namenode.visitor.INodeCountVisitor.Counts; import org.apache.hadoop.hdfs.server.namenode.visitor.INodeCountVisitor.Counts;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo; import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.logging.HadoopLoggerUtils;
import org.apache.hadoop.util.GSet; import org.apache.hadoop.util.GSet;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner; import org.apache.hadoop.util.ToolRunner;
import org.apache.log4j.Level;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -110,13 +110,13 @@ public class FsImageValidation {
} }
static void initLogLevels() { static void initLogLevels() {
Util.setLogLevel(FSImage.class, Level.TRACE); Util.setLogLevel(FSImage.class, "TRACE");
Util.setLogLevel(FileJournalManager.class, Level.TRACE); Util.setLogLevel(FileJournalManager.class, "TRACE");
Util.setLogLevel(GSet.class, Level.OFF); Util.setLogLevel(GSet.class, "OFF");
Util.setLogLevel(BlockManager.class, Level.OFF); Util.setLogLevel(BlockManager.class, "OFF");
Util.setLogLevel(DatanodeManager.class, Level.OFF); Util.setLogLevel(DatanodeManager.class, "OFF");
Util.setLogLevel(TopMetrics.class, Level.OFF); Util.setLogLevel(TopMetrics.class, "OFF");
} }
static class Util { static class Util {
@ -127,11 +127,10 @@ public class FsImageValidation {
+ ", max=" + StringUtils.byteDesc(runtime.maxMemory()); + ", max=" + StringUtils.byteDesc(runtime.maxMemory());
} }
static void setLogLevel(Class<?> clazz, Level level) { static void setLogLevel(Class<?> clazz, String level) {
final org.apache.log4j.Logger logger = org.apache.log4j.Logger.getLogger(clazz); HadoopLoggerUtils.setLogLevel(clazz.getName(), level);
logger.setLevel(level);
LOG.info("setLogLevel {} to {}, getEffectiveLevel() = {}", clazz.getName(), level, LOG.info("setLogLevel {} to {}, getEffectiveLevel() = {}", clazz.getName(), level,
logger.getEffectiveLevel()); HadoopLoggerUtils.getEffectiveLevel(clazz.getName()));
} }
static String toCommaSeparatedNumber(long n) { static String toCommaSeparatedNumber(long n) {

View File

@ -1,75 +0,0 @@
/**
* 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.hdfs;
import java.util.ArrayList;
import java.util.List;
import org.apache.log4j.AppenderSkeleton;
import org.apache.log4j.spi.LoggingEvent;
import org.apache.log4j.spi.ThrowableInformation;
/**
* Used to verify that certain exceptions or messages are present in log output.
*/
public class LogVerificationAppender extends AppenderSkeleton {
private final List<LoggingEvent> log = new ArrayList<LoggingEvent>();
@Override
public boolean requiresLayout() {
return false;
}
@Override
protected void append(final LoggingEvent loggingEvent) {
log.add(loggingEvent);
}
@Override
public void close() {
}
public List<LoggingEvent> getLog() {
return new ArrayList<LoggingEvent>(log);
}
public int countExceptionsWithMessage(final String text) {
int count = 0;
for (LoggingEvent e: getLog()) {
ThrowableInformation t = e.getThrowableInformation();
if (t != null) {
String m = t.getThrowable().getMessage();
if (m.contains(text)) {
count++;
}
}
}
return count;
}
public int countLinesWithMessage(final String text) {
int count = 0;
for (LoggingEvent e: getLog()) {
String msg = e.getRenderedMessage();
if (msg != null && msg.contains(text)) {
count++;
}
}
return count;
}
}

View File

@ -33,7 +33,8 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter; import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.logging.LogCapturer;
import org.junit.Test; import org.junit.Test;
public class TestDFSRename { public class TestDFSRename {
@ -189,8 +190,8 @@ public class TestDFSRename {
final DistributedFileSystem dfs = cluster.getFileSystem(); final DistributedFileSystem dfs = cluster.getFileSystem();
Path path = new Path("/test"); Path path = new Path("/test");
dfs.mkdirs(path); dfs.mkdirs(path);
GenericTestUtils.LogCapturer auditLog = LogCapturer auditLog =
GenericTestUtils.LogCapturer.captureLogs(FSNamesystem.AUDIT_LOG); LogCapturer.captureLogs(FSNamesystem.AUDIT_LOG);
dfs.rename(path, new Path("/dir1"), dfs.rename(path, new Path("/dir1"),
new Rename[] {Rename.OVERWRITE, Rename.TO_TRASH}); new Rename[] {Rename.OVERWRITE, Rename.TO_TRASH});
String auditOut = auditLog.getOutput(); String auditOut = auditLog.getOutput();

View File

@ -45,9 +45,9 @@ import org.apache.hadoop.hdfs.server.namenode.FSImage;
import org.apache.hadoop.hdfs.server.namenode.FSImageFormat; import org.apache.hadoop.hdfs.server.namenode.FSImageFormat;
import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil; import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
import org.apache.hadoop.hdfs.server.namenode.IllegalReservedPathException; import org.apache.hadoop.hdfs.server.namenode.IllegalReservedPathException;
import org.apache.hadoop.logging.LogCapturer;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.apache.log4j.Logger;
import org.junit.Test; import org.junit.Test;
import static org.junit.Assert.*; import static org.junit.Assert.*;
@ -317,9 +317,7 @@ public class TestDFSUpgradeFromImage {
"imageMD5Digest", "22222222222222222222222222222222"); "imageMD5Digest", "22222222222222222222222222222222");
// Attach our own log appender so we can verify output // Attach our own log appender so we can verify output
final LogVerificationAppender appender = new LogVerificationAppender(); LogCapturer logCapturer = LogCapturer.captureLogs(LoggerFactory.getLogger("root"));
final Logger logger = Logger.getRootLogger();
logger.addAppender(appender);
// Upgrade should now fail // Upgrade should now fail
try { try {
@ -331,9 +329,10 @@ public class TestDFSUpgradeFromImage {
if (!msg.contains("Failed to load FSImage file")) { if (!msg.contains("Failed to load FSImage file")) {
throw ioe; throw ioe;
} }
int md5failures = appender.countExceptionsWithMessage( int md5failures = org.apache.commons.lang3.StringUtils.countMatches(logCapturer.getOutput(),
" is corrupt with MD5 checksum of "); " is corrupt with MD5 checksum of ");
assertEquals("Upgrade did not fail with bad MD5", 1, md5failures); assertEquals("Upgrade did not fail with bad MD5", 1, md5failures);
logCapturer.stopCapturing();
} }
} }

View File

@ -26,7 +26,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.GenericTestUtils.LogCapturer; import org.apache.hadoop.logging.LogCapturer;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.BeforeClass; import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
@ -48,7 +48,7 @@ public class TestDataStream {
@Test(timeout = 60000) @Test(timeout = 60000)
public void testDfsClient() throws IOException, InterruptedException { public void testDfsClient() throws IOException, InterruptedException {
LogCapturer logs = GenericTestUtils.LogCapturer.captureLogs(LoggerFactory LogCapturer logs = LogCapturer.captureLogs(LoggerFactory
.getLogger(DataStreamer.class)); .getLogger(DataStreamer.class));
byte[] toWrite = new byte[PACKET_SIZE]; byte[] toWrite = new byte[PACKET_SIZE];
new Random(1).nextBytes(toWrite); new Random(1).nextBytes(toWrite);

View File

@ -51,7 +51,7 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.GenericTestUtils.LogCapturer; import org.apache.hadoop.logging.LogCapturer;
import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey; import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
import org.junit.After; import org.junit.After;
import org.junit.Before; import org.junit.Before;
@ -168,9 +168,9 @@ public class TestEncryptedTransfer {
FileChecksum checksum = writeUnencryptedAndThenRestartEncryptedCluster(); FileChecksum checksum = writeUnencryptedAndThenRestartEncryptedCluster();
LogCapturer logs = GenericTestUtils.LogCapturer.captureLogs( LogCapturer logs = LogCapturer.captureLogs(
LoggerFactory.getLogger(SaslDataTransferServer.class)); LoggerFactory.getLogger(SaslDataTransferServer.class));
LogCapturer logs1 = GenericTestUtils.LogCapturer.captureLogs( LogCapturer logs1 = LogCapturer.captureLogs(
LoggerFactory.getLogger(DataTransferSaslUtil.class)); LoggerFactory.getLogger(DataTransferSaslUtil.class));
try { try {
assertEquals(PLAIN_TEXT, DFSTestUtil.readFile(fs, TEST_PATH)); assertEquals(PLAIN_TEXT, DFSTestUtil.readFile(fs, TEST_PATH));
@ -239,7 +239,7 @@ public class TestEncryptedTransfer {
Mockito.doReturn(false).when(spyClient).shouldEncryptData(); Mockito.doReturn(false).when(spyClient).shouldEncryptData();
DFSClientAdapter.setDFSClient((DistributedFileSystem) fs, spyClient); DFSClientAdapter.setDFSClient((DistributedFileSystem) fs, spyClient);
LogCapturer logs = GenericTestUtils.LogCapturer.captureLogs( LogCapturer logs = LogCapturer.captureLogs(
LoggerFactory.getLogger(DataNode.class)); LoggerFactory.getLogger(DataNode.class));
try { try {
assertEquals(PLAIN_TEXT, DFSTestUtil.readFile(fs, TEST_PATH)); assertEquals(PLAIN_TEXT, DFSTestUtil.readFile(fs, TEST_PATH));
@ -457,9 +457,9 @@ public class TestEncryptedTransfer {
fs = getFileSystem(conf); fs = getFileSystem(conf);
LogCapturer logs = GenericTestUtils.LogCapturer.captureLogs( LogCapturer logs = LogCapturer.captureLogs(
LoggerFactory.getLogger(SaslDataTransferServer.class)); LoggerFactory.getLogger(SaslDataTransferServer.class));
LogCapturer logs1 = GenericTestUtils.LogCapturer.captureLogs( LogCapturer logs1 = LogCapturer.captureLogs(
LoggerFactory.getLogger(DataTransferSaslUtil.class)); LoggerFactory.getLogger(DataTransferSaslUtil.class));
try { try {
writeTestDataToFile(fs); writeTestDataToFile(fs);

View File

@ -54,7 +54,7 @@ import org.apache.hadoop.http.HttpConfig.Policy;
import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.GenericTestUtils.LogCapturer; import org.apache.hadoop.logging.LogCapturer;
import org.junit.After; import org.junit.After;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Rule; import org.junit.Rule;
@ -138,7 +138,7 @@ public class TestSaslDataTransfer extends SaslDataTransferTestCase {
HdfsConfiguration clientConf = new HdfsConfiguration(clusterConf); HdfsConfiguration clientConf = new HdfsConfiguration(clusterConf);
clientConf.set(DFS_DATA_TRANSFER_PROTECTION_KEY, ""); clientConf.set(DFS_DATA_TRANSFER_PROTECTION_KEY, "");
LogCapturer logs = GenericTestUtils.LogCapturer.captureLogs( LogCapturer logs = LogCapturer.captureLogs(
LoggerFactory.getLogger(DataNode.class)); LoggerFactory.getLogger(DataNode.class));
try { try {
doTest(clientConf); doTest(clientConf);

View File

@ -30,7 +30,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream; import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream;
import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion; import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion;
import org.apache.hadoop.test.GenericTestUtils.LogCapturer; import org.apache.hadoop.logging.LogCapturer;
import org.apache.hadoop.test.PathUtils; import org.apache.hadoop.test.PathUtils;
import org.junit.After; import org.junit.After;
import org.junit.Before; import org.junit.Before;

View File

@ -56,7 +56,7 @@ import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
import org.apache.hadoop.hdfs.server.namenode.ha.ObserverReadProxyProvider; import org.apache.hadoop.hdfs.server.namenode.ha.ObserverReadProxyProvider;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.GenericTestUtils.LogCapturer; import org.apache.hadoop.logging.LogCapturer;
import org.junit.Test; import org.junit.Test;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;

View File

@ -28,6 +28,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.logging.LogCapturer;
import org.apache.hadoop.test.Whitebox; import org.apache.hadoop.test.Whitebox;
import org.assertj.core.api.Assertions; import org.assertj.core.api.Assertions;
@ -235,8 +236,8 @@ public class TestBlockManagerSafeMode {
public void testCheckSafeMode9() throws Exception { public void testCheckSafeMode9() throws Exception {
Configuration conf = new HdfsConfiguration(); Configuration conf = new HdfsConfiguration();
conf.setLong(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_RECHECK_INTERVAL_KEY, 3000); conf.setLong(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_RECHECK_INTERVAL_KEY, 3000);
GenericTestUtils.LogCapturer logs = LogCapturer logs =
GenericTestUtils.LogCapturer.captureLogs(BlockManagerSafeMode.LOG); LogCapturer.captureLogs(BlockManagerSafeMode.LOG);
BlockManagerSafeMode blockManagerSafeMode = new BlockManagerSafeMode(bm, BlockManagerSafeMode blockManagerSafeMode = new BlockManagerSafeMode(bm,
fsn, true, conf); fsn, true, conf);
String content = logs.getOutput(); String content = logs.getOutput();
@ -247,8 +248,8 @@ public class TestBlockManagerSafeMode {
public void testCheckSafeMode10(){ public void testCheckSafeMode10(){
Configuration conf = new HdfsConfiguration(); Configuration conf = new HdfsConfiguration();
conf.setLong(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_RECHECK_INTERVAL_KEY, -1); conf.setLong(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_RECHECK_INTERVAL_KEY, -1);
GenericTestUtils.LogCapturer logs = LogCapturer logs =
GenericTestUtils.LogCapturer.captureLogs(BlockManagerSafeMode.LOG); LogCapturer.captureLogs(BlockManagerSafeMode.LOG);
BlockManagerSafeMode blockManagerSafeMode = new BlockManagerSafeMode(bm, BlockManagerSafeMode blockManagerSafeMode = new BlockManagerSafeMode(bm,
fsn, true, conf); fsn, true, conf);
String content = logs.getOutput(); String content = logs.getOutput();

View File

@ -58,7 +58,7 @@ import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStat
import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks; import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
import org.apache.hadoop.metrics2.MetricsRecordBuilder; import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.GenericTestUtils.LogCapturer; import org.apache.hadoop.logging.LogCapturer;
import org.junit.Test; import org.junit.Test;
import org.mockito.Mockito; import org.mockito.Mockito;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -575,7 +575,7 @@ public class TestPendingReconstruction {
new MiniDFSCluster.Builder(conf).numDataNodes(2).build(); new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
cluster.waitActive(); cluster.waitActive();
DFSTestUtil.setNameNodeLogLevel(Level.DEBUG); DFSTestUtil.setNameNodeLogLevel(Level.DEBUG);
LogCapturer logs = GenericTestUtils.LogCapturer LogCapturer logs = LogCapturer
.captureLogs(LoggerFactory.getLogger("BlockStateChange")); .captureLogs(LoggerFactory.getLogger("BlockStateChange"));
BlockManager bm = cluster.getNamesystem().getBlockManager(); BlockManager bm = cluster.getNamesystem().getBlockManager();
try { try {

View File

@ -21,7 +21,6 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSI
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull; import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.any;
@ -41,6 +40,7 @@ import java.util.Set;
import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.AddBlockFlag; import org.apache.hadoop.hdfs.AddBlockFlag;
import org.apache.hadoop.fs.ContentSummary; import org.apache.hadoop.fs.ContentSummary;
@ -49,7 +49,6 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.LogVerificationAppender;
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.TestBlockStoragePolicy; import org.apache.hadoop.hdfs.TestBlockStoragePolicy;
import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.Block;
@ -67,16 +66,15 @@ import org.apache.hadoop.hdfs.server.namenode.INodeFile;
import org.apache.hadoop.hdfs.server.namenode.Namesystem; import org.apache.hadoop.hdfs.server.namenode.Namesystem;
import org.apache.hadoop.hdfs.server.namenode.TestINodeFile; import org.apache.hadoop.hdfs.server.namenode.TestINodeFile;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.logging.LogCapturer;
import org.apache.hadoop.net.Node; import org.apache.hadoop.net.Node;
import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.ReflectionUtils;
import org.apache.log4j.Level;
import org.apache.log4j.Logger;
import org.apache.log4j.spi.LoggingEvent;
import org.junit.Rule; import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
import org.junit.rules.ExpectedException; import org.junit.rules.ExpectedException;
import org.junit.runner.RunWith; import org.junit.runner.RunWith;
import org.junit.runners.Parameterized; import org.junit.runners.Parameterized;
import org.slf4j.LoggerFactory;
@RunWith(Parameterized.class) @RunWith(Parameterized.class)
public class TestReplicationPolicy extends BaseReplicationPolicyTest { public class TestReplicationPolicy extends BaseReplicationPolicyTest {
@ -507,26 +505,26 @@ public class TestReplicationPolicy extends BaseReplicationPolicyTest {
2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 2* HdfsServerConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
(HdfsServerConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L, 0L, 0L, 0, 0); (HdfsServerConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
} }
final LogVerificationAppender appender = new LogVerificationAppender(); final LogCapturer logCapturer = LogCapturer.captureLogs(LoggerFactory.getLogger("root"));
final Logger logger = Logger.getRootLogger();
logger.addAppender(appender);
// try to choose NUM_OF_DATANODES which is more than actually available // try to choose NUM_OF_DATANODES which is more than actually available
// nodes. // nodes.
DatanodeStorageInfo[] targets = chooseTarget(dataNodes.length); DatanodeStorageInfo[] targets = chooseTarget(dataNodes.length);
assertEquals(targets.length, dataNodes.length - 2); assertEquals(targets.length, dataNodes.length - 2);
final List<LoggingEvent> log = appender.getLog(); boolean isFound = false;
assertNotNull(log); for (String logLine : logCapturer.getOutput().split("\n")) {
assertFalse(log.size() == 0); // Suppose to place replicas on each node but two data nodes are not
final LoggingEvent lastLogEntry = log.get(log.size() - 1); // available for placing replica, so here we expect a short of 2
if(logLine.contains("WARN") && logLine.contains("in need of 2")) {
assertTrue(Level.WARN.isGreaterOrEqual(lastLogEntry.getLevel())); isFound = true;
// Suppose to place replicas on each node but two data nodes are not break;
// available for placing replica, so here we expect a short of 2 }
assertTrue(((String)lastLogEntry.getMessage()).contains("in need of 2")); }
assertTrue("Could not find the block placement log specific to 2 datanodes not being "
+ "available for placing replicas", isFound);
logCapturer.stopCapturing();
resetHeartbeatForStorages(); resetHeartbeatForStorages();
} }
@ -1710,17 +1708,14 @@ public class TestReplicationPolicy extends BaseReplicationPolicyTest {
@Test @Test
public void testChosenFailureForStorageType() { public void testChosenFailureForStorageType() {
final LogVerificationAppender appender = new LogVerificationAppender(); final LogCapturer logCapturer = LogCapturer.captureLogs(LoggerFactory.getLogger("root"));
final Logger logger = Logger.getRootLogger();
logger.addAppender(appender);
DatanodeStorageInfo[] targets = replicator.chooseTarget(filename, 1, DatanodeStorageInfo[] targets = replicator.chooseTarget(filename, 1,
dataNodes[0], new ArrayList<DatanodeStorageInfo>(), false, null, dataNodes[0], new ArrayList<DatanodeStorageInfo>(), false, null,
BLOCK_SIZE, TestBlockStoragePolicy.POLICY_SUITE.getPolicy( BLOCK_SIZE, TestBlockStoragePolicy.POLICY_SUITE.getPolicy(
HdfsConstants.StoragePolicy.COLD.value()), null); HdfsConstants.StoragePolicy.COLD.value()), null);
assertEquals(0, targets.length); assertEquals(0, targets.length);
assertNotEquals(0, assertNotEquals(0,
appender.countLinesWithMessage("NO_REQUIRED_STORAGE_TYPE")); StringUtils.countMatches(logCapturer.getOutput(), "NO_REQUIRED_STORAGE_TYPE"));
} }
@Test @Test

View File

@ -27,7 +27,6 @@ import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
import java.util.Collections; import java.util.Collections;
import java.util.List;
import java.util.Random; import java.util.Random;
import java.util.concurrent.TimeoutException; import java.util.concurrent.TimeoutException;
@ -39,19 +38,15 @@ import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.server.namenode.PatternMatchingAppender; import org.apache.hadoop.logging.LogCapturer;
import org.apache.hadoop.metrics2.util.MBeans; import org.apache.hadoop.metrics2.util.MBeans;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.log4j.Appender;
import org.apache.log4j.AsyncAppender;
import org.junit.After; import org.junit.After;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Rule; import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
import org.junit.rules.Timeout; import org.junit.rules.Timeout;
import java.util.function.Supplier;
/** /**
* Test periodic logging of DataNode metrics. * Test periodic logging of DataNode metrics.
*/ */
@ -128,13 +123,13 @@ public class TestDataNodeMetricsLogger {
} }
@Test @Test
@SuppressWarnings("unchecked")
public void testMetricsLoggerIsAsync() throws IOException { public void testMetricsLoggerIsAsync() throws IOException {
startDNForTest(true); startDNForTest(true);
assertNotNull(dn); assertNotNull(dn);
org.apache.log4j.Logger logger = org.apache.log4j.Logger.getLogger(DataNode.METRICS_LOG_NAME); assertTrue(Collections.list(
@SuppressWarnings("unchecked") org.apache.log4j.Logger.getLogger(DataNode.METRICS_LOG_NAME).getAllAppenders())
List<Appender> appenders = Collections.list(logger.getAllAppenders()); .get(0) instanceof org.apache.log4j.AsyncAppender);
assertTrue(appenders.get(0) instanceof AsyncAppender);
} }
/** /**
@ -149,27 +144,15 @@ public class TestDataNodeMetricsLogger {
metricsProvider); metricsProvider);
startDNForTest(true); startDNForTest(true);
assertNotNull(dn); assertNotNull(dn);
final PatternMatchingAppender appender = LogCapturer logCapturer =
(PatternMatchingAppender) org.apache.log4j.Logger.getLogger(DataNode.METRICS_LOG_NAME) LogCapturer.captureLogs(LoggerFactory.getLogger(DataNode.METRICS_LOG_NAME));
.getAppender("PATTERNMATCHERAPPENDER");
// Ensure that the supplied pattern was matched. // Ensure that the supplied pattern was matched.
GenericTestUtils.waitFor(new Supplier<Boolean>() { GenericTestUtils.waitFor(() -> logCapturer.getOutput().contains("FakeMetric"),
@Override 1000, 60000);
public Boolean get() { logCapturer.stopCapturing();
return appender.isMatched();
}
}, 1000, 60000);
dn.shutdown(); dn.shutdown();
} }
private void addAppender(org.apache.log4j.Logger logger, Appender appender) {
@SuppressWarnings("unchecked")
List<Appender> appenders = Collections.list(logger.getAllAppenders());
((AsyncAppender) appenders.get(0)).addAppender(appender);
}
public interface TestFakeMetricMXBean { public interface TestFakeMetricMXBean {
int getFakeMetric(); int getFakeMetric();
} }

View File

@ -27,7 +27,6 @@ import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull; import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import java.io.ByteArrayOutputStream;
import java.io.File; import java.io.File;
import java.io.FileOutputStream; import java.io.FileOutputStream;
import java.io.IOException; import java.io.IOException;
@ -77,10 +76,9 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.LazyPersistTestCase
import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.logging.LogCapturer;
import org.apache.hadoop.util.AutoCloseableLock; import org.apache.hadoop.util.AutoCloseableLock;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
import org.apache.log4j.SimpleLayout;
import org.apache.log4j.WriterAppender;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import org.mockito.Mockito; import org.mockito.Mockito;
@ -414,14 +412,9 @@ public class TestDirectoryScanner {
@Test(timeout=600000) @Test(timeout=600000)
public void testScanDirectoryStructureWarn() throws Exception { public void testScanDirectoryStructureWarn() throws Exception {
LogCapturer logCapturer = LogCapturer.captureLogs(LoggerFactory.getLogger("root"));
//add a logger stream to check what has printed to log //add a logger stream to check what has printed to log
ByteArrayOutputStream loggerStream = new ByteArrayOutputStream();
org.apache.log4j.Logger rootLogger =
org.apache.log4j.Logger.getRootLogger();
GenericTestUtils.setRootLogLevel(Level.INFO); GenericTestUtils.setRootLogLevel(Level.INFO);
WriterAppender writerAppender =
new WriterAppender(new SimpleLayout(), loggerStream);
rootLogger.addAppender(writerAppender);
Configuration conf = getConfiguration(); Configuration conf = getConfiguration();
cluster = new MiniDFSCluster cluster = new MiniDFSCluster
@ -452,7 +445,7 @@ public class TestDirectoryScanner {
scan(1, 1, 0, 1, 0, 0, 0); scan(1, 1, 0, 1, 0, 0, 0);
//ensure the warn log not appear and missing block log do appear //ensure the warn log not appear and missing block log do appear
String logContent = new String(loggerStream.toByteArray()); String logContent = logCapturer.getOutput();
String missingBlockWarn = "Deleted a metadata file" + String missingBlockWarn = "Deleted a metadata file" +
" for the deleted block"; " for the deleted block";
String dirStructureWarnLog = " found in invalid directory." + String dirStructureWarnLog = " found in invalid directory." +
@ -464,6 +457,7 @@ public class TestDirectoryScanner {
LOG.info("check pass"); LOG.info("check pass");
} finally { } finally {
logCapturer.stopCapturing();
if (scanner != null) { if (scanner != null) {
scanner.shutdown(); scanner.shutdown();
scanner = null; scanner = null;
@ -526,7 +520,7 @@ public class TestDirectoryScanner {
client = cluster.getFileSystem().getClient(); client = cluster.getFileSystem().getClient();
conf.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THREADS_KEY, 1); conf.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THREADS_KEY, 1);
// log trace // log trace
GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer. LogCapturer logCapturer = LogCapturer.
captureLogs(NameNode.stateChangeLog); captureLogs(NameNode.stateChangeLog);
// Add files with 5 blocks // Add files with 5 blocks
createFile(GenericTestUtils.getMethodName(), BLOCK_LENGTH * 5, false); createFile(GenericTestUtils.getMethodName(), BLOCK_LENGTH * 5, false);

View File

@ -18,6 +18,8 @@
package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl; package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
import net.jcip.annotations.NotThreadSafe; import net.jcip.annotations.NotThreadSafe;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports; import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
import static org.apache.hadoop.test.MetricsAsserts.getMetrics; import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
@ -51,7 +53,6 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.LogVerificationAppender;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry; import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.Block;
@ -79,10 +80,10 @@ import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.nativeio.NativeIO; import org.apache.hadoop.io.nativeio.NativeIO;
import org.apache.hadoop.io.nativeio.NativeIO.POSIX.CacheManipulator; import org.apache.hadoop.io.nativeio.NativeIO.POSIX.CacheManipulator;
import org.apache.hadoop.io.nativeio.NativeIO.POSIX.NoMlockCacheManipulator; import org.apache.hadoop.io.nativeio.NativeIO.POSIX.NoMlockCacheManipulator;
import org.apache.hadoop.logging.LogCapturer;
import org.apache.hadoop.metrics2.MetricsRecordBuilder; import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.MetricsAsserts; import org.apache.hadoop.test.MetricsAsserts;
import org.apache.log4j.Logger;
import org.junit.After; import org.junit.After;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.Assert; import org.junit.Assert;
@ -393,9 +394,7 @@ public class TestFsDatasetCache {
} }
// nth file should hit a capacity exception // nth file should hit a capacity exception
final LogVerificationAppender appender = new LogVerificationAppender(); LogCapturer logCapturer = LogCapturer.captureLogs(LoggerFactory.getLogger("root"));
final Logger logger = Logger.getRootLogger();
logger.addAppender(appender);
setHeartbeatResponse(cacheBlocks(fileLocs[numFiles-1])); setHeartbeatResponse(cacheBlocks(fileLocs[numFiles-1]));
GenericTestUtils.waitFor(new Supplier<Boolean>() { GenericTestUtils.waitFor(new Supplier<Boolean>() {
@ -403,11 +402,12 @@ public class TestFsDatasetCache {
public Boolean get() { public Boolean get() {
// check the log reported by FsDataSetCache // check the log reported by FsDataSetCache
// in the case that cache capacity is exceeded. // in the case that cache capacity is exceeded.
int lines = appender.countLinesWithMessage( int lines = StringUtils.countMatches(logCapturer.getOutput(),
"could not reserve more bytes in the cache: "); "could not reserve more bytes in the cache: ");
return lines > 0; return lines > 0;
} }
}, 500, 30000); }, 500, 30000);
logCapturer.stopCapturing();
// Also check the metrics for the failure // Also check the metrics for the failure
assertTrue("Expected more than 0 failed cache attempts", assertTrue("Expected more than 0 failed cache attempts",
fsd.getNumBlocksFailedToCache() > 0); fsd.getNumBlocksFailedToCache() > 0);

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.hadoop.hdfs.server.diskbalancer; package org.apache.hadoop.hdfs.server.diskbalancer;
import org.apache.hadoop.logging.LogCapturer;
import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.util.Preconditions;
import java.util.function.Supplier; import java.util.function.Supplier;
import org.apache.commons.codec.digest.DigestUtils; import org.apache.commons.codec.digest.DigestUtils;
@ -321,7 +322,7 @@ public class TestDiskBalancer {
0); 0);
DFSTestUtil.waitReplication(fs, filePath, (short) 1); DFSTestUtil.waitReplication(fs, filePath, (short) 1);
GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer LogCapturer logCapturer = LogCapturer
.captureLogs(DiskBalancer.LOG); .captureLogs(DiskBalancer.LOG);
try { try {

View File

@ -1,58 +0,0 @@
/**
* 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.hdfs.server.namenode;
import java.util.regex.Pattern;
import org.apache.log4j.AppenderSkeleton;
import org.apache.log4j.spi.LoggingEvent;
/**
* An appender that matches logged messages against the given
* regular expression.
*/
public class PatternMatchingAppender extends AppenderSkeleton {
private final Pattern pattern;
private volatile boolean matched;
public PatternMatchingAppender() {
this.pattern = Pattern.compile("^.*FakeMetric.*$");
this.matched = false;
}
public boolean isMatched() {
return matched;
}
@Override
protected void append(LoggingEvent event) {
if (pattern.matcher(event.getMessage().toString()).matches()) {
matched = true;
}
}
@Override
public void close() {
}
@Override
public boolean requiresLayout() {
return false;
}
}

View File

@ -37,7 +37,7 @@ import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.authorize.ProxyServers; import org.apache.hadoop.security.authorize.ProxyServers;
import org.apache.hadoop.security.authorize.ProxyUsers; import org.apache.hadoop.security.authorize.ProxyUsers;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.GenericTestUtils.LogCapturer; import org.apache.hadoop.logging.LogCapturer;
import org.apache.hadoop.util.Lists; import org.apache.hadoop.util.Lists;
import org.junit.Before; import org.junit.Before;

View File

@ -41,7 +41,7 @@ import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.test.GenericTestUtils.LogCapturer; import org.apache.hadoop.logging.LogCapturer;
import java.io.IOException; import java.io.IOException;
import java.security.PrivilegedExceptionAction; import java.security.PrivilegedExceptionAction;

View File

@ -24,7 +24,6 @@ import java.io.InputStream;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.List;
import java.util.regex.Pattern; import java.util.regex.Pattern;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
@ -39,12 +38,9 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.web.WebHdfsConstants; import org.apache.hadoop.hdfs.web.WebHdfsConstants;
import org.apache.hadoop.hdfs.web.WebHdfsTestUtil; import org.apache.hadoop.hdfs.web.WebHdfsTestUtil;
import org.apache.hadoop.hdfs.web.WebHdfsFileSystem; import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
import org.apache.hadoop.logging.LogCapturer;
import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
import org.apache.log4j.Appender;
import org.apache.log4j.AsyncAppender;
import org.apache.log4j.Logger;
import org.junit.After; import org.junit.After;
import org.junit.AfterClass; import org.junit.AfterClass;
@ -107,6 +103,7 @@ public class TestAuditLogs {
UserGroupInformation userGroupInfo; UserGroupInformation userGroupInfo;
@Before @Before
@SuppressWarnings("unchecked")
public void setupCluster() throws Exception { public void setupCluster() throws Exception {
// must configure prior to instantiating the namesystem because it // must configure prior to instantiating the namesystem because it
// will reconfigure the logger if async is enabled // will reconfigure the logger if async is enabled
@ -122,11 +119,9 @@ public class TestAuditLogs {
util.createFiles(fs, fileName); util.createFiles(fs, fileName);
// make sure the appender is what it's supposed to be // make sure the appender is what it's supposed to be
Logger logger = org.apache.log4j.Logger.getLogger( assertTrue(Collections.list(org.apache.log4j.Logger.getLogger(
"org.apache.hadoop.hdfs.server.namenode.FSNamesystem.audit"); "org.apache.hadoop.hdfs.server.namenode.FSNamesystem.audit").getAllAppenders())
@SuppressWarnings("unchecked") .get(0) instanceof org.apache.log4j.AsyncAppender);
List<Appender> appenders = Collections.list(logger.getAllAppenders());
assertTrue(appenders.get(0) instanceof AsyncAppender);
fnames = util.getFileNames(fileName); fnames = util.getFileNames(fileName);
util.waitReplication(fs, fileName, (short)3); util.waitReplication(fs, fileName, (short)3);

View File

@ -82,7 +82,7 @@ import org.apache.hadoop.io.Text;
import org.apache.hadoop.metrics2.MetricsRecordBuilder; import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.GenericTestUtils.DelayAnswer; import org.apache.hadoop.test.GenericTestUtils.DelayAnswer;
import org.apache.hadoop.test.GenericTestUtils.LogCapturer; import org.apache.hadoop.logging.LogCapturer;
import org.apache.hadoop.test.PathUtils; import org.apache.hadoop.test.PathUtils;
import org.apache.hadoop.util.ExitUtil; import org.apache.hadoop.util.ExitUtil;
import org.apache.hadoop.util.ExitUtil.ExitException; import org.apache.hadoop.util.ExitUtil.ExitException;
@ -863,7 +863,7 @@ public class TestCheckpoint {
savedSd = sd; savedSd = sd;
} }
LogCapturer logs = GenericTestUtils.LogCapturer.captureLogs( LogCapturer logs = LogCapturer.captureLogs(
LoggerFactory.getLogger(Storage.class)); LoggerFactory.getLogger(Storage.class));
try { try {
// try to lock the storage that's already locked // try to lock the storage that's already locked

View File

@ -49,7 +49,7 @@ import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB; import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.GenericTestUtils.LogCapturer; import org.apache.hadoop.logging.LogCapturer;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;

View File

@ -83,6 +83,7 @@ import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.hdfs.util.XMLUtils.InvalidXmlException; import org.apache.hadoop.hdfs.util.XMLUtils.InvalidXmlException;
import org.apache.hadoop.hdfs.util.XMLUtils.Stanza; import org.apache.hadoop.hdfs.util.XMLUtils.Stanza;
import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.logging.LogCapturer;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.PathUtils; import org.apache.hadoop.test.PathUtils;
import org.apache.hadoop.util.ExitUtil; import org.apache.hadoop.util.ExitUtil;
@ -90,9 +91,6 @@ import org.apache.hadoop.util.ExitUtil.ExitException;
import org.apache.hadoop.util.Lists; import org.apache.hadoop.util.Lists;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
import org.apache.log4j.AppenderSkeleton;
import org.apache.log4j.LogManager;
import org.apache.log4j.spi.LoggingEvent;
import org.junit.Test; import org.junit.Test;
import org.junit.runner.RunWith; import org.junit.runner.RunWith;
import org.junit.runners.Parameterized; import org.junit.runners.Parameterized;
@ -1717,36 +1715,13 @@ public class TestEditLog {
} }
} }
class TestAppender extends AppenderSkeleton {
private final List<LoggingEvent> log = new ArrayList<>();
@Override
public boolean requiresLayout() {
return false;
}
@Override
protected void append(final LoggingEvent loggingEvent) {
log.add(loggingEvent);
}
@Override
public void close() {
}
public List<LoggingEvent> getLog() {
return new ArrayList<>(log);
}
}
/** /**
* *
* @throws Exception * @throws Exception
*/ */
@Test @Test
public void testReadActivelyUpdatedLog() throws Exception { public void testReadActivelyUpdatedLog() throws Exception {
final TestAppender appender = new TestAppender(); final LogCapturer logCapturer = LogCapturer.captureLogs(LoggerFactory.getLogger("root"));
LogManager.getRootLogger().addAppender(appender);
Configuration conf = new HdfsConfiguration(); Configuration conf = new HdfsConfiguration();
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY, true); conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY, true);
// Set single handler thread, so all transactions hit same thread-local ops. // Set single handler thread, so all transactions hit same thread-local ops.
@ -1794,21 +1769,16 @@ public class TestEditLog {
rwf.close(); rwf.close();
events.poll(); events.poll();
String pattern = "Caught exception after reading (.*) ops"; for (String logLine : logCapturer.getOutput().split("\n")) {
Pattern r = Pattern.compile(pattern); if (logLine != null && logLine.contains("Caught exception after reading")) {
final List<LoggingEvent> log = appender.getLog();
for (LoggingEvent event : log) {
Matcher m = r.matcher(event.getRenderedMessage());
if (m.find()) {
fail("Should not try to read past latest syned edit log op"); fail("Should not try to read past latest syned edit log op");
} }
} }
} finally { } finally {
if (cluster != null) { if (cluster != null) {
cluster.shutdown(); cluster.shutdown();
} }
LogManager.getRootLogger().removeAppender(appender); logCapturer.stopCapturing();
} }
} }

View File

@ -26,6 +26,8 @@ import java.io.IOException;
import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.logging.LogCapturer;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
@ -118,8 +120,8 @@ public class TestEditsDoubleBuffer {
op3.setTransactionId(3); op3.setTransactionId(3);
buffer.writeOp(op3, fakeLogVersion); buffer.writeOp(op3, fakeLogVersion);
GenericTestUtils.LogCapturer logs = LogCapturer logs =
GenericTestUtils.LogCapturer.captureLogs(EditsDoubleBuffer.LOG); LogCapturer.captureLogs(EditsDoubleBuffer.LOG);
try { try {
buffer.close(); buffer.close();
fail(); fail();

View File

@ -64,7 +64,7 @@ import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.io.erasurecode.ECSchema;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.GenericTestUtils.LogCapturer; import org.apache.hadoop.logging.LogCapturer;
import org.apache.hadoop.test.PathUtils; import org.apache.hadoop.test.PathUtils;
import org.apache.hadoop.util.FakeTimer; import org.apache.hadoop.util.FakeTimer;
import org.slf4j.event.Level; import org.slf4j.event.Level;

View File

@ -25,7 +25,7 @@ import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.hadoop.metrics2.lib.MetricsRegistry; import org.apache.hadoop.metrics2.lib.MetricsRegistry;
import org.apache.hadoop.metrics2.lib.MutableRatesWithAggregation; import org.apache.hadoop.metrics2.lib.MutableRatesWithAggregation;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.GenericTestUtils.LogCapturer; import org.apache.hadoop.logging.LogCapturer;
import org.apache.hadoop.test.MetricsAsserts; import org.apache.hadoop.test.MetricsAsserts;
import org.apache.hadoop.util.FakeTimer; import org.apache.hadoop.util.FakeTimer;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;

View File

@ -29,6 +29,8 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.logging.LogCapturer;
import org.junit.After; import org.junit.After;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
@ -58,7 +60,7 @@ public class TestFSNamesystemLockReport {
private MiniDFSCluster cluster; private MiniDFSCluster cluster;
private FileSystem fs; private FileSystem fs;
private UserGroupInformation userGroupInfo; private UserGroupInformation userGroupInfo;
private GenericTestUtils.LogCapturer logs; private LogCapturer logs;
@Before @Before
public void setUp() throws Exception { public void setUp() throws Exception {
@ -76,7 +78,7 @@ public class TestFSNamesystemLockReport {
userGroupInfo = UserGroupInformation.createUserForTesting("bob", userGroupInfo = UserGroupInformation.createUserForTesting("bob",
new String[] {"hadoop"}); new String[] {"hadoop"});
logs = GenericTestUtils.LogCapturer.captureLogs(FSNamesystem.LOG); logs = LogCapturer.captureLogs(FSNamesystem.LOG);
GenericTestUtils GenericTestUtils
.setLogLevel(LoggerFactory.getLogger(FSNamesystem.class.getName()), .setLogLevel(LoggerFactory.getLogger(FSNamesystem.class.getName()),
org.slf4j.event.Level.INFO); org.slf4j.event.Level.INFO);

View File

@ -115,7 +115,7 @@ import org.apache.hadoop.net.NetworkTopology;
import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.GenericTestUtils.LogCapturer; import org.apache.hadoop.logging.LogCapturer;
import org.apache.hadoop.util.ToolRunner; import org.apache.hadoop.util.ToolRunner;
import org.junit.After; import org.junit.After;
import org.junit.AfterClass; import org.junit.AfterClass;

View File

@ -18,15 +18,13 @@
package org.apache.hadoop.hdfs.server.namenode; package org.apache.hadoop.hdfs.server.namenode;
import java.util.function.Supplier;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.logging.LogCapturer;
import org.apache.hadoop.metrics2.util.MBeans; import org.apache.hadoop.metrics2.util.MBeans;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.log4j.Appender;
import org.apache.log4j.AsyncAppender;
import org.junit.Rule; import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
@ -34,7 +32,6 @@ import org.junit.rules.Timeout;
import java.io.IOException; import java.io.IOException;
import java.util.Collections; import java.util.Collections;
import java.util.List;
import java.util.concurrent.TimeoutException; import java.util.concurrent.TimeoutException;
import static org.apache.hadoop.hdfs.DFSConfigKeys.*; import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
@ -64,12 +61,12 @@ public class TestNameNodeMetricsLogger {
} }
@Test @Test
@SuppressWarnings("unchecked")
public void testMetricsLoggerIsAsync() throws IOException { public void testMetricsLoggerIsAsync() throws IOException {
makeNameNode(true); makeNameNode(true);
org.apache.log4j.Logger logger = org.apache.log4j.Logger.getLogger(NameNode.METRICS_LOG_NAME); org.apache.log4j.Logger logger = org.apache.log4j.Logger.getLogger(NameNode.METRICS_LOG_NAME);
@SuppressWarnings("unchecked") assertTrue(Collections.list(logger.getAllAppenders()).get(0)
List<Appender> appenders = Collections.list(logger.getAllAppenders()); instanceof org.apache.log4j.AsyncAppender);
assertTrue(appenders.get(0) instanceof AsyncAppender);
} }
/** /**
@ -80,20 +77,14 @@ public class TestNameNodeMetricsLogger {
public void testMetricsLogOutput() public void testMetricsLogOutput()
throws IOException, InterruptedException, TimeoutException { throws IOException, InterruptedException, TimeoutException {
TestFakeMetric metricsProvider = new TestFakeMetric(); TestFakeMetric metricsProvider = new TestFakeMetric();
MBeans.register(this.getClass().getSimpleName(), MBeans.register(this.getClass().getSimpleName(), "DummyMetrics", metricsProvider);
"DummyMetrics", metricsProvider);
makeNameNode(true); // Log metrics early and often. makeNameNode(true); // Log metrics early and often.
final PatternMatchingAppender appender = LogCapturer logCapturer =
(PatternMatchingAppender) org.apache.log4j.Logger.getLogger(NameNode.METRICS_LOG_NAME) LogCapturer.captureLogs(LoggerFactory.getLogger(NameNode.METRICS_LOG_NAME));
.getAppender("PATTERNMATCHERAPPENDER");
// Ensure that the supplied pattern was matched. GenericTestUtils.waitFor(() -> logCapturer.getOutput().contains("FakeMetric"),
GenericTestUtils.waitFor(new Supplier<Boolean>() { 1000, 60000);
@Override logCapturer.stopCapturing();
public Boolean get() {
return appender.isMatched();
}
}, 1000, 60000);
} }
/** /**

View File

@ -28,7 +28,8 @@ import java.util.Collection;
import org.junit.Test; import org.junit.Test;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
import org.apache.hadoop.logging.LogCapturer;
public class TestNameNodeResourcePolicy { public class TestNameNodeResourcePolicy {

View File

@ -52,7 +52,6 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.LogVerificationAppender;
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.StripedFileTestUtil; import org.apache.hadoop.hdfs.StripedFileTestUtil;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@ -69,12 +68,12 @@ import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
import org.apache.hadoop.hdfs.util.HostsFileWriter; import org.apache.hadoop.hdfs.util.HostsFileWriter;
import org.apache.hadoop.hdfs.util.MD5FileUtils; import org.apache.hadoop.hdfs.util.MD5FileUtils;
import org.apache.hadoop.io.MD5Hash; import org.apache.hadoop.io.MD5Hash;
import org.apache.hadoop.logging.LogCapturer;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.PathUtils; import org.apache.hadoop.test.PathUtils;
import org.apache.hadoop.util.ExitUtil.ExitException; import org.apache.hadoop.util.ExitUtil.ExitException;
import org.apache.hadoop.util.ExitUtil; import org.apache.hadoop.util.ExitUtil;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.apache.log4j.Logger;
import org.junit.After; import org.junit.After;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
@ -524,10 +523,8 @@ public class TestStartup {
// Corrupt the md5 files in all the namedirs // Corrupt the md5 files in all the namedirs
corruptFSImageMD5(true); corruptFSImageMD5(true);
// Attach our own log appender so we can verify output // Attach our own log appender so we can verify output
final LogVerificationAppender appender = new LogVerificationAppender(); LogCapturer logCapturer = LogCapturer.captureLogs(LoggerFactory.getLogger("root"));
final Logger logger = Logger.getRootLogger();
logger.addAppender(appender);
// Try to start a new cluster // Try to start a new cluster
LOG.info("\n===========================================\n" + LOG.info("\n===========================================\n" +
@ -541,10 +538,13 @@ public class TestStartup {
} catch (IOException ioe) { } catch (IOException ioe) {
GenericTestUtils.assertExceptionContains( GenericTestUtils.assertExceptionContains(
"Failed to load FSImage file", ioe); "Failed to load FSImage file", ioe);
int md5failures = appender.countExceptionsWithMessage(
" is corrupt with MD5 checksum of "); int md5failures =
org.apache.commons.lang3.StringUtils.countMatches(logCapturer.getOutput(),
" is corrupt with MD5 checksum of ");
// Two namedirs, so should have seen two failures // Two namedirs, so should have seen two failures
assertEquals(2, md5failures); assertEquals(2, md5failures);
logCapturer.stopCapturing();
} }
} finally { } finally {
if (cluster != null) { if (cluster != null) {

View File

@ -43,7 +43,7 @@ import org.apache.hadoop.hdfs.server.namenode.NNStorage;
import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter; import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.GenericTestUtils.LogCapturer; import org.apache.hadoop.logging.LogCapturer;
import org.junit.After; import org.junit.After;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
@ -197,7 +197,7 @@ public class TestBootstrapStandby {
// Trying to bootstrap standby should now fail since the edit // Trying to bootstrap standby should now fail since the edit
// logs aren't available in the shared dir. // logs aren't available in the shared dir.
LogCapturer logs = GenericTestUtils.LogCapturer.captureLogs( LogCapturer logs = LogCapturer.captureLogs(
LoggerFactory.getLogger(BootstrapStandby.class)); LoggerFactory.getLogger(BootstrapStandby.class));
try { try {
assertEquals(BootstrapStandby.ERR_CODE_LOGS_UNAVAILABLE, forceBootstrap(1)); assertEquals(BootstrapStandby.ERR_CODE_LOGS_UNAVAILABLE, forceBootstrap(1));

View File

@ -44,6 +44,7 @@ import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.logging.LogCapturer;
import org.apache.hadoop.test.Whitebox; import org.apache.hadoop.test.Whitebox;
import org.junit.After; import org.junit.After;
import org.junit.Before; import org.junit.Before;
@ -143,7 +144,7 @@ public class TestDelegationTokensWithHA {
() -> (DistributedFileSystem) FileSystem.get(conf)); () -> (DistributedFileSystem) FileSystem.get(conf));
GenericTestUtils.setLogLevel(ObserverReadProxyProvider.LOG, Level.DEBUG); GenericTestUtils.setLogLevel(ObserverReadProxyProvider.LOG, Level.DEBUG);
GenericTestUtils.LogCapturer logCapture = GenericTestUtils.LogCapturer LogCapturer logCapture = LogCapturer
.captureLogs(ObserverReadProxyProvider.LOG); .captureLogs(ObserverReadProxyProvider.LOG);
try { try {
dfs.access(new Path("/"), FsAction.READ); dfs.access(new Path("/"), FsAction.READ);

View File

@ -37,7 +37,6 @@ import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.LogVerificationAppender;
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.MiniDFSNNTopology; import org.apache.hadoop.hdfs.MiniDFSNNTopology;
import org.apache.hadoop.hdfs.server.common.Util; import org.apache.hadoop.hdfs.server.common.Util;
@ -48,12 +47,12 @@ import org.apache.hadoop.io.compress.CompressionCodecFactory;
import org.apache.hadoop.io.compress.CompressionOutputStream; import org.apache.hadoop.io.compress.CompressionOutputStream;
import org.apache.hadoop.io.compress.GzipCodec; import org.apache.hadoop.io.compress.GzipCodec;
import org.apache.hadoop.ipc.StandbyException; import org.apache.hadoop.ipc.StandbyException;
import org.apache.hadoop.logging.LogCapturer;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.GenericTestUtils.DelayAnswer; import org.apache.hadoop.test.GenericTestUtils.DelayAnswer;
import org.apache.hadoop.test.PathUtils; import org.apache.hadoop.test.PathUtils;
import org.apache.hadoop.util.Lists; import org.apache.hadoop.util.Lists;
import org.apache.hadoop.util.ThreadUtil; import org.apache.hadoop.util.ThreadUtil;
import org.apache.log4j.spi.LoggingEvent;
import org.junit.After; import org.junit.After;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
@ -299,39 +298,38 @@ public class TestStandbyCheckpoints {
@Test(timeout = 30000) @Test(timeout = 30000)
public void testCheckpointBeforeNameNodeInitializationIsComplete() public void testCheckpointBeforeNameNodeInitializationIsComplete()
throws Exception { throws Exception {
final LogVerificationAppender appender = new LogVerificationAppender(); LogCapturer logCapturer = LogCapturer.captureLogs(LoggerFactory.getLogger("root"));
final org.apache.log4j.Logger logger = org.apache.log4j.Logger
.getRootLogger();
logger.addAppender(appender);
// Transition 2 to observer try {
cluster.transitionToObserver(2); // Transition 2 to observer
doEdits(0, 10); cluster.transitionToObserver(2);
// After a rollEditLog, Standby(nn1)'s next checkpoint would be doEdits(0, 10);
// ahead of observer(nn2). // After a rollEditLog, Standby(nn1)'s next checkpoint would be
nns[0].getRpcServer().rollEditLog(); // ahead of observer(nn2).
nns[0].getRpcServer().rollEditLog();
NameNode nn2 = nns[2]; NameNode nn2 = nns[2];
FSImage nnFSImage = NameNodeAdapter.getAndSetFSImageInHttpServer(nn2, null); FSImage nnFSImage = NameNodeAdapter.getAndSetFSImageInHttpServer(nn2, null);
// After standby creating a checkpoint, it will try to push the image to // After standby creating a checkpoint, it will try to push the image to
// active and all observer, updating it's own txid to the most recent. // active and all observer, updating it's own txid to the most recent.
HATestUtil.waitForCheckpoint(cluster, 1, ImmutableList.of(12)); HATestUtil.waitForCheckpoint(cluster, 1, ImmutableList.of(12));
HATestUtil.waitForCheckpoint(cluster, 0, ImmutableList.of(12)); HATestUtil.waitForCheckpoint(cluster, 0, ImmutableList.of(12));
NameNodeAdapter.getAndSetFSImageInHttpServer(nn2, nnFSImage); NameNodeAdapter.getAndSetFSImageInHttpServer(nn2, nnFSImage);
cluster.transitionToStandby(2); cluster.transitionToStandby(2);
logger.removeAppender(appender);
for (LoggingEvent event : appender.getLog()) { for (String logLine : logCapturer.getOutput().split("\n")) {
String message = event.getRenderedMessage(); if (logLine != null && logLine.contains("PutImage failed") && logLine.contains(
if (message.contains("PutImage failed") && "FSImage has not been set in the NameNode.")) {
message.contains("FSImage has not been set in the NameNode.")) { //Logs have the expected exception.
//Logs have the expected exception. return;
return; }
} }
fail("Expected exception not present in logs.");
} finally {
logCapturer.stopCapturing();
} }
fail("Expected exception not present in logs.");
} }
/** /**

View File

@ -93,7 +93,7 @@ import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authentication.util.KerberosName; import org.apache.hadoop.security.authentication.util.KerberosName;
import org.apache.hadoop.security.ssl.KeyStoreTestUtil; import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.GenericTestUtils.LogCapturer; import org.apache.hadoop.logging.LogCapturer;
import org.apache.hadoop.test.LambdaTestUtils; import org.apache.hadoop.test.LambdaTestUtils;
import org.apache.hadoop.util.ExitUtil; import org.apache.hadoop.util.ExitUtil;
import org.junit.After; import org.junit.After;
@ -1372,7 +1372,7 @@ public class TestExternalStoragePolicySatisfier {
Path filePath = new Path("/zeroSizeFile"); Path filePath = new Path("/zeroSizeFile");
DFSTestUtil.createFile(fs, filePath, 1024, (short) 5, 0); DFSTestUtil.createFile(fs, filePath, 1024, (short) 5, 0);
fs.setReplication(filePath, (short) 3); fs.setReplication(filePath, (short) 3);
LogCapturer logs = GenericTestUtils.LogCapturer.captureLogs( LogCapturer logs = LogCapturer.captureLogs(
LoggerFactory.getLogger(BlockStorageMovementAttemptedItems.class)); LoggerFactory.getLogger(BlockStorageMovementAttemptedItems.class));
fs.setStoragePolicy(filePath, "COLD"); fs.setStoragePolicy(filePath, "COLD");
fs.satisfyStoragePolicy(filePath); fs.satisfyStoragePolicy(filePath);

View File

@ -22,9 +22,6 @@ log4j.appender.stdout=org.apache.log4j.ConsoleAppender
log4j.appender.stdout.layout=org.apache.log4j.PatternLayout log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} (%F:%M(%L)) - %m%n log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} (%F:%M(%L)) - %m%n
# Only to be used for testing
log4j.appender.PATTERNMATCHERAPPENDER=org.apache.hadoop.hdfs.server.namenode.PatternMatchingAppender
# #
# NameNode metrics logging. # NameNode metrics logging.
# The default is to retain two namenode-metrics.log files up to 64MB each. # The default is to retain two namenode-metrics.log files up to 64MB each.
@ -32,10 +29,10 @@ log4j.appender.PATTERNMATCHERAPPENDER=org.apache.hadoop.hdfs.server.namenode.Pat
# TODO : While migrating to log4j2, replace AsyncRFAAppender with AsyncAppender as # TODO : While migrating to log4j2, replace AsyncRFAAppender with AsyncAppender as
# log4j2 properties support wrapping of other appenders to AsyncAppender using appender ref # log4j2 properties support wrapping of other appenders to AsyncAppender using appender ref
namenode.metrics.logger=INFO,ASYNCNNMETRICSRFA,PATTERNMATCHERAPPENDER namenode.metrics.logger=INFO,ASYNCNNMETRICSRFA
log4j.logger.NameNodeMetricsLog=${namenode.metrics.logger} log4j.logger.NameNodeMetricsLog=${namenode.metrics.logger}
log4j.additivity.NameNodeMetricsLog=false log4j.additivity.NameNodeMetricsLog=false
log4j.appender.ASYNCNNMETRICSRFA=org.apache.hadoop.hdfs.util.AsyncRFAAppender log4j.appender.ASYNCNNMETRICSRFA=org.apache.hadoop.logging.appenders.AsyncRFAAppender
log4j.appender.ASYNCNNMETRICSRFA.conversionPattern=%d{ISO8601} %m%n log4j.appender.ASYNCNNMETRICSRFA.conversionPattern=%d{ISO8601} %m%n
log4j.appender.ASYNCNNMETRICSRFA.maxFileSize=64MB log4j.appender.ASYNCNNMETRICSRFA.maxFileSize=64MB
log4j.appender.ASYNCNNMETRICSRFA.fileName=${hadoop.log.dir}/namenode-metrics.log log4j.appender.ASYNCNNMETRICSRFA.fileName=${hadoop.log.dir}/namenode-metrics.log
@ -48,10 +45,10 @@ log4j.appender.ASYNCNNMETRICSRFA.maxBackupIndex=1
# TODO : While migrating to log4j2, replace AsyncRFAAppender with AsyncAppender as # TODO : While migrating to log4j2, replace AsyncRFAAppender with AsyncAppender as
# log4j2 properties support wrapping of other appenders to AsyncAppender using appender ref # log4j2 properties support wrapping of other appenders to AsyncAppender using appender ref
datanode.metrics.logger=INFO,ASYNCDNMETRICSRFA,PATTERNMATCHERAPPENDER datanode.metrics.logger=INFO,ASYNCDNMETRICSRFA
log4j.logger.DataNodeMetricsLog=${datanode.metrics.logger} log4j.logger.DataNodeMetricsLog=${datanode.metrics.logger}
log4j.additivity.DataNodeMetricsLog=false log4j.additivity.DataNodeMetricsLog=false
log4j.appender.ASYNCDNMETRICSRFA=org.apache.hadoop.hdfs.util.AsyncRFAAppender log4j.appender.ASYNCDNMETRICSRFA=org.apache.hadoop.logging.appenders.AsyncRFAAppender
log4j.appender.ASYNCDNMETRICSRFA.conversionPattern=%d{ISO8601} %m%n log4j.appender.ASYNCDNMETRICSRFA.conversionPattern=%d{ISO8601} %m%n
log4j.appender.ASYNCDNMETRICSRFA.maxFileSize=64MB log4j.appender.ASYNCDNMETRICSRFA.maxFileSize=64MB
log4j.appender.ASYNCDNMETRICSRFA.fileName=${hadoop.log.dir}/datanode-metrics.log log4j.appender.ASYNCDNMETRICSRFA.fileName=${hadoop.log.dir}/datanode-metrics.log
@ -72,7 +69,7 @@ hdfs.audit.log.maxfilesize=256MB
hdfs.audit.log.maxbackupindex=20 hdfs.audit.log.maxbackupindex=20
log4j.logger.org.apache.hadoop.hdfs.server.namenode.FSNamesystem.audit=${hdfs.audit.logger} log4j.logger.org.apache.hadoop.hdfs.server.namenode.FSNamesystem.audit=${hdfs.audit.logger}
log4j.additivity.org.apache.hadoop.hdfs.server.namenode.FSNamesystem.audit=false log4j.additivity.org.apache.hadoop.hdfs.server.namenode.FSNamesystem.audit=false
log4j.appender.ASYNCAUDITAPPENDER=org.apache.hadoop.hdfs.util.AsyncRFAAppender log4j.appender.ASYNCAUDITAPPENDER=org.apache.hadoop.logging.appenders.AsyncRFAAppender
log4j.appender.ASYNCAUDITAPPENDER.blocking=false log4j.appender.ASYNCAUDITAPPENDER.blocking=false
log4j.appender.ASYNCAUDITAPPENDER.bufferSize=256 log4j.appender.ASYNCAUDITAPPENDER.bufferSize=256
log4j.appender.ASYNCAUDITAPPENDER.conversionPattern=%m%n log4j.appender.ASYNCAUDITAPPENDER.conversionPattern=%m%n

View File

@ -124,6 +124,12 @@
<artifactId>assertj-core</artifactId> <artifactId>assertj-core</artifactId>
<scope>test</scope> <scope>test</scope>
</dependency> </dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-logging</artifactId>
<scope>test</scope>
<type>test-jar</type>
</dependency>
</dependencies> </dependencies>
<build> <build>

View File

@ -36,9 +36,10 @@ import java.util.HashMap;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.concurrent.CopyOnWriteArrayList;
import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableMap; import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableMap;
import org.apache.hadoop.logging.LogCapturer;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptFailEvent; import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptFailEvent;
import org.apache.hadoop.yarn.util.resource.CustomResourceTypesConfigurationProvider; import org.apache.hadoop.yarn.util.resource.CustomResourceTypesConfigurationProvider;
import org.junit.After; import org.junit.After;
@ -107,12 +108,10 @@ import org.apache.hadoop.yarn.util.Clock;
import org.apache.hadoop.yarn.util.ControlledClock; import org.apache.hadoop.yarn.util.ControlledClock;
import org.apache.hadoop.yarn.util.SystemClock; import org.apache.hadoop.yarn.util.SystemClock;
import org.apache.hadoop.yarn.util.resource.ResourceUtils; import org.apache.hadoop.yarn.util.resource.ResourceUtils;
import org.apache.log4j.AppenderSkeleton;
import org.apache.log4j.Level;
import org.apache.log4j.Logger;
import org.apache.log4j.spi.LoggingEvent;
import org.junit.Test; import org.junit.Test;
import org.mockito.ArgumentCaptor; import org.mockito.ArgumentCaptor;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableList; import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableList;
@ -128,29 +127,6 @@ public class TestTaskAttempt{
} }
} }
private static class TestAppender extends AppenderSkeleton {
private final List<LoggingEvent> logEvents = new CopyOnWriteArrayList<>();
@Override
public boolean requiresLayout() {
return false;
}
@Override
public void close() {
}
@Override
protected void append(LoggingEvent arg0) {
logEvents.add(arg0);
}
private List<LoggingEvent> getLogEvents() {
return logEvents;
}
}
@BeforeClass @BeforeClass
public static void setupBeforeClass() { public static void setupBeforeClass() {
ResourceUtils.resetResourceTypes(new Configuration()); ResourceUtils.resetResourceTypes(new Configuration());
@ -1724,11 +1700,10 @@ public class TestTaskAttempt{
for (String memoryName : ImmutableList.of( for (String memoryName : ImmutableList.of(
MRJobConfig.RESOURCE_TYPE_NAME_MEMORY, MRJobConfig.RESOURCE_TYPE_NAME_MEMORY,
MRJobConfig.RESOURCE_TYPE_ALTERNATIVE_NAME_MEMORY)) { MRJobConfig.RESOURCE_TYPE_ALTERNATIVE_NAME_MEMORY)) {
TestAppender testAppender = new TestAppender(); final Logger logger = LoggerFactory.getLogger(TaskAttemptImpl.class);
final Logger logger = Logger.getLogger(TaskAttemptImpl.class); LogCapturer logCapturer = LogCapturer.captureLogs(logger);
try { try {
TaskAttemptImpl.RESOURCE_REQUEST_CACHE.clear(); TaskAttemptImpl.RESOURCE_REQUEST_CACHE.clear();
logger.addAppender(testAppender);
EventHandler eventHandler = mock(EventHandler.class); EventHandler eventHandler = mock(EventHandler.class);
Clock clock = SystemClock.getInstance(); Clock clock = SystemClock.getInstance();
JobConf jobConf = new JobConf(); JobConf jobConf = new JobConf();
@ -1741,13 +1716,11 @@ public class TestTaskAttempt{
getResourceInfoFromContainerRequest(taImpl, eventHandler). getResourceInfoFromContainerRequest(taImpl, eventHandler).
getMemorySize(); getMemorySize();
assertEquals(3072, memorySize); assertEquals(3072, memorySize);
assertTrue(testAppender.getLogEvents().stream() assertTrue(logCapturer.getOutput().contains(
.anyMatch(e -> e.getLevel() == Level.WARN && ("Configuration " + "Configuration " + "mapreduce.reduce.resource." + memoryName + "=3Gi is "
"mapreduce.reduce.resource." + memoryName + "=3Gi is " + + "overriding the mapreduce.reduce.memory.mb=2048 configuration"));
"overriding the mapreduce.reduce.memory.mb=2048 configuration")
.equals(e.getMessage())));
} finally { } finally {
logger.removeAppender(testAppender); logCapturer.stopCapturing();
} }
} }
} }
@ -1809,10 +1782,9 @@ public class TestTaskAttempt{
@Test @Test
public void testReducerCpuRequestOverriding() { public void testReducerCpuRequestOverriding() {
TestAppender testAppender = new TestAppender(); final Logger logger = LoggerFactory.getLogger(TaskAttemptImpl.class);
final Logger logger = Logger.getLogger(TaskAttemptImpl.class); final LogCapturer logCapturer = LogCapturer.captureLogs(logger);
try { try {
logger.addAppender(testAppender);
EventHandler eventHandler = mock(EventHandler.class); EventHandler eventHandler = mock(EventHandler.class);
Clock clock = SystemClock.getInstance(); Clock clock = SystemClock.getInstance();
JobConf jobConf = new JobConf(); JobConf jobConf = new JobConf();
@ -1825,13 +1797,11 @@ public class TestTaskAttempt{
getResourceInfoFromContainerRequest(taImpl, eventHandler). getResourceInfoFromContainerRequest(taImpl, eventHandler).
getVirtualCores(); getVirtualCores();
assertEquals(7, vCores); assertEquals(7, vCores);
assertTrue(testAppender.getLogEvents().stream().anyMatch( assertTrue(logCapturer.getOutput().contains(
e -> e.getLevel() == Level.WARN && ("Configuration " + "Configuration " + "mapreduce.reduce.resource.vcores=7 is overriding the "
"mapreduce.reduce.resource.vcores=7 is overriding the " + + "mapreduce.reduce.cpu.vcores=9 configuration"));
"mapreduce.reduce.cpu.vcores=9 configuration").equals(
e.getMessage())));
} finally { } finally {
logger.removeAppender(testAppender); logCapturer.stopCapturing();
} }
} }

View File

@ -72,6 +72,12 @@
<artifactId>assertj-core</artifactId> <artifactId>assertj-core</artifactId>
<scope>test</scope> <scope>test</scope>
</dependency> </dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-logging</artifactId>
<scope>test</scope>
<type>test-jar</type>
</dependency>
</dependencies> </dependencies>
<build> <build>

View File

@ -23,12 +23,10 @@ import java.io.BufferedReader;
import java.io.DataOutputStream; import java.io.DataOutputStream;
import java.io.File; import java.io.File;
import java.io.FileInputStream; import java.io.FileInputStream;
import java.io.Flushable;
import java.io.IOException; import java.io.IOException;
import java.io.InputStream; import java.io.InputStream;
import java.io.InputStreamReader; import java.io.InputStreamReader;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Enumeration;
import java.util.List; import java.util.List;
import java.util.concurrent.Executors; import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledExecutorService;
@ -44,16 +42,13 @@ import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.SecureIOUtils; import org.apache.hadoop.io.SecureIOUtils;
import org.apache.hadoop.logging.HadoopLoggerUtils;
import org.apache.hadoop.mapreduce.JobID; import org.apache.hadoop.mapreduce.JobID;
import org.apache.hadoop.mapreduce.util.ProcessTree; import org.apache.hadoop.mapreduce.util.ProcessTree;
import org.apache.hadoop.util.Shell; import org.apache.hadoop.util.Shell;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.ShutdownHookManager; import org.apache.hadoop.util.ShutdownHookManager;
import org.apache.hadoop.util.concurrent.HadoopExecutors; import org.apache.hadoop.util.concurrent.HadoopExecutors;
import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.log4j.Appender;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.apache.hadoop.thirdparty.com.google.common.base.Charsets;
@ -276,42 +271,7 @@ public class TaskLog {
} }
// flush & close all appenders // flush & close all appenders
LogManager.shutdown(); HadoopLoggerUtils.shutdownLogManager();
}
@SuppressWarnings("unchecked")
public static synchronized void syncLogs() {
// flush standard streams
//
System.out.flush();
System.err.flush();
// flush flushable appenders
//
final Logger rootLogger = Logger.getRootLogger();
flushAppenders(rootLogger);
final Enumeration<Logger> allLoggers = rootLogger.getLoggerRepository().
getCurrentLoggers();
while (allLoggers.hasMoreElements()) {
final Logger l = allLoggers.nextElement();
flushAppenders(l);
}
}
@SuppressWarnings("unchecked")
private static void flushAppenders(Logger l) {
final Enumeration<Appender> allAppenders = l.getAllAppenders();
while (allAppenders.hasMoreElements()) {
final Appender a = allAppenders.nextElement();
if (a instanceof Flushable) {
try {
((Flushable) a).flush();
} catch (IOException ioe) {
System.err.println(a + ": Failed to flush!"
+ StringUtils.stringifyException(ioe));
}
}
}
} }
public static ScheduledExecutorService createLogSyncer() { public static ScheduledExecutorService createLogSyncer() {
@ -336,7 +296,7 @@ public class TaskLog {
new Runnable() { new Runnable() {
@Override @Override
public void run() { public void run() {
TaskLog.syncLogs(); HadoopLoggerUtils.syncLogs();
} }
}, 0L, 5L, TimeUnit.SECONDS); }, 0L, 5L, TimeUnit.SECONDS);
return scheduler; return scheduler;

View File

@ -28,24 +28,19 @@ import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.spy; import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.when; import static org.mockito.Mockito.when;
import java.io.ByteArrayOutputStream;
import java.io.IOException; import java.io.IOException;
import java.io.LineNumberReader;
import java.io.StringReader;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import static org.junit.Assert.*; import static org.junit.Assert.*;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.logging.LogCapturer;
import org.apache.hadoop.mapred.TaskReport; import org.apache.hadoop.mapred.TaskReport;
import org.apache.hadoop.mapreduce.JobStatus.State; import org.apache.hadoop.mapreduce.JobStatus.State;
import org.apache.hadoop.mapreduce.protocol.ClientProtocol; import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
import org.apache.log4j.Layout;
import org.apache.log4j.Level;
import org.apache.log4j.Logger;
import org.apache.log4j.WriterAppender;
import org.mockito.stubbing.Answer; import org.mockito.stubbing.Answer;
import org.slf4j.LoggerFactory;
/** /**
* Test to make sure that command line output for * Test to make sure that command line output for
@ -73,55 +68,53 @@ public class TestJobMonitorAndPrint {
@Test @Test
public void testJobMonitorAndPrint() throws Exception { public void testJobMonitorAndPrint() throws Exception {
JobStatus jobStatus_1 = new JobStatus(new JobID("job_000", 1), 1f, 0.1f, LogCapturer logCapturer = LogCapturer.captureLogs(LoggerFactory.getLogger(Job.class));
0.1f, 0f, State.RUNNING, JobPriority.HIGH, "tmp-user", "tmp-jobname", try {
"tmp-queue", "tmp-jobfile", "tmp-url", true); JobStatus jobStatus_1 =
JobStatus jobStatus_2 = new JobStatus(new JobID("job_000", 1), 1f, 1f, new JobStatus(new JobID("job_000", 1), 1f, 0.1f, 0.1f, 0f, State.RUNNING,
1f, 1f, State.SUCCEEDED, JobPriority.HIGH, "tmp-user", "tmp-jobname", JobPriority.HIGH, "tmp-user", "tmp-jobname", "tmp-queue", "tmp-jobfile", "tmp-url",
"tmp-queue", "tmp-jobfile", "tmp-url", true); true);
JobStatus jobStatus_2 =
new JobStatus(new JobID("job_000", 1), 1f, 1f, 1f, 1f, State.SUCCEEDED, JobPriority.HIGH,
"tmp-user", "tmp-jobname", "tmp-queue", "tmp-jobfile", "tmp-url", true);
doAnswer((Answer<TaskCompletionEvent[]>) invocation -> doAnswer((Answer<TaskCompletionEvent[]>) invocation -> TaskCompletionEvent.EMPTY_ARRAY).when(
TaskCompletionEvent.EMPTY_ARRAY).when(job) job).getTaskCompletionEvents(anyInt(), anyInt());
.getTaskCompletionEvents(anyInt(), anyInt());
doReturn(new TaskReport[5]).when(job).getTaskReports(isA(TaskType.class)); doReturn(new TaskReport[5]).when(job).getTaskReports(isA(TaskType.class));
when(clientProtocol.getJobStatus(any(JobID.class))).thenReturn(jobStatus_1, jobStatus_2); when(clientProtocol.getJobStatus(any(JobID.class))).thenReturn(jobStatus_1, jobStatus_2);
// setup the logger to capture all logs
Layout layout =
Logger.getRootLogger().getAppender("stdout").getLayout();
ByteArrayOutputStream os = new ByteArrayOutputStream();
WriterAppender appender = new WriterAppender(layout, os);
appender.setThreshold(Level.ALL);
Logger qlogger = Logger.getLogger(Job.class);
qlogger.addAppender(appender);
job.monitorAndPrintJob(); job.monitorAndPrintJob();
qlogger.removeAppender(appender); boolean foundHundred = false;
LineNumberReader r = new LineNumberReader(new StringReader(os.toString())); boolean foundComplete = false;
String line; boolean foundUber = false;
boolean foundHundred = false; String uberModeMatch = "uber mode : true";
boolean foundComplete = false; String progressMatch = "map 100% reduce 100%";
boolean foundUber = false; String completionMatch = "completed successfully";
String uberModeMatch = "uber mode : true"; for (String logLine : logCapturer.getOutput().split("\n")) {
String progressMatch = "map 100% reduce 100%"; if (logLine.contains(uberModeMatch)) {
String completionMatch = "completed successfully"; foundUber = true;
while ((line = r.readLine()) != null) { }
if (line.contains(uberModeMatch)) { if (logLine.contains(progressMatch)) {
foundUber = true; foundHundred = true;
}
if (logLine.contains(completionMatch)) {
foundComplete = true;
}
if (foundUber && foundHundred && foundComplete) {
break;
}
} }
foundHundred = line.contains(progressMatch); assertTrue(foundUber);
if (foundHundred) assertTrue(foundHundred);
break; assertTrue(foundComplete);
}
line = r.readLine();
foundComplete = line.contains(completionMatch);
assertTrue(foundUber);
assertTrue(foundHundred);
assertTrue(foundComplete);
System.out.println("The output of job.toString() is : \n" + job.toString()); System.out.println("The output of job.toString() is : \n" + job.toString());
assertTrue(job.toString().contains("Number of maps: 5\n")); assertTrue(job.toString().contains("Number of maps: 5\n"));
assertTrue(job.toString().contains("Number of reduces: 5\n")); assertTrue(job.toString().contains("Number of reduces: 5\n"));
} finally {
logCapturer.stopCapturing();
}
} }
} }

View File

@ -128,6 +128,12 @@
<artifactId>assertj-core</artifactId> <artifactId>assertj-core</artifactId>
<scope>test</scope> <scope>test</scope>
</dependency> </dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-logging</artifactId>
<scope>test</scope>
<type>test-jar</type>
</dependency>
</dependencies> </dependencies>
<profiles> <profiles>

View File

@ -34,7 +34,6 @@ import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when; import static org.mockito.Mockito.when;
import java.io.ByteArrayOutputStream;
import java.io.File; import java.io.File;
import java.io.FileOutputStream; import java.io.FileOutputStream;
import java.io.IOException; import java.io.IOException;
@ -45,7 +44,6 @@ import java.security.PrivilegedExceptionAction;
import java.util.Arrays; import java.util.Arrays;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.concurrent.CopyOnWriteArrayList;
import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableMap; import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableMap;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
@ -55,6 +53,7 @@ import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Text;
import org.apache.hadoop.logging.LogCapturer;
import org.apache.hadoop.mapreduce.JobID; import org.apache.hadoop.mapreduce.JobID;
import org.apache.hadoop.mapreduce.JobPriority; import org.apache.hadoop.mapreduce.JobPriority;
import org.apache.hadoop.mapreduce.JobStatus.State; import org.apache.hadoop.mapreduce.JobStatus.State;
@ -110,13 +109,6 @@ import org.apache.hadoop.yarn.server.utils.BuilderUtils;
import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.Records;
import org.apache.hadoop.yarn.util.resource.CustomResourceTypesConfigurationProvider; import org.apache.hadoop.yarn.util.resource.CustomResourceTypesConfigurationProvider;
import org.apache.hadoop.yarn.util.resource.ResourceUtils; import org.apache.hadoop.yarn.util.resource.ResourceUtils;
import org.apache.log4j.Appender;
import org.apache.log4j.AppenderSkeleton;
import org.apache.log4j.Layout;
import org.apache.log4j.Level;
import org.apache.log4j.SimpleLayout;
import org.apache.log4j.WriterAppender;
import org.apache.log4j.spi.LoggingEvent;
import org.junit.After; import org.junit.After;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
@ -144,29 +136,6 @@ public class TestYARNRunner {
MRJobConfig.DEFAULT_TASK_PROFILE_PARAMS.lastIndexOf("%")); MRJobConfig.DEFAULT_TASK_PROFILE_PARAMS.lastIndexOf("%"));
private static final String CUSTOM_RESOURCE_NAME = "a-custom-resource"; private static final String CUSTOM_RESOURCE_NAME = "a-custom-resource";
private static class TestAppender extends AppenderSkeleton {
private final List<LoggingEvent> logEvents = new CopyOnWriteArrayList<>();
@Override
public boolean requiresLayout() {
return false;
}
@Override
public void close() {
}
@Override
protected void append(LoggingEvent arg0) {
logEvents.add(arg0);
}
private List<LoggingEvent> getLogEvents() {
return logEvents;
}
}
private YARNRunner yarnRunner; private YARNRunner yarnRunner;
private ResourceMgrDelegate resourceMgrDelegate; private ResourceMgrDelegate resourceMgrDelegate;
private YarnConfiguration conf; private YarnConfiguration conf;
@ -549,38 +518,48 @@ public class TestYARNRunner {
assertTrue("AM admin command opts is after user command opts.", adminIndex < userIndex); assertTrue("AM admin command opts is after user command opts.", adminIndex < userIndex);
} }
} }
@Test(timeout=20000) @Test(timeout=20000)
public void testWarnCommandOpts() throws Exception { public void testWarnCommandOpts() throws Exception {
org.apache.log4j.Logger logger = LogCapturer logCapturer = LogCapturer.captureLogs(LoggerFactory.getLogger(YARNRunner.class));
org.apache.log4j.Logger.getLogger(YARNRunner.class); try {
JobConf jobConf = new JobConf();
ByteArrayOutputStream bout = new ByteArrayOutputStream();
Layout layout = new SimpleLayout(); jobConf.set(MRJobConfig.MR_AM_ADMIN_COMMAND_OPTS,
Appender appender = new WriterAppender(layout, bout); "-Djava.net.preferIPv4Stack=true -Djava.library.path=foo");
logger.addAppender(appender); jobConf.set(MRJobConfig.MR_AM_COMMAND_OPTS, "-Xmx1024m -Djava.library.path=bar");
JobConf jobConf = new JobConf(); YARNRunner yarnRunner = new YARNRunner(jobConf);
jobConf.set(MRJobConfig.MR_AM_ADMIN_COMMAND_OPTS, "-Djava.net.preferIPv4Stack=true -Djava.library.path=foo"); @SuppressWarnings("unused")
jobConf.set(MRJobConfig.MR_AM_COMMAND_OPTS, "-Xmx1024m -Djava.library.path=bar"); ApplicationSubmissionContext submissionContext = buildSubmitContext(yarnRunner, jobConf);
YARNRunner yarnRunner = new YARNRunner(jobConf); boolean isFoundOne = false;
boolean isFoundTwo = false;
@SuppressWarnings("unused") for (String logLine : logCapturer.getOutput().split("\n")) {
ApplicationSubmissionContext submissionContext = if (logLine == null) {
buildSubmitContext(yarnRunner, jobConf); continue;
}
String logMsg = bout.toString(); if (logLine.contains("WARN") && logLine.contains("Usage of -Djava.library.path in "
assertTrue(logMsg.contains("WARN - Usage of -Djava.library.path in " + + "yarn.app.mapreduce.am.admin-command-opts can cause programs to no "
"yarn.app.mapreduce.am.admin-command-opts can cause programs to no " + + "longer function if hadoop native libraries are used. These values "
"longer function if hadoop native libraries are used. These values " + + "should be set as part of the LD_LIBRARY_PATH in the app master JVM "
"should be set as part of the LD_LIBRARY_PATH in the app master JVM " + + "env using yarn.app.mapreduce.am.admin.user.env config settings.")) {
"env using yarn.app.mapreduce.am.admin.user.env config settings.")); isFoundOne = true;
assertTrue(logMsg.contains("WARN - Usage of -Djava.library.path in " + }
"yarn.app.mapreduce.am.command-opts can cause programs to no longer " + if (logLine.contains("WARN") && logLine.contains("Usage of -Djava.library.path in "
"function if hadoop native libraries are used. These values should " + + "yarn.app.mapreduce.am.command-opts can cause programs to no longer "
"be set as part of the LD_LIBRARY_PATH in the app master JVM env " + + "function if hadoop native libraries are used. These values should "
"using yarn.app.mapreduce.am.env config settings.")); + "be set as part of the LD_LIBRARY_PATH in the app master JVM env "
+ "using yarn.app.mapreduce.am.env config settings.")) {
isFoundTwo = true;
}
}
assertTrue(isFoundOne);
assertTrue(isFoundTwo);
} finally {
logCapturer.stopCapturing();
}
} }
@Test(timeout=20000) @Test(timeout=20000)
@ -996,10 +975,7 @@ public class TestYARNRunner {
for (String memoryName : ImmutableList.of( for (String memoryName : ImmutableList.of(
MRJobConfig.RESOURCE_TYPE_NAME_MEMORY, MRJobConfig.RESOURCE_TYPE_NAME_MEMORY,
MRJobConfig.RESOURCE_TYPE_ALTERNATIVE_NAME_MEMORY)) { MRJobConfig.RESOURCE_TYPE_ALTERNATIVE_NAME_MEMORY)) {
TestAppender testAppender = new TestAppender(); LogCapturer logCapturer = LogCapturer.captureLogs(LoggerFactory.getLogger(YARNRunner.class));
org.apache.log4j.Logger logger =
org.apache.log4j.Logger.getLogger(YARNRunner.class);
logger.addAppender(testAppender);
try { try {
JobConf jobConf = new JobConf(); JobConf jobConf = new JobConf();
jobConf.set(MRJobConfig.MR_AM_RESOURCE_PREFIX + memoryName, "3 Gi"); jobConf.set(MRJobConfig.MR_AM_RESOURCE_PREFIX + memoryName, "3 Gi");
@ -1017,13 +993,17 @@ public class TestYARNRunner {
long memorySize = resourceRequest.getCapability().getMemorySize(); long memorySize = resourceRequest.getCapability().getMemorySize();
Assert.assertEquals(3072, memorySize); Assert.assertEquals(3072, memorySize);
assertTrue(testAppender.getLogEvents().stream().anyMatch( boolean isLogFound = false;
e -> e.getLevel() == Level.WARN && ("Configuration " + for (String logLine : logCapturer.getOutput().split("\n")) {
"yarn.app.mapreduce.am.resource." + memoryName + "=3Gi is " + if (logLine != null && logLine.contains("WARN") && logLine.contains(
"overriding the yarn.app.mapreduce.am.resource.mb=2048 " + "Configuration " + "yarn.app.mapreduce.am.resource." + memoryName + "=3Gi is "
"configuration").equals(e.getMessage()))); + "overriding the yarn.app.mapreduce.am.resource.mb=2048 " + "configuration")) {
isLogFound = true;
}
}
assertTrue("Log line could not be found", isLogFound);
} finally { } finally {
logger.removeAppender(testAppender); logCapturer.stopCapturing();
} }
} }
} }

View File

@ -29,8 +29,6 @@ import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapred.HadoopTestCase; import org.apache.hadoop.mapred.HadoopTestCase;
import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.JobConf;
import org.apache.log4j.Level;
import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
@ -76,12 +74,10 @@ public class TestChild extends HadoopTestCase {
mapJavaOpts, mapJavaOpts,
mapJavaOpts, MAP_OPTS_VAL); mapJavaOpts, MAP_OPTS_VAL);
} }
Level logLevel = String logLevel = conf.get(JobConf.MAPRED_MAP_TASK_LOG_LEVEL, "INFO");
Level.toLevel(conf.get(JobConf.MAPRED_MAP_TASK_LOG_LEVEL, assertEquals(JobConf.MAPRED_MAP_TASK_LOG_LEVEL + "has value of " + logLevel, logLevel,
Level.INFO.toString())); "OFF");
assertEquals(JobConf.MAPRED_MAP_TASK_LOG_LEVEL + "has value of " +
logLevel, logLevel, Level.OFF);
} }
} }
@ -108,12 +104,10 @@ public class TestChild extends HadoopTestCase {
reduceJavaOpts, reduceJavaOpts,
reduceJavaOpts, REDUCE_OPTS_VAL); reduceJavaOpts, REDUCE_OPTS_VAL);
} }
Level logLevel = String logLevel = conf.get(JobConf.MAPRED_REDUCE_TASK_LOG_LEVEL, "INFO");
Level.toLevel(conf.get(JobConf.MAPRED_REDUCE_TASK_LOG_LEVEL, assertEquals(JobConf.MAPRED_REDUCE_TASK_LOG_LEVEL + "has value of " + logLevel, logLevel,
Level.INFO.toString())); "OFF");
assertEquals(JobConf.MAPRED_REDUCE_TASK_LOG_LEVEL + "has value of " +
logLevel, logLevel, Level.OFF);
} }
} }
@ -127,9 +121,9 @@ public class TestChild extends HadoopTestCase {
conf.set(JobConf.MAPRED_MAP_TASK_JAVA_OPTS, MAP_OPTS_VAL); conf.set(JobConf.MAPRED_MAP_TASK_JAVA_OPTS, MAP_OPTS_VAL);
conf.set(JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS, REDUCE_OPTS_VAL); conf.set(JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS, REDUCE_OPTS_VAL);
} }
conf.set(JobConf.MAPRED_MAP_TASK_LOG_LEVEL, Level.OFF.toString()); conf.set(JobConf.MAPRED_MAP_TASK_LOG_LEVEL, "OFF");
conf.set(JobConf.MAPRED_REDUCE_TASK_LOG_LEVEL, Level.OFF.toString()); conf.set(JobConf.MAPRED_REDUCE_TASK_LOG_LEVEL, "OFF");
Job job = MapReduceTestUtil.createJob(conf, inDir, outDir, Job job = MapReduceTestUtil.createJob(conf, inDir, outDir,
numMaps, numReds); numMaps, numReds);

View File

@ -25,6 +25,7 @@ import java.net.InetSocketAddress;
import java.security.PrivilegedAction; import java.security.PrivilegedAction;
import java.security.PrivilegedExceptionAction; import java.security.PrivilegedExceptionAction;
import org.apache.hadoop.logging.HadoopLoggerUtils;
import org.apache.hadoop.security.token.SecretManager; import org.apache.hadoop.security.token.SecretManager;
import org.apache.hadoop.test.LambdaTestUtils; import org.apache.hadoop.test.LambdaTestUtils;
import org.junit.Assert; import org.junit.Assert;
@ -50,8 +51,6 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.ipc.YarnRPC; import org.apache.hadoop.yarn.ipc.YarnRPC;
import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.hadoop.yarn.util.ConverterUtils;
import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.Records;
import org.apache.log4j.Level;
import org.apache.log4j.LogManager;
import org.junit.Test; import org.junit.Test;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -64,8 +63,7 @@ public class TestJHSSecurity {
@Test @Test
public void testDelegationToken() throws Exception { public void testDelegationToken() throws Exception {
org.apache.log4j.Logger rootLogger = LogManager.getRootLogger(); HadoopLoggerUtils.setLogLevel("root", "DEBUG");
rootLogger.setLevel(Level.DEBUG);
final YarnConfiguration conf = new YarnConfiguration(new JobConf()); final YarnConfiguration conf = new YarnConfiguration(new JobConf());
// Just a random principle // Just a random principle

View File

@ -99,7 +99,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.WorkflowPriorityMappingsManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.WorkflowPriorityMappingsManager;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.WorkflowPriorityMappingsManager.WorkflowPriorityMapping; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.WorkflowPriorityMappingsManager.WorkflowPriorityMapping;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
import org.apache.log4j.Level;
import org.junit.After; import org.junit.After;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.Assert; import org.junit.Assert;
@ -557,9 +556,9 @@ public class TestMRJobs {
systemClasses); systemClasses);
} }
sleepConf.set(MRJobConfig.IO_SORT_MB, TEST_IO_SORT_MB); sleepConf.set(MRJobConfig.IO_SORT_MB, TEST_IO_SORT_MB);
sleepConf.set(MRJobConfig.MR_AM_LOG_LEVEL, Level.ALL.toString()); sleepConf.set(MRJobConfig.MR_AM_LOG_LEVEL, "ALL");
sleepConf.set(MRJobConfig.MAP_LOG_LEVEL, Level.ALL.toString()); sleepConf.set(MRJobConfig.MAP_LOG_LEVEL, "ALL");
sleepConf.set(MRJobConfig.REDUCE_LOG_LEVEL, Level.ALL.toString()); sleepConf.set(MRJobConfig.REDUCE_LOG_LEVEL, "ALL");
sleepConf.set(MRJobConfig.MAP_JAVA_OPTS, "-verbose:class"); sleepConf.set(MRJobConfig.MAP_JAVA_OPTS, "-verbose:class");
final SleepJob sleepJob = new SleepJob(); final SleepJob sleepJob = new SleepJob();
sleepJob.setConf(sleepConf); sleepJob.setConf(sleepConf);
@ -856,11 +855,11 @@ public class TestMRJobs {
final SleepJob sleepJob = new SleepJob(); final SleepJob sleepJob = new SleepJob();
final JobConf sleepConf = new JobConf(mrCluster.getConfig()); final JobConf sleepConf = new JobConf(mrCluster.getConfig());
sleepConf.set(MRJobConfig.MAP_LOG_LEVEL, Level.ALL.toString()); sleepConf.set(MRJobConfig.MAP_LOG_LEVEL, "ALL");
final long userLogKb = 4; final long userLogKb = 4;
sleepConf.setLong(MRJobConfig.TASK_USERLOG_LIMIT, userLogKb); sleepConf.setLong(MRJobConfig.TASK_USERLOG_LIMIT, userLogKb);
sleepConf.setInt(MRJobConfig.TASK_LOG_BACKUPS, 3); sleepConf.setInt(MRJobConfig.TASK_LOG_BACKUPS, 3);
sleepConf.set(MRJobConfig.MR_AM_LOG_LEVEL, Level.ALL.toString()); sleepConf.set(MRJobConfig.MR_AM_LOG_LEVEL, "ALL");
final long amLogKb = 7; final long amLogKb = 7;
sleepConf.setLong(MRJobConfig.MR_AM_LOG_KB, amLogKb); sleepConf.setLong(MRJobConfig.MR_AM_LOG_KB, amLogKb);
sleepConf.setInt(MRJobConfig.MR_AM_LOG_BACKUPS, 7); sleepConf.setInt(MRJobConfig.MR_AM_LOG_BACKUPS, 7);

View File

@ -1944,6 +1944,18 @@
<artifactId>log4j-web</artifactId> <artifactId>log4j-web</artifactId>
<version>${log4j2.version}</version> <version>${log4j2.version}</version>
</dependency> </dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-logging</artifactId>
<version>${hadoop.version}</version>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-logging</artifactId>
<version>${hadoop.version}</version>
<scope>test</scope>
<type>test-jar</type>
</dependency>
</dependencies> </dependencies>
</dependencyManagement> </dependencyManagement>

View File

@ -349,7 +349,12 @@
<artifactId>hamcrest-library</artifactId> <artifactId>hamcrest-library</artifactId>
<scope>test</scope> <scope>test</scope>
</dependency> </dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-logging</artifactId>
<scope>test</scope>
<type>test-jar</type>
</dependency>
</dependencies> </dependencies>
<profiles> <profiles>

View File

@ -32,7 +32,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.azure.NativeAzureFileSystem.FolderRenamePending; import org.apache.hadoop.fs.azure.NativeAzureFileSystem.FolderRenamePending;
import org.apache.hadoop.test.GenericTestUtils.LogCapturer; import org.apache.hadoop.logging.LogCapturer;
import org.junit.Before; import org.junit.Before;
import org.junit.Rule; import org.junit.Rule;
import org.junit.Test; import org.junit.Test;

View File

@ -23,7 +23,7 @@ import java.util.StringTokenizer;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.test.GenericTestUtils.LogCapturer; import org.apache.hadoop.logging.LogCapturer;
import org.junit.Test; import org.junit.Test;
import org.slf4j.Logger; import org.slf4j.Logger;

View File

@ -81,6 +81,12 @@
<artifactId>hadoop-hdfs-client</artifactId> <artifactId>hadoop-hdfs-client</artifactId>
<scope>provided</scope> <scope>provided</scope>
</dependency> </dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-logging</artifactId>
<scope>test</scope>
<type>test-jar</type>
</dependency>
<dependency> <dependency>
<groupId>org.apache.hadoop</groupId> <groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-hdfs</artifactId> <artifactId>hadoop-hdfs</artifactId>

View File

@ -41,6 +41,7 @@ import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.Counter; import org.apache.hadoop.mapreduce.Counter;
import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.logging.LogCapturer;
import org.apache.hadoop.tools.CopyListingFileStatus; import org.apache.hadoop.tools.CopyListingFileStatus;
import org.apache.hadoop.tools.DistCp; import org.apache.hadoop.tools.DistCp;
import org.apache.hadoop.tools.DistCpConstants; import org.apache.hadoop.tools.DistCpConstants;
@ -701,8 +702,8 @@ public abstract class AbstractContractDistCpTest
GenericTestUtils GenericTestUtils
.createFiles(remoteFS, source, getDepth(), getWidth(), getWidth()); .createFiles(remoteFS, source, getDepth(), getWidth(), getWidth());
GenericTestUtils.LogCapturer log = LogCapturer log =
GenericTestUtils.LogCapturer.captureLogs(SimpleCopyListing.LOG); LogCapturer.captureLogs(SimpleCopyListing.LOG);
String options = "-useiterator -update -delete" + getDefaultCLIOptions(); String options = "-useiterator -update -delete" + getDefaultCLIOptions();
DistCpTestUtils.assertRunDistCp(DistCpConstants.SUCCESS, source.toString(), DistCpTestUtils.assertRunDistCp(DistCpConstants.SUCCESS, source.toString(),

View File

@ -27,11 +27,10 @@ import java.util.regex.Pattern;
import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.logging.HadoopLoggerUtils;
import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.tools.rumen.datatypes.*; import org.apache.hadoop.tools.rumen.datatypes.*;
import org.apache.log4j.Level;
import org.apache.log4j.Logger;
/** /**
* A default parser for MapReduce job configuration properties. * A default parser for MapReduce job configuration properties.
@ -83,7 +82,7 @@ public class MapReduceJobPropertiesParser implements JobPropertyParser {
// turn off the warning w.r.t deprecated mapreduce keys // turn off the warning w.r.t deprecated mapreduce keys
static { static {
Logger.getLogger(Configuration.class).setLevel(Level.OFF); HadoopLoggerUtils.setLogLevel(Configuration.class.getName(), "OFF");
} }
// Accepts a key if there is a corresponding key in the current mapreduce // Accepts a key if there is a corresponding key in the current mapreduce

View File

@ -576,16 +576,6 @@
<Bug pattern="SQL_PREPARED_STATEMENT_GENERATED_FROM_NONCONSTANT_STRING" /> <Bug pattern="SQL_PREPARED_STATEMENT_GENERATED_FROM_NONCONSTANT_STRING" />
</Match> </Match>
<!-- Following fields are used in ErrorsAndWarningsBlock, which is not a part of analysis of findbugs -->
<Match>
<Class name="org.apache.hadoop.yarn.util.Log4jWarningErrorMetricsAppender$Element" />
<Or>
<Field name="count" />
<Field name="timestampSeconds" />
</Or>
<Bug pattern="URF_UNREAD_PUBLIC_OR_PROTECTED_FIELD" />
</Match>
<Match> <Match>
<Class name="org.apache.hadoop.yarn.api.records.ResourceRequest" /> <Class name="org.apache.hadoop.yarn.api.records.ResourceRequest" />
<Method name="equals" /> <Method name="equals" />

Some files were not shown because too many files have changed in this diff Show More