Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -246,6 +246,11 @@ static void reportError(Exception exception, Task task,
LOG.error(
"Fast fail the job because the cluster storage capacity was exceeded.");
fastFailJob = true;
} else {
LOG.warn(
"The cluster storage capacity was exceeded, but fast fail is disabled. "
+ "Set {} to true to enable fast fail.",
MRJobConfig.JOB_DFS_STORAGE_CAPACITY_KILL_LIMIT_EXCEED);
}
}
umbilical.fatalError(taskid, StringUtils.stringifyException(exception),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,15 +21,18 @@

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.ClusterStorageCapacityExceededException;
import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.slf4j.LoggerFactory;

import static org.mockito.Mockito.any;
import static org.mockito.Mockito.anyString;
import static org.mockito.Mockito.eq;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
import static org.junit.jupiter.api.Assertions.assertTrue;

/**
* Tests the behavior of YarnChild.
Expand Down Expand Up @@ -91,7 +94,8 @@ public void testReportErrorWhenCapacityExceptionHappenAndFastFailDisabled()
new RuntimeException(new ClusterStorageCapacityExceededException());
conf.setBoolean(KILL_LIMIT_EXCEED_CONF_NAME, false);

verifyReportError(exception, false);
verifyReportError(exception, false,
"The cluster storage capacity was exceeded, but fast fail is disabled.");
}

@Test
Expand Down Expand Up @@ -120,4 +124,23 @@ private void verifyReportError(Exception exception, boolean fastFail)
verify(umbilical).fatalError(any(), anyString(),
eq(fastFail));
}

private void verifyReportError(Exception exception, boolean fastFail,
String expectedLogMessage) throws IOException {
LogCapturer logCapturer = LogCapturer.captureLogs(
LoggerFactory.getLogger(YarnChild.class));
try {
verifyReportError(exception, fastFail);
assertTrue(logCapturer.getOutput().contains(expectedLogMessage));
if (fastFail) {
assertTrue(logCapturer.getOutput().contains(
"Fast fail the job because the cluster storage capacity was exceeded."));
} else {
assertTrue(logCapturer.getOutput().contains(
KILL_LIMIT_EXCEED_CONF_NAME));
}
} finally {
logCapturer.stopCapturing();
}
}
}
Loading