|
@@ -22,7 +22,6 @@ import java.io.File;
|
|
|
import java.io.FileNotFoundException;
|
|
|
import java.io.IOException;
|
|
|
import java.net.URI;
|
|
|
-import java.net.URISyntaxException;
|
|
|
import java.net.URLEncoder;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
@@ -41,7 +40,6 @@ import org.apache.commons.cli2.commandline.Parser;
|
|
|
import org.apache.commons.cli2.option.PropertyOption;
|
|
|
import org.apache.commons.cli2.resource.ResourceConstants;
|
|
|
import org.apache.commons.cli2.util.HelpFormatter;
|
|
|
-import org.apache.commons.cli2.validation.FileValidator;
|
|
|
import org.apache.commons.cli2.validation.InvalidArgumentException;
|
|
|
import org.apache.commons.cli2.validation.Validator;
|
|
|
import org.apache.commons.logging.*;
|
|
@@ -61,6 +59,7 @@ import org.apache.hadoop.mapred.KeyValueTextInputFormat;
|
|
|
import org.apache.hadoop.mapred.RunningJob;
|
|
|
import org.apache.hadoop.mapred.SequenceFileInputFormat;
|
|
|
import org.apache.hadoop.mapred.SequenceFileAsTextInputFormat;
|
|
|
+import org.apache.hadoop.mapred.TextInputFormat;
|
|
|
import org.apache.hadoop.mapred.TextOutputFormat;
|
|
|
import org.apache.hadoop.filecache.*;
|
|
|
import org.apache.hadoop.util.*;
|
|
@@ -73,8 +72,7 @@ public class StreamJob {
|
|
|
|
|
|
protected static final Log LOG = LogFactory.getLog(StreamJob.class.getName());
|
|
|
final static String REDUCE_NONE = "NONE";
|
|
|
- private boolean reducerNone_;
|
|
|
-
|
|
|
+
|
|
|
/** -----------Streaming CLI Implementation **/
|
|
|
private DefaultOptionBuilder builder =
|
|
|
new DefaultOptionBuilder("-","-", false);
|
|
@@ -214,7 +212,6 @@ public class StreamJob {
|
|
|
|
|
|
inputSpecs_.addAll(cmdLine.getValues("-input"));
|
|
|
output_ = (String) cmdLine.getValue("-output");
|
|
|
- mapsideoutURI_ = (String) cmdLine.getValue("-mapsideoutput");
|
|
|
|
|
|
mapCmd_ = (String)cmdLine.getValue("-mapper");
|
|
|
comCmd_ = (String)cmdLine.getValue("-combiner");
|
|
@@ -450,20 +447,17 @@ public class StreamJob {
|
|
|
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(" -combiner <cmd> The streaming command to run");
|
|
|
- System.out.println(" -reducer <cmd> The streaming command to run");
|
|
|
+ System.out.println(" -mapper <cmd|JavaClassName> The streaming command to run");
|
|
|
+ System.out.println(" -combiner <JavaClassName> Combiner has to be a Java class");
|
|
|
+ System.out.println(" -reducer <cmd|JavaClassName> The streaming command to run");
|
|
|
System.out.println(" -file <file> File/dir to be shipped in the Job jar file");
|
|
|
- //Only advertise the standard way: [--config dir] in our launcher
|
|
|
- //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>|local Optional. Override DFS configuration");
|
|
|
System.out.println(" -jt <h:p>|local Optional. Override JobTracker configuration");
|
|
|
System.out.println(" -additionalconfspec specfile Optional.");
|
|
|
- System.out.println(" -inputformat KeyValueTextInputFormat(default)|SequenceFileInputFormat|XmlTextInputFormat Optional.");
|
|
|
- System.out.println(" -outputformat specfile Optional.");
|
|
|
- System.out.println(" -partitioner specfile Optional.");
|
|
|
- System.out.println(" -numReduceTasks specfile Optional.");
|
|
|
+ System.out.println(" -inputformat TextInputFormat(default)|SequenceFileAsTextInputFormat|JavaClassName Optional.");
|
|
|
+ System.out.println(" -outputformat TextOutputFormat(default)|JavaClassName Optional.");
|
|
|
+ System.out.println(" -partitioner JavaClassName Optional.");
|
|
|
+ System.out.println(" -numReduceTasks <num> Optional.");
|
|
|
System.out.println(" -inputreader <spec> Optional.");
|
|
|
System.out.println(" -jobconf <n>=<v> Optional. Add or override a JobConf property");
|
|
|
System.out.println(" -cmdenv <n>=<v> Optional. Pass env.var to streaming commands");
|
|
@@ -478,10 +472,7 @@ public class StreamJob {
|
|
|
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");
|
|
|
- System.out.println("Custom Map input format: -inputreader package.MyRecordReader,n=v,n=v ");
|
|
|
- System.out
|
|
|
- .println(" comma-separated name-values can be specified to configure the InputFormat");
|
|
|
- System.out.println(" Ex: -inputreader 'StreamXmlRecordReader,begin=<doc>,end=</doc>'");
|
|
|
+ System.out.println("Custom input format: -inputformat package.MyInputFormat ");
|
|
|
System.out.println("Map output format, reduce input/output format:");
|
|
|
System.out.println(" Format defined by what the mapper command outputs. Line-oriented");
|
|
|
System.out.println();
|
|
@@ -489,34 +480,21 @@ public class StreamJob {
|
|
|
System.out.println(" working directory when the mapper and reducer are run.");
|
|
|
System.out.println(" The location of this working directory is unspecified.");
|
|
|
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("To set the number of reduce tasks (num. of output files):");
|
|
|
+ System.out.println(" -jobconf mapred.reduce.tasks=10");
|
|
|
System.out.println("To skip the sort/combine/shuffle/sort/reduce step:");
|
|
|
- System.out.println(" Use -reducer " + REDUCE_NONE);
|
|
|
+ System.out.println(" Use -numReduceTasks 0");
|
|
|
System.out
|
|
|
.println(" A Task's Map output then becomes a 'side-effect output' rather than a reduce input");
|
|
|
System.out
|
|
|
.println(" This speeds up processing, This also feels more like \"in-place\" processing");
|
|
|
System.out.println(" because the input filename and the map input order are preserved");
|
|
|
- System.out.println("To specify a single side-effect output file");
|
|
|
- System.out.println(" -mapsideoutput [file:/C:/win|file:/unix/|socket://host:port]");//-output for side-effects will be soon deprecated
|
|
|
- System.out.println(" If the jobtracker is local this is a local file");
|
|
|
- System.out.println(" This currently requires -reducer NONE");
|
|
|
+ System.out.println(" This equivalent -reducer NONE");
|
|
|
System.out.println();
|
|
|
- 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 speed up the last reduces:");
|
|
|
System.out.println(" -jobconf mapred.speculative.execution=true");
|
|
|
- System.out.println(" Do not use this along -reducer " + REDUCE_NONE);
|
|
|
System.out.println("To name the job (appears in the JobTracker Web UI):");
|
|
|
System.out.println(" -jobconf mapred.job.name='My Job' ");
|
|
|
- System.out.println("To specify that line-oriented input is in gzip format:");
|
|
|
- System.out
|
|
|
- .println("(at this time ALL input files must be gzipped and this is not recognized based on file extension)");
|
|
|
- System.out.println(" -jobconf stream.recordreader.compression=gzip ");
|
|
|
System.out.println("To change the local temp directory:");
|
|
|
System.out.println(" -jobconf dfs.data.dir=/tmp/dfs");
|
|
|
System.out.println(" -jobconf stream.tmpdir=/tmp/streaming");
|
|
@@ -681,8 +659,6 @@ public class StreamJob {
|
|
|
config_.addFinalResource(new Path(pathName));
|
|
|
}
|
|
|
|
|
|
- testMerge_ = (-1 != userJobConfProps_.toString().indexOf("stream.testmerge"));
|
|
|
-
|
|
|
// general MapRed job properties
|
|
|
jobConf_ = new JobConf(config_);
|
|
|
|
|
@@ -695,25 +671,32 @@ public class StreamJob {
|
|
|
// (to resolve local vs. dfs drive letter differences)
|
|
|
// (mapred.working.dir will be lazily initialized ONCE and depends on FS)
|
|
|
for (int i = 0; i < inputSpecs_.size(); i++) {
|
|
|
- addInputSpec((String) inputSpecs_.get(i), i);
|
|
|
+ jobConf_.addInputPath(new Path(((String) inputSpecs_.get(i))));
|
|
|
}
|
|
|
- jobConf_.setBoolean("stream.inputtagged", inputTagged_);
|
|
|
jobConf_.set("stream.numinputspecs", "" + inputSpecs_.size());
|
|
|
|
|
|
String defaultPackage = this.getClass().getPackage().getName();
|
|
|
Class c;
|
|
|
Class fmt = null;
|
|
|
if (inReaderSpec_ == null && inputFormatSpec_ == null) {
|
|
|
- fmt = KeyValueTextInputFormat.class;
|
|
|
+ fmt = TextInputFormat.class;
|
|
|
} else if (inputFormatSpec_ != null) {
|
|
|
- if (inputFormatSpec_.equals(KeyValueTextInputFormat.class.getName())
|
|
|
- || inputFormatSpec_.equals(KeyValueTextInputFormat.class.getCanonicalName())) {
|
|
|
- fmt = KeyValueTextInputFormat.class;
|
|
|
- } else if (inputFormatSpec_.equals(SequenceFileInputFormat.class.getName())
|
|
|
- || inputFormatSpec_.equals(SequenceFileInputFormat.class.getCanonicalName())) {
|
|
|
- fmt = SequenceFileInputFormat.class;
|
|
|
- } else if (inputFormatSpec_.equals(SequenceFileAsTextInputFormat.class.getName())
|
|
|
- || inputFormatSpec_.equals(SequenceFileAsTextInputFormat.class.getCanonicalName())) {
|
|
|
+ if (inputFormatSpec_.equals(TextInputFormat.class.getName())
|
|
|
+ || inputFormatSpec_.equals(TextInputFormat.class.getCanonicalName())) {
|
|
|
+ fmt = TextInputFormat.class;
|
|
|
+ } else if (inputFormatSpec_.equals(KeyValueTextInputFormat.class
|
|
|
+ .getName())
|
|
|
+ || inputFormatSpec_.equals(KeyValueTextInputFormat.class
|
|
|
+ .getCanonicalName())) {
|
|
|
+ } else if (inputFormatSpec_.equals(SequenceFileInputFormat.class
|
|
|
+ .getName())
|
|
|
+ || inputFormatSpec_
|
|
|
+ .equals(org.apache.hadoop.mapred.SequenceFileInputFormat.class
|
|
|
+ .getCanonicalName())) {
|
|
|
+ } else if (inputFormatSpec_.equals(SequenceFileAsTextInputFormat.class
|
|
|
+ .getName())
|
|
|
+ || inputFormatSpec_.equals(SequenceFileAsTextInputFormat.class
|
|
|
+ .getCanonicalName())) {
|
|
|
fmt = SequenceFileAsTextInputFormat.class;
|
|
|
} else {
|
|
|
c = StreamUtil.goodClassOrNull(inputFormatSpec_, defaultPackage);
|
|
@@ -725,14 +708,7 @@ public class StreamJob {
|
|
|
}
|
|
|
}
|
|
|
if (fmt == null) {
|
|
|
- if (testMerge_ && false == hasSimpleInputSpecs_) {
|
|
|
- // this ignores -inputreader
|
|
|
- fmt = MergerInputFormat.class;
|
|
|
- } else {
|
|
|
- // need to keep this case to support custom -inputreader
|
|
|
- // and their parameters ,n=v,n=v
|
|
|
- fmt = StreamInputFormat.class;
|
|
|
- }
|
|
|
+ fmt = StreamInputFormat.class;
|
|
|
}
|
|
|
|
|
|
jobConf_.setInputFormat(fmt);
|
|
@@ -757,13 +733,10 @@ public class StreamJob {
|
|
|
c = StreamUtil.goodClassOrNull(comCmd_, defaultPackage);
|
|
|
if (c != null) {
|
|
|
jobConf_.setCombinerClass(c);
|
|
|
- } else {
|
|
|
- jobConf_.setCombinerClass(PipeCombiner.class);
|
|
|
- jobConf_.set("stream.combine.streamprocessor", URLEncoder.encode(comCmd_, "UTF-8"));
|
|
|
- }
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
- reducerNone_ = false;
|
|
|
+ boolean reducerNone_ = false;
|
|
|
if (redCmd_ != null) {
|
|
|
reducerNone_ = redCmd_.equals(REDUCE_NONE);
|
|
|
if (redCmd_.compareToIgnoreCase("aggregate") == 0) {
|
|
@@ -801,9 +774,7 @@ public class StreamJob {
|
|
|
}
|
|
|
|
|
|
setUserJobConfProps(false);
|
|
|
- // output setup is done late so we can customize for reducerNone_
|
|
|
- //jobConf_.setOutputDir(new File(output_));
|
|
|
- setOutputSpec();
|
|
|
+ jobConf_.setOutputPath(new Path(output_));
|
|
|
fmt = null;
|
|
|
if (outputFormatSpec_!= null) {
|
|
|
c = StreamUtil.goodClassOrNull(outputFormatSpec_, defaultPackage);
|
|
@@ -812,11 +783,7 @@ public class StreamJob {
|
|
|
}
|
|
|
}
|
|
|
if (fmt == null) {
|
|
|
- if (testMerge_) {
|
|
|
- fmt = MuxOutputFormat.class;
|
|
|
- } else {
|
|
|
- fmt = TextOutputFormat.class;
|
|
|
- }
|
|
|
+ fmt = TextOutputFormat.class;
|
|
|
}
|
|
|
jobConf_.setOutputFormat(fmt);
|
|
|
|
|
@@ -831,6 +798,9 @@ public class StreamJob {
|
|
|
int numReduceTasks = Integer.parseInt(numReduceTasksSpec_);
|
|
|
jobConf_.setNumReduceTasks(numReduceTasks);
|
|
|
}
|
|
|
+ if (reducerNone_) {
|
|
|
+ jobConf_.setNumReduceTasks(0);
|
|
|
+ }
|
|
|
|
|
|
// last, allow user to override anything
|
|
|
// (although typically used with properties we didn't touch)
|
|
@@ -880,78 +850,6 @@ public class StreamJob {
|
|
|
}
|
|
|
msg("====");
|
|
|
}
|
|
|
-
|
|
|
- /** InputSpec-s encode: a glob pattern x additional column files x additional joins */
|
|
|
- protected void addInputSpec(String inSpec, int index) {
|
|
|
- if (!testMerge_) {
|
|
|
- jobConf_.addInputPath(new Path(inSpec));
|
|
|
- } else {
|
|
|
- CompoundDirSpec spec = new CompoundDirSpec(inSpec, true);
|
|
|
- msg("Parsed -input:\n" + spec.toTableString());
|
|
|
- if (index == 0) {
|
|
|
- hasSimpleInputSpecs_ = (spec.paths_.length == 0);
|
|
|
- msg("hasSimpleInputSpecs_=" + hasSimpleInputSpecs_);
|
|
|
- }
|
|
|
- String primary = spec.primarySpec();
|
|
|
- if (!seenPrimary_.add(primary)) {
|
|
|
- // this won't detect glob overlaps and noncanonical path variations
|
|
|
- fail("Primary used in multiple -input spec: " + primary);
|
|
|
- }
|
|
|
- jobConf_.addInputPath(new Path(primary));
|
|
|
- // during Job execution, will reparse into a CompoundDirSpec
|
|
|
- jobConf_.set("stream.inputspecs." + index, inSpec);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /** uses output_ and mapsideoutURI_ */
|
|
|
- protected void setOutputSpec() throws IOException {
|
|
|
- CompoundDirSpec spec = new CompoundDirSpec(output_, false);
|
|
|
- msg("Parsed -output:\n" + spec.toTableString());
|
|
|
- String primary = spec.primarySpec();
|
|
|
- String channel0;
|
|
|
- // TODO simplify cases, encapsulate in a StreamJobConf
|
|
|
- if (!reducerNone_) {
|
|
|
- channel0 = primary;
|
|
|
- } else {
|
|
|
- if (mapsideoutURI_ != null) {
|
|
|
- // user can override in case this is in a difft filesystem..
|
|
|
- try {
|
|
|
- URI uri = new URI(mapsideoutURI_);
|
|
|
- if (uri.getScheme() == null || uri.getScheme().equals("file")) { // || uri.getScheme().equals("hdfs")
|
|
|
- if (!new Path(uri.getSchemeSpecificPart()).isAbsolute()) {
|
|
|
- fail("Must be absolute: " + mapsideoutURI_);
|
|
|
- }
|
|
|
- } else if (uri.getScheme().equals("socket")) {
|
|
|
- // ok
|
|
|
- } else {
|
|
|
- fail("Invalid scheme: " + uri.getScheme() + " for -mapsideoutput " + mapsideoutURI_);
|
|
|
- }
|
|
|
- } catch (URISyntaxException e) {
|
|
|
- throw (IOException) new IOException().initCause(e);
|
|
|
- }
|
|
|
- }
|
|
|
- // an empty reduce output named "part-00002" will go here and not collide.
|
|
|
- channel0 = primary + ".NONE";
|
|
|
- // the side-effect of the first split of an input named "part-00002"
|
|
|
- // will go in this directory
|
|
|
- jobConf_.set("stream.sideoutput.dir", primary);
|
|
|
- // oops if user overrides low-level this isn't set yet :-(
|
|
|
- boolean localjt = StreamUtil.isLocalJobTracker(jobConf_);
|
|
|
- // just a guess user may prefer remote..
|
|
|
- jobConf_.setBoolean("stream.sideoutput.localfs", localjt);
|
|
|
- }
|
|
|
- // a path in fs.name.default filesystem
|
|
|
- System.out.println(channel0);
|
|
|
- System.out.println(new Path(channel0));
|
|
|
- jobConf_.setOutputPath(new Path(channel0));
|
|
|
- // will reparse remotely
|
|
|
- jobConf_.set("stream.outputspec", output_);
|
|
|
- if (null != mapsideoutURI_) {
|
|
|
- // a path in "jobtracker's filesystem"
|
|
|
- // overrides sideoutput.dir
|
|
|
- jobConf_.set("stream.sideoutput.uri", mapsideoutURI_);
|
|
|
- }
|
|
|
- }
|
|
|
|
|
|
protected String getJobTrackerHostPort() {
|
|
|
return jobConf_.get("mapred.job.tracker");
|
|
@@ -1099,15 +997,12 @@ public class StreamJob {
|
|
|
|
|
|
// command-line arguments
|
|
|
protected ArrayList inputSpecs_ = new ArrayList(); // <String>
|
|
|
- protected boolean inputTagged_ = false;
|
|
|
protected TreeSet seenPrimary_ = new TreeSet(); // <String>
|
|
|
protected boolean hasSimpleInputSpecs_;
|
|
|
protected ArrayList packageFiles_ = new ArrayList(); // <String>
|
|
|
protected ArrayList shippedCanonFiles_ = new ArrayList(); // <String>
|
|
|
- //protected ArrayList userJobConfProps_ = new ArrayList(); // <String> name=value
|
|
|
protected TreeMap<String, String> userJobConfProps_ = new TreeMap<String, String>();
|
|
|
protected String output_;
|
|
|
- protected String mapsideoutURI_;
|
|
|
protected String mapCmd_;
|
|
|
protected String comCmd_;
|
|
|
protected String redCmd_;
|
|
@@ -1125,8 +1020,6 @@ public class StreamJob {
|
|
|
protected String numReduceTasksSpec_;
|
|
|
protected String additionalConfSpec_;
|
|
|
|
|
|
- protected boolean testMerge_;
|
|
|
-
|
|
|
// Use to communicate config to the external processes (ex env.var.HADOOP_USER)
|
|
|
// encoding "a=b c=d"
|
|
|
protected String addTaskEnvironment_;
|