|
@@ -20,13 +20,13 @@ package org.apache.hadoop.mapred;
|
|
|
|
|
|
import java.io.DataOutputStream;
|
|
import java.io.DataOutputStream;
|
|
import java.io.File;
|
|
import java.io.File;
|
|
-import java.io.FileReader;
|
|
|
|
-import java.io.FileWriter;
|
|
|
|
|
|
+import java.io.FileOutputStream;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
import java.net.URI;
|
|
import java.net.URI;
|
|
import java.util.Arrays;
|
|
import java.util.Arrays;
|
|
import java.util.HashMap;
|
|
import java.util.HashMap;
|
|
import java.util.Map;
|
|
import java.util.Map;
|
|
|
|
+import java.util.Random;
|
|
|
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
import org.apache.commons.logging.LogFactory;
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.Log;
|
|
@@ -49,8 +49,7 @@ import org.apache.hadoop.mapreduce.split.JobSplit;
|
|
import org.junit.After;
|
|
import org.junit.After;
|
|
import org.junit.Test;
|
|
import org.junit.Test;
|
|
|
|
|
|
-import static org.junit.Assert.assertTrue;
|
|
|
|
-import static org.junit.Assert.assertEquals;
|
|
|
|
|
|
+import static org.junit.Assert.*;
|
|
|
|
|
|
/**
|
|
/**
|
|
* Verify the logs' truncation functionality.
|
|
* Verify the logs' truncation functionality.
|
|
@@ -58,6 +57,7 @@ import static org.junit.Assert.assertEquals;
|
|
public class TestTaskLogsTruncater {
|
|
public class TestTaskLogsTruncater {
|
|
|
|
|
|
static final Log LOG = LogFactory.getLog(TestTaskLogsTruncater.class);
|
|
static final Log LOG = LogFactory.getLog(TestTaskLogsTruncater.class);
|
|
|
|
+ private static int truncatedMsgSize = TaskLogsTruncater.TRUNCATED_MSG.getBytes().length;
|
|
|
|
|
|
/**
|
|
/**
|
|
* clean-up any stale directories after enabling writable permissions for all
|
|
* clean-up any stale directories after enabling writable permissions for all
|
|
@@ -74,9 +74,8 @@ public class TestTaskLogsTruncater {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- void writeRealBytes(TaskAttemptID firstAttemptID,
|
|
|
|
- TaskAttemptID attemptID, LogName logName, long numBytes, char data)
|
|
|
|
- throws IOException {
|
|
|
|
|
|
+ private void writeBytes(TaskAttemptID firstAttemptID, TaskAttemptID attemptID,
|
|
|
|
+ LogName logName, long numBytes, boolean random, char data) throws IOException {
|
|
|
|
|
|
File logFile = TaskLog.getTaskLogFile(firstAttemptID, false, logName);
|
|
File logFile = TaskLog.getTaskLogFile(firstAttemptID, false, logName);
|
|
File logLocation = logFile.getParentFile();
|
|
File logLocation = logFile.getParentFile();
|
|
@@ -99,16 +98,33 @@ public class TestTaskLogsTruncater {
|
|
// Need to call up front to set currenttaskid.
|
|
// Need to call up front to set currenttaskid.
|
|
TaskLog.syncLogs(logLocation.toString(), attemptID, false);
|
|
TaskLog.syncLogs(logLocation.toString(), attemptID, false);
|
|
|
|
|
|
- FileWriter writer = new FileWriter(logFile, true);
|
|
|
|
|
|
+ FileOutputStream outputStream = new FileOutputStream(logFile, true);
|
|
|
|
+ Random r = new Random();
|
|
for (long i = 0; i < numBytes; i++) {
|
|
for (long i = 0; i < numBytes; i++) {
|
|
- writer.write(data);
|
|
|
|
|
|
+ if(random) {
|
|
|
|
+ outputStream.write(r.nextInt());
|
|
|
|
+ } else {
|
|
|
|
+ outputStream.write(data);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
- writer.close();
|
|
|
|
|
|
+ outputStream.close();
|
|
TaskLog.syncLogs(logLocation.toString(), attemptID, false);
|
|
TaskLog.syncLogs(logLocation.toString(), attemptID, false);
|
|
- LOG.info("Written " + numBytes + " real bytes to the log file "
|
|
|
|
|
|
+ LOG.info("Written " + logFile.length() + " real bytes to the log file "
|
|
+ logFile);
|
|
+ logFile);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ private void writeRandomBytes(TaskAttemptID firstAttemptID,
|
|
|
|
+ TaskAttemptID attemptID, LogName logName, long numBytes)
|
|
|
|
+ throws IOException {
|
|
|
|
+ writeBytes(firstAttemptID, attemptID, logName, numBytes, true, ' ');
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void writeRealChars(TaskAttemptID firstAttemptID,
|
|
|
|
+ TaskAttemptID attemptID, LogName logName, long numChars, char data)
|
|
|
|
+ throws IOException {
|
|
|
|
+ writeBytes(firstAttemptID, attemptID, logName, numChars, false, data);
|
|
|
|
+ }
|
|
|
|
+
|
|
private static Map<LogName, Long> getAllLogsFileLengths(
|
|
private static Map<LogName, Long> getAllLogsFileLengths(
|
|
TaskAttemptID tid, boolean isCleanup) throws IOException {
|
|
TaskAttemptID tid, boolean isCleanup) throws IOException {
|
|
Map<LogName, Long> allLogsFileLengths = new HashMap<LogName, Long>();
|
|
Map<LogName, Long> allLogsFileLengths = new HashMap<LogName, Long>();
|
|
@@ -158,7 +174,7 @@ public class TestTaskLogsTruncater {
|
|
|
|
|
|
// Let the tasks write logs within retain-size
|
|
// Let the tasks write logs within retain-size
|
|
for (LogName log : LogName.values()) {
|
|
for (LogName log : LogName.values()) {
|
|
- writeRealBytes(attemptID, attemptID, log, 500, 'H');
|
|
|
|
|
|
+ writeRandomBytes(attemptID, attemptID, log, 500);
|
|
}
|
|
}
|
|
File logIndex = TaskLog.getIndexFile(attemptID, false);
|
|
File logIndex = TaskLog.getIndexFile(attemptID, false);
|
|
long indexModificationTimeStamp = logIndex.lastModified();
|
|
long indexModificationTimeStamp = logIndex.lastModified();
|
|
@@ -219,7 +235,7 @@ public class TestTaskLogsTruncater {
|
|
|
|
|
|
// Let the tasks write some logs
|
|
// Let the tasks write some logs
|
|
for (LogName log : LogName.values()) {
|
|
for (LogName log : LogName.values()) {
|
|
- writeRealBytes(attemptID, attemptID, log, 1500, 'H');
|
|
|
|
|
|
+ writeRandomBytes(attemptID, attemptID, log, 1500);
|
|
}
|
|
}
|
|
|
|
|
|
File attemptDir = TaskLog.getAttemptDir(attemptID, false);
|
|
File attemptDir = TaskLog.getAttemptDir(attemptID, false);
|
|
@@ -259,7 +275,7 @@ public class TestTaskLogsTruncater {
|
|
|
|
|
|
// Let the tasks write logs more than retain-size
|
|
// Let the tasks write logs more than retain-size
|
|
for (LogName log : LogName.values()) {
|
|
for (LogName log : LogName.values()) {
|
|
- writeRealBytes(attemptID, attemptID, log, 1500, 'H');
|
|
|
|
|
|
+ writeRandomBytes(attemptID, attemptID, log, 1500);
|
|
}
|
|
}
|
|
|
|
|
|
File attemptDir = TaskLog.getAttemptDir(attemptID, false);
|
|
File attemptDir = TaskLog.getAttemptDir(attemptID, false);
|
|
@@ -275,18 +291,18 @@ public class TestTaskLogsTruncater {
|
|
Map<LogName, Long> logLengths = getAllLogsFileLengths(attemptID, false);
|
|
Map<LogName, Long> logLengths = getAllLogsFileLengths(attemptID, false);
|
|
for (LogName log : LogName.values()) {
|
|
for (LogName log : LogName.values()) {
|
|
File logFile = TaskLog.getTaskLogFile(attemptID, false, log);
|
|
File logFile = TaskLog.getTaskLogFile(attemptID, false, log);
|
|
- assertEquals(1000, logFile.length());
|
|
|
|
|
|
+ assertEquals(1000 + truncatedMsgSize, logFile.length());
|
|
// The index file should also be proper.
|
|
// The index file should also be proper.
|
|
- assertEquals(1000, logLengths.get(log).longValue());
|
|
|
|
|
|
+ assertEquals(1000 + truncatedMsgSize, logLengths.get(log).longValue());
|
|
}
|
|
}
|
|
|
|
|
|
// truncate once again
|
|
// truncate once again
|
|
logLengths = getAllLogsFileLengths(attemptID, false);
|
|
logLengths = getAllLogsFileLengths(attemptID, false);
|
|
for (LogName log : LogName.values()) {
|
|
for (LogName log : LogName.values()) {
|
|
File logFile = TaskLog.getTaskLogFile(attemptID, false, log);
|
|
File logFile = TaskLog.getTaskLogFile(attemptID, false, log);
|
|
- assertEquals(1000, logFile.length());
|
|
|
|
|
|
+ assertEquals(1000 + truncatedMsgSize, logFile.length());
|
|
// The index file should also be proper.
|
|
// The index file should also be proper.
|
|
- assertEquals(1000, logLengths.get(log).longValue());
|
|
|
|
|
|
+ assertEquals(1000 + truncatedMsgSize, logLengths.get(log).longValue());
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -310,8 +326,8 @@ public class TestTaskLogsTruncater {
|
|
0);
|
|
0);
|
|
|
|
|
|
// Let the tasks write logs more than retain-size
|
|
// Let the tasks write logs more than retain-size
|
|
- writeRealBytes(attemptID, attemptID, LogName.SYSLOG, 1500, 'H');
|
|
|
|
- writeRealBytes(attemptID, attemptID, LogName.STDERR, 500, 'H');
|
|
|
|
|
|
+ writeRandomBytes(attemptID, attemptID, LogName.SYSLOG, 1500);
|
|
|
|
+ writeRandomBytes(attemptID, attemptID, LogName.STDERR, 500);
|
|
|
|
|
|
File attemptDir = TaskLog.getAttemptDir(attemptID, false);
|
|
File attemptDir = TaskLog.getAttemptDir(attemptID, false);
|
|
assertTrue(attemptDir + " doesn't exist!", attemptDir.exists());
|
|
assertTrue(attemptDir + " doesn't exist!", attemptDir.exists());
|
|
@@ -325,21 +341,29 @@ public class TestTaskLogsTruncater {
|
|
|
|
|
|
Map<LogName, Long> logLengths = getAllLogsFileLengths(attemptID, false);
|
|
Map<LogName, Long> logLengths = getAllLogsFileLengths(attemptID, false);
|
|
File logFile = TaskLog.getTaskLogFile(attemptID, false, LogName.SYSLOG);
|
|
File logFile = TaskLog.getTaskLogFile(attemptID, false, LogName.SYSLOG);
|
|
- assertEquals(1000, logFile.length());
|
|
|
|
|
|
+ assertEquals(1000 + truncatedMsgSize, logFile.length());
|
|
// The index file should also be proper.
|
|
// The index file should also be proper.
|
|
- assertEquals(1000, logLengths.get(LogName.SYSLOG).longValue());
|
|
|
|
|
|
+ assertEquals(1000 + truncatedMsgSize, logLengths.get(LogName.SYSLOG)
|
|
|
|
+ .longValue());
|
|
|
|
+ String syslog = TestMiniMRMapRedDebugScript.readTaskLog(LogName.SYSLOG,
|
|
|
|
+ attemptID, false);
|
|
|
|
+ assertTrue(syslog.startsWith(TaskLogsTruncater.TRUNCATED_MSG));
|
|
logFile = TaskLog.getTaskLogFile(attemptID, false, LogName.STDERR);
|
|
logFile = TaskLog.getTaskLogFile(attemptID, false, LogName.STDERR);
|
|
assertEquals(500, logFile.length());
|
|
assertEquals(500, logFile.length());
|
|
// The index file should also be proper.
|
|
// The index file should also be proper.
|
|
assertEquals(500, logLengths.get(LogName.STDERR).longValue());
|
|
assertEquals(500, logLengths.get(LogName.STDERR).longValue());
|
|
|
|
+ String stderr = TestMiniMRMapRedDebugScript.readTaskLog(LogName.STDERR,
|
|
|
|
+ attemptID, false);
|
|
|
|
+ assertFalse(stderr.startsWith(TaskLogsTruncater.TRUNCATED_MSG));
|
|
|
|
|
|
// truncate once again
|
|
// truncate once again
|
|
logManager.addLogEvent(new JvmFinishedEvent(jvmInfo));
|
|
logManager.addLogEvent(new JvmFinishedEvent(jvmInfo));
|
|
logLengths = getAllLogsFileLengths(attemptID, false);
|
|
logLengths = getAllLogsFileLengths(attemptID, false);
|
|
logFile = TaskLog.getTaskLogFile(attemptID, false, LogName.SYSLOG);
|
|
logFile = TaskLog.getTaskLogFile(attemptID, false, LogName.SYSLOG);
|
|
- assertEquals(1000, logFile.length());
|
|
|
|
|
|
+ assertEquals(1000 + truncatedMsgSize, logFile.length());
|
|
// The index file should also be proper.
|
|
// The index file should also be proper.
|
|
- assertEquals(1000, logLengths.get(LogName.SYSLOG).longValue());
|
|
|
|
|
|
+ assertEquals(1000 + truncatedMsgSize, logLengths.get(LogName.SYSLOG)
|
|
|
|
+ .longValue());
|
|
logFile = TaskLog.getTaskLogFile(attemptID, false, LogName.STDERR);
|
|
logFile = TaskLog.getTaskLogFile(attemptID, false, LogName.STDERR);
|
|
assertEquals(500, logFile.length());
|
|
assertEquals(500, logFile.length());
|
|
// The index file should also be proper.
|
|
// The index file should also be proper.
|
|
@@ -365,7 +389,7 @@ public class TestTaskLogsTruncater {
|
|
0);
|
|
0);
|
|
|
|
|
|
// Let the tasks write logs more than retain-size
|
|
// Let the tasks write logs more than retain-size
|
|
- writeRealBytes(attempt1, attempt1, LogName.SYSLOG, 200, 'A');
|
|
|
|
|
|
+ writeRealChars(attempt1, attempt1, LogName.SYSLOG, 200, 'A');
|
|
|
|
|
|
File attemptDir = TaskLog.getAttemptDir(attempt1, false);
|
|
File attemptDir = TaskLog.getAttemptDir(attempt1, false);
|
|
assertTrue(attemptDir + " doesn't exist!", attemptDir.exists());
|
|
assertTrue(attemptDir + " doesn't exist!", attemptDir.exists());
|
|
@@ -375,13 +399,13 @@ public class TestTaskLogsTruncater {
|
|
Task task2 = new MapTask(null, attempt2, 0, new JobSplit.TaskSplitIndex(),
|
|
Task task2 = new MapTask(null, attempt2, 0, new JobSplit.TaskSplitIndex(),
|
|
0);
|
|
0);
|
|
// Let attempt2 also write some logs
|
|
// Let attempt2 also write some logs
|
|
- writeRealBytes(attempt1, attempt2, LogName.SYSLOG, 100, 'B');
|
|
|
|
|
|
+ writeRealChars(attempt1, attempt2, LogName.SYSLOG, 100, 'B');
|
|
// Start yet another attempt in the same JVM
|
|
// Start yet another attempt in the same JVM
|
|
TaskAttemptID attempt3 = new TaskAttemptID(baseTaskID, attemptsCount++);
|
|
TaskAttemptID attempt3 = new TaskAttemptID(baseTaskID, attemptsCount++);
|
|
Task task3 = new MapTask(null, attempt3, 0, new JobSplit.TaskSplitIndex(),
|
|
Task task3 = new MapTask(null, attempt3, 0, new JobSplit.TaskSplitIndex(),
|
|
0);
|
|
0);
|
|
// Let attempt3 also write some logs
|
|
// Let attempt3 also write some logs
|
|
- writeRealBytes(attempt1, attempt3, LogName.SYSLOG, 225, 'C');
|
|
|
|
|
|
+ writeRealChars(attempt1, attempt3, LogName.SYSLOG, 225, 'C');
|
|
// Finish the JVM.
|
|
// Finish the JVM.
|
|
JVMInfo jvmInfo = new JVMInfo(attemptDir,
|
|
JVMInfo jvmInfo = new JVMInfo(attemptDir,
|
|
Arrays.asList((new Task[] { task1, task2, task3 })));
|
|
Arrays.asList((new Task[] { task1, task2, task3 })));
|
|
@@ -390,46 +414,50 @@ public class TestTaskLogsTruncater {
|
|
// The log-file should now be truncated.
|
|
// The log-file should now be truncated.
|
|
assertTrue(attemptDir.exists());
|
|
assertTrue(attemptDir.exists());
|
|
File logFile = TaskLog.getTaskLogFile(attempt1, false, LogName.SYSLOG);
|
|
File logFile = TaskLog.getTaskLogFile(attempt1, false, LogName.SYSLOG);
|
|
- assertEquals(400, logFile.length());
|
|
|
|
|
|
+ assertEquals(400 + (2 * truncatedMsgSize), logFile.length());
|
|
// The index files should also be proper.
|
|
// The index files should also be proper.
|
|
- assertEquals(150, getAllLogsFileLengths(attempt1, false).get(
|
|
|
|
- LogName.SYSLOG).longValue());
|
|
|
|
- assertEquals(100, getAllLogsFileLengths(attempt2, false).get(
|
|
|
|
- LogName.SYSLOG).longValue());
|
|
|
|
- assertEquals(150, getAllLogsFileLengths(attempt3, false).get(
|
|
|
|
- LogName.SYSLOG).longValue());
|
|
|
|
-
|
|
|
|
- // assert the data.
|
|
|
|
- FileReader reader =
|
|
|
|
- new FileReader(TaskLog.getTaskLogFile(attempt1, false, LogName.SYSLOG));
|
|
|
|
- int ch, bytesRead = 0;
|
|
|
|
- boolean dataValid = true;
|
|
|
|
- while ((ch = reader.read()) != -1) {
|
|
|
|
- bytesRead++;
|
|
|
|
- if (bytesRead <= 150) {
|
|
|
|
- if ((char) ch != 'A') {
|
|
|
|
- LOG.warn("Truncation didn't happen properly. At "
|
|
|
|
- + (bytesRead + 1) + "th byte, expected 'A' but found "
|
|
|
|
- + (char) ch);
|
|
|
|
- dataValid = false;
|
|
|
|
- }
|
|
|
|
- } else if (bytesRead <= 250) {
|
|
|
|
- if ((char) ch != 'B') {
|
|
|
|
- LOG.warn("Truncation didn't happen properly. At "
|
|
|
|
- + (bytesRead + 1) + "th byte, expected 'B' but found "
|
|
|
|
- + (char) ch);
|
|
|
|
- dataValid = false;
|
|
|
|
- }
|
|
|
|
- } else if ((char) ch != 'C') {
|
|
|
|
- LOG.warn("Truncation didn't happen properly. At " + (bytesRead + 1)
|
|
|
|
- + "th byte, expected 'C' but found " + (char) ch);
|
|
|
|
- dataValid = false;
|
|
|
|
|
|
+ assertEquals(150 + truncatedMsgSize, getAllLogsFileLengths(attempt1, false)
|
|
|
|
+ .get(LogName.SYSLOG).longValue());
|
|
|
|
+ assertEquals(100, getAllLogsFileLengths(attempt2, false)
|
|
|
|
+ .get(LogName.SYSLOG).longValue());
|
|
|
|
+ assertEquals(150 + truncatedMsgSize, getAllLogsFileLengths(attempt3, false)
|
|
|
|
+ .get(LogName.SYSLOG).longValue());
|
|
|
|
+
|
|
|
|
+ // assert data for attempt1
|
|
|
|
+ String syslog = TestMiniMRMapRedDebugScript.readTaskLog(LogName.SYSLOG,
|
|
|
|
+ attempt1, false);
|
|
|
|
+ assertTrue(syslog.startsWith(TaskLogsTruncater.TRUNCATED_MSG));
|
|
|
|
+ String truncatedLog = syslog.substring(truncatedMsgSize);
|
|
|
|
+ for (int i = 0 ; i < 150; i++) {
|
|
|
|
+ assertEquals("Truncation didn't happen properly. At "
|
|
|
|
+ + (i + 1) + "th byte, expected 'A' but found "
|
|
|
|
+ + truncatedLog.charAt(i), 'A', truncatedLog.charAt(i));
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // assert data for attempt2
|
|
|
|
+ syslog = TestMiniMRMapRedDebugScript.readTaskLog(LogName.SYSLOG,
|
|
|
|
+ attempt2, false);
|
|
|
|
+ for (int i = 0 ; i < 100; i++) {
|
|
|
|
+ assertEquals("Truncation didn't happen properly. At "
|
|
|
|
+ + (i + 1) + "th byte, expected 'B' but found "
|
|
|
|
+ + truncatedLog.charAt(i), 'B', syslog.charAt(i));
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ // assert data for attempt3
|
|
|
|
+ syslog = TestMiniMRMapRedDebugScript.readTaskLog(LogName.SYSLOG,
|
|
|
|
+ attempt3, false);
|
|
|
|
+ assertTrue(syslog.startsWith(TaskLogsTruncater.TRUNCATED_MSG));
|
|
|
|
+ truncatedLog = syslog.substring(truncatedMsgSize);
|
|
|
|
+ for (int i = 0 ; i < 150; i++) {
|
|
|
|
+ assertEquals("Truncation didn't happen properly. At "
|
|
|
|
+ + (i + 1) + "th byte, expected 'C' but found "
|
|
|
|
+ + truncatedLog.charAt(i), 'C', truncatedLog.charAt(i));
|
|
}
|
|
}
|
|
- assertTrue("Log-truncation didn't happen properly!", dataValid);
|
|
|
|
|
|
|
|
logManager.addLogEvent(new JvmFinishedEvent(jvmInfo));
|
|
logManager.addLogEvent(new JvmFinishedEvent(jvmInfo));
|
|
- assertEquals(400, logFile.length());
|
|
|
|
|
|
+ // First and third attempts' logs are only truncated, so include 2*length of
|
|
|
|
+ // TRUNCATED_MSG header
|
|
|
|
+ assertEquals(400 + 2 * truncatedMsgSize, logFile.length());
|
|
}
|
|
}
|
|
|
|
|
|
private static String TEST_ROOT_DIR =
|
|
private static String TEST_ROOT_DIR =
|
|
@@ -500,13 +528,12 @@ public class TestTaskLogsTruncater {
|
|
TaskLog.getTaskLogFile(tce.getTaskAttemptId(), false,
|
|
TaskLog.getTaskLogFile(tce.getTaskAttemptId(), false,
|
|
TaskLog.LogName.STDOUT).length();
|
|
TaskLog.LogName.STDOUT).length();
|
|
assertTrue("STDOUT log file length for " + tce.getTaskAttemptId()
|
|
assertTrue("STDOUT log file length for " + tce.getTaskAttemptId()
|
|
- + " is " + length + " and not <=10000", length <= 10000);
|
|
|
|
|
|
+ + " is " + length + " and not <=" + 10000 + truncatedMsgSize,
|
|
|
|
+ length <= 10000 + truncatedMsgSize);
|
|
if (tce.isMap) {
|
|
if (tce.isMap) {
|
|
String stderr = TestMiniMRMapRedDebugScript.readTaskLog(
|
|
String stderr = TestMiniMRMapRedDebugScript.readTaskLog(
|
|
LogName.STDERR, tce.getTaskAttemptId(), false);
|
|
LogName.STDERR, tce.getTaskAttemptId(), false);
|
|
System.out.println("STDERR log:" + stderr);
|
|
System.out.println("STDERR log:" + stderr);
|
|
- assertTrue(stderr.length() > 0);
|
|
|
|
- assertTrue(stderr.length() < 10000);
|
|
|
|
assertTrue(stderr.equals(STDERR_LOG));
|
|
assertTrue(stderr.equals(STDERR_LOG));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -591,9 +618,9 @@ public class TestTaskLogsTruncater {
|
|
TaskLog.LogName.DEBUGOUT);
|
|
TaskLog.LogName.DEBUGOUT);
|
|
if (debugOutFile.exists()) {
|
|
if (debugOutFile.exists()) {
|
|
long length = debugOutFile.length();
|
|
long length = debugOutFile.length();
|
|
- assertTrue("DEBUGOUT log file length for "
|
|
|
|
- + tce.getTaskAttemptId() + " is " + length
|
|
|
|
- + " and not =10000", length == 10000);
|
|
|
|
|
|
+ assertTrue("DEBUGOUT log file length for " + tce.getTaskAttemptId()
|
|
|
|
+ + " is " + length + " and not " + 10000 + truncatedMsgSize,
|
|
|
|
+ length == 10000 + truncatedMsgSize);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|