|
@@ -21,6 +21,9 @@ package org.apache.hadoop.mapreduce.jobhistory;
|
|
|
import static org.junit.Assert.assertFalse;
|
|
|
import static org.junit.Assert.assertTrue;
|
|
|
import static org.mockito.Matchers.any;
|
|
|
+import static org.mockito.Mockito.doNothing;
|
|
|
+import static org.mockito.Mockito.doReturn;
|
|
|
+import static org.mockito.Mockito.doThrow;
|
|
|
import static org.mockito.Mockito.mock;
|
|
|
import static org.mockito.Mockito.spy;
|
|
|
import static org.mockito.Mockito.times;
|
|
@@ -62,6 +65,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.JobStateInternal;
|
|
|
import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
|
|
|
import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
|
|
|
import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
|
|
|
+import org.apache.hadoop.mapreduce.v2.util.MRWebAppUtil;
|
|
|
import org.apache.hadoop.security.authorize.AccessControlList;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
@@ -920,6 +924,104 @@ public class TestJobHistoryEventHandler {
|
|
|
jheh.lastEventHandled.getHistoryEvent()
|
|
|
instanceof JobUnsuccessfulCompletionEvent);
|
|
|
}
|
|
|
+
|
|
|
+ @Test (timeout=50000)
|
|
|
+ public void testSetTrackingURLAfterHistoryIsWritten() throws Exception {
|
|
|
+ TestParams t = new TestParams(true);
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+
|
|
|
+ JHEvenHandlerForTest realJheh =
|
|
|
+ new JHEvenHandlerForTest(t.mockAppContext, 0, false);
|
|
|
+ JHEvenHandlerForTest jheh = spy(realJheh);
|
|
|
+ jheh.init(conf);
|
|
|
+
|
|
|
+ try {
|
|
|
+ jheh.start();
|
|
|
+ handleEvent(jheh, new JobHistoryEvent(t.jobId, new AMStartedEvent(
|
|
|
+ t.appAttemptId, 200, t.containerId, "nmhost", 3000, 4000, -1)));
|
|
|
+ verify(jheh, times(0)).processDoneFiles(any(JobId.class));
|
|
|
+ verify(t.mockAppContext, times(0)).setHistoryUrl(any(String.class));
|
|
|
+
|
|
|
+ // Job finishes and successfully writes history
|
|
|
+ handleEvent(jheh, new JobHistoryEvent(t.jobId, new JobFinishedEvent(
|
|
|
+ TypeConverter.fromYarn(t.jobId), 0, 0, 0, 0, 0, new Counters(),
|
|
|
+ new Counters(), new Counters())));
|
|
|
+
|
|
|
+ verify(jheh, times(1)).processDoneFiles(any(JobId.class));
|
|
|
+ String historyUrl = MRWebAppUtil.getApplicationWebURLOnJHSWithScheme(
|
|
|
+ conf, t.mockAppContext.getApplicationID());
|
|
|
+ verify(t.mockAppContext, times(1)).setHistoryUrl(historyUrl);
|
|
|
+ } finally {
|
|
|
+ jheh.stop();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test (timeout=50000)
|
|
|
+ public void testDontSetTrackingURLIfHistoryWriteFailed() throws Exception {
|
|
|
+ TestParams t = new TestParams(true);
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+
|
|
|
+ JHEvenHandlerForTest realJheh =
|
|
|
+ new JHEvenHandlerForTest(t.mockAppContext, 0, false);
|
|
|
+ JHEvenHandlerForTest jheh = spy(realJheh);
|
|
|
+ jheh.init(conf);
|
|
|
+
|
|
|
+ try {
|
|
|
+ jheh.start();
|
|
|
+ doReturn(false).when(jheh).moveToDoneNow(any(Path.class),
|
|
|
+ any(Path.class));
|
|
|
+ doNothing().when(jheh).moveTmpToDone(any(Path.class));
|
|
|
+ handleEvent(jheh, new JobHistoryEvent(t.jobId, new AMStartedEvent(
|
|
|
+ t.appAttemptId, 200, t.containerId, "nmhost", 3000, 4000, -1)));
|
|
|
+ verify(jheh, times(0)).processDoneFiles(any(JobId.class));
|
|
|
+ verify(t.mockAppContext, times(0)).setHistoryUrl(any(String.class));
|
|
|
+
|
|
|
+ // Job finishes, but doesn't successfully write history
|
|
|
+ handleEvent(jheh, new JobHistoryEvent(t.jobId, new JobFinishedEvent(
|
|
|
+ TypeConverter.fromYarn(t.jobId), 0, 0, 0, 0, 0, new Counters(),
|
|
|
+ new Counters(), new Counters())));
|
|
|
+ verify(jheh, times(1)).processDoneFiles(any(JobId.class));
|
|
|
+ verify(t.mockAppContext, times(0)).setHistoryUrl(any(String.class));
|
|
|
+
|
|
|
+ } finally {
|
|
|
+ jheh.stop();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ @Test (timeout=50000)
|
|
|
+ public void testDontSetTrackingURLIfHistoryWriteThrows() throws Exception {
|
|
|
+ TestParams t = new TestParams(true);
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+
|
|
|
+ JHEvenHandlerForTest realJheh =
|
|
|
+ new JHEvenHandlerForTest(t.mockAppContext, 0, false);
|
|
|
+ JHEvenHandlerForTest jheh = spy(realJheh);
|
|
|
+ jheh.init(conf);
|
|
|
+
|
|
|
+ try {
|
|
|
+ jheh.start();
|
|
|
+ doThrow(new YarnRuntimeException(new IOException()))
|
|
|
+ .when(jheh).processDoneFiles(any(JobId.class));
|
|
|
+ handleEvent(jheh, new JobHistoryEvent(t.jobId, new AMStartedEvent(
|
|
|
+ t.appAttemptId, 200, t.containerId, "nmhost", 3000, 4000, -1)));
|
|
|
+ verify(jheh, times(0)).processDoneFiles(any(JobId.class));
|
|
|
+ verify(t.mockAppContext, times(0)).setHistoryUrl(any(String.class));
|
|
|
+
|
|
|
+ // Job finishes, but doesn't successfully write history
|
|
|
+ try {
|
|
|
+ handleEvent(jheh, new JobHistoryEvent(t.jobId, new JobFinishedEvent(
|
|
|
+ TypeConverter.fromYarn(t.jobId), 0, 0, 0, 0, 0, new Counters(),
|
|
|
+ new Counters(), new Counters())));
|
|
|
+ throw new RuntimeException(
|
|
|
+ "processDoneFiles didn't throw, but should have");
|
|
|
+ } catch (YarnRuntimeException yre) {
|
|
|
+ // Exception expected, do nothing
|
|
|
+ }
|
|
|
+ verify(jheh, times(1)).processDoneFiles(any(JobId.class));
|
|
|
+ verify(t.mockAppContext, times(0)).setHistoryUrl(any(String.class));
|
|
|
+ } finally {
|
|
|
+ jheh.stop();
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
class JHEvenHandlerForTest extends JobHistoryEventHandler {
|