|
@@ -223,4 +223,74 @@ public class TestShuffleScheduler {
|
|
|
+ " Aggregated copy rate(10 of 10 at 2.00 MB/s)", progress.toString());
|
|
|
|
|
|
}
|
|
|
+
|
|
|
+ @SuppressWarnings("rawtypes")
|
|
|
+ @Test
|
|
|
+ public <K, V> void TestSucceedAndFailedCopyMap() throws Exception {
|
|
|
+ JobConf job = new JobConf();
|
|
|
+ job.setNumMapTasks(2);
|
|
|
+ //mock creation
|
|
|
+ TaskUmbilicalProtocol mockUmbilical = mock(TaskUmbilicalProtocol.class);
|
|
|
+ Reporter mockReporter = mock(Reporter.class);
|
|
|
+ FileSystem mockFileSystem = mock(FileSystem.class);
|
|
|
+ Class<? extends org.apache.hadoop.mapred.Reducer> combinerClass = job.getCombinerClass();
|
|
|
+ @SuppressWarnings("unchecked") // needed for mock with generic
|
|
|
+ CombineOutputCollector<K, V> mockCombineOutputCollector =
|
|
|
+ (CombineOutputCollector<K, V>) mock(CombineOutputCollector.class);
|
|
|
+ org.apache.hadoop.mapreduce.TaskAttemptID mockTaskAttemptID =
|
|
|
+ mock(org.apache.hadoop.mapreduce.TaskAttemptID.class);
|
|
|
+ LocalDirAllocator mockLocalDirAllocator = mock(LocalDirAllocator.class);
|
|
|
+ CompressionCodec mockCompressionCodec = mock(CompressionCodec.class);
|
|
|
+ Counter mockCounter = mock(Counter.class);
|
|
|
+ TaskStatus mockTaskStatus = mock(TaskStatus.class);
|
|
|
+ Progress mockProgress = mock(Progress.class);
|
|
|
+ MapOutputFile mockMapOutputFile = mock(MapOutputFile.class);
|
|
|
+ Task mockTask = mock(Task.class);
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
+ MapOutput<K, V> output = mock(MapOutput.class);
|
|
|
+
|
|
|
+ ShuffleConsumerPlugin.Context<K, V> context =
|
|
|
+ new ShuffleConsumerPlugin.Context<K, V>(
|
|
|
+ mockTaskAttemptID, job, mockFileSystem,
|
|
|
+ mockUmbilical, mockLocalDirAllocator,
|
|
|
+ mockReporter, mockCompressionCodec,
|
|
|
+ combinerClass, mockCombineOutputCollector,
|
|
|
+ mockCounter, mockCounter, mockCounter,
|
|
|
+ mockCounter, mockCounter, mockCounter,
|
|
|
+ mockTaskStatus, mockProgress, mockProgress,
|
|
|
+ mockTask, mockMapOutputFile, null);
|
|
|
+ TaskStatus status = new TaskStatus() {
|
|
|
+ @Override
|
|
|
+ public boolean getIsMap() {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ @Override
|
|
|
+ public void addFetchFailedMap(TaskAttemptID mapTaskId) {
|
|
|
+ }
|
|
|
+ };
|
|
|
+ Progress progress = new Progress();
|
|
|
+ ShuffleSchedulerImpl<K, V> scheduler = new ShuffleSchedulerImpl<K, V>(job,
|
|
|
+ status, null, null, progress, context.getShuffledMapsCounter(),
|
|
|
+ context.getReduceShuffleBytes(), context.getFailedShuffleCounter());
|
|
|
+
|
|
|
+ MapHost host1 = new MapHost("host1", null);
|
|
|
+ TaskAttemptID failedAttemptID = new TaskAttemptID(
|
|
|
+ new org.apache.hadoop.mapred.TaskID(
|
|
|
+ new JobID("test",0), TaskType.MAP, 0), 0);
|
|
|
+
|
|
|
+ TaskAttemptID succeedAttemptID = new TaskAttemptID(
|
|
|
+ new org.apache.hadoop.mapred.TaskID(
|
|
|
+ new JobID("test",0), TaskType.MAP, 1), 1);
|
|
|
+
|
|
|
+ // handle output fetch failure for failedAttemptID, part I
|
|
|
+ scheduler.hostFailed(host1.getHostName());
|
|
|
+
|
|
|
+ // handle output fetch succeed for succeedAttemptID
|
|
|
+ long bytes = (long)500 * 1024 * 1024;
|
|
|
+ scheduler.copySucceeded(succeedAttemptID, host1, bytes, 0, 500000, output);
|
|
|
+
|
|
|
+ // handle output fetch failure for failedAttemptID, part II
|
|
|
+ // for MAPREDUCE-6361: verify no NPE exception get thrown out
|
|
|
+ scheduler.copyFailed(failedAttemptID, host1, true, false);
|
|
|
+ }
|
|
|
}
|