|
@@ -25,9 +25,7 @@ import static org.junit.Assert.fail;
|
|
import java.io.DataOutputStream;
|
|
import java.io.DataOutputStream;
|
|
import java.io.File;
|
|
import java.io.File;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
-import java.util.HashMap;
|
|
|
|
import java.util.Iterator;
|
|
import java.util.Iterator;
|
|
-import java.util.Map;
|
|
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.LogFactory;
|
|
import org.apache.commons.logging.LogFactory;
|
|
@@ -41,11 +39,9 @@ import org.apache.hadoop.io.LongWritable;
|
|
import org.apache.hadoop.io.Text;
|
|
import org.apache.hadoop.io.Text;
|
|
import org.apache.hadoop.io.Writable;
|
|
import org.apache.hadoop.io.Writable;
|
|
import org.apache.hadoop.io.WritableComparable;
|
|
import org.apache.hadoop.io.WritableComparable;
|
|
-import org.apache.hadoop.mapred.lib.IdentityReducer;
|
|
|
|
import org.apache.hadoop.mapreduce.Job;
|
|
import org.apache.hadoop.mapreduce.Job;
|
|
import org.apache.hadoop.mapreduce.MRJobConfig;
|
|
import org.apache.hadoop.mapreduce.MRJobConfig;
|
|
import org.apache.hadoop.mapreduce.v2.MiniMRYarnCluster;
|
|
import org.apache.hadoop.mapreduce.v2.MiniMRYarnCluster;
|
|
-import org.apache.hadoop.mapreduce.v2.util.MRApps;
|
|
|
|
import org.apache.hadoop.util.Shell;
|
|
import org.apache.hadoop.util.Shell;
|
|
import org.junit.AfterClass;
|
|
import org.junit.AfterClass;
|
|
import org.junit.BeforeClass;
|
|
import org.junit.BeforeClass;
|
|
@@ -111,29 +107,6 @@ public class TestMiniMRChildTask {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Map class which checks if hadoop lib location
|
|
|
|
- * is in the execution path
|
|
|
|
- */
|
|
|
|
- public static class ExecutionEnvCheckMapClass extends MapReduceBase
|
|
|
|
- implements Mapper<LongWritable, Text, Text, IntWritable> {
|
|
|
|
- public void map (LongWritable key, Text value,
|
|
|
|
- OutputCollector<Text, IntWritable> output,
|
|
|
|
- Reporter reporter) throws IOException {
|
|
|
|
- }
|
|
|
|
- public void configure(JobConf job) {
|
|
|
|
- String executionEnvPathVariable = System.getenv(Shell.WINDOWS ? "PATH"
|
|
|
|
- : "LD_LIBRARY_PATH");
|
|
|
|
- String hadoopHome = System.getenv("HADOOP_COMMON_HOME");
|
|
|
|
- if (hadoopHome == null) {
|
|
|
|
- hadoopHome = "";
|
|
|
|
- }
|
|
|
|
- String hadoopLibLocation = hadoopHome
|
|
|
|
- + (Shell.WINDOWS ? "\\bin" : "/lib/native");
|
|
|
|
- assertTrue(executionEnvPathVariable.contains(hadoopLibLocation));
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
|
|
|
|
// configure a job
|
|
// configure a job
|
|
private void configure(JobConf conf, Path inDir, Path outDir, String input,
|
|
private void configure(JobConf conf, Path inDir, Path outDir, String input,
|
|
@@ -167,57 +140,31 @@ public class TestMiniMRChildTask {
|
|
conf.set("test.build.data", TEST_ROOT_DIR);
|
|
conf.set("test.build.data", TEST_ROOT_DIR);
|
|
}
|
|
}
|
|
|
|
|
|
- /**
|
|
|
|
- * Launch tests
|
|
|
|
- * @param conf Configuration of the mapreduce job.
|
|
|
|
- * @param inDir input path
|
|
|
|
- * @param outDir output path
|
|
|
|
- * @param input Input text
|
|
|
|
- * @throws IOException
|
|
|
|
- */
|
|
|
|
- public void launchTest(JobConf conf,
|
|
|
|
- Path inDir,
|
|
|
|
- Path outDir,
|
|
|
|
- String input)
|
|
|
|
- throws IOException, InterruptedException, ClassNotFoundException {
|
|
|
|
-
|
|
|
|
- FileSystem outFs = outDir.getFileSystem(conf);
|
|
|
|
-
|
|
|
|
- // Launch job with default option for temp dir.
|
|
|
|
- // i.e. temp dir is ./tmp
|
|
|
|
- Job job = Job.getInstance(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);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
private static void checkEnv(String envName, String expValue, String mode) {
|
|
private static void checkEnv(String envName, String expValue, String mode) {
|
|
- String envValue = System.getenv(envName).trim();
|
|
|
|
|
|
+ String envValue = System.getenv(envName);
|
|
if ("append".equals(mode)) {
|
|
if ("append".equals(mode)) {
|
|
if (envValue == null || !envValue.contains(File.pathSeparator)) {
|
|
if (envValue == null || !envValue.contains(File.pathSeparator)) {
|
|
throw new RuntimeException("Missing env variable");
|
|
throw new RuntimeException("Missing env variable");
|
|
} else {
|
|
} else {
|
|
- String parts[] = envValue.split(File.pathSeparator);
|
|
|
|
|
|
+ String[] parts = envValue.trim().split(File.pathSeparator);
|
|
// check if the value is appended
|
|
// check if the value is appended
|
|
if (!parts[parts.length - 1].equals(expValue)) {
|
|
if (!parts[parts.length - 1].equals(expValue)) {
|
|
throw new RuntimeException("Wrong env variable in append mode");
|
|
throw new RuntimeException("Wrong env variable in append mode");
|
|
}
|
|
}
|
|
}
|
|
}
|
|
} else {
|
|
} else {
|
|
- if (envValue == null || !envValue.equals(expValue)) {
|
|
|
|
|
|
+ if (envValue == null || !envValue.trim().equals(expValue)) {
|
|
throw new RuntimeException("Wrong env variable in noappend mode");
|
|
throw new RuntimeException("Wrong env variable in noappend mode");
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
// Mappers that simply checks if the desired user env are present or not
|
|
// Mappers that simply checks if the desired user env are present or not
|
|
- static class EnvCheckMapper extends MapReduceBase implements
|
|
|
|
|
|
+ private static class EnvCheckMapper extends MapReduceBase implements
|
|
Mapper<WritableComparable, Writable, WritableComparable, Writable> {
|
|
Mapper<WritableComparable, Writable, WritableComparable, Writable> {
|
|
-
|
|
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ @SuppressWarnings("deprecation")
|
|
public void configure(JobConf job) {
|
|
public void configure(JobConf job) {
|
|
boolean oldConfigs = job.getBoolean(OLD_CONFIGS, false);
|
|
boolean oldConfigs = job.getBoolean(OLD_CONFIGS, false);
|
|
if (oldConfigs) {
|
|
if (oldConfigs) {
|
|
@@ -236,38 +183,12 @@ public class TestMiniMRChildTask {
|
|
mapJavaOpts, MAP_OPTS_VAL);
|
|
mapJavaOpts, MAP_OPTS_VAL);
|
|
}
|
|
}
|
|
|
|
|
|
- String path = job.get("path");
|
|
|
|
-
|
|
|
|
- // check if the pwd is there in LD_LIBRARY_PATH
|
|
|
|
- String pwd = System.getenv("PWD");
|
|
|
|
-
|
|
|
|
- assertTrue("LD doesnt contain pwd",
|
|
|
|
- System.getenv("LD_LIBRARY_PATH").contains(pwd));
|
|
|
|
-
|
|
|
|
- // check if X=$X:/abc works for LD_LIBRARY_PATH
|
|
|
|
- checkEnv("LD_LIBRARY_PATH", "/tmp", "append");
|
|
|
|
// check if X=y works for an already existing parameter
|
|
// check if X=y works for an already existing parameter
|
|
checkEnv("LANG", "en_us_8859_1", "noappend");
|
|
checkEnv("LANG", "en_us_8859_1", "noappend");
|
|
// check if X=/tmp for a new env variable
|
|
// check if X=/tmp for a new env variable
|
|
checkEnv("MY_PATH", "/tmp", "noappend");
|
|
checkEnv("MY_PATH", "/tmp", "noappend");
|
|
// check if X=$X:/tmp works for a new env var and results into :/tmp
|
|
// check if X=$X:/tmp works for a new env var and results into :/tmp
|
|
checkEnv("NEW_PATH", File.pathSeparator + "/tmp", "noappend");
|
|
checkEnv("NEW_PATH", File.pathSeparator + "/tmp", "noappend");
|
|
- // check if X=$(tt's X var):/tmp for an old env variable inherited from
|
|
|
|
- // the tt
|
|
|
|
- if (Shell.WINDOWS) {
|
|
|
|
- // On Windows, PATH is replaced one more time as part of default config
|
|
|
|
- // of "mapreduce.admin.user.env", i.e. on Windows,
|
|
|
|
- // "mapreduce.admin.user.env" is set to
|
|
|
|
- // "PATH=%PATH%;%HADOOP_COMMON_HOME%\\bin"
|
|
|
|
- String hadoopHome = System.getenv("HADOOP_COMMON_HOME");
|
|
|
|
- if (hadoopHome == null) {
|
|
|
|
- hadoopHome = "";
|
|
|
|
- }
|
|
|
|
- String hadoopLibLocation = hadoopHome + "\\bin";
|
|
|
|
- path += File.pathSeparator + hadoopLibLocation;
|
|
|
|
- path += File.pathSeparator + path;
|
|
|
|
- }
|
|
|
|
- checkEnv("PATH", path + File.pathSeparator + "/tmp", "noappend");
|
|
|
|
|
|
|
|
String jobLocalDir = job.get(MRJobConfig.JOB_LOCAL_DIR);
|
|
String jobLocalDir = job.get(MRJobConfig.JOB_LOCAL_DIR);
|
|
assertNotNull(MRJobConfig.JOB_LOCAL_DIR + " is null",
|
|
assertNotNull(MRJobConfig.JOB_LOCAL_DIR + " is null",
|
|
@@ -281,10 +202,12 @@ public class TestMiniMRChildTask {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- static class EnvCheckReducer extends MapReduceBase
|
|
|
|
- implements Reducer<WritableComparable, Writable, WritableComparable, Writable> {
|
|
|
|
-
|
|
|
|
|
|
+ private static class EnvCheckReducer extends MapReduceBase
|
|
|
|
+ implements Reducer<WritableComparable, Writable,
|
|
|
|
+ WritableComparable, Writable> {
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
|
|
+ @SuppressWarnings("deprecation")
|
|
public void configure(JobConf job) {
|
|
public void configure(JobConf job) {
|
|
boolean oldConfigs = job.getBoolean(OLD_CONFIGS, false);
|
|
boolean oldConfigs = job.getBoolean(OLD_CONFIGS, false);
|
|
if (oldConfigs) {
|
|
if (oldConfigs) {
|
|
@@ -303,39 +226,12 @@ public class TestMiniMRChildTask {
|
|
reduceJavaOpts, REDUCE_OPTS_VAL);
|
|
reduceJavaOpts, REDUCE_OPTS_VAL);
|
|
}
|
|
}
|
|
|
|
|
|
- String path = job.get("path");
|
|
|
|
-
|
|
|
|
- // check if the pwd is there in LD_LIBRARY_PATH
|
|
|
|
- String pwd = System.getenv("PWD");
|
|
|
|
-
|
|
|
|
- assertTrue("LD doesnt contain pwd",
|
|
|
|
- System.getenv("LD_LIBRARY_PATH").contains(pwd));
|
|
|
|
-
|
|
|
|
- // check if X=$X:/abc works for LD_LIBRARY_PATH
|
|
|
|
- checkEnv("LD_LIBRARY_PATH", "/tmp", "append");
|
|
|
|
// check if X=y works for an already existing parameter
|
|
// check if X=y works for an already existing parameter
|
|
checkEnv("LANG", "en_us_8859_1", "noappend");
|
|
checkEnv("LANG", "en_us_8859_1", "noappend");
|
|
// check if X=/tmp for a new env variable
|
|
// check if X=/tmp for a new env variable
|
|
checkEnv("MY_PATH", "/tmp", "noappend");
|
|
checkEnv("MY_PATH", "/tmp", "noappend");
|
|
// check if X=$X:/tmp works for a new env var and results into :/tmp
|
|
// check if X=$X:/tmp works for a new env var and results into :/tmp
|
|
checkEnv("NEW_PATH", File.pathSeparator + "/tmp", "noappend");
|
|
checkEnv("NEW_PATH", File.pathSeparator + "/tmp", "noappend");
|
|
- // check if X=$(tt's X var):/tmp for an old env variable inherited from
|
|
|
|
- // the tt
|
|
|
|
- if (Shell.WINDOWS) {
|
|
|
|
- // On Windows, PATH is replaced one more time as part of default config
|
|
|
|
- // of "mapreduce.admin.user.env", i.e. on Windows,
|
|
|
|
- // "mapreduce.admin.user.env"
|
|
|
|
- // is set to "PATH=%PATH%;%HADOOP_COMMON_HOME%\\bin"
|
|
|
|
- String hadoopHome = System.getenv("HADOOP_COMMON_HOME");
|
|
|
|
- if (hadoopHome == null) {
|
|
|
|
- hadoopHome = "";
|
|
|
|
- }
|
|
|
|
- String hadoopLibLocation = hadoopHome + "\\bin";
|
|
|
|
- path += File.pathSeparator + hadoopLibLocation;
|
|
|
|
- path += File.pathSeparator + path;
|
|
|
|
- }
|
|
|
|
- checkEnv("PATH", path + File.pathSeparator + "/tmp", "noappend");
|
|
|
|
-
|
|
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
@@ -392,73 +288,10 @@ public class TestMiniMRChildTask {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- /**
|
|
|
|
- * To test OS dependent setting of default execution path for a MapRed task.
|
|
|
|
- * Mainly that we can use MRJobConfig.DEFAULT_MAPRED_ADMIN_USER_ENV to set -
|
|
|
|
- * for WINDOWS: %HADOOP_COMMON_HOME%\bin is expected to be included in PATH -
|
|
|
|
- * for Linux: $HADOOP_COMMON_HOME/lib/native is expected to be included in
|
|
|
|
- * LD_LIBRARY_PATH
|
|
|
|
- */
|
|
|
|
- @Test
|
|
|
|
- public void testMapRedExecutionEnv() {
|
|
|
|
- // for windows, test if the env variable can be set
|
|
|
|
- // this may be removed as part of MAPREDUCE-6588
|
|
|
|
- if (Shell.WINDOWS) {
|
|
|
|
- try {
|
|
|
|
- // Application environment
|
|
|
|
- Map<String, String> environment = new HashMap<String, String>();
|
|
|
|
- String setupHadoopHomeCommand =
|
|
|
|
- "HADOOP_COMMON_HOME=C:\\fake\\PATH\\to\\hadoop\\common\\home";
|
|
|
|
- MRApps.setEnvFromInputString(environment, setupHadoopHomeCommand, conf);
|
|
|
|
-
|
|
|
|
- // Add the env variables passed by the admin
|
|
|
|
- MRApps.setEnvFromInputString(environment, conf.get(
|
|
|
|
- MRJobConfig.MAPRED_ADMIN_USER_ENV,
|
|
|
|
- MRJobConfig.DEFAULT_MAPRED_ADMIN_USER_ENV), conf);
|
|
|
|
-
|
|
|
|
- String executionPaths = environment.get("PATH");
|
|
|
|
- String toFind =
|
|
|
|
- "C:\\fake\\PATH\\to\\hadoop\\common\\home\\bin";
|
|
|
|
-
|
|
|
|
- // Ensure execution PATH/LD_LIBRARY_PATH set up pointing to hadoop lib
|
|
|
|
- assertTrue("execution path does not include the hadoop lib location "
|
|
|
|
- + toFind, executionPaths.contains(toFind));
|
|
|
|
- } catch (Exception e) {
|
|
|
|
- e.printStackTrace();
|
|
|
|
- fail("Exception in testing execution environment for MapReduce task");
|
|
|
|
- tearDown();
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- // now launch a mapreduce job to ensure that the child
|
|
|
|
- // also gets the configured setting for hadoop lib
|
|
|
|
- try {
|
|
|
|
-
|
|
|
|
- JobConf conf = new JobConf(mr.getConfig());
|
|
|
|
- // initialize input, output directories
|
|
|
|
- Path rootDir = new Path(System.getProperty("test.build.data",
|
|
|
|
- "build/test/data"));
|
|
|
|
- Path inDir = new Path(rootDir, "input");
|
|
|
|
- Path outDir = new Path(rootDir, "output");
|
|
|
|
- String input = "The input";
|
|
|
|
-
|
|
|
|
- // set config to use the ExecutionEnvCheckMapClass map class
|
|
|
|
- configure(conf, inDir, outDir, input,
|
|
|
|
- ExecutionEnvCheckMapClass.class, IdentityReducer.class);
|
|
|
|
- launchTest(conf, inDir, outDir, input);
|
|
|
|
-
|
|
|
|
- } catch(Exception e) {
|
|
|
|
- e.printStackTrace();
|
|
|
|
- fail("Exception in testing propagation of env setting to child task");
|
|
|
|
- tearDown();
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
/**
|
|
/**
|
|
* Test to test if the user set env variables reflect in the child
|
|
* Test to test if the user set env variables reflect in the child
|
|
* processes. Mainly
|
|
* processes. Mainly
|
|
* - x=y (x can be a already existing env variable or a new variable)
|
|
* - 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)
|
|
|
|
*/
|
|
*/
|
|
@Test
|
|
@Test
|
|
public void testTaskEnv(){
|
|
public void testTaskEnv(){
|
|
@@ -482,10 +315,9 @@ public class TestMiniMRChildTask {
|
|
* Test to test if the user set *old* env variables reflect in the child
|
|
* Test to test if the user set *old* env variables reflect in the child
|
|
* processes. Mainly
|
|
* processes. Mainly
|
|
* - x=y (x can be a already existing env variable or a new variable)
|
|
* - 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)
|
|
|
|
*/
|
|
*/
|
|
@Test
|
|
@Test
|
|
- public void testTaskOldEnv(){
|
|
|
|
|
|
+ public void testTaskOldEnv() {
|
|
try {
|
|
try {
|
|
JobConf conf = new JobConf(mr.getConfig());
|
|
JobConf conf = new JobConf(mr.getConfig());
|
|
String baseDir = System.getProperty("test.build.data", "build/test/data");
|
|
String baseDir = System.getProperty("test.build.data", "build/test/data");
|
|
@@ -495,23 +327,23 @@ public class TestMiniMRChildTask {
|
|
FileSystem outFs = outDir.getFileSystem(conf);
|
|
FileSystem outFs = outDir.getFileSystem(conf);
|
|
runTestTaskEnv(conf, inDir, outDir, true);
|
|
runTestTaskEnv(conf, inDir, outDir, true);
|
|
outFs.delete(outDir, true);
|
|
outFs.delete(outDir, true);
|
|
- } catch(Exception e) {
|
|
|
|
|
|
+ } catch (Exception e) {
|
|
e.printStackTrace();
|
|
e.printStackTrace();
|
|
fail("Exception in testing child env");
|
|
fail("Exception in testing child env");
|
|
tearDown();
|
|
tearDown();
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
- void runTestTaskEnv(JobConf conf, Path inDir, Path outDir, boolean oldConfigs)
|
|
|
|
- throws IOException, InterruptedException, ClassNotFoundException {
|
|
|
|
|
|
+
|
|
|
|
+ @SuppressWarnings("deprecation")
|
|
|
|
+ private void runTestTaskEnv(JobConf config, Path inDir, Path outDir,
|
|
|
|
+ boolean oldConfigs)
|
|
|
|
+ throws IOException, InterruptedException, ClassNotFoundException {
|
|
String input = "The input";
|
|
String input = "The input";
|
|
- configure(conf, inDir, outDir, input,
|
|
|
|
|
|
+ configure(config, inDir, outDir, input,
|
|
EnvCheckMapper.class, EnvCheckReducer.class);
|
|
EnvCheckMapper.class, EnvCheckReducer.class);
|
|
// test
|
|
// test
|
|
// - new SET of new var (MY_PATH)
|
|
// - new SET of new var (MY_PATH)
|
|
// - set of old var (LANG)
|
|
// - set of old var (LANG)
|
|
- // - append to an old var from modified env (LD_LIBRARY_PATH)
|
|
|
|
- // - append to an old var from tt's env (PATH)
|
|
|
|
// - append to a new var (NEW_PATH)
|
|
// - append to a new var (NEW_PATH)
|
|
String mapTaskEnvKey = JobConf.MAPRED_MAP_TASK_ENV;
|
|
String mapTaskEnvKey = JobConf.MAPRED_MAP_TASK_ENV;
|
|
String reduceTaskEnvKey = JobConf.MAPRED_MAP_TASK_ENV;
|
|
String reduceTaskEnvKey = JobConf.MAPRED_MAP_TASK_ENV;
|
|
@@ -519,29 +351,24 @@ public class TestMiniMRChildTask {
|
|
String reduceTaskJavaOptsKey = JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS;
|
|
String reduceTaskJavaOptsKey = JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS;
|
|
String mapTaskJavaOpts = MAP_OPTS_VAL;
|
|
String mapTaskJavaOpts = MAP_OPTS_VAL;
|
|
String reduceTaskJavaOpts = REDUCE_OPTS_VAL;
|
|
String reduceTaskJavaOpts = REDUCE_OPTS_VAL;
|
|
- conf.setBoolean(OLD_CONFIGS, oldConfigs);
|
|
|
|
|
|
+ config.setBoolean(OLD_CONFIGS, oldConfigs);
|
|
if (oldConfigs) {
|
|
if (oldConfigs) {
|
|
mapTaskEnvKey = reduceTaskEnvKey = JobConf.MAPRED_TASK_ENV;
|
|
mapTaskEnvKey = reduceTaskEnvKey = JobConf.MAPRED_TASK_ENV;
|
|
mapTaskJavaOptsKey = reduceTaskJavaOptsKey = JobConf.MAPRED_TASK_JAVA_OPTS;
|
|
mapTaskJavaOptsKey = reduceTaskJavaOptsKey = JobConf.MAPRED_TASK_JAVA_OPTS;
|
|
mapTaskJavaOpts = reduceTaskJavaOpts = TASK_OPTS_VAL;
|
|
mapTaskJavaOpts = reduceTaskJavaOpts = TASK_OPTS_VAL;
|
|
}
|
|
}
|
|
- conf.set(
|
|
|
|
|
|
+ config.set(
|
|
mapTaskEnvKey,
|
|
mapTaskEnvKey,
|
|
- Shell.WINDOWS ? "MY_PATH=/tmp,LANG=en_us_8859_1,LD_LIBRARY_PATH=%LD_LIBRARY_PATH%;/tmp,"
|
|
|
|
- + "PATH=%PATH%;/tmp,NEW_PATH=%NEW_PATH%;/tmp"
|
|
|
|
- : "MY_PATH=/tmp,LANG=en_us_8859_1,LD_LIBRARY_PATH=$LD_LIBRARY_PATH:/tmp,"
|
|
|
|
- + "PATH=$PATH:/tmp,NEW_PATH=$NEW_PATH:/tmp");
|
|
|
|
- conf.set(
|
|
|
|
|
|
+ Shell.WINDOWS ? "MY_PATH=/tmp,LANG=en_us_8859_1,NEW_PATH=%MY_PATH%;/tmp"
|
|
|
|
+ : "MY_PATH=/tmp,LANG=en_us_8859_1,NEW_PATH=$NEW_PATH:/tmp");
|
|
|
|
+ config.set(
|
|
reduceTaskEnvKey,
|
|
reduceTaskEnvKey,
|
|
- Shell.WINDOWS ? "MY_PATH=/tmp,LANG=en_us_8859_1,LD_LIBRARY_PATH=%LD_LIBRARY_PATH%;/tmp,"
|
|
|
|
- + "PATH=%PATH%;/tmp,NEW_PATH=%NEW_PATH%;/tmp"
|
|
|
|
- : "MY_PATH=/tmp,LANG=en_us_8859_1,LD_LIBRARY_PATH=$LD_LIBRARY_PATH:/tmp,"
|
|
|
|
- + "PATH=$PATH:/tmp,NEW_PATH=$NEW_PATH:/tmp");
|
|
|
|
- conf.set("path", System.getenv("PATH"));
|
|
|
|
- conf.set(mapTaskJavaOptsKey, mapTaskJavaOpts);
|
|
|
|
- conf.set(reduceTaskJavaOptsKey, reduceTaskJavaOpts);
|
|
|
|
|
|
+ Shell.WINDOWS ? "MY_PATH=/tmp,LANG=en_us_8859_1,NEW_PATH=%MY_PATH%;/tmp"
|
|
|
|
+ : "MY_PATH=/tmp,LANG=en_us_8859_1,NEW_PATH=$NEW_PATH:/tmp");
|
|
|
|
+ config.set(mapTaskJavaOptsKey, mapTaskJavaOpts);
|
|
|
|
+ config.set(reduceTaskJavaOptsKey, reduceTaskJavaOpts);
|
|
|
|
|
|
- Job job = Job.getInstance(conf);
|
|
|
|
|
|
+ Job job = Job.getInstance(config);
|
|
job.addFileToClassPath(APP_JAR);
|
|
job.addFileToClassPath(APP_JAR);
|
|
job.setJarByClass(TestMiniMRChildTask.class);
|
|
job.setJarByClass(TestMiniMRChildTask.class);
|
|
job.setMaxMapAttempts(1); // speed up failures
|
|
job.setMaxMapAttempts(1); // speed up failures
|