YARN-2901. Add errors and warning metrics page to RM, NM web UI. (Varun Vasudev via wangda)

This commit is contained in:
Wangda Tan 2015-04-02 17:23:20 -07:00
parent b21c72777a
commit bad070fe15
13 changed files with 1114 additions and 8 deletions

View File

@ -266,3 +266,12 @@ log4j.appender.RMSUMMARY.layout.ConversionPattern=%d{ISO8601} %p %c{2}: %m%n
#log4j.appender.nodemanagerrequestlog=org.apache.hadoop.http.HttpRequestLogAppender
#log4j.appender.nodemanagerrequestlog.Filename=${hadoop.log.dir}/jetty-nodemanager-yyyy_mm_dd.log
#log4j.appender.nodemanagerrequestlog.RetainDays=3
# Appender for viewing information for errors and warnings
yarn.ewma.cleanupInterval=300
yarn.ewma.messageAgeLimitSeconds=86400
yarn.ewma.maxUniqueMessages=250
log4j.appender.EWMA=org.apache.hadoop.yarn.util.Log4jWarningErrorMetricsAppender
log4j.appender.EWMA.cleanupInterval=${yarn.ewma.cleanupInterval}
log4j.appender.EWMA.messageAgeLimitSeconds=${yarn.ewma.messageAgeLimitSeconds}
log4j.appender.EWMA.maxUniqueMessages=${yarn.ewma.maxUniqueMessages}

View File

@ -101,6 +101,9 @@ Release 2.8.0 - UNRELEASED
YARN-3248. Display count of nodes blacklisted by apps in the web UI.
(Varun Vasudev via xgong)
YARN-2901. Add errors and warning metrics page to RM, NM web UI.
(Varun Vasudev via wangda)
OPTIMIZATIONS
YARN-3339. TestDockerContainerExecutor should pull a single image and not

View File

