|
@@ -18,6 +18,8 @@
|
|
|
|
|
|
package org.apache.hadoop.mapreduce.jobhistory;
|
|
|
|
|
|
+import static junit.framework.Assert.assertFalse;
|
|
|
+import static junit.framework.Assert.assertTrue;
|
|
|
import static org.mockito.Matchers.any;
|
|
|
import static org.mockito.Mockito.mock;
|
|
|
import static org.mockito.Mockito.spy;
|
|
@@ -48,6 +50,8 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
|
import org.apache.hadoop.yarn.util.BuilderUtils;
|
|
|
import org.junit.Test;
|
|
|
+import org.mockito.Mockito;
|
|
|
+import org.mockito.verification.VerificationMode;
|
|
|
|
|
|
public class TestJobHistoryEventHandler {
|
|
|
|
|
@@ -277,6 +281,68 @@ public class TestJobHistoryEventHandler {
|
|
|
JobId jobId = MRBuilderUtils.newJobId(appId, 1);
|
|
|
AppContext mockAppContext = mockAppContext(jobId);
|
|
|
}
|
|
|
+
|
|
|
+ private JobHistoryEvent getEventToEnqueue(JobId jobId) {
|
|
|
+ JobHistoryEvent toReturn = Mockito.mock(JobHistoryEvent.class);
|
|
|
+ HistoryEvent he = Mockito.mock(HistoryEvent.class);
|
|
|
+ Mockito.when(he.getEventType()).thenReturn(EventType.JOB_STATUS_CHANGED);
|
|
|
+ Mockito.when(toReturn.getHistoryEvent()).thenReturn(he);
|
|
|
+ Mockito.when(toReturn.getJobID()).thenReturn(jobId);
|
|
|
+ return toReturn;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ /**
|
|
|
+ * Tests that in case of SIGTERM, the JHEH stops without processing its event
|
|
|
+ * queue (because we must stop quickly lest we get SIGKILLed) and processes
|
|
|
+ * a JobUnsuccessfulEvent for jobs which were still running (so that they may
|
|
|
+ * show up in the JobHistoryServer)
|
|
|
+ */
|
|
|
+ public void testSigTermedFunctionality() throws IOException {
|
|
|
+ AppContext mockedContext = Mockito.mock(AppContext.class);
|
|
|
+ JHEventHandlerForSigtermTest jheh =
|
|
|
+ new JHEventHandlerForSigtermTest(mockedContext, 0);
|
|
|
+
|
|
|
+ JobId jobId = Mockito.mock(JobId.class);
|
|
|
+ jheh.addToFileMap(jobId);
|
|
|
+
|
|
|
+ //Submit 4 events and check that they're handled in the absence of a signal
|
|
|
+ final int numEvents = 4;
|
|
|
+ JobHistoryEvent events[] = new JobHistoryEvent[numEvents];
|
|
|
+ for(int i=0; i < numEvents; ++i) {
|
|
|
+ events[i] = getEventToEnqueue(jobId);
|
|
|
+ jheh.handle(events[i]);
|
|
|
+ }
|
|
|
+ jheh.stop();
|
|
|
+ //Make sure events were handled
|
|
|
+ assertTrue("handleEvent should've been called only 4 times but was "
|
|
|
+ + jheh.eventsHandled, jheh.eventsHandled == 4);
|
|
|
+
|
|
|
+ //Create a new jheh because the last stop closed the eventWriter etc.
|
|
|
+ jheh = new JHEventHandlerForSigtermTest(mockedContext, 0);
|
|
|
+
|
|
|
+ // Make constructor of JobUnsuccessfulCompletionEvent pass
|
|
|
+ Job job = Mockito.mock(Job.class);
|
|
|
+ Mockito.when(mockedContext.getJob(jobId)).thenReturn(job);
|
|
|
+ // Make TypeConverter(JobID) pass
|
|
|
+ ApplicationId mockAppId = Mockito.mock(ApplicationId.class);
|
|
|
+ Mockito.when(mockAppId.getClusterTimestamp()).thenReturn(1000l);
|
|
|
+ Mockito.when(jobId.getAppId()).thenReturn(mockAppId);
|
|
|
+
|
|
|
+ jheh.addToFileMap(jobId);
|
|
|
+ jheh.setSignalled(true);
|
|
|
+ for(int i=0; i < numEvents; ++i) {
|
|
|
+ events[i] = getEventToEnqueue(jobId);
|
|
|
+ jheh.handle(events[i]);
|
|
|
+ }
|
|
|
+ jheh.stop();
|
|
|
+ //Make sure events were handled, 4 + 1 finish event
|
|
|
+ assertTrue("handleEvent should've been called only 5 times but was "
|
|
|
+ + jheh.eventsHandled, jheh.eventsHandled == 5);
|
|
|
+ assertTrue("Last event handled wasn't JobUnsuccessfulCompletionEvent",
|
|
|
+ jheh.lastEventHandled.getHistoryEvent()
|
|
|
+ instanceof JobUnsuccessfulCompletionEvent);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
class JHEvenHandlerForTest extends JobHistoryEventHandler {
|
|
@@ -307,4 +373,28 @@ class JHEvenHandlerForTest extends JobHistoryEventHandler {
|
|
|
public EventWriter getEventWriter() {
|
|
|
return this.eventWriter;
|
|
|
}
|
|
|
+}
|
|
|
+
|
|
|
+/**
|
|
|
+ * Class to help with testSigTermedFunctionality
|
|
|
+ */
|
|
|
+class JHEventHandlerForSigtermTest extends JobHistoryEventHandler {
|
|
|
+ private MetaInfo metaInfo;
|
|
|
+ public JHEventHandlerForSigtermTest(AppContext context, int startCount) {
|
|
|
+ super(context, startCount);
|
|
|
+ }
|
|
|
+
|
|
|
+ public void addToFileMap(JobId jobId) {
|
|
|
+ metaInfo = Mockito.mock(MetaInfo.class);
|
|
|
+ Mockito.when(metaInfo.isWriterActive()).thenReturn(true);
|
|
|
+ fileMap.put(jobId, metaInfo);
|
|
|
+ }
|
|
|
+
|
|
|
+ JobHistoryEvent lastEventHandled;
|
|
|
+ int eventsHandled = 0;
|
|
|
+ @Override
|
|
|
+ protected void handleEvent(JobHistoryEvent event) {
|
|
|
+ this.lastEventHandled = event;
|
|
|
+ this.eventsHandled++;
|
|
|
+ }
|
|
|
}
|