|
@@ -23,6 +23,8 @@ import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
|
import java.util.Iterator;
|
|
|
|
|
|
+import org.apache.commons.logging.*;
|
|
|
+
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
|
|
|
import org.apache.hadoop.io.UTF8;
|
|
@@ -32,16 +34,14 @@ import org.apache.hadoop.mapred.JobConf;
|
|
|
import org.apache.hadoop.mapred.JobClient;
|
|
|
import org.apache.hadoop.mapred.RunningJob;
|
|
|
|
|
|
-import org.apache.hadoop.util.LogFormatter;
|
|
|
-
|
|
|
/** All the client-side work happens here.
|
|
|
* (Jar packaging, MapRed job submission and monitoring)
|
|
|
* @author Michel Tourn
|
|
|
*/
|
|
|
public class StreamJob
|
|
|
{
|
|
|
- protected static final Logger LOG = LogFormatter.getLogger(StreamJob.class.getName());
|
|
|
-
|
|
|
+ protected static final Log LOG = LogFactory.getLog(StreamJob.class.getName());
|
|
|
+
|
|
|
public StreamJob(String[] argv, boolean mayExit)
|
|
|
{
|
|
|
argv_ = argv;
|
|
@@ -72,9 +72,10 @@ public class StreamJob
|
|
|
void preProcessArgs()
|
|
|
{
|
|
|
verbose_ = false;
|
|
|
+ addTaskEnvironment_ = "";
|
|
|
}
|
|
|
|
|
|
- void postProcessArgs()
|
|
|
+ void postProcessArgs() throws IOException
|
|
|
{
|
|
|
if(cluster_ == null) {
|
|
|
// hadoop-default.xml is standard, hadoop-local.xml is not.
|
|
@@ -87,22 +88,35 @@ public class StreamJob
|
|
|
if(output_ == null) {
|
|
|
fail("Required argument: -output ");
|
|
|
}
|
|
|
- // careful with class names..
|
|
|
- mapCmd_ = packageOrTrimNoShip(mapCmd_);
|
|
|
- redCmd_ = packageOrTrimNoShip(redCmd_);
|
|
|
+ msg("addTaskEnvironment=" + addTaskEnvironment_);
|
|
|
+
|
|
|
+ Iterator it = packageFiles_.iterator();
|
|
|
+ while(it.hasNext()) {
|
|
|
+ File f = new File((String)it.next());
|
|
|
+ if(f.isFile()) {
|
|
|
+ shippedCanonFiles_.add(f.getCanonicalPath());
|
|
|
+ }
|
|
|
+ }
|
|
|
+ msg("shippedCanonFiles_=" + shippedCanonFiles_);
|
|
|
|
|
|
- // TBD -D format or sthg on cmdline.
|
|
|
- // Plus maybe a standard list originating on client or server
|
|
|
- addTaskEnvironment_ = "";
|
|
|
+ // careful with class names..
|
|
|
+ mapCmd_ = unqualifyIfLocalPath(mapCmd_);
|
|
|
+ redCmd_ = unqualifyIfLocalPath(redCmd_);
|
|
|
+ }
|
|
|
+
|
|
|
+ void validateNameEqValue(String neqv)
|
|
|
+ {
|
|
|
+ String[] nv = neqv.split("=", 2);
|
|
|
+ if(nv.length < 2) {
|
|
|
+ fail("Invalid name=value spec: " + neqv);
|
|
|
+ }
|
|
|
+ msg("Recording name=value: name=" + nv[0] + " value=" + nv[1]);
|
|
|
}
|
|
|
|
|
|
- String packageOrTrimNoShip(String cmd)
|
|
|
+ String unqualifyIfLocalPath(String cmd) throws IOException
|
|
|
{
|
|
|
if(cmd == null) {
|
|
|
//
|
|
|
- } else if(cmd.startsWith(NOSHIP)) {
|
|
|
- // don't package the file, but keep the abolute path
|
|
|
- cmd = cmd.substring(NOSHIP.length());
|
|
|
} else {
|
|
|
String prog = cmd;
|
|
|
String args = "";
|
|
@@ -111,18 +125,23 @@ public class StreamJob
|
|
|
prog = cmd.substring(0, s);
|
|
|
args = cmd.substring(s+1);
|
|
|
}
|
|
|
- packageFiles_.add(new File(prog).getAbsolutePath());
|
|
|
- // Change path to simple filename.
|
|
|
- // That way when PipeMapRed calls Runtime.exec(),
|
|
|
- // it will look for the excutable in Task's working dir.
|
|
|
- // And this is where TaskRunner unjars our job jar.
|
|
|
- prog = new File(prog).getName();
|
|
|
- if(args.length() > 0) {
|
|
|
- cmd = prog + " " + args;
|
|
|
- } else {
|
|
|
- cmd = prog;
|
|
|
+ String progCanon = new File(prog).getCanonicalPath();
|
|
|
+ boolean shipped = shippedCanonFiles_.contains(progCanon);
|
|
|
+ msg("shipped: " + shipped + " " + progCanon);
|
|
|
+ if(shipped) {
|
|
|
+ // Change path to simple filename.
|
|
|
+ // That way when PipeMapRed calls Runtime.exec(),
|
|
|
+ // it will look for the excutable in Task's working dir.
|
|
|
+ // And this is where TaskRunner unjars our job jar.
|
|
|
+ prog = new File(prog).getName();
|
|
|
+ if(args.length() > 0) {
|
|
|
+ cmd = prog + " " + args;
|
|
|
+ } else {
|
|
|
+ cmd = prog;
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
+ msg("cmd=" + cmd);
|
|
|
return cmd;
|
|
|
}
|
|
|
|
|
@@ -130,17 +149,20 @@ public class StreamJob
|
|
|
{
|
|
|
return new File(getHadoopClientHome() + "/conf", hadoopAliasConf_).getAbsolutePath();
|
|
|
}
|
|
|
+
|
|
|
|
|
|
void parseArgv()
|
|
|
{
|
|
|
if(argv_.length==0) {
|
|
|
- exitUsage();
|
|
|
+ exitUsage(false);
|
|
|
}
|
|
|
int i=0;
|
|
|
while(i < argv_.length) {
|
|
|
String s;
|
|
|
if(argv_[i].equals("-verbose")) {
|
|
|
verbose_ = true;
|
|
|
+ } else if(argv_[i].equals("-info")) {
|
|
|
+ detailedUsage_ = true;
|
|
|
} else if(argv_[i].equals("-debug")) {
|
|
|
debug_++;
|
|
|
} else if((s = optionArg(argv_, i, "-input", false)) != null) {
|
|
@@ -155,7 +177,7 @@ public class StreamJob
|
|
|
} else if((s = optionArg(argv_, i, "-reducer", redCmd_ != null)) != null) {
|
|
|
i++;
|
|
|
redCmd_ = s;
|
|
|
- } else if((s = optionArg(argv_, i, "-files", false)) != null) {
|
|
|
+ } else if((s = optionArg(argv_, i, "-file", false)) != null) {
|
|
|
i++;
|
|
|
packageFiles_.add(s);
|
|
|
} else if((s = optionArg(argv_, i, "-cluster", cluster_ != null)) != null) {
|
|
@@ -164,15 +186,35 @@ public class StreamJob
|
|
|
} else if((s = optionArg(argv_, i, "-config", false)) != null) {
|
|
|
i++;
|
|
|
configPath_.add(s);
|
|
|
+ } else if((s = optionArg(argv_, i, "-dfs", false)) != null) {
|
|
|
+ i++;
|
|
|
+ userJobConfProps_.add("fs.default.name="+s);
|
|
|
+ } else if((s = optionArg(argv_, i, "-jt", false)) != null) {
|
|
|
+ i++;
|
|
|
+ userJobConfProps_.add("mapred.job.tracker="+s);
|
|
|
+ } else if((s = optionArg(argv_, i, "-jobconf", false)) != null) {
|
|
|
+ i++;
|
|
|
+ validateNameEqValue(s);
|
|
|
+ userJobConfProps_.add(s);
|
|
|
+ } else if((s = optionArg(argv_, i, "-cmdenv", false)) != null) {
|
|
|
+ i++;
|
|
|
+ validateNameEqValue(s);
|
|
|
+ if(addTaskEnvironment_.length() > 0) {
|
|
|
+ addTaskEnvironment_ += " ";
|
|
|
+ }
|
|
|
+ addTaskEnvironment_ += s;
|
|
|
} else if((s = optionArg(argv_, i, "-inputreader", inReaderSpec_ != null)) != null) {
|
|
|
i++;
|
|
|
inReaderSpec_ = s;
|
|
|
} else {
|
|
|
System.err.println("Unexpected argument: " + argv_[i]);
|
|
|
- exitUsage();
|
|
|
+ exitUsage(false);
|
|
|
}
|
|
|
i++;
|
|
|
}
|
|
|
+ if(detailedUsage_) {
|
|
|
+ exitUsage(true);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
String optionArg(String[] args, int index, String arg, boolean argSet)
|
|
@@ -196,22 +238,32 @@ public class StreamJob
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- public void exitUsage()
|
|
|
+ public void exitUsage(boolean detailed)
|
|
|
{
|
|
|
// 1 2 3 4 5 6 7
|
|
|
//1234567890123456789012345678901234567890123456789012345678901234567890123456789
|
|
|
- System.out.println("Usage: bin/hadoop jar build/hadoop-streaming.jar [options]");
|
|
|
+ System.out.println("Usage: $HADOOP_HOME/bin/hadoop jar build/hadoop-streaming.jar [options]");
|
|
|
System.out.println("Options:");
|
|
|
- System.out.println(" -input <path> DFS input file(s) for the Map step");
|
|
|
- System.out.println(" -output <path> DFS output directory for the Reduce step");
|
|
|
- System.out.println(" -mapper <cmd> The streaming command to run");
|
|
|
- System.out.println(" -reducer <cmd> The streaming command to run");
|
|
|
- System.out.println(" -files <file> Additional files to be shipped in the Job jar file");
|
|
|
- System.out.println(" -cluster <name> Default uses hadoop-default.xml and hadoop-site.xml");
|
|
|
- System.out.println(" -config <file> Optional. One or more paths to xml config files");
|
|
|
- System.out.println(" -inputreader <spec> Optional. See below");
|
|
|
+ System.out.println(" -input <path> DFS input file(s) for the Map step");
|
|
|
+ System.out.println(" -output <path> DFS output directory for the Reduce step");
|
|
|
+ System.out.println(" -mapper <cmd> The streaming command to run");
|
|
|
+ System.out.println(" -combiner <cmd> Not implemented. But you can pipe the mapper output");
|
|
|
+ System.out.println(" -reducer <cmd> The streaming command to run");
|
|
|
+ System.out.println(" -file <file> File/dir to be shipped in the Job jar file");
|
|
|
+ System.out.println(" -cluster <name> Default uses hadoop-default.xml and hadoop-site.xml");
|
|
|
+ System.out.println(" -config <file> Optional. One or more paths to xml config files");
|
|
|
+ System.out.println(" -dfs <h:p> Optional. Override DFS configuration");
|
|
|
+ System.out.println(" -jt <h:p> Optional. Override JobTracker configuration");
|
|
|
+ System.out.println(" -inputreader <spec> Optional.");
|
|
|
+ System.out.println(" -jobconf <n>=<v> Optional.");
|
|
|
+ System.out.println(" -cmdenv <n>=<v> Optional. Pass env.var to streaming commands");
|
|
|
System.out.println(" -verbose");
|
|
|
System.out.println();
|
|
|
+ if(!detailed) {
|
|
|
+ System.out.println("For more details about these options:");
|
|
|
+ System.out.println("Use $HADOOP_HOME/bin/hadoop jar build/hadoop-streaming.jar -info");
|
|
|
+ fail("");
|
|
|
+ }
|
|
|
System.out.println("In -input: globbing on <path> is supported and can have multiple -input");
|
|
|
System.out.println("Default Map input format: a line is a record in UTF-8");
|
|
|
System.out.println(" the key part ends at first TAB, the rest of the line is the value");
|
|
@@ -220,21 +272,34 @@ public class StreamJob
|
|
|
System.out.println(" Ex: -inputreader 'StreamXmlRecordReader,begin=<doc>,end=</doc>'");
|
|
|
System.out.println("Map output format, reduce input/output format:");
|
|
|
System.out.println(" Format defined by what mapper command outputs. Line-oriented");
|
|
|
- System.out.println("Mapper and Reducer <cmd> syntax: ");
|
|
|
- System.out.println(" If the mapper or reducer programs are prefixed with " + NOSHIP + " then ");
|
|
|
- System.out.println(" the paths are assumed to be valid absolute paths on the task tracker machines");
|
|
|
- System.out.println(" and are NOT packaged with the Job jar file.");
|
|
|
+ System.out.println();
|
|
|
System.out.println("Use -cluster <name> to switch between \"local\" Hadoop and one or more remote ");
|
|
|
System.out.println(" Hadoop clusters. ");
|
|
|
System.out.println(" The default is to use the normal hadoop-default.xml and hadoop-site.xml");
|
|
|
System.out.println(" Else configuration will use $HADOOP_HOME/conf/hadoop-<name>.xml");
|
|
|
System.out.println();
|
|
|
- System.out.println("Example: hadoopStreaming -mapper \"noship:/usr/local/bin/perl5 filter.pl\"");
|
|
|
- System.out.println(" -files /local/filter.pl -input \"/logs/0604*/*\" [...]");
|
|
|
+ System.out.println("To set the number of reduce tasks (num. of output files):");
|
|
|
+ System.out.println(" -jobconf mapred.reduce.tasks=10");
|
|
|
+ System.out.println("To change the local temp directory:");
|
|
|
+ System.out.println(" -jobconf dfs.data.dir=/tmp");
|
|
|
+ System.out.println("Additional local temp directories with -cluster local:");
|
|
|
+ System.out.println(" -jobconf mapred.local.dir=/tmp/local");
|
|
|
+ System.out.println(" -jobconf mapred.system.dir=/tmp/system");
|
|
|
+ System.out.println(" -jobconf mapred.temp.dir=/tmp/temp");
|
|
|
+ System.out.println("For more details about jobconf parameters see:");
|
|
|
+ System.out.println(" http://wiki.apache.org/lucene-hadoop/JobConfFile");
|
|
|
+ System.out.println("To set an environement variable in a streaming command:");
|
|
|
+ System.out.println(" -cmdenv EXAMPLE_DIR=/home/example/dictionaries/");
|
|
|
+ System.out.println();
|
|
|
+ System.out.println("Shortcut to run from any directory:");
|
|
|
+ System.out.println(" setenv HSTREAMING \"$HADOOP_HOME/bin/hadoop jar $HADOOP_HOME/build/hadoop-streaming.jar\"");
|
|
|
+ System.out.println();
|
|
|
+ System.out.println("Example: $HSTREAMING -mapper \"/usr/local/bin/perl5 filter.pl\"");
|
|
|
+ System.out.println(" -file /local/filter.pl -input \"/logs/0604*/*\" [...]");
|
|
|
System.out.println(" Ships a script, invokes the non-shipped perl interpreter");
|
|
|
System.out.println(" Shipped files go to the working directory so filter.pl is found by perl");
|
|
|
System.out.println(" Input files are all the daily logs for days in month 2006-04");
|
|
|
- fail("");
|
|
|
+ fail("");
|
|
|
}
|
|
|
|
|
|
public void fail(String message)
|
|
@@ -291,7 +356,7 @@ public class StreamJob
|
|
|
msg("Found runtime classes in: " + runtimeClasses);
|
|
|
}
|
|
|
if(isLocalHadoop()) {
|
|
|
- // don't package class files (they might get unpackaged in . and then
|
|
|
+ // don't package class files (they might get unpackaged in "." and then
|
|
|
// hide the intended CLASSPATH entry)
|
|
|
// we still package everything else (so that scripts and executable are found in
|
|
|
// Task workdir like distributed Hadoop)
|
|
@@ -393,7 +458,17 @@ public class StreamJob
|
|
|
if(jar_ != null) {
|
|
|
jobConf_.setJar(jar_);
|
|
|
}
|
|
|
- //jobConf_.mtdump();System.exit(1);
|
|
|
+
|
|
|
+ // last, allow user to override anything
|
|
|
+ // (although typically used with properties we didn't touch)
|
|
|
+ it = userJobConfProps_.iterator();
|
|
|
+ while(it.hasNext()) {
|
|
|
+ String prop = (String)it.next();
|
|
|
+ String[] nv = prop.split("=", 2);
|
|
|
+ msg("JobConf: set(" + nv[0] + ", " + nv[1]+")");
|
|
|
+ jobConf_.set(nv[0], nv[1]);
|
|
|
+ }
|
|
|
+
|
|
|
}
|
|
|
|
|
|
protected String getJobTrackerHostPort()
|
|
@@ -432,7 +507,7 @@ public class StreamJob
|
|
|
running_ = jc_.submitJob(jobConf_);
|
|
|
jobId_ = running_.getJobID();
|
|
|
|
|
|
- LOG.info("getLocalDirs(): " + Arrays.toString(jobConf_.getLocalDirs()));
|
|
|
+ LOG.info("getLocalDirs(): " + Arrays.asList(jobConf_.getLocalDirs()));
|
|
|
LOG.info("Running job: " + jobId_);
|
|
|
jobInfo();
|
|
|
|
|
@@ -467,11 +542,10 @@ public class StreamJob
|
|
|
}
|
|
|
|
|
|
|
|
|
- public final static String NOSHIP = "noship:";
|
|
|
-
|
|
|
protected boolean mayExit_;
|
|
|
protected String[] argv_;
|
|
|
protected boolean verbose_;
|
|
|
+ protected boolean detailedUsage_;
|
|
|
protected int debug_;
|
|
|
|
|
|
protected Environment env_;
|
|
@@ -483,8 +557,10 @@ public class StreamJob
|
|
|
protected JobClient jc_;
|
|
|
|
|
|
// command-line arguments
|
|
|
- protected ArrayList inputGlobs_ = new ArrayList(); // <String>
|
|
|
- protected ArrayList packageFiles_ = new ArrayList(); // <String>
|
|
|
+ protected ArrayList inputGlobs_ = new ArrayList(); // <String>
|
|
|
+ protected ArrayList packageFiles_ = new ArrayList(); // <String>
|
|
|
+ protected ArrayList shippedCanonFiles_= new ArrayList(); // <String>
|
|
|
+ protected ArrayList userJobConfProps_ = new ArrayList(); // <String>
|
|
|
protected String output_;
|
|
|
protected String mapCmd_;
|
|
|
protected String redCmd_;
|