@ -0,0 +1,447 @@
/**
* 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.yarn.util;
import org.apache.hadoop.classification.InterfaceAudience;
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.Level;
import org.apache.log4j.Logger;
import org.apache.log4j.spi.LoggingEvent;
import java.util.*;
@InterfaceAudience.Private
@InterfaceStability.Unstable
public class Log4jWarningErrorMetricsAppender extends AppenderSkeleton {
public static final String LOG_METRICS_APPENDER = "RM_LOG_METRICS_APPENDER";
static final int MAX_MESSAGE_SIZE = 2048;
static public class Element {
public Long count;
public Long timestampSeconds;
Element(Long count, Long timestampSeconds) {
this.count = count;
this.timestampSeconds = timestampSeconds;
}
}
static class PurgeElement implements Comparable<PurgeElement> {
String message;
Long timestamp;
PurgeElement(String message, Long timestamp) {
this.message = message;
this.timestamp = timestamp;
}
public int compareTo(PurgeElement e) {
if (e == null) {
throw new NullPointerException("Null element passed to compareTo");
}
int ret = this.timestamp.compareTo(e.timestamp);
if (ret != 0) {
return ret;
}
return this.message.compareTo(e.message);
}
@Override
public boolean equals(Object e) {
if (e == null || !(e instanceof PurgeElement)) {
return false;
}
if (e == this) {
return true;
}
PurgeElement el = (PurgeElement) e;
return (this.message.equals(el.message))
&& (this.timestamp.equals(el.timestamp));
}
@Override
public int hashCode() {
return this.timestamp.hashCode();
}
}
Map<String, SortedMap<Long, Integer>> errors;
Map<String, SortedMap<Long, Integer>> warnings;
SortedMap<Long, Integer> errorsTimestampCount;
SortedMap<Long, Integer> warningsTimestampCount;
SortedSet<PurgeElement> errorsPurgeInformation;
SortedSet<PurgeElement> warningsPurgeInformation;
Timer cleanupTimer;
long cleanupInterval;
long messageAgeLimitSeconds;
int maxUniqueMessages;
final Object lock = new Object();
/**
* Create an appender to keep track of the errors and warnings logged by the
* system. It will keep purge messages older than 2 days. It will store upto
* the last 500 unique errors and the last 500 unique warnings. The thread to
* purge message will run every 5 minutes, unless the 500 message limit is hit
* earlier.
*/
public Log4jWarningErrorMetricsAppender() {
this(5 * 60, 24 * 60 * 60, 250);
}
/**
* Create an appender to keep track of the errors and warnings logged by the
* system.
*
* @param cleanupIntervalSeconds
* the interval at which old messages are purged to prevent the
* message stores from growing unbounded
* @param messageAgeLimitSeconds
* the maximum age of a message in seconds before it is purged from
* the store
* @param maxUniqueMessages
* the maximum number of unique messages of each type we keep before
* we start purging
*/
public Log4jWarningErrorMetricsAppender(int cleanupIntervalSeconds,
long messageAgeLimitSeconds, int maxUniqueMessages) {
super();
errors = new HashMap<>();
warnings = new HashMap<>();
errorsTimestampCount = new TreeMap<>();
warningsTimestampCount = new TreeMap<>();
errorsPurgeInformation = new TreeSet<>();
warningsPurgeInformation = new TreeSet<>();
cleanupTimer = new Timer();
cleanupInterval = cleanupIntervalSeconds * 1000;
cleanupTimer.schedule(new ErrorAndWarningsCleanup(), cleanupInterval);
this.messageAgeLimitSeconds = messageAgeLimitSeconds;
this.maxUniqueMessages = maxUniqueMessages;
this.setName(LOG_METRICS_APPENDER);
this.setThreshold(Level.WARN);
}
/**
* {@inheritDoc}
*/
@Override
protected void append(LoggingEvent event) {
String message = event.getRenderedMessage();
String[] throwableStr = event.getThrowableStrRep();
if (throwableStr != null) {
message = message + "\n" + StringUtils.join("\n", throwableStr);
message =
org.apache.commons.lang.StringUtils.left(message, MAX_MESSAGE_SIZE);
}
int level = event.getLevel().toInt();
if (level == Level.WARN_INT || level == Level.ERROR_INT) {
// store second level information
Long eventTimeSeconds = event.getTimeStamp() / 1000;
Map<String, SortedMap<Long, Integer>> map;
SortedMap<Long, Integer> timestampsCount;
SortedSet<PurgeElement> purgeInformation;
if (level == Level.WARN_INT) {
map = warnings;
timestampsCount = warningsTimestampCount;
purgeInformation = warningsPurgeInformation;
} else {
map = errors;
timestampsCount = errorsTimestampCount;
purgeInformation = errorsPurgeInformation;
}
updateMessageDetails(message, eventTimeSeconds, map, timestampsCount,
purgeInformation);
}
}
private void updateMessageDetails(String message, Long eventTimeSeconds,
Map<String, SortedMap<Long, Integer>> map,
SortedMap<Long, Integer> timestampsCount,
SortedSet<PurgeElement> purgeInformation) {
synchronized (lock) {
if (map.containsKey(message)) {
SortedMap<Long, Integer> tmp = map.get(message);
Long lastMessageTime = tmp.lastKey();
int value = 1;
if (tmp.containsKey(eventTimeSeconds)) {
value = tmp.get(eventTimeSeconds) + 1;
}
tmp.put(eventTimeSeconds, value);
purgeInformation.remove(new PurgeElement(message, lastMessageTime));
} else {
SortedMap<Long, Integer> value = new TreeMap<>();
value.put(eventTimeSeconds, 1);
map.put(message, value);
if (map.size() > maxUniqueMessages * 2) {
cleanupTimer.cancel();
cleanupTimer = new Timer();
cleanupTimer.schedule(new ErrorAndWarningsCleanup(), 0);
}
}
purgeInformation.add(new PurgeElement(message, eventTimeSeconds));
int newValue = 1;
if (timestampsCount.containsKey(eventTimeSeconds)) {
newValue = timestampsCount.get(eventTimeSeconds) + 1;
}
timestampsCount.put(eventTimeSeconds, newValue);
}
}
/**
* {@inheritDoc}
*/
@Override
public void close() {
cleanupTimer.cancel();
}
/**
* {@inheritDoc}
*/
@Override
public boolean requiresLayout() {
return false;
}
/**
* Get the counts of errors in the time periods provided. Note that the counts
* provided by this function may differ from the ones provided by
* getErrorMessagesAndCounts since the message store is purged at regular
* intervals to prevent it from growing without bounds, while the store for
* the counts is purged less frequently.
*
* @param cutoffs
* list of timestamp cutoffs(in seconds) for which the counts are
* desired
* @return list of error counts in the time periods corresponding to cutoffs
*/
public List<Integer> getErrorCounts(List<Long> cutoffs) {
return this.getCounts(errorsTimestampCount, cutoffs);
}
/**
* Get the counts of warnings in the time periods provided. Note that the
* counts provided by this function may differ from the ones provided by
* getWarningMessagesAndCounts since the message store is purged at regular
* intervals to prevent it from growing without bounds, while the store for
* the counts is purged less frequently.
*
* @param cutoffs
* list of timestamp cutoffs(in seconds) for which the counts are
* desired
* @return list of warning counts in the time periods corresponding to cutoffs
*/
public List<Integer> getWarningCounts(List<Long> cutoffs) {
return this.getCounts(warningsTimestampCount, cutoffs);
}
private List<Integer> getCounts(SortedMap<Long, Integer> map,
List<Long> cutoffs) {
List<Integer> ret = new ArrayList<>();
Long largestCutoff = Collections.min(cutoffs);
for (int i = 0; i < cutoffs.size(); ++i) {
ret.add(0);
}
synchronized (lock) {
Map<Long, Integer> submap = map.tailMap(largestCutoff);
for (Map.Entry<Long, Integer> entry : submap.entrySet()) {
for (int i = 0; i < cutoffs.size(); ++i) {
if (entry.getKey() >= cutoffs.get(i)) {
int tmp = ret.get(i);
ret.set(i, tmp + entry.getValue());
}
}
}
}
return ret;
}
/**
* Get the errors and the number of occurrences for each of the errors for the
* time cutoffs provided. Note that the counts provided by this function may
* differ from the ones provided by getErrorCounts since the message store is
* purged at regular intervals to prevent it from growing without bounds,
* while the store for the counts is purged less frequently.
*
* @param cutoffs
* list of timestamp cutoffs(in seconds) for which the counts are
* desired
* @return list of maps corresponding for each cutoff provided; each map
* contains the error and the number of times the error occurred in
* the time period
*/
public List<Map<String, Element>>
getErrorMessagesAndCounts(List<Long> cutoffs) {
return this.getElementsAndCounts(errors, cutoffs, errorsPurgeInformation);
}
/**
* Get the warning and the number of occurrences for each of the warnings for
* the time cutoffs provided. Note that the counts provided by this function
* may differ from the ones provided by getWarningCounts since the message
* store is purged at regular intervals to prevent it from growing without
* bounds, while the store for the counts is purged less frequently.
*
* @param cutoffs
* list of timestamp cutoffs(in seconds) for which the counts are
* desired
* @return list of maps corresponding for each cutoff provided; each map
* contains the warning and the number of times the error occurred in
* the time period
*/
public List<Map<String, Element>> getWarningMessagesAndCounts(
List<Long> cutoffs) {
return this.getElementsAndCounts(warnings, cutoffs, warningsPurgeInformation);
}
private List<Map<String, Element>> getElementsAndCounts(
Map<String, SortedMap<Long, Integer>> map, List<Long> cutoffs,
SortedSet<PurgeElement> purgeInformation) {
if (purgeInformation.size() > maxUniqueMessages) {
ErrorAndWarningsCleanup cleanup = new ErrorAndWarningsCleanup();
long cutoff = Time.now() - (messageAgeLimitSeconds * 1000);
cutoff = (cutoff / 1000);
cleanup.cleanupMessages(map, purgeInformation, cutoff, maxUniqueMessages);
}
List<Map<String, Element>> ret = new ArrayList<>(cutoffs.size());
for (int i = 0; i < cutoffs.size(); ++i) {
ret.add(new HashMap<String, Element>());
}
synchronized (lock) {
for (Map.Entry<String, SortedMap<Long, Integer>> element : map.entrySet()) {
for (int i = 0; i < cutoffs.size(); ++i) {
Map<String, Element> retMap = ret.get(i);
SortedMap<Long, Integer> qualifyingTimes =
element.getValue().tailMap(cutoffs.get(i));
long count = 0;
for (Map.Entry<Long, Integer> entry : qualifyingTimes.entrySet()) {
count += entry.getValue();
}
if (!qualifyingTimes.isEmpty()) {
retMap.put(element.getKey(),
new Element(count, qualifyingTimes.lastKey()));
}
}
}
}
return ret;
}
// getters and setters for log4j
public long getCleanupInterval() {
return cleanupInterval;
}
public void setCleanupInterval(long cleanupInterval) {
this.cleanupInterval = cleanupInterval;
}
public long getMessageAgeLimitSeconds() {
return messageAgeLimitSeconds;
}
public void setMessageAgeLimitSeconds(long messageAgeLimitSeconds) {
this.messageAgeLimitSeconds = messageAgeLimitSeconds;
}
public int getMaxUniqueMessages() {
return maxUniqueMessages;
}
public void setMaxUniqueMessages(int maxUniqueMessages) {
this.maxUniqueMessages = maxUniqueMessages;
}
class ErrorAndWarningsCleanup extends TimerTask {
@Override
public void run() {
long cutoff = Time.now() - (messageAgeLimitSeconds * 1000);
cutoff = (cutoff / 1000);
cleanupMessages(errors, errorsPurgeInformation, cutoff, maxUniqueMessages);
cleanupMessages(warnings, warningsPurgeInformation, cutoff,
maxUniqueMessages);
cleanupCounts(errorsTimestampCount, cutoff);
cleanupCounts(warningsTimestampCount, cutoff);
try {
cleanupTimer.schedule(new ErrorAndWarningsCleanup(), cleanupInterval);
} catch (IllegalStateException ie) {
// don't do anything since new timer is already scheduled
}
}
void cleanupMessages(Map<String, SortedMap<Long, Integer>> map,
SortedSet<PurgeElement> purgeInformation, long cutoff,
int mapTargetSize) {
PurgeElement el = new PurgeElement("", cutoff);
synchronized (lock) {
SortedSet<PurgeElement> removeSet = purgeInformation.headSet(el);
Iterator<PurgeElement> it = removeSet.iterator();
while (it.hasNext()) {
PurgeElement p = it.next();
map.remove(p.message);
it.remove();
}
// don't keep more mapTargetSize keys
if (purgeInformation.size() > mapTargetSize) {
Object[] array = purgeInformation.toArray();
int cutoffIndex = purgeInformation.size() - mapTargetSize;
for (int i = 0; i < cutoffIndex; ++i) {
PurgeElement p = (PurgeElement) array[i];
map.remove(p.message);
purgeInformation.remove(p);
}
}
}
}
void cleanupCounts(SortedMap<Long, Integer> map, long cutoff) {
synchronized (lock) {
Iterator<Map.Entry<Long, Integer>> it = map.entrySet().iterator();
while (it.hasNext()) {
Map.Entry<Long, Integer> element = it.next();
if (element.getKey() < cutoff) {
it.remove();
}
}
}
}
}
// helper function
public static Log4jWarningErrorMetricsAppender findAppender() {
Enumeration appenders = Logger.getRootLogger().getAllAppenders();
while(appenders.hasMoreElements()) {
Object obj = appenders.nextElement();
if(obj instanceof Log4jWarningErrorMetricsAppender) {
return (Log4jWarningErrorMetricsAppender) obj;
}
}
return null;
}
}

View File

@ -0,0 +1,260 @@
/**
* 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.yarn.util;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.util.Time;
import org.apache.log4j.Level;
import org.apache.log4j.Logger;
import org.junit.Assert;
import org.junit.Test;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
public class TestLog4jWarningErrorMetricsAppender {
Log4jWarningErrorMetricsAppender appender;
Log logger = LogFactory.getLog(TestLog4jWarningErrorMetricsAppender.class);
List<Long> cutoff = new ArrayList<>();
void setupAppender(int cleanupIntervalSeconds, long messageAgeLimitSeconds,
int maxUniqueMessages) {
removeAppender();
appender =
new Log4jWarningErrorMetricsAppender(cleanupIntervalSeconds,
messageAgeLimitSeconds, maxUniqueMessages);
Logger.getRootLogger().addAppender(appender);
}
void removeAppender() {
Logger.getRootLogger().removeAppender(appender);
}
void logMessages(Level level, String message, int count) {
for (int i = 0; i < count; ++i) {
switch (level.toInt()) {
case Level.FATAL_INT:
logger.fatal(message);
break;
case Level.ERROR_INT:
logger.error(message);
break;
case Level.WARN_INT:
logger.warn(message);
break;
case Level.INFO_INT:
logger.info(message);
break;
case Level.DEBUG_INT:
logger.debug(message);
break;
case Level.TRACE_INT:
logger.trace(message);
break;
}
}
}
@Test
public void testPurge() throws Exception {
setupAppender(2, 1, 1);
logMessages(Level.ERROR, "test message 1", 1);
cutoff.clear();
cutoff.add(0L);
Assert.assertEquals(1, appender.getErrorCounts(cutoff).size());
Assert.assertEquals(1, appender.getErrorCounts(cutoff).get(0).longValue());
Assert.assertEquals(1, appender.getErrorMessagesAndCounts(cutoff).get(0)
.size());
Thread.sleep(2000);
Assert.assertEquals(1, appender.getErrorCounts(cutoff).size());
Assert.assertEquals(0, appender.getErrorCounts(cutoff).get(0).longValue());
Assert.assertEquals(0, appender.getErrorMessagesAndCounts(cutoff).get(0)
.size());
setupAppender(2, 1000, 2);
logMessages(Level.ERROR, "test message 1", 3);
logMessages(Level.ERROR, "test message 2", 2);
Assert.assertEquals(1, appender.getErrorCounts(cutoff).size());
Assert.assertEquals(5, appender.getErrorCounts(cutoff).get(0).longValue());
Assert.assertEquals(2, appender.getErrorMessagesAndCounts(cutoff).get(0)
.size());
logMessages(Level.ERROR, "test message 3", 3);
Thread.sleep(2000);
Assert.assertEquals(8, appender.getErrorCounts(cutoff).get(0).longValue());
Assert.assertEquals(2, appender.getErrorMessagesAndCounts(cutoff).get(0)
.size());
}
@Test
public void testErrorCounts() throws Exception {
cutoff.clear();
setupAppender(100, 100, 100);
cutoff.add(0L);
logMessages(Level.ERROR, "test message 1", 2);
logMessages(Level.ERROR, "test message 2", 3);
Assert.assertEquals(1, appender.getErrorCounts(cutoff).size());
Assert.assertEquals(1, appender.getWarningCounts(cutoff).size());
Assert.assertEquals(5, appender.getErrorCounts(cutoff).get(0).longValue());
Assert
.assertEquals(0, appender.getWarningCounts(cutoff).get(0).longValue());
Thread.sleep(1000);
cutoff.add(Time.now() / 1000);
logMessages(Level.ERROR, "test message 3", 2);
Assert.assertEquals(2, appender.getErrorCounts(cutoff).size());
Assert.assertEquals(2, appender.getWarningCounts(cutoff).size());
Assert.assertEquals(7, appender.getErrorCounts(cutoff).get(0).longValue());
Assert.assertEquals(2, appender.getErrorCounts(cutoff).get(1).longValue());
Assert
.assertEquals(0, appender.getWarningCounts(cutoff).get(0).longValue());
Assert
.assertEquals(0, appender.getWarningCounts(cutoff).get(1).longValue());
}
@Test
public void testWarningCounts() throws Exception {
cutoff.clear();
setupAppender(100, 100, 100);
cutoff.add(0L);
logMessages(Level.WARN, "test message 1", 2);
logMessages(Level.WARN, "test message 2", 3);
Assert.assertEquals(1, appender.getErrorCounts(cutoff).size());
Assert.assertEquals(1, appender.getWarningCounts(cutoff).size());
Assert.assertEquals(0, appender.getErrorCounts(cutoff).get(0).longValue());
Assert
.assertEquals(5, appender.getWarningCounts(cutoff).get(0).longValue());
Thread.sleep(1000);
cutoff.add(Time.now() / 1000);
logMessages(Level.WARN, "test message 3", 2);
Assert.assertEquals(2, appender.getErrorCounts(cutoff).size());
Assert.assertEquals(2, appender.getWarningCounts(cutoff).size());
Assert.assertEquals(0, appender.getErrorCounts(cutoff).get(0).longValue());
Assert.assertEquals(0, appender.getErrorCounts(cutoff).get(1).longValue());
Assert
.assertEquals(7, appender.getWarningCounts(cutoff).get(0).longValue());
Assert
.assertEquals(2, appender.getWarningCounts(cutoff).get(1).longValue());
}
@Test
public void testWarningMessages() throws Exception {
cutoff.clear();
setupAppender(100, 100, 100);
cutoff.add(0L);
logMessages(Level.WARN, "test message 1", 2);
logMessages(Level.WARN, "test message 2", 3);
Assert.assertEquals(1, appender.getErrorMessagesAndCounts(cutoff).size());
Assert.assertEquals(1, appender.getWarningMessagesAndCounts(cutoff).size());
Map<String, Log4jWarningErrorMetricsAppender.Element> errorsMap =
appender.getErrorMessagesAndCounts(cutoff).get(0);
Map<String, Log4jWarningErrorMetricsAppender.Element> warningsMap =
appender.getWarningMessagesAndCounts(cutoff).get(0);
Assert.assertEquals(0, errorsMap.size());
Assert.assertEquals(2, warningsMap.size());
Assert.assertTrue(warningsMap.containsKey("test message 1"));
Assert.assertTrue(warningsMap.containsKey("test message 2"));
Log4jWarningErrorMetricsAppender.Element msg1Info = warningsMap.get("test message 1");
Log4jWarningErrorMetricsAppender.Element msg2Info = warningsMap.get("test message 2");
Assert.assertEquals(2, msg1Info.count.intValue());
Assert.assertEquals(3, msg2Info.count.intValue());
Thread.sleep(1000);
cutoff.add(Time.now() / 1000);
logMessages(Level.WARN, "test message 3", 2);
Assert.assertEquals(2, appender.getErrorMessagesAndCounts(cutoff).size());
Assert.assertEquals(2, appender.getWarningMessagesAndCounts(cutoff).size());
errorsMap = appender.getErrorMessagesAndCounts(cutoff).get(0);
warningsMap = appender.getWarningMessagesAndCounts(cutoff).get(0);
Assert.assertEquals(0, errorsMap.size());
Assert.assertEquals(3, warningsMap.size());
Assert.assertTrue(warningsMap.containsKey("test message 3"));
errorsMap = appender.getErrorMessagesAndCounts(cutoff).get(1);
warningsMap = appender.getWarningMessagesAndCounts(cutoff).get(1);
Assert.assertEquals(0, errorsMap.size());
Assert.assertEquals(1, warningsMap.size());
Assert.assertTrue(warningsMap.containsKey("test message 3"));
Log4jWarningErrorMetricsAppender.Element msg3Info = warningsMap.get("test message 3");
Assert.assertEquals(2, msg3Info.count.intValue());
}
@Test
public void testErrorMessages() throws Exception {
cutoff.clear();
setupAppender(100, 100, 100);
cutoff.add(0L);
logMessages(Level.ERROR, "test message 1", 2);
logMessages(Level.ERROR, "test message 2", 3);
Assert.assertEquals(1, appender.getErrorMessagesAndCounts(cutoff).size());
Assert.assertEquals(1, appender.getWarningMessagesAndCounts(cutoff).size());
Map<String, Log4jWarningErrorMetricsAppender.Element> errorsMap =
appender.getErrorMessagesAndCounts(cutoff).get(0);
Map<String, Log4jWarningErrorMetricsAppender.Element> warningsMap =
appender.getWarningMessagesAndCounts(cutoff).get(0);
Assert.assertEquals(2, errorsMap.size());
Assert.assertEquals(0, warningsMap.size());
Assert.assertTrue(errorsMap.containsKey("test message 1"));
Assert.assertTrue(errorsMap.containsKey("test message 2"));
Log4jWarningErrorMetricsAppender.Element msg1Info = errorsMap.get("test message 1");
Log4jWarningErrorMetricsAppender.Element msg2Info = errorsMap.get("test message 2");
Assert.assertEquals(2, msg1Info.count.intValue());
Assert.assertEquals(3, msg2Info.count.intValue());
Thread.sleep(1000);
cutoff.add(Time.now() / 1000);
logMessages(Level.ERROR, "test message 3", 2);
Assert.assertEquals(2, appender.getErrorMessagesAndCounts(cutoff).size());
Assert.assertEquals(2, appender.getWarningMessagesAndCounts(cutoff).size());
errorsMap = appender.getErrorMessagesAndCounts(cutoff).get(0);
warningsMap = appender.getWarningMessagesAndCounts(cutoff).get(0);
Assert.assertEquals(3, errorsMap.size());
Assert.assertEquals(0, warningsMap.size());
Assert.assertTrue(errorsMap.containsKey("test message 3"));
errorsMap = appender.getErrorMessagesAndCounts(cutoff).get(1);
warningsMap = appender.getWarningMessagesAndCounts(cutoff).get(1);
Assert.assertEquals(1, errorsMap.size());
Assert.assertEquals(0, warningsMap.size());
Assert.assertTrue(errorsMap.containsKey("test message 3"));
Log4jWarningErrorMetricsAppender.Element msg3Info = errorsMap.get("test message 3");
Assert.assertEquals(2, msg3Info.count.intValue());
}
@Test
public void testInfoDebugTrace() {
cutoff.clear();
setupAppender(100, 100, 100);
cutoff.add(0L);
logMessages(Level.INFO, "test message 1", 2);
logMessages(Level.DEBUG, "test message 2", 2);
logMessages(Level.TRACE, "test message 3", 2);
Assert.assertEquals(1, appender.getErrorMessagesAndCounts(cutoff).size());
Assert.assertEquals(1, appender.getWarningMessagesAndCounts(cutoff).size());
Assert.assertEquals(1, appender.getErrorCounts(cutoff).size());
Assert.assertEquals(1, appender.getWarningCounts(cutoff).size());
Assert.assertEquals(0, appender.getErrorCounts(cutoff).get(0).longValue());
Assert
.assertEquals(0, appender.getWarningCounts(cutoff).get(0).longValue());
Assert.assertEquals(0, appender.getErrorMessagesAndCounts(cutoff).get(0)
.size());
Assert.assertEquals(0, appender.getWarningMessagesAndCounts(cutoff).get(0)
.size());
}
}

View File

@ -0,0 +1,233 @@
/**
* 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.yarn.server.webapp;
import com.google.inject.Inject;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.util.Time;
import org.apache.hadoop.yarn.util.Log4jWarningErrorMetricsAppender;
import org.apache.hadoop.yarn.util.Times;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
public class ErrorsAndWarningsBlock extends HtmlBlock {
long cutoffPeriodSeconds;
@Inject
ErrorsAndWarningsBlock(ViewContext ctx) {
super(ctx);
// default is to show all errors and warnings
cutoffPeriodSeconds = Time.now() / 1000;
String value = ctx.requestContext().get("cutoff", "");
try {
cutoffPeriodSeconds = Integer.parseInt(value);
if (cutoffPeriodSeconds <= 0) {
cutoffPeriodSeconds = Time.now() / 1000;
}
} catch (NumberFormatException ne) {
cutoffPeriodSeconds = Time.now() / 1000;
}
}
@Override
protected void render(Block html) {
Log log = LogFactory.getLog(ErrorsAndWarningsBlock.class);
if (log instanceof Log4JLogger) {
html._(ErrorMetrics.class);
html._(WarningMetrics.class);
html.div().button().$onclick("reloadPage()").b("View data for the last ")
._().select().$id("cutoff").option().$value("60")._("1 min")._()
.option().$value("300")._("5 min")._().option().$value("900")
._("15 min")._().option().$value("3600")._("1 hour")._().option()
.$value("21600")._("6 hours")._().option().$value("43200")
._("12 hours")._().option().$value("86400")._("24 hours")._()._()._();
String script = "function reloadPage() {"
+ " var timePeriod = $(\"#cutoff\").val();"
+ " document.location.href = '/cluster/errors-and-warnings?cutoff=' + timePeriod"
+ "}";
script = script
+ "; function toggleContent(element) {"
+ " $(element).parent().siblings('.toggle-content').fadeToggle();"
+ "}";
html.script().$type("text/javascript")._(script)._();
html.style(".toggle-content { display: none; }");
Log4jWarningErrorMetricsAppender appender =
Log4jWarningErrorMetricsAppender.findAppender();
if (appender == null) {
return;
}
List<Long> cutoff = new ArrayList<>();
Hamlet.TBODY<Hamlet.TABLE<Hamlet>> errorsTable =
html.table("#messages").thead().tr().th(".message", "Message")
.th(".type", "Type").th(".count", "Count")
.th(".lasttime", "Latest Message Time")._()._().tbody();
// cutoff has to be in seconds
cutoff.add((Time.now() - cutoffPeriodSeconds * 1000) / 1000);
List<Map<String, Log4jWarningErrorMetricsAppender.Element>> errorsData =
appender.getErrorMessagesAndCounts(cutoff);
List<Map<String, Log4jWarningErrorMetricsAppender.Element>> warningsData =
appender.getWarningMessagesAndCounts(cutoff);
Map<String, List<Map<String, Log4jWarningErrorMetricsAppender.Element>>> sources =
new HashMap<>();
sources.put("Error", errorsData);
sources.put("Warning", warningsData);
int maxDisplayLength = 80;
for (Map.Entry<String, List<Map<String, Log4jWarningErrorMetricsAppender.Element>>> source : sources
.entrySet()) {
String type = source.getKey();
List<Map<String, Log4jWarningErrorMetricsAppender.Element>> data =
source.getValue();
if (data.size() > 0) {
Map<String, Log4jWarningErrorMetricsAppender.Element> map = data.get(0);
for (Map.Entry<String, Log4jWarningErrorMetricsAppender.Element> entry : map
.entrySet()) {
String message = entry.getKey();
Hamlet.TR<Hamlet.TBODY<Hamlet.TABLE<Hamlet>>> row =
errorsTable.tr();
Hamlet.TD<Hamlet.TR<Hamlet.TBODY<Hamlet.TABLE<Hamlet>>>> cell =
row.td();
if (message.length() > maxDisplayLength || message.contains("\n")) {
String displayMessage = entry.getKey().split("\n")[0];
if (displayMessage.length() > maxDisplayLength) {
displayMessage = displayMessage.substring(0, maxDisplayLength);
}
cell.pre().a().$href("#").$onclick("toggleContent(this);")
.$style("white-space: pre")._(displayMessage)._()._().div()
.$class("toggle-content").pre()._(message)._()._()._();
} else {
cell.pre()._(message)._()._();
}
Log4jWarningErrorMetricsAppender.Element ele = entry.getValue();
row.td(type).td(String.valueOf(ele.count))
.td(Times.format(ele.timestampSeconds * 1000))._();
}
}
}
errorsTable._()._();
}
}
public static class MetricsBase extends HtmlBlock {
List<Long> cutoffs;
List<Integer> values;
String tableHeading;
Log4jWarningErrorMetricsAppender appender;
MetricsBase(ViewContext ctx) {
super(ctx);
cutoffs = new ArrayList<>();
// cutoff has to be in seconds
long now = Time.now();
cutoffs.add((now - 60 * 1000) / 1000);
cutoffs.add((now - 300 * 1000) / 1000);
cutoffs.add((now - 900 * 1000) / 1000);
cutoffs.add((now - 3600 * 1000) / 1000);
cutoffs.add((now - 21600 * 1000) / 1000);
cutoffs.add((now - 43200 * 1000) / 1000);
cutoffs.add((now - 84600 * 1000) / 1000);
Log log = LogFactory.getLog(ErrorsAndWarningsBlock.class);
if (log instanceof Log4JLogger) {
appender =
Log4jWarningErrorMetricsAppender.findAppender();
}
}
List<Long> getCutoffs() {
return this.cutoffs;
}
@Override
protected void render(Block html) {
Log log = LogFactory.getLog(ErrorsAndWarningsBlock.class);
if (log instanceof Log4JLogger) {
Hamlet.DIV<Hamlet> div =
html.div().$class("metrics").$style("padding-bottom: 20px");
div.h3(tableHeading).table("#metricsoverview").thead()
.$class("ui-widget-header").tr().th().$class("ui-state-default")
._("Last 1 minute")._().th().$class("ui-state-default")
._("Last 5 minutes")._().th().$class("ui-state-default")
._("Last 15 minutes")._().th().$class("ui-state-default")
._("Last 1 hour")._().th().$class("ui-state-default")
._("Last 6 hours")._().th().$class("ui-state-default")
._("Last 12 hours")._().th().$class("ui-state-default")
._("Last 24 hours")._()._()._().tbody().$class("ui-widget-content")
.tr().td(String.valueOf(values.get(0)))
.td(String.valueOf(values.get(1))).td(String.valueOf(values.get(2)))
.td(String.valueOf(values.get(3))).td(String.valueOf(values.get(4)))
.td(String.valueOf(values.get(5))).td(String.valueOf(values.get(6)))
._()._()._();
div._();
}
}
}
public static class ErrorMetrics extends MetricsBase {
@Inject
ErrorMetrics(ViewContext ctx) {
super(ctx);
tableHeading = "Error Metrics";
}
@Override
protected void render(Block html) {
if (appender == null) {
return;
}
values = appender.getErrorCounts(getCutoffs());
super.render(html);
}
}
public static class WarningMetrics extends MetricsBase {
@Inject
WarningMetrics(ViewContext ctx) {
super(ctx);
tableHeading = "Warning Metrics";
}
@Override
protected void render(Block html) {
if (appender == null) {
return;
}
values = appender.getWarningCounts(getCutoffs());
super.render(html);
}
}
}

View File

@ -75,6 +75,10 @@ public class NMController extends Controller implements YarnWebParams {
render(ContainerPage.class);
}
public void errorsAndWarnings() {
render(NMErrorsAndWarningsPage.class);
}
public void logs() {
String containerIdStr = $(CONTAINER_ID);
ContainerId containerId = null;

View File

@ -0,0 +1,55 @@
/**
* 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.yarn.server.nodemanager.webapp;
import org.apache.hadoop.yarn.webapp.SubView;
import org.apache.hadoop.yarn.server.webapp.ErrorsAndWarningsBlock;
import org.apache.hadoop.yarn.webapp.view.HtmlPage;
import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*;
public class NMErrorsAndWarningsPage extends NMView {
@Override
protected Class<? extends SubView> content() {
return ErrorsAndWarningsBlock.class;
}
@Override
protected void preHead(HtmlPage.Page.HTML<HtmlPage._> html) {
commonPreHead(html);
String title = "Errors and Warnings in the NodeManager";
setTitle(title);
String tableId = "messages";
set(DATATABLES_ID, tableId);
set(initID(DATATABLES, tableId), tablesInit());
setTableStyles(html, tableId, ".message {width:50em}",
".count {width:8em}", ".lasttime {width:16em}");
}
private String tablesInit() {
StringBuilder b = tableInit().append(", aoColumnDefs: [");
b.append("{'sType': 'string', 'aTargets': [ 0 ]}");
b.append(", {'sType': 'string', 'bSearchable': true, 'aTargets': [ 1 ]}");
b.append(", {'sType': 'numeric', 'bSearchable': false, 'aTargets': [ 2 ]}");
b.append(", {'sType': 'date', 'aTargets': [ 3 ] }]");
b.append(", aaSorting: [[3, 'desc']]}");
return b.toString();
}
}

View File

@ -18,8 +18,13 @@
package org.apache.hadoop.yarn.server.nodemanager.webapp;
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.yarn.util.Log4jWarningErrorMetricsAppender;
import org.apache.hadoop.yarn.webapp.YarnWebParams;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
@ -37,9 +42,18 @@ public class NavBlock extends HtmlBlock implements YarnWebParams {
@Override
protected void render(Block html) {
boolean addErrorsAndWarningsLink = false;
Log log = LogFactory.getLog(NMErrorsAndWarningsPage.class);
if (log instanceof Log4JLogger) {
Log4jWarningErrorMetricsAppender appender = Log4jWarningErrorMetricsAppender.findAppender();
if (appender != null) {
addErrorsAndWarningsLink = true;
}
}
String RMWebAppURL =
WebAppUtils.getResolvedRemoteRMWebAppURLWithScheme(this.conf);
html
WebAppUtils.getResolvedRMWebAppURLWithScheme(this.conf);
Hamlet.UL<Hamlet.DIV<Hamlet>> ul = html
.div("#nav")
.h3()._("ResourceManager")._()
.ul()
@ -59,7 +73,11 @@ public class NavBlock extends HtmlBlock implements YarnWebParams {
.li().a("/conf", "Configuration")._()
.li().a("/logs", "Local logs")._()
.li().a("/stacks", "Server stacks")._()
.li().a("/jmx?qry=Hadoop:*", "Server metrics")._()._()._();
.li().a("/jmx?qry=Hadoop:*", "Server metrics")._();
if (addErrorsAndWarningsLink) {
ul.li().a(url("errors-and-warnings"), "Errors/Warnings")._();
}
ul._()._();
}
}

View File

@ -126,6 +126,7 @@ public class WebServer extends AbstractService {
route(
pajoin("/containerlogs", CONTAINER_ID, APP_OWNER, CONTAINER_LOG_TYPE),
NMController.class, "logs");
route("/errors-and-warnings", NMController.class, "errorsAndWarnings");
}
}

View File

@ -18,7 +18,11 @@
package org.apache.hadoop.yarn.server.resourcemanager.webapp;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.util.Log4jWarningErrorMetricsAppender;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV;
import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.LI;
@ -28,6 +32,15 @@ import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
public class NavBlock extends HtmlBlock {
@Override public void render(Block html) {
boolean addErrorsAndWarningsLink = false;
Log log = LogFactory.getLog(RMErrorsAndWarningsPage.class);
if (log instanceof Log4JLogger) {
Log4jWarningErrorMetricsAppender appender =
Log4jWarningErrorMetricsAppender.findAppender();
if (appender != null) {
addErrorsAndWarningsLink = true;
}
}
UL<DIV<Hamlet>> mainList = html.
div("#nav").
h3("Cluster").
@ -44,13 +57,17 @@ public class NavBlock extends HtmlBlock {
li().a(url("apps", state.toString()), state.toString())._();
}
subAppsList._()._();
mainList.
UL<DIV<Hamlet>> tools = mainList.
li().a(url("scheduler"), "Scheduler")._()._().
h3("Tools").
ul().
li().a("/conf", "Configuration")._().
h3("Tools").ul();
tools.li().a("/conf", "Configuration")._().
li().a("/logs", "Local logs")._().
li().a("/stacks", "Server stacks")._().
li().a("/jmx?qry=Hadoop:*", "Server metrics")._()._()._();
li().a("/jmx?qry=Hadoop:*", "Server metrics")._();
if (addErrorsAndWarningsLink) {
tools.li().a(url("errors-and-warnings"), "Errors/Warnings")._();
}
tools._()._();
}
}

View File

@ -0,0 +1,54 @@
/**
* 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.yarn.server.resourcemanager.webapp;
import org.apache.hadoop.yarn.webapp.SubView;
import org.apache.hadoop.yarn.server.webapp.ErrorsAndWarningsBlock;
import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*;
public class RMErrorsAndWarningsPage extends RmView {
@Override
protected Class<? extends SubView> content() {
return ErrorsAndWarningsBlock.class;
}
@Override
protected void preHead(Page.HTML<_> html) {
commonPreHead(html);
String title = "Errors and Warnings in the ResourceManager";
setTitle(title);
String tableId = "messages";
set(DATATABLES_ID, tableId);
set(initID(DATATABLES, tableId), tablesInit());
setTableStyles(html, tableId, ".message {width:50em}",
".count {width:8em}", ".lasttime {width:16em}");
}
private String tablesInit() {
StringBuilder b = tableInit().append(", aoColumnDefs: [");
b.append("{'sType': 'string', 'aTargets': [ 0 ]}");
b.append(", {'sType': 'string', 'bSearchable': true, 'aTargets': [ 1 ]}");
b.append(", {'sType': 'numeric', 'bSearchable': false, 'aTargets': [ 2 ]}");
b.append(", {'sType': 'date', 'aTargets': [ 3 ] }]");
b.append(", aaSorting: [[3, 'desc']]}");
return b.toString();
}
}

View File

@ -67,6 +67,7 @@ public class RMWebApp extends WebApp implements YarnWebParams {
route(pajoin("/appattempt", APPLICATION_ATTEMPT_ID), RmController.class,
"appattempt");
route(pajoin("/container", CONTAINER_ID), RmController.class, "container");
route("/errors-and-warnings", RmController.class, "errorsAndWarnings");
}
@Override

View File

@ -105,4 +105,8 @@ public class RmController extends Controller {
setTitle("Node Labels");
render(NodeLabelsPage.class);
}
public void errorsAndWarnings() {
render(RMErrorsAndWarningsPage.class);
}
}