HDFS-8880. NameNode metrics logging. (Arpit Agarwal)

This commit is contained in:
Arpit Agarwal 2015-08-17 16:55:46 -07:00
parent ec183faadc
commit a88f31ebf3
8 changed files with 445 additions and 5 deletions

View File

@ -148,6 +148,19 @@ log4j.appender.RFAAUDIT.layout.ConversionPattern=%d{ISO8601} %p %c{2}: %m%n
log4j.appender.RFAAUDIT.MaxFileSize=${hdfs.audit.log.maxfilesize} log4j.appender.RFAAUDIT.MaxFileSize=${hdfs.audit.log.maxfilesize}
log4j.appender.RFAAUDIT.MaxBackupIndex=${hdfs.audit.log.maxbackupindex} log4j.appender.RFAAUDIT.MaxBackupIndex=${hdfs.audit.log.maxbackupindex}
#
# NameNode metrics logging.
# The default is to retain two namenode-metrics.log files up to 64MB each.
#
log4j.logger.NameNodeMetricsLog=INFO,NNMETRICSRFA
log4j.additivity.NameNodeMetricsLog=false
log4j.appender.NNMETRICSRFA=org.apache.log4j.RollingFileAppender
log4j.appender.NNMETRICSRFA.File=${hadoop.log.dir}/namenode-metrics.log
log4j.appender.NNMETRICSRFA.layout=org.apache.log4j.PatternLayout
log4j.appender.NNMETRICSRFA.layout.ConversionPattern=%d{ISO8601} %m%n
log4j.appender.NNMETRICSRFA.MaxBackupIndex=1
log4j.appender.NNMETRICSRFA.MaxFileSize=64MB
# #
# mapred audit logging # mapred audit logging
# #

View File

@ -18,6 +18,8 @@
package org.apache.hadoop.metrics2.util; package org.apache.hadoop.metrics2.util;
import java.lang.management.ManagementFactory; import java.lang.management.ManagementFactory;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import javax.management.InstanceAlreadyExistsException; import javax.management.InstanceAlreadyExistsException;
import javax.management.MBeanServer; import javax.management.MBeanServer;
@ -38,6 +40,13 @@ import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
@InterfaceStability.Stable @InterfaceStability.Stable
public class MBeans { public class MBeans {
private static final Log LOG = LogFactory.getLog(MBeans.class); private static final Log LOG = LogFactory.getLog(MBeans.class);
private static final String DOMAIN_PREFIX = "Hadoop:";
private static final String SERVICE_PREFIX = "service=";
private static final String NAME_PREFIX = "name=";
private static final Pattern MBEAN_NAME_PATTERN = Pattern.compile(
"^" + DOMAIN_PREFIX + SERVICE_PREFIX + "([^,]+)," +
NAME_PREFIX + "(.+)$");
/** /**
* Register the MBean using our standard MBeanName format * Register the MBean using our standard MBeanName format
@ -72,6 +81,26 @@ public class MBeans {
return null; return null;
} }
public static String getMbeanNameService(final ObjectName objectName) {
Matcher matcher = MBEAN_NAME_PATTERN.matcher(objectName.toString());
if (matcher.matches()) {
return matcher.group(1);
} else {
throw new IllegalArgumentException(
objectName + " is not a valid Hadoop mbean");
}
}
public static String getMbeanNameName(final ObjectName objectName) {
Matcher matcher = MBEAN_NAME_PATTERN.matcher(objectName.toString());
if (matcher.matches()) {
return matcher.group(2);
} else {
throw new IllegalArgumentException(
objectName + " is not a valid Hadoop mbean");
}
}
static public void unregister(ObjectName mbeanName) { static public void unregister(ObjectName mbeanName) {
LOG.debug("Unregistering "+ mbeanName); LOG.debug("Unregistering "+ mbeanName);
final MBeanServer mbs = ManagementFactory.getPlatformMBeanServer(); final MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
@ -88,13 +117,13 @@ public class MBeans {
} }
static private ObjectName getMBeanName(String serviceName, String nameName) { static private ObjectName getMBeanName(String serviceName, String nameName) {
ObjectName name = null; String nameStr = DOMAIN_PREFIX + SERVICE_PREFIX + serviceName + "," +
String nameStr = "Hadoop:service="+ serviceName +",name="+ nameName; NAME_PREFIX + nameName;
try { try {
name = DefaultMetricsSystem.newMBeanName(nameStr); return DefaultMetricsSystem.newMBeanName(nameStr);
} catch (Exception e) { } catch (Exception e) {
LOG.warn("Error creating MBean object name: "+ nameStr, e); LOG.warn("Error creating MBean object name: "+ nameStr, e);
return null;
} }
return name;
} }
} }

View File

@ -799,6 +799,8 @@ Release 2.8.0 - UNRELEASED
HDFS-8801. Convert BlockInfoUnderConstruction as a feature. HDFS-8801. Convert BlockInfoUnderConstruction as a feature.
(Jing Zhao via wheat9) (Jing Zhao via wheat9)
HDFS-8880. NameNode metrics logging. (Arpit Agarwal)
OPTIMIZATIONS OPTIMIZATIONS
HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

View File

@ -344,6 +344,10 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final String DFS_NAMENODE_AUDIT_LOG_ASYNC_KEY = "dfs.namenode.audit.log.async"; public static final String DFS_NAMENODE_AUDIT_LOG_ASYNC_KEY = "dfs.namenode.audit.log.async";
public static final boolean DFS_NAMENODE_AUDIT_LOG_ASYNC_DEFAULT = false; public static final boolean DFS_NAMENODE_AUDIT_LOG_ASYNC_DEFAULT = false;
public static final String DFS_NAMENODE_AUDIT_LOG_DEBUG_CMDLIST = "dfs.namenode.audit.log.debug.cmdlist"; public static final String DFS_NAMENODE_AUDIT_LOG_DEBUG_CMDLIST = "dfs.namenode.audit.log.debug.cmdlist";
public static final String DFS_NAMENODE_METRICS_LOGGER_PERIOD_SECONDS_KEY =
"dfs.namenode.metrics.logger.period.seconds";
public static final int DFS_NAMENODE_METRICS_LOGGER_PERIOD_SECONDS_DEFAULT =
600;
public static final String DFS_BALANCER_MOVEDWINWIDTH_KEY = "dfs.balancer.movedWinWidth"; public static final String DFS_BALANCER_MOVEDWINWIDTH_KEY = "dfs.balancer.movedWinWidth";
public static final long DFS_BALANCER_MOVEDWINWIDTH_DEFAULT = 5400*1000L; public static final long DFS_BALANCER_MOVEDWINWIDTH_DEFAULT = 5400*1000L;

View File

@ -21,6 +21,9 @@ import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Joiner; import com.google.common.base.Joiner;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.HadoopIllegalArgumentException; import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
@ -74,20 +77,34 @@ import org.apache.hadoop.util.GenericOptionsParser;
import org.apache.hadoop.util.JvmPauseMonitor; import org.apache.hadoop.util.JvmPauseMonitor;
import org.apache.hadoop.util.ServicePlugin; import org.apache.hadoop.util.ServicePlugin;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.apache.log4j.Appender;
import org.apache.log4j.AsyncAppender;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import javax.management.Attribute;
import javax.management.AttributeList;
import javax.management.MBeanAttributeInfo;
import javax.management.MBeanInfo;
import javax.management.MBeanServer;
import javax.management.MalformedObjectNameException;
import javax.management.ObjectName; import javax.management.ObjectName;
import java.io.IOException; import java.io.IOException;
import java.io.PrintStream; import java.io.PrintStream;
import java.lang.management.ManagementFactory;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
import java.net.URI; import java.net.URI;
import java.security.PrivilegedExceptionAction; import java.security.PrivilegedExceptionAction;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.List; import java.util.List;
import java.util.Set;
import java.util.concurrent.ScheduledThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY;
@ -113,6 +130,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HTTP_BIND_HOST_K
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_KERBEROS_INTERNAL_SPNEGO_PRINCIPAL_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_KERBEROS_INTERNAL_SPNEGO_PRINCIPAL_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_METRICS_LOGGER_PERIOD_SECONDS_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_METRICS_LOGGER_PERIOD_SECONDS_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_PLUGINS_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_PLUGINS_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
@ -304,6 +323,9 @@ public class NameNode implements NameNodeStatusMXBean {
public static final HAState ACTIVE_STATE = new ActiveState(); public static final HAState ACTIVE_STATE = new ActiveState();
public static final HAState STANDBY_STATE = new StandbyState(); public static final HAState STANDBY_STATE = new StandbyState();
public static final Log MetricsLog =
LogFactory.getLog("NameNodeMetricsLog");
protected FSNamesystem namesystem; protected FSNamesystem namesystem;
protected final Configuration conf; protected final Configuration conf;
protected final NamenodeRole role; protected final NamenodeRole role;
@ -329,6 +351,8 @@ public class NameNode implements NameNodeStatusMXBean {
private JvmPauseMonitor pauseMonitor; private JvmPauseMonitor pauseMonitor;
private ObjectName nameNodeStatusBeanName; private ObjectName nameNodeStatusBeanName;
SpanReceiverHost spanReceiverHost; SpanReceiverHost spanReceiverHost;
ScheduledThreadPoolExecutor metricsLoggerTimer;
/** /**
* The namenode address that clients will use to access this namenode * The namenode address that clients will use to access this namenode
* or the name service. For HA configurations using logical URI, it * or the name service. For HA configurations using logical URI, it
@ -662,6 +686,68 @@ public class NameNode implements NameNodeStatusMXBean {
metrics.getJvmMetrics().setPauseMonitor(pauseMonitor); metrics.getJvmMetrics().setPauseMonitor(pauseMonitor);
startCommonServices(conf); startCommonServices(conf);
startMetricsLogger(conf);
}
/**
* Start a timer to periodically write NameNode metrics to the log
* file. This behavior can be disabled by configuration.
* @param conf
*/
protected void startMetricsLogger(Configuration conf) {
long metricsLoggerPeriodSec =
conf.getInt(DFS_NAMENODE_METRICS_LOGGER_PERIOD_SECONDS_KEY,
DFS_NAMENODE_METRICS_LOGGER_PERIOD_SECONDS_DEFAULT);
if (metricsLoggerPeriodSec <= 0) {
return;
}
makeMetricsLoggerAsync();
// Schedule the periodic logging.
metricsLoggerTimer = new ScheduledThreadPoolExecutor(1);
metricsLoggerTimer.setExecuteExistingDelayedTasksAfterShutdownPolicy(
false);
metricsLoggerTimer.scheduleWithFixedDelay(new MetricsLoggerTask(),
metricsLoggerPeriodSec,
metricsLoggerPeriodSec,
TimeUnit.SECONDS);
}
/**
* Make the metrics logger async and add all pre-existing appenders
* to the async appender.
*/
private static void makeMetricsLoggerAsync() {
if (!(MetricsLog instanceof Log4JLogger)) {
LOG.warn(
"Metrics logging will not be async since the logger is not log4j");
return;
}
org.apache.log4j.Logger logger = ((Log4JLogger) MetricsLog).getLogger();
logger.setAdditivity(false); // Don't pollute NN logs with metrics dump
@SuppressWarnings("unchecked")
List<Appender> appenders = Collections.list(logger.getAllAppenders());
// failsafe against trying to async it more than once
if (!appenders.isEmpty() && !(appenders.get(0) instanceof AsyncAppender)) {
AsyncAppender asyncAppender = new AsyncAppender();
// change logger to have an async appender containing all the
// previously configured appenders
for (Appender appender : appenders) {
logger.removeAppender(appender);
asyncAppender.addAppender(appender);
}
logger.addAppender(asyncAppender);
}
}
protected void stopMetricsLogger() {
if (metricsLoggerTimer != null) {
metricsLoggerTimer.shutdown();
metricsLoggerTimer = null;
}
} }
/** /**
@ -865,6 +951,7 @@ public class NameNode implements NameNodeStatusMXBean {
} catch (ServiceFailedException e) { } catch (ServiceFailedException e) {
LOG.warn("Encountered exception while exiting state ", e); LOG.warn("Encountered exception while exiting state ", e);
} finally { } finally {
stopMetricsLogger();
stopCommonServices(); stopCommonServices();
if (metrics != null) { if (metrics != null) {
metrics.shutdown(); metrics.shutdown();
@ -1830,4 +1917,91 @@ public class NameNode implements NameNodeStatusMXBean {
break; break;
} }
} }
private static class MetricsLoggerTask implements Runnable {
private static final int MAX_LOGGED_VALUE_LEN = 128;
private static ObjectName OBJECT_NAME = null;
static {
try {
OBJECT_NAME = new ObjectName("Hadoop:*");
} catch (MalformedObjectNameException m) {
// This should not occur in practice since we pass
// a valid pattern to the constructor above.
}
}
/**
* Write NameNode metrics to the metrics appender when invoked.
*/
@Override
public void run() {
// Skip querying metrics if there are no known appenders.
if (!MetricsLog.isInfoEnabled() ||
!hasAppenders(MetricsLog) ||
OBJECT_NAME == null) {
return;
}
MetricsLog.info(" >> Begin NameNode metrics dump");
final MBeanServer server = ManagementFactory.getPlatformMBeanServer();
// Iterate over each MBean.
for (final ObjectName mbeanName : server.queryNames(OBJECT_NAME, null)) {
try {
MBeanInfo mBeanInfo = server.getMBeanInfo(mbeanName);
final String mBeanNameName = MBeans.getMbeanNameName(mbeanName);
final Set<String> attributeNames = getFilteredAttributes(mBeanInfo);
final AttributeList attributes =
server.getAttributes(mbeanName,
attributeNames.toArray(new String[attributeNames.size()]));
for (Object o : attributes) {
final Attribute attribute = (Attribute) o;
final Object value = attribute.getValue();
final String valueStr =
(value != null) ? value.toString() : "null";
// Truncate the value if it is too long
MetricsLog.info(mBeanNameName + ":" + attribute.getName() + "=" +
(valueStr.length() < MAX_LOGGED_VALUE_LEN ? valueStr :
valueStr.substring(0, MAX_LOGGED_VALUE_LEN) + "..."));
}
} catch (Exception e) {
MetricsLog.error("Failed to get NameNode metrics for mbean " +
mbeanName.toString(), e);
}
}
MetricsLog.info(" << End NameNode metrics dump");
}
private static boolean hasAppenders(Log logger) {
if (!(logger instanceof Log4JLogger)) {
// Don't bother trying to determine the presence of appenders.
return true;
}
Log4JLogger log4JLogger = ((Log4JLogger) MetricsLog);
return log4JLogger.getLogger().getAllAppenders().hasMoreElements();
}
/**
* Get the list of attributes for the MBean, filtering out a few
* attribute types.
*/
private static Set<String> getFilteredAttributes(
MBeanInfo mBeanInfo) {
Set<String> attributeNames = new HashSet<>();
for (MBeanAttributeInfo attributeInfo : mBeanInfo.getAttributes()) {
if (!attributeInfo.getType().equals(
"javax.management.openmbean.TabularData") &&
!attributeInfo.getType().equals(
"javax.management.openmbean.CompositeData") &&
!attributeInfo.getType().equals(
"[Ljavax.management.openmbean.CompositeData;")) {
attributeNames.add(attributeInfo.getName());
}
}
return attributeNames;
}
}
} }

View File

@ -1575,6 +1575,18 @@
</description> </description>
</property> </property>
<property>
<name>dfs.namenode.metrics.logger.period.seconds</name>
<value>600</value>
<description>
This setting controls how frequently the NameNode logs its metrics. The
logging configuration must also define one or more appenders for
NameNodeMetricsLog for the metrics to be logged.
NameNode metrics logging is disabled if this value is set to zero or
less than zero.
</description>
</property>
<property> <property>
<name>dfs.metrics.percentiles.intervals</name> <name>dfs.metrics.percentiles.intervals</name>
<value></value> <value></value>

View File

@ -0,0 +1,193 @@
/**
* 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
* <p/>
* http://www.apache.org/licenses/LICENSE-2.0
* <p/>
* 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 com.google.common.base.Supplier;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.metrics2.util.MBeans;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.log4j.Appender;
import org.apache.log4j.AppenderSkeleton;
import org.apache.log4j.AsyncAppender;
import org.apache.log4j.spi.LoggingEvent;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.Timeout;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
import java.util.Random;
import java.util.concurrent.TimeoutException;
import java.util.regex.Pattern;
import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
import static org.junit.Assert.*;
import static org.mockito.Mockito.mock;
/**
* Test periodic logging of NameNode metrics.
*/
public class TestNameNodeMetricsLogger {
static final Log LOG = LogFactory.getLog(TestNameNodeMetricsLogger.class);
static final Random random = new Random(System.currentTimeMillis());
@Rule
public Timeout timeout = new Timeout(300000);
@Test
public void testMetricsLoggerOnByDefault() throws IOException {
NameNode nn = makeNameNode(true);
assertNotNull(nn.metricsLoggerTimer);
}
@Test
public void testDisableMetricsLogger() throws IOException {
NameNode nn = makeNameNode(false);
assertNull(nn.metricsLoggerTimer);
}
@Test
public void testMetricsLoggerIsAsync() throws IOException {
makeNameNode(true);
org.apache.log4j.Logger logger =
((Log4JLogger) NameNode.MetricsLog).getLogger();
@SuppressWarnings("unchecked")
List<Appender> appenders = Collections.list(logger.getAllAppenders());
assertTrue(appenders.get(0) instanceof AsyncAppender);
}
/**
* Publish a fake metric under the "Hadoop:" domain and ensure it is
* logged by the metrics logger.
*/
@Test
public void testMetricsLogOutput()
throws IOException, InterruptedException, TimeoutException {
TestFakeMetric metricsProvider = new TestFakeMetric();
MBeans.register(this.getClass().getSimpleName(),
"DummyMetrics", metricsProvider);
makeNameNode(true); // Log metrics early and often.
final PatternMatchingAppender appender =
new PatternMatchingAppender("^.*FakeMetric42.*$");
addAppender(NameNode.MetricsLog, appender);
// Ensure that the supplied pattern was matched.
GenericTestUtils.waitFor(new Supplier<Boolean>() {
@Override
public Boolean get() {
return appender.isMatched();
}
}, 1000, 60000);
}
/**
* Create a NameNode object that listens on a randomly chosen port
* number.
*
* @param enableMetricsLogging true if periodic metrics logging is to be
* enabled, false otherwise.
*/
private NameNode makeNameNode(boolean enableMetricsLogging)
throws IOException {
Configuration conf = new HdfsConfiguration();
conf.set(FS_DEFAULT_NAME_KEY, "hdfs://localhost:" + getRandomPort());
conf.set(DFS_NAMENODE_HTTP_ADDRESS_KEY, "0.0.0.0:" + getRandomPort());
conf.setInt(DFS_NAMENODE_METRICS_LOGGER_PERIOD_SECONDS_KEY,
enableMetricsLogging ? 1 : 0); // If enabled, log early and log often
return new TestNameNode(conf);
}
private int getRandomPort() {
return 10000 + random.nextInt(50000);
}
private void addAppender(Log log, Appender appender) {
org.apache.log4j.Logger logger = ((Log4JLogger) log).getLogger();
@SuppressWarnings("unchecked")
List<Appender> appenders = Collections.list(logger.getAllAppenders());
((AsyncAppender) appenders.get(0)).addAppender(appender);
}
/**
* A NameNode that stubs out the NameSystem for testing.
*/
private static class TestNameNode extends NameNode {
@Override
protected void loadNamesystem(Configuration conf) throws IOException {
this.namesystem = mock(FSNamesystem.class);
}
public TestNameNode(Configuration conf) throws IOException {
super(conf);
}
}
public interface TestFakeMetricMXBean {
int getFakeMetric42();
}
/**
* MBean for testing
*/
public static class TestFakeMetric implements TestFakeMetricMXBean {
@Override
public int getFakeMetric42() {
return 0;
}
}
/**
* An appender that matches logged messages against the given
* regular expression.
*/
public static class PatternMatchingAppender extends AppenderSkeleton {
private final Pattern pattern;
private volatile boolean matched;
public PatternMatchingAppender(String pattern) {
this.pattern = Pattern.compile(pattern);
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

@ -21,3 +21,16 @@ log4j.threshold=ALL
log4j.appender.stdout=org.apache.log4j.ConsoleAppender 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
#
# NameNode metrics logging.
# The default is to retain two namenode-metrics.log files up to 64MB each.
#
log4j.logger.NameNodeMetricsLog=INFO,NNMETRICSRFA
log4j.additivity.NameNodeMetricsLog=false
log4j.appender.NNMETRICSRFA=org.apache.log4j.RollingFileAppender
log4j.appender.NNMETRICSRFA.File=${hadoop.log.dir}/namenode-metrics.log
log4j.appender.NNMETRICSRFA.layout=org.apache.log4j.PatternLayout
log4j.appender.NNMETRICSRFA.layout.ConversionPattern=%d{ISO8601} %m%n
log4j.appender.NNMETRICSRFA.MaxBackupIndex=1
log4j.appender.NNMETRICSRFA.MaxFileSize=64MB