|
@@ -18,17 +18,26 @@
|
|
|
|
|
|
package org.apache.hadoop.mapred;
|
|
package org.apache.hadoop.mapred;
|
|
|
|
|
|
|
|
+import static org.apache.hadoop.fs.CreateFlag.CREATE;
|
|
import static org.mockito.Matchers.isA;
|
|
import static org.mockito.Matchers.isA;
|
|
import static org.mockito.Mockito.doAnswer;
|
|
import static org.mockito.Mockito.doAnswer;
|
|
import static org.mockito.Mockito.mock;
|
|
import static org.mockito.Mockito.mock;
|
|
import static org.mockito.Mockito.when;
|
|
import static org.mockito.Mockito.when;
|
|
|
|
|
|
|
|
+import java.io.File;
|
|
|
|
+import java.io.IOException;
|
|
|
|
+import java.util.EnumSet;
|
|
import java.util.HashMap;
|
|
import java.util.HashMap;
|
|
import java.util.Map;
|
|
import java.util.Map;
|
|
import java.util.concurrent.CountDownLatch;
|
|
import java.util.concurrent.CountDownLatch;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.LogFactory;
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
|
+import org.apache.hadoop.conf.Configuration;
|
|
|
|
+import org.apache.hadoop.fs.FileContext;
|
|
|
|
+import org.apache.hadoop.fs.FileSystem;
|
|
|
|
+import org.apache.hadoop.fs.Path;
|
|
|
|
+import org.apache.hadoop.mapreduce.MRConfig;
|
|
import org.apache.hadoop.mapreduce.TypeConverter;
|
|
import org.apache.hadoop.mapreduce.TypeConverter;
|
|
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
|
|
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
|
|
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
|
|
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
|
|
@@ -46,6 +55,9 @@ import org.apache.hadoop.yarn.api.records.Container;
|
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
|
import org.apache.hadoop.yarn.event.Event;
|
|
import org.apache.hadoop.yarn.event.Event;
|
|
import org.apache.hadoop.yarn.event.EventHandler;
|
|
import org.apache.hadoop.yarn.event.EventHandler;
|
|
|
|
+import org.junit.AfterClass;
|
|
|
|
+import org.junit.Assert;
|
|
|
|
+import org.junit.BeforeClass;
|
|
import org.junit.Test;
|
|
import org.junit.Test;
|
|
import org.mockito.invocation.InvocationOnMock;
|
|
import org.mockito.invocation.InvocationOnMock;
|
|
import org.mockito.stubbing.Answer;
|
|
import org.mockito.stubbing.Answer;
|
|
@@ -53,6 +65,36 @@ import org.mockito.stubbing.Answer;
|
|
public class TestLocalContainerLauncher {
|
|
public class TestLocalContainerLauncher {
|
|
private static final Log LOG =
|
|
private static final Log LOG =
|
|
LogFactory.getLog(TestLocalContainerLauncher.class);
|
|
LogFactory.getLog(TestLocalContainerLauncher.class);
|
|
|
|
+ private static File testWorkDir;
|
|
|
|
+ private static final String[] localDirs = new String[2];
|
|
|
|
+
|
|
|
|
+ private static void delete(File dir) throws IOException {
|
|
|
|
+ Configuration conf = new Configuration();
|
|
|
|
+ FileSystem fs = FileSystem.getLocal(conf);
|
|
|
|
+ Path p = fs.makeQualified(new Path(dir.getAbsolutePath()));
|
|
|
|
+ fs.delete(p, true);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @BeforeClass
|
|
|
|
+ public static void setupTestDirs() throws IOException {
|
|
|
|
+ testWorkDir = new File("target",
|
|
|
|
+ TestLocalContainerLauncher.class.getCanonicalName());
|
|
|
|
+ testWorkDir.delete();
|
|
|
|
+ testWorkDir.mkdirs();
|
|
|
|
+ testWorkDir = testWorkDir.getAbsoluteFile();
|
|
|
|
+ for (int i = 0; i < localDirs.length; i++) {
|
|
|
|
+ final File dir = new File(testWorkDir, "local-" + i);
|
|
|
|
+ dir.mkdirs();
|
|
|
|
+ localDirs[i] = dir.toString();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @AfterClass
|
|
|
|
+ public static void cleanupTestDirs() throws IOException {
|
|
|
|
+ if (testWorkDir != null) {
|
|
|
|
+ delete(testWorkDir);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
|
|
@SuppressWarnings("rawtypes")
|
|
@SuppressWarnings("rawtypes")
|
|
@Test(timeout=10000)
|
|
@Test(timeout=10000)
|
|
@@ -141,4 +183,35 @@ public class TestLocalContainerLauncher {
|
|
when(container.getNodeId()).thenReturn(nodeId);
|
|
when(container.getNodeId()).thenReturn(nodeId);
|
|
return container;
|
|
return container;
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+
|
|
|
|
+ @Test
|
|
|
|
+ public void testRenameMapOutputForReduce() throws Exception {
|
|
|
|
+ final JobConf conf = new JobConf();
|
|
|
|
+
|
|
|
|
+ final MROutputFiles mrOutputFiles = new MROutputFiles();
|
|
|
|
+ mrOutputFiles.setConf(conf);
|
|
|
|
+
|
|
|
|
+ // make sure both dirs are distinct
|
|
|
|
+ //
|
|
|
|
+ conf.set(MRConfig.LOCAL_DIR, localDirs[0].toString());
|
|
|
|
+ final Path mapOut = mrOutputFiles.getOutputFileForWrite(1);
|
|
|
|
+ conf.set(MRConfig.LOCAL_DIR, localDirs[1].toString());
|
|
|
|
+ final Path mapOutIdx = mrOutputFiles.getOutputIndexFileForWrite(1);
|
|
|
|
+ Assert.assertNotEquals("Paths must be different!",
|
|
|
|
+ mapOut.getParent(), mapOutIdx.getParent());
|
|
|
|
+
|
|
|
|
+ // make both dirs part of LOCAL_DIR
|
|
|
|
+ conf.setStrings(MRConfig.LOCAL_DIR, localDirs);
|
|
|
|
+
|
|
|
|
+ final FileContext lfc = FileContext.getLocalFSFileContext(conf);
|
|
|
|
+ lfc.create(mapOut, EnumSet.of(CREATE)).close();
|
|
|
|
+ lfc.create(mapOutIdx, EnumSet.of(CREATE)).close();
|
|
|
|
+
|
|
|
|
+ final JobId jobId = MRBuilderUtils.newJobId(12345L, 1, 2);
|
|
|
|
+ final TaskId tid = MRBuilderUtils.newTaskId(jobId, 0, TaskType.MAP);
|
|
|
|
+ final TaskAttemptId taid = MRBuilderUtils.newTaskAttemptId(tid, 0);
|
|
|
|
+
|
|
|
|
+ LocalContainerLauncher.renameMapOutputForReduce(conf, taid, mrOutputFiles);
|
|
|
|
+ }
|
|
}
|
|
}
|