|
@@ -24,6 +24,8 @@ import static org.junit.Assert.assertNull;
|
|
import static org.junit.Assert.assertTrue;
|
|
import static org.junit.Assert.assertTrue;
|
|
import static org.mockito.Matchers.any;
|
|
import static org.mockito.Matchers.any;
|
|
import static org.mockito.Mockito.mock;
|
|
import static org.mockito.Mockito.mock;
|
|
|
|
+import static org.mockito.Mockito.never;
|
|
|
|
+import static org.mockito.Mockito.times;
|
|
import static org.mockito.Mockito.verify;
|
|
import static org.mockito.Mockito.verify;
|
|
import static org.mockito.Mockito.when;
|
|
import static org.mockito.Mockito.when;
|
|
|
|
|
|
@@ -41,7 +43,9 @@ import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
|
|
import org.apache.hadoop.mapreduce.v2.app.AppContext;
|
|
import org.apache.hadoop.mapreduce.v2.app.AppContext;
|
|
import org.apache.hadoop.mapreduce.v2.app.TaskHeartbeatHandler;
|
|
import org.apache.hadoop.mapreduce.v2.app.TaskHeartbeatHandler;
|
|
import org.apache.hadoop.mapreduce.v2.app.job.Job;
|
|
import org.apache.hadoop.mapreduce.v2.app.job.Job;
|
|
|
|
+import org.apache.hadoop.mapreduce.v2.app.rm.RMHeartbeatHandler;
|
|
import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
|
|
import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
|
|
|
|
+import org.apache.hadoop.yarn.SystemClock;
|
|
import org.apache.hadoop.yarn.factories.RecordFactory;
|
|
import org.apache.hadoop.yarn.factories.RecordFactory;
|
|
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
|
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
|
import org.junit.Test;
|
|
import org.junit.Test;
|
|
@@ -51,8 +55,9 @@ public class TestTaskAttemptListenerImpl {
|
|
|
|
|
|
public MockTaskAttemptListenerImpl(AppContext context,
|
|
public MockTaskAttemptListenerImpl(AppContext context,
|
|
JobTokenSecretManager jobTokenSecretManager,
|
|
JobTokenSecretManager jobTokenSecretManager,
|
|
|
|
+ RMHeartbeatHandler rmHeartbeatHandler,
|
|
TaskHeartbeatHandler hbHandler) {
|
|
TaskHeartbeatHandler hbHandler) {
|
|
- super(context, jobTokenSecretManager);
|
|
|
|
|
|
+ super(context, jobTokenSecretManager, rmHeartbeatHandler);
|
|
this.taskHeartbeatHandler = hbHandler;
|
|
this.taskHeartbeatHandler = hbHandler;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -76,9 +81,12 @@ public class TestTaskAttemptListenerImpl {
|
|
public void testGetTask() throws IOException {
|
|
public void testGetTask() throws IOException {
|
|
AppContext appCtx = mock(AppContext.class);
|
|
AppContext appCtx = mock(AppContext.class);
|
|
JobTokenSecretManager secret = mock(JobTokenSecretManager.class);
|
|
JobTokenSecretManager secret = mock(JobTokenSecretManager.class);
|
|
|
|
+ RMHeartbeatHandler rmHeartbeatHandler =
|
|
|
|
+ mock(RMHeartbeatHandler.class);
|
|
TaskHeartbeatHandler hbHandler = mock(TaskHeartbeatHandler.class);
|
|
TaskHeartbeatHandler hbHandler = mock(TaskHeartbeatHandler.class);
|
|
MockTaskAttemptListenerImpl listener =
|
|
MockTaskAttemptListenerImpl listener =
|
|
- new MockTaskAttemptListenerImpl(appCtx, secret, hbHandler);
|
|
|
|
|
|
+ new MockTaskAttemptListenerImpl(appCtx, secret,
|
|
|
|
+ rmHeartbeatHandler, hbHandler);
|
|
Configuration conf = new Configuration();
|
|
Configuration conf = new Configuration();
|
|
listener.init(conf);
|
|
listener.init(conf);
|
|
listener.start();
|
|
listener.start();
|
|
@@ -152,9 +160,11 @@ public class TestTaskAttemptListenerImpl {
|
|
AppContext appCtx = mock(AppContext.class);
|
|
AppContext appCtx = mock(AppContext.class);
|
|
when(appCtx.getJob(any(JobId.class))).thenReturn(mockJob);
|
|
when(appCtx.getJob(any(JobId.class))).thenReturn(mockJob);
|
|
JobTokenSecretManager secret = mock(JobTokenSecretManager.class);
|
|
JobTokenSecretManager secret = mock(JobTokenSecretManager.class);
|
|
|
|
+ RMHeartbeatHandler rmHeartbeatHandler =
|
|
|
|
+ mock(RMHeartbeatHandler.class);
|
|
final TaskHeartbeatHandler hbHandler = mock(TaskHeartbeatHandler.class);
|
|
final TaskHeartbeatHandler hbHandler = mock(TaskHeartbeatHandler.class);
|
|
TaskAttemptListenerImpl listener =
|
|
TaskAttemptListenerImpl listener =
|
|
- new TaskAttemptListenerImpl(appCtx, secret) {
|
|
|
|
|
|
+ new TaskAttemptListenerImpl(appCtx, secret, rmHeartbeatHandler) {
|
|
@Override
|
|
@Override
|
|
protected void registerHeartbeatHandler(Configuration conf) {
|
|
protected void registerHeartbeatHandler(Configuration conf) {
|
|
taskHeartbeatHandler = hbHandler;
|
|
taskHeartbeatHandler = hbHandler;
|
|
@@ -191,4 +201,46 @@ public class TestTaskAttemptListenerImpl {
|
|
return tce;
|
|
return tce;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ @Test
|
|
|
|
+ public void testCommitWindow() throws IOException {
|
|
|
|
+ SystemClock clock = new SystemClock();
|
|
|
|
+
|
|
|
|
+ org.apache.hadoop.mapreduce.v2.app.job.Task mockTask =
|
|
|
|
+ mock(org.apache.hadoop.mapreduce.v2.app.job.Task.class);
|
|
|
|
+ when(mockTask.canCommit(any(TaskAttemptId.class))).thenReturn(true);
|
|
|
|
+ Job mockJob = mock(Job.class);
|
|
|
|
+ when(mockJob.getTask(any(TaskId.class))).thenReturn(mockTask);
|
|
|
|
+ AppContext appCtx = mock(AppContext.class);
|
|
|
|
+ when(appCtx.getJob(any(JobId.class))).thenReturn(mockJob);
|
|
|
|
+ when(appCtx.getClock()).thenReturn(clock);
|
|
|
|
+ JobTokenSecretManager secret = mock(JobTokenSecretManager.class);
|
|
|
|
+ RMHeartbeatHandler rmHeartbeatHandler =
|
|
|
|
+ mock(RMHeartbeatHandler.class);
|
|
|
|
+ final TaskHeartbeatHandler hbHandler = mock(TaskHeartbeatHandler.class);
|
|
|
|
+ TaskAttemptListenerImpl listener =
|
|
|
|
+ new TaskAttemptListenerImpl(appCtx, secret, rmHeartbeatHandler) {
|
|
|
|
+ @Override
|
|
|
|
+ protected void registerHeartbeatHandler(Configuration conf) {
|
|
|
|
+ taskHeartbeatHandler = hbHandler;
|
|
|
|
+ }
|
|
|
|
+ };
|
|
|
|
+
|
|
|
|
+ Configuration conf = new Configuration();
|
|
|
|
+ listener.init(conf);
|
|
|
|
+ listener.start();
|
|
|
|
+
|
|
|
|
+ // verify commit not allowed when RM heartbeat has not occurred recently
|
|
|
|
+ TaskAttemptID tid = new TaskAttemptID("12345", 1, TaskType.REDUCE, 1, 0);
|
|
|
|
+ boolean canCommit = listener.canCommit(tid);
|
|
|
|
+ assertFalse(canCommit);
|
|
|
|
+ verify(mockTask, never()).canCommit(any(TaskAttemptId.class));
|
|
|
|
+
|
|
|
|
+ // verify commit allowed when RM heartbeat is recent
|
|
|
|
+ when(rmHeartbeatHandler.getLastHeartbeatTime()).thenReturn(clock.getTime());
|
|
|
|
+ canCommit = listener.canCommit(tid);
|
|
|
|
+ assertTrue(canCommit);
|
|
|
|
+ verify(mockTask, times(1)).canCommit(any(TaskAttemptId.class));
|
|
|
|
+
|
|
|
|
+ listener.stop();
|
|
|
|
+ }
|
|
}
|
|
}
|