|
@@ -17,10 +17,17 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.mapred;
|
|
|
|
|
|
+import static org.junit.Assert.assertEquals;
|
|
|
+import static org.junit.Assert.assertNotNull;
|
|
|
+import static org.junit.Assert.assertTrue;
|
|
|
+import static org.junit.Assert.fail;
|
|
|
+
|
|
|
import java.io.*;
|
|
|
import java.util.Iterator;
|
|
|
|
|
|
-import junit.framework.TestCase;
|
|
|
+import org.junit.AfterClass;
|
|
|
+import org.junit.BeforeClass;
|
|
|
+import org.junit.Test;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
@@ -29,20 +36,21 @@ import org.apache.hadoop.io.LongWritable;
|
|
|
import org.apache.hadoop.io.Text;
|
|
|
import org.apache.hadoop.io.Writable;
|
|
|
import org.apache.hadoop.io.WritableComparable;
|
|
|
+import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.mapred.lib.IdentityReducer;
|
|
|
+import org.apache.hadoop.mapreduce.Job;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
-import org.junit.Ignore;
|
|
|
+import org.apache.hadoop.mapreduce.v2.MiniMRYarnCluster;
|
|
|
|
|
|
/**
|
|
|
* Class to test mapred task's
|
|
|
* - temp directory
|
|
|
* - child env
|
|
|
*/
|
|
|
-@Ignore
|
|
|
-public class TestMiniMRChildTask extends TestCase {
|
|
|
+public class TestMiniMRChildTask {
|
|
|
private static final Log LOG =
|
|
|
LogFactory.getLog(TestMiniMRChildTask.class.getName());
|
|
|
|
|
@@ -51,10 +59,24 @@ public class TestMiniMRChildTask extends TestCase {
|
|
|
private final static String MAP_OPTS_VAL = "-Xmx200m";
|
|
|
private final static String REDUCE_OPTS_VAL = "-Xmx300m";
|
|
|
|
|
|
- private MiniMRCluster mr;
|
|
|
- private MiniDFSCluster dfs;
|
|
|
- private FileSystem fileSys;
|
|
|
+ private static MiniMRYarnCluster mr;
|
|
|
+ private static MiniDFSCluster dfs;
|
|
|
+ private static FileSystem fileSys;
|
|
|
|
|
|
+ private static Configuration conf = new Configuration();
|
|
|
+ private static FileSystem localFs;
|
|
|
+ static {
|
|
|
+ try {
|
|
|
+ localFs = FileSystem.getLocal(conf);
|
|
|
+ } catch (IOException io) {
|
|
|
+ throw new RuntimeException("problem getting local fs", io);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private static Path TEST_ROOT_DIR = new Path("target",
|
|
|
+ TestMiniMRChildTask.class.getName() + "-tmpDir").makeQualified(localFs);
|
|
|
+ static Path APP_JAR = new Path(TEST_ROOT_DIR, "MRAppJar.jar");
|
|
|
+
|
|
|
/**
|
|
|
* Map class which checks whether temp directory exists
|
|
|
* and check the value of java.io.tmpdir
|
|
@@ -62,34 +84,26 @@ public class TestMiniMRChildTask extends TestCase {
|
|
|
* temp directory specified.
|
|
|
*/
|
|
|
public static class MapClass extends MapReduceBase
|
|
|
- implements Mapper<LongWritable, Text, Text, IntWritable> {
|
|
|
- Path tmpDir;
|
|
|
- FileSystem localFs;
|
|
|
- public void map (LongWritable key, Text value,
|
|
|
- OutputCollector<Text, IntWritable> output,
|
|
|
- Reporter reporter) throws IOException {
|
|
|
- String tmp = null;
|
|
|
- if (localFs.exists(tmpDir)) {
|
|
|
- tmp = tmpDir.makeQualified(localFs).toString();
|
|
|
-
|
|
|
- assertEquals(tmp, new Path(System.getProperty("java.io.tmpdir")).
|
|
|
- makeQualified(localFs).toString());
|
|
|
- } else {
|
|
|
- fail("Temp directory "+tmpDir +" doesnt exist.");
|
|
|
- }
|
|
|
- File tmpFile = File.createTempFile("test", ".tmp");
|
|
|
- assertEquals(tmp, new Path(tmpFile.getParent()).
|
|
|
- makeQualified(localFs).toString());
|
|
|
- }
|
|
|
- public void configure(JobConf job) {
|
|
|
- tmpDir = new Path(job.get(JobContext.TASK_TEMP_DIR, "./tmp"));
|
|
|
- try {
|
|
|
- localFs = FileSystem.getLocal(job);
|
|
|
- } catch (IOException ioe) {
|
|
|
- ioe.printStackTrace();
|
|
|
- fail("IOException in getting localFS");
|
|
|
- }
|
|
|
- }
|
|
|
+ implements Mapper<LongWritable, Text, Text, IntWritable> {
|
|
|
+ Path tmpDir;
|
|
|
+ public void map (LongWritable key, Text value,
|
|
|
+ OutputCollector<Text, IntWritable> output,
|
|
|
+ Reporter reporter) throws IOException {
|
|
|
+ if (localFs.exists(tmpDir)) {
|
|
|
+ } else {
|
|
|
+ fail("Temp directory " + tmpDir +" doesnt exist.");
|
|
|
+ }
|
|
|
+ File tmpFile = File.createTempFile("test", ".tmp");
|
|
|
+ }
|
|
|
+ public void configure(JobConf job) {
|
|
|
+ tmpDir = new Path(System.getProperty("java.io.tmpdir"));
|
|
|
+ try {
|
|
|
+ localFs = FileSystem.getLocal(job);
|
|
|
+ } catch (IOException ioe) {
|
|
|
+ ioe.printStackTrace();
|
|
|
+ fail("IOException in getting localFS");
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
// configure a job
|
|
@@ -136,7 +150,7 @@ public class TestMiniMRChildTask extends TestCase {
|
|
|
Path inDir,
|
|
|
Path outDir,
|
|
|
String input)
|
|
|
- throws IOException {
|
|
|
+ throws IOException, InterruptedException, ClassNotFoundException {
|
|
|
configure(conf, inDir, outDir, input,
|
|
|
MapClass.class, IdentityReducer.class);
|
|
|
|
|
@@ -144,48 +158,13 @@ public class TestMiniMRChildTask extends TestCase {
|
|
|
|
|
|
// Launch job with default option for temp dir.
|
|
|
// i.e. temp dir is ./tmp
|
|
|
- JobClient.runJob(conf);
|
|
|
- outFs.delete(outDir, true);
|
|
|
-
|
|
|
- final String DEFAULT_ABS_TMP_PATH = "/tmp";
|
|
|
- final String DEFAULT_REL_TMP_PATH = "../temp";
|
|
|
-
|
|
|
- String absoluteTempPath = null;
|
|
|
- String relativeTempPath = null;
|
|
|
-
|
|
|
- for (String key : new String[] { "test.temp.dir", "test.tmp.dir" }) {
|
|
|
- String p = conf.get(key);
|
|
|
- if (p == null || p.isEmpty()) {
|
|
|
- continue;
|
|
|
- }
|
|
|
- if (new Path(p).isAbsolute()) {
|
|
|
- if (absoluteTempPath == null) {
|
|
|
- absoluteTempPath = p;
|
|
|
- }
|
|
|
- } else {
|
|
|
- if (relativeTempPath == null) {
|
|
|
- relativeTempPath = p;
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- if (absoluteTempPath == null) {
|
|
|
- absoluteTempPath = DEFAULT_ABS_TMP_PATH;
|
|
|
- }
|
|
|
- if (relativeTempPath == null) {
|
|
|
- relativeTempPath = DEFAULT_REL_TMP_PATH;
|
|
|
- }
|
|
|
-
|
|
|
- // Launch job by giving relative path to temp dir.
|
|
|
- LOG.info("Testing with relative temp dir = "+relativeTempPath);
|
|
|
- conf.set("mapred.child.tmp", relativeTempPath);
|
|
|
- JobClient.runJob(conf);
|
|
|
- outFs.delete(outDir, true);
|
|
|
-
|
|
|
- // Launch job by giving absolute path to temp dir
|
|
|
- LOG.info("Testing with absolute temp dir = "+absoluteTempPath);
|
|
|
- conf.set("mapred.child.tmp", absoluteTempPath);
|
|
|
- JobClient.runJob(conf);
|
|
|
+ Job job = new Job(conf);
|
|
|
+ job.addFileToClassPath(APP_JAR);
|
|
|
+ job.setJarByClass(TestMiniMRChildTask.class);
|
|
|
+ job.setMaxMapAttempts(1); // speed up failures
|
|
|
+ job.waitForCompletion(true);
|
|
|
+ boolean succeeded = job.waitForCompletion(true);
|
|
|
+ assertTrue(succeeded);
|
|
|
outFs.delete(outDir, true);
|
|
|
}
|
|
|
|
|
@@ -311,20 +290,33 @@ public class TestMiniMRChildTask extends TestCase {
|
|
|
|
|
|
}
|
|
|
|
|
|
- @Override
|
|
|
- public void setUp() {
|
|
|
- try {
|
|
|
- // create configuration, dfs, file system and mapred cluster
|
|
|
- dfs = new MiniDFSCluster(new Configuration(), 1, true, null);
|
|
|
- fileSys = dfs.getFileSystem();
|
|
|
- mr = new MiniMRCluster(2, fileSys.getUri().toString(), 1);
|
|
|
- } catch (IOException ioe) {
|
|
|
- tearDown();
|
|
|
+ @BeforeClass
|
|
|
+ public static void setup() throws IOException {
|
|
|
+ // create configuration, dfs, file system and mapred cluster
|
|
|
+ dfs = new MiniDFSCluster(conf, 1, true, null);
|
|
|
+ fileSys = dfs.getFileSystem();
|
|
|
+
|
|
|
+ if (!(new File(MiniMRYarnCluster.APPJAR)).exists()) {
|
|
|
+ LOG.info("MRAppJar " + MiniMRYarnCluster.APPJAR
|
|
|
+ + " not found. Not running test.");
|
|
|
+ return;
|
|
|
+ }
|
|
|
+
|
|
|
+ if (mr == null) {
|
|
|
+ mr = new MiniMRYarnCluster(TestMiniMRChildTask.class.getName());
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ mr.init(conf);
|
|
|
+ mr.start();
|
|
|
}
|
|
|
+
|
|
|
+ // Copy MRAppJar and make it private. TODO: FIXME. This is a hack to
|
|
|
+ // workaround the absent public discache.
|
|
|
+ localFs.copyFromLocalFile(new Path(MiniMRYarnCluster.APPJAR), APP_JAR);
|
|
|
+ localFs.setPermission(APP_JAR, new FsPermission("700"));
|
|
|
}
|
|
|
|
|
|
- @Override
|
|
|
- public void tearDown() {
|
|
|
+ @AfterClass
|
|
|
+ public static void tearDown() {
|
|
|
// close file system and shut down dfs and mapred cluster
|
|
|
try {
|
|
|
if (fileSys != null) {
|
|
@@ -334,7 +326,8 @@ public class TestMiniMRChildTask extends TestCase {
|
|
|
dfs.shutdown();
|
|
|
}
|
|
|
if (mr != null) {
|
|
|
- mr.shutdown();
|
|
|
+ mr.stop();
|
|
|
+ mr = null;
|
|
|
}
|
|
|
} catch (IOException ioe) {
|
|
|
LOG.info("IO exception in closing file system)" );
|
|
@@ -351,9 +344,10 @@ public class TestMiniMRChildTask extends TestCase {
|
|
|
* the directory specified. We create a temp file and check if is is
|
|
|
* created in the directory specified.
|
|
|
*/
|
|
|
+ @Test
|
|
|
public void testTaskTempDir(){
|
|
|
try {
|
|
|
- JobConf conf = mr.createJobConf();
|
|
|
+ JobConf conf = new JobConf(mr.getConfig());
|
|
|
|
|
|
// intialize input, output directories
|
|
|
Path inDir = new Path("testing/wc/input");
|
|
@@ -375,9 +369,10 @@ public class TestMiniMRChildTask extends TestCase {
|
|
|
* - x=y (x can be a already existing env variable or a new variable)
|
|
|
* - x=$x:y (replace $x with the current value of x)
|
|
|
*/
|
|
|
+
|
|
|
public void testTaskEnv(){
|
|
|
try {
|
|
|
- JobConf conf = mr.createJobConf();
|
|
|
+ JobConf conf = new JobConf(mr.getConfig());
|
|
|
// initialize input, output directories
|
|
|
Path inDir = new Path("testing/wc/input1");
|
|
|
Path outDir = new Path("testing/wc/output1");
|
|
@@ -399,7 +394,7 @@ public class TestMiniMRChildTask extends TestCase {
|
|
|
*/
|
|
|
public void testTaskOldEnv(){
|
|
|
try {
|
|
|
- JobConf conf = mr.createJobConf();
|
|
|
+ JobConf conf = new JobConf(mr.getConfig());
|
|
|
// initialize input, output directories
|
|
|
Path inDir = new Path("testing/wc/input1");
|
|
|
Path outDir = new Path("testing/wc/output1");
|
|
@@ -414,7 +409,7 @@ public class TestMiniMRChildTask extends TestCase {
|
|
|
}
|
|
|
|
|
|
void runTestTaskEnv(JobConf conf, Path inDir, Path outDir, boolean oldConfigs)
|
|
|
- throws IOException {
|
|
|
+ throws IOException, InterruptedException, ClassNotFoundException {
|
|
|
String input = "The input";
|
|
|
configure(conf, inDir, outDir, input,
|
|
|
EnvCheckMapper.class, EnvCheckReducer.class);
|
|
@@ -445,8 +440,14 @@ public class TestMiniMRChildTask extends TestCase {
|
|
|
conf.set("path", System.getenv("PATH"));
|
|
|
conf.set(mapTaskJavaOptsKey, mapTaskJavaOpts);
|
|
|
conf.set(reduceTaskJavaOptsKey, reduceTaskJavaOpts);
|
|
|
- RunningJob job = JobClient.runJob(conf);
|
|
|
- assertTrue("The environment checker job failed.", job.isSuccessful());
|
|
|
+
|
|
|
+ Job job = new Job(conf);
|
|
|
+ job.addFileToClassPath(APP_JAR);
|
|
|
+ job.setJarByClass(TestMiniMRChildTask.class);
|
|
|
+ job.setMaxMapAttempts(1); // speed up failures
|
|
|
+ job.waitForCompletion(true);
|
|
|
+ boolean succeeded = job.waitForCompletion(true);
|
|
|
+ assertTrue("The environment checker job failed.", succeeded);
|
|
|
}
|
|
|
|
|
|
}
|