blob: 8f918edd1719102d4533e36f245034700722d66f [file] [log] [blame]
<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
<html>
<head>
<META http-equiv="Content-Type" content="text/html; charset=UTF-8">
<meta content="Apache Forrest" name="Generator">
<meta name="Forrest-version" content="0.8">
<meta name="Forrest-skin-name" content="pelt">
<meta name="http-equiv" content="Content-Type">
<meta name="content" content="text/html;">
<meta name="charset" content="utf-8">
<title>Hadoop Streaming</title>
<link type="text/css" href="skin/basic.css" rel="stylesheet">
<link media="screen" type="text/css" href="skin/screen.css" rel="stylesheet">
<link media="print" type="text/css" href="skin/print.css" rel="stylesheet">
<link type="text/css" href="skin/profile.css" rel="stylesheet">
<script src="skin/getBlank.js" language="javascript" type="text/javascript"></script><script src="skin/getMenu.js" language="javascript" type="text/javascript"></script><script src="skin/fontsize.js" language="javascript" type="text/javascript"></script>
<link rel="shortcut icon" href="images/favicon.ico">
</head>
<body onload="init()">
<script type="text/javascript">ndeSetTextSize();</script>
<div id="top">
<!--+
|breadtrail
+-->
<div class="breadtrail">
<a href="http://www.apache.org/">Apache</a> &gt; <a href="http://hadoop.apache.org/">Hadoop</a> &gt; <a href="http://hadoop.apache.org/core/">Core</a><script src="skin/breadcrumbs.js" language="JavaScript" type="text/javascript"></script>
</div>
<!--+
|header
+-->
<div class="header">
<!--+
|start group logo
+-->
<div class="grouplogo">
<a href="http://hadoop.apache.org/"><img class="logoImage" alt="Hadoop" src="images/hadoop-logo.jpg" title="Apache Hadoop"></a>
</div>
<!--+
|end group logo
+-->
<!--+
|start Project Logo
+-->
<div class="projectlogo">
<a href="http://hadoop.apache.org/core/"><img class="logoImage" alt="Hadoop" src="images/core-logo.gif" title="Scalable Computing Platform"></a>
</div>
<!--+
|end Project Logo
+-->
<!--+
|start Search
+-->
<div class="searchbox">
<form action="http://www.google.com/search" method="get" class="roundtopsmall">
<input value="hadoop.apache.org" name="sitesearch" type="hidden"><input onFocus="getBlank (this, 'Search the site with google');" size="25" name="q" id="query" type="text" value="Search the site with google">&nbsp;
<input name="Search" value="Search" type="submit">
</form>
</div>
<!--+
|end search
+-->
<!--+
|start Tabs
+-->
<ul id="tabs">
<li>
<a class="unselected" href="http://hadoop.apache.org/core/">Project</a>
</li>
<li>
<a class="unselected" href="http://wiki.apache.org/hadoop">Wiki</a>
</li>
<li class="current">
<a class="selected" href="index.html">Hadoop 0.17 Documentation</a>
</li>
</ul>
<!--+
|end Tabs
+-->
</div>
</div>
<div id="main">
<div id="publishedStrip">
<!--+
|start Subtabs
+-->
<div id="level2tabs"></div>
<!--+
|end Endtabs
+-->
<script type="text/javascript"><!--
document.write("Last Published: " + document.lastModified);
// --></script>
</div>
<!--+
|breadtrail
+-->
<div class="breadtrail">
&nbsp;
</div>
<!--+
|start Menu, mainarea
+-->
<!--+
|start Menu
+-->
<div id="menu">
<div onclick="SwitchMenu('menu_selected_1.1', 'skin/')" id="menu_selected_1.1Title" class="menutitle" style="background-image: url('skin/images/chapter_open.gif');">Documentation</div>
<div id="menu_selected_1.1" class="selectedmenuitemgroup" style="display: block;">
<div class="menuitem">
<a href="index.html">Overview</a>
</div>
<div class="menuitem">
<a href="quickstart.html">Quickstart</a>
</div>
<div class="menuitem">
<a href="cluster_setup.html">Cluster Setup</a>
</div>
<div class="menuitem">
<a href="hdfs_design.html">HDFS Architecture</a>
</div>
<div class="menuitem">
<a href="hdfs_user_guide.html">HDFS User Guide</a>
</div>
<div class="menuitem">
<a href="hdfs_shell.html">HDFS Shell Guide</a>
</div>
<div class="menuitem">
<a href="hdfs_permissions_guide.html">HDFS Permissions Guide</a>
</div>
<div class="menuitem">
<a href="mapred_tutorial.html">Map-Reduce Tutorial</a>
</div>
<div class="menuitem">
<a href="native_libraries.html">Native Hadoop Libraries</a>
</div>
<div class="menupage">
<div class="menupagetitle">Streaming</div>
</div>
<div class="menuitem">
<a href="hod.html">Hadoop On Demand</a>
</div>
<div class="menuitem">
<a href="api/index.html">API Docs</a>
</div>
<div class="menuitem">
<a href="http://wiki.apache.org/hadoop/">Wiki</a>
</div>
<div class="menuitem">
<a href="http://wiki.apache.org/hadoop/FAQ">FAQ</a>
</div>
<div class="menuitem">
<a href="http://hadoop.apache.org/core/mailing_lists.html">Mailing Lists</a>
</div>
<div class="menuitem">
<a href="releasenotes.html">Release Notes</a>
</div>
<div class="menuitem">
<a href="changes.html">All Changes</a>
</div>
</div>
<div id="credit"></div>
<div id="roundbottom">
<img style="display: none" class="corner" height="15" width="15" alt="" src="skin/images/rc-b-l-15-1body-2menu-3menu.png"></div>
<!--+
|alternative credits
+-->
<div id="credit2"></div>
</div>
<!--+
|end Menu
+-->
<!--+
|start content
+-->
<div id="content">
<div title="Portable Document Format" class="pdflink">
<a class="dida" href="streaming.pdf"><img alt="PDF -icon" src="skin/images/pdfdoc.gif" class="skin"><br>
PDF</a>
</div>
<h1>Hadoop Streaming</h1>
<div id="minitoc-area">
<ul class="minitoc">
<li>
<a href="#Hadoop+Streaming">Hadoop Streaming</a>
</li>
<li>
<a href="#How+Does+Streaming+Work">How Does Streaming Work </a>
</li>
<li>
<a href="#Package+Files+With+Job+Submissions">Package Files With Job Submissions</a>
</li>
<li>
<a href="#Streaming+Options+and+Usage">Streaming Options and Usage </a>
<ul class="minitoc">
<li>
<a href="#Mapper-Only+Jobs">Mapper-Only Jobs </a>
</li>
<li>
<a href="#Specifying+Other+Plugins+for+Jobs">Specifying Other Plugins for Jobs </a>
</li>
<li>
<a href="#Large+files+and+archives+in+Hadoop+Streaming">Large files and archives in Hadoop Streaming </a>
</li>
<li>
<a href="#Specifying+Additional+Configuration+Variables+for+Jobs">Specifying Additional Configuration Variables for Jobs </a>
</li>
<li>
<a href="#Other+Supported+Options">Other Supported Options </a>
</li>
</ul>
</li>
<li>
<a href="#More+usage+examples">More usage examples </a>
<ul class="minitoc">
<li>
<a href="#Customizing+the+Way+to+Split+Lines+into+Key%2FValue+Pairs">Customizing the Way to Split Lines into Key/Value Pairs </a>
</li>
<li>
<a href="#A+Useful+Partitioner+Class+%28secondary+sort%2C+the+-partitioner+org.apache.hadoop.mapred.lib.KeyFieldBasedPartitioner+option%29">A Useful Partitioner Class (secondary sort, the -partitioner org.apache.hadoop.mapred.lib.KeyFieldBasedPartitioner option) </a>
</li>
<li>
<a href="#Working+with+the+Hadoop+Aggregate+Package+%28the+-reduce+aggregate+option%29">Working with the Hadoop Aggregate Package (the -reduce aggregate option) </a>
</li>
<li>
<a href="#Field+Selection+%28+similar+to+unix+%27cut%27+command%29">Field Selection ( similar to unix 'cut' command) </a>
</li>
</ul>
</li>
<li>
<a href="#Frequently+Asked+Questions">Frequently Asked Questions </a>
<ul class="minitoc">
<li>
<a href="#How+do+I+use+Hadoop+Streaming+to+run+an+arbitrary+set+of+%28semi-%29independent+tasks%3F">How do I use Hadoop Streaming to run an arbitrary set of (semi-)independent tasks? </a>
</li>
<li>
<a href="#How+do+I+process+files%2C+one+per+map%3F">How do I process files, one per map? </a>
</li>
<li>
<a href="#How+many+reducers+should+I+use%3F">How many reducers should I use? </a>
</li>
<li>
<a href="#If+I+set+up+an+alias+in+my+shell+script%2C+will+that+work+after+-mapper%2C+i.e.+say+I+do%3A+alias+c1%3D%27cut+-f1%27.+Will+-mapper+%22c1%22+work%3F">If I set up an alias in my shell script, will that work after -mapper, i.e. say I do: alias c1='cut -f1'. Will -mapper "c1" work? </a>
</li>
<li>
<a href="#Can+I+use+UNIX+pipes%3F+For+example%2C+will+-mapper+%22cut+-f1+%7C+sed+s%2Ffoo%2Fbar%2Fg%22+work%3F">Can I use UNIX pipes? For example, will -mapper "cut -f1 | sed s/foo/bar/g" work?</a>
</li>
<li>
<a href="#When+I+run+a+streaming+job+by">When I run a streaming job by distributing large executables (for example, 3.6G) through the -file option, I get a "No space left on device" error. What do I do? </a>
</li>
<li>
<a href="#How+do+I+specify+multiple+input+directories%3F">How do I specify multiple input directories? </a>
</li>
<li>
<a href="#How+do+I+generate+output+files+with+gzip+format%3F">How do I generate output files with gzip format? </a>
</li>
<li>
<a href="#How+do+I+provide+my+own+input%2Foutput+format+with+streaming%3F">How do I provide my own input/output format with streaming? </a>
</li>
<li>
<a href="#How+do+I+parse+XML+documents+using+streaming%3F">How do I parse XML documents using streaming? </a>
</li>
</ul>
</li>
</ul>
</div>
<a name="N10019"></a><a name="Hadoop+Streaming"></a>
<h2 class="h3">Hadoop Streaming</h2>
<div class="section">
<p>
Hadoop streaming is a utility that comes with the Hadoop distribution. The utility allows you to create and run map/reduce jobs with any executable or script as the mapper and/or the reducer. For example:
</p>
<pre class="code">
$HADOOP_HOME/bin/hadoop jar $HADOOP_HOME/hadoop-streaming.jar \
-input myInputDirs \
-output myOutputDir \
-mapper /bin/cat \
-reducer /bin/wc
</pre>
</div>
<a name="N10027"></a><a name="How+Does+Streaming+Work"></a>
<h2 class="h3">How Does Streaming Work </h2>
<div class="section">
<p>
In the above example, both the mapper and the reducer are executables that read the input from stdin (line by line) and emit the output to stdout. The utility will create a map/reduce job, submit the job to an appropriate cluster, and monitor the progress of the job until it completes.
</p>
<p>
When an executable is specified for mappers, each mapper task will launch the executable as a separate process when the mapper is initialized. As the mapper task runs, it converts its inputs into lines and feed the lines to the stdin of the process. In the meantime, the mapper collects the line oriented outputs from the stdout of the process and converts each line into a key/value pair, which is collected as the output of the mapper. By default, the
<em>prefix of a line up to the first tab character</em> is the <strong>key</strong> and the the rest of the line (excluding the tab character) will be the <strong>value</strong>.
If there is no tab character in the line, then entire line is considered as key and the value is null. However, this can be customized, as discussed later.
</p>
<p>
When an executable is specified for reducers, each reducer task will launch the executable as a separate process then the reducer is initialized. As the reducer task runs, it converts its input key/values pairs into lines and feeds the lines to the stdin of the process. In the meantime, the reducer collects the line oriented outputs from the stdout of the process, converts each line into a key/value pair, which is collected as the output of the reducer. By default, the prefix of a line up to the first tab character is the key and the the rest of the line (excluding the tab character) is the value. However, this can be customized, as discussed later.
</p>
<p>
This is the basis for the communication protocol between the map/reduce framework and the streaming mapper/reducer.
</p>
<p>
You can supply a Java class as the mapper and/or the reducer. The above example is equivalent to:
</p>
<pre class="code">
$HADOOP_HOME/bin/hadoop jar $HADOOP_HOME/hadoop-streaming.jar \
-input myInputDirs \
-output myOutputDir \
-mapper org.apache.hadoop.mapred.lib.IdentityMapper \
-reducer /bin/wc
</pre>
</div>
<a name="N10047"></a><a name="Package+Files+With+Job+Submissions"></a>
<h2 class="h3">Package Files With Job Submissions</h2>
<div class="section">
<p>
You can specify any executable as the mapper and/or the reducer. The executables do not need to pre-exist on the machines in the cluster; however, if they don't, you will need to use "-file" option to tell the framework to pack your executable files as a part of job submission. For example:
</p>
<pre class="code">
$HADOOP_HOME/bin/hadoop jar $HADOOP_HOME/hadoop-streaming.jar \
-input myInputDirs \
-output myOutputDir \
-mapper myPythonScript.py \
-reducer /bin/wc \
-file myPythonScript.py
</pre>
<p>
The above example specifies a user defined Python executable as the mapper. The option "-file myPythonScript.py" causes the python executable shipped to the cluster machines as a part of job submission.
</p>
<p>
In addition to executable files, you can also package other auxiliary files (such as dictionaries, configuration files, etc) that may be used by the mapper and/or the reducer. For example:
</p>
<pre class="code">
$HADOOP_HOME/bin/hadoop jar $HADOOP_HOME/hadoop-streaming.jar \
-input myInputDirs \
-output myOutputDir \
-mapper myPythonScript.py \
-reducer /bin/wc \
-file myPythonScript.py \
-file myDictionary.txt
</pre>
</div>
<a name="N1005F"></a><a name="Streaming+Options+and+Usage"></a>
<h2 class="h3">Streaming Options and Usage </h2>
<div class="section">
<a name="N10065"></a><a name="Mapper-Only+Jobs"></a>
<h3 class="h4">Mapper-Only Jobs </h3>
<p>
Often, you may want to process input data using a map function only. To do this, simply set mapred.reduce.tasks to zero. The map/reduce framework will not create any reducer tasks. Rather, the outputs of the mapper tasks will be the final output of the job.
</p>
<p>
To be backward compatible, Hadoop Streaming also supports the "-reduce NONE" option, which is equivalent to "-jobconf mapred.reduce.tasks=0".
</p>
<a name="N10071"></a><a name="Specifying+Other+Plugins+for+Jobs"></a>
<h3 class="h4">Specifying Other Plugins for Jobs </h3>
<p>
Just as with a normal map/reduce job, you can specify other plugins for a streaming job:
</p>
<pre class="code">
-inputformat JavaClassName
-outputformat JavaClassName
-partitioner JavaClassName
-combiner JavaClassName
</pre>
<p>
The class you supply for the input format should return key/value pairs of Text class. If you do not specify an input format class, the TextInputFormat is used as the default. Since the TextInputFormat returns keys of LongWritable class, which are actually not part of the input data, the keys will be discarded; only the values will be piped to the streaming mapper.
</p>
<p>
The class you supply for the output format is expected to take key/value pairs of Text class. If you do not specify an output format class, the TextOutputFormat is used as the default.
</p>
<a name="N10084"></a><a name="Large+files+and+archives+in+Hadoop+Streaming"></a>
<h3 class="h4">Large files and archives in Hadoop Streaming </h3>
<p>
The -cacheFile and -cacheArchive options allow you to make files and archives available to the tasks. The argument is a URI to the file or archive that you have already uploaded to HDFS. These files and archives are cached across jobs. You can retrieve the host and fs_port values from the fs.default.name config variable.
</p>
<p>
Here are examples of the -cacheFile option:
</p>
<pre class="code">
-cacheFile hdfs://host:fs_port/user/testfile.txt#testlink
</pre>
<p>
In the above example, the part of the url after # is used as the symlink name that is created in the current working directory of tasks. So the tasks will have a symlink called testlink in the cwd that points to a local copy of testfile.txt. Multiple entries can be specified as:
</p>
<pre class="code">
-cacheFile hdfs://host:fs_port/user/testfile1.txt#testlink1 -cacheFile hdfs://host:fs_port/user/testfile2.txt#testlink2
</pre>
<p>
The -cacheArchive option allows you to copy jars locally to the cwd of tasks and automatically unjar the files. For example:
</p>
<pre class="code">
-cacheArchive hdfs://host:fs_port/user/testfile.jar#testlink3
</pre>
<p>
In the example above, a symlink testlink3 is created in the current working directory of tasks. This symlink points to the directory that stores the unjarred contents of the uploaded jar file.
</p>
<p>
Here's another example of the -cacheArchive option. Here, the input.txt file has two lines specifying the names of the two files: testlink/cache.txt and testlink/cache2.txt. "testlink" is a symlink to the archived directory, which has the files "cache.txt" and "cache2.txt".
</p>
<pre class="code">
$HADOOP_HOME/bin/hadoop jar $HADOOP_HOME/hadoop-streaming.jar \
-input "/user/me/samples/cachefile/input.txt" \
-mapper "xargs cat" \
-reducer "cat" \
-output "/user/me/samples/cachefile/out" \
-cacheArchive 'hdfs://hadoop-nn1.example.com:8020/user/me/samples/cachefile/cachedir.jar#testlink' \
-jobconf mapred.map.tasks=1 \
-jobconf mapred.reduce.tasks=1 \
-jobconf mapred.job.name="Experiment"
$ ls test_jar/
cache.txt cache2.txt
$ jar cvf cachedir.jar -C test_jar/ .
added manifest
adding: cache.txt(in = 30) (out= 29)(deflated 3%)
adding: cache2.txt(in = 37) (out= 35)(deflated 5%)
$ hadoop dfs -put cachedir.jar samples/cachefile
$ hadoop dfs -cat /user/me/samples/cachefile/input.txt
testlink/cache.txt
testlink/cache2.txt
$ cat test_jar/cache.txt
This is just the cache string
$ cat test_jar/cache2.txt
This is just the second cache string
$ hadoop dfs -ls /user/me/samples/cachefile/out
Found 1 items
/user/me/samples/cachefile/out/part-00000 &lt;r 3&gt; 69
$ hadoop dfs -cat /user/me/samples/cachefile/out/part-00000
This is just the cache string
This is just the second cache string
</pre>
<a name="N100AD"></a><a name="Specifying+Additional+Configuration+Variables+for+Jobs"></a>
<h3 class="h4">Specifying Additional Configuration Variables for Jobs </h3>
<p>
You can specify additional configuration variables by using "-jobconf &lt;n&gt;=&lt;v&gt;". For example:
</p>
<pre class="code">
$HADOOP_HOME/bin/hadoop jar $HADOOP_HOME/hadoop-streaming.jar \
-input myInputDirs \
-output myOutputDir \
-mapper org.apache.hadoop.mapred.lib.IdentityMapper\
-reducer /bin/wc \
-jobconf mapred.reduce.tasks=2
</pre>
<p>
The -jobconf mapred.reduce.tasks=2 in the above example specifies to use two reducers for the job.
</p>
<p>
For more details on the jobconf parameters see: <a href="http://wiki.apache.org/hadoop/JobConfFile">http://wiki.apache.org/hadoop/JobConfFile</a>
</p>
<a name="N100C4"></a><a name="Other+Supported+Options"></a>
<h3 class="h4">Other Supported Options </h3>
<p>
Other options you may specify for a streaming job are described here:
</p>
<table class="ForrestTable" cellspacing="1" cellpadding="4">
<tr>
<th colspan="1" rowspan="1">Parameter</th><th colspan="1" rowspan="1">Optional/Required </th><th colspan="1" rowspan="1">Description </th>
</tr>
<tr>
<td colspan="1" rowspan="1"> -cluster name </td><td colspan="1" rowspan="1"> Optional </td><td colspan="1" rowspan="1"> Switch between local Hadoop and one or more remote clusters </td>
</tr>
<tr>
<td colspan="1" rowspan="1"> -dfs host:port or local </td><td colspan="1" rowspan="1"> Optional </td><td colspan="1" rowspan="1"> Override the DFS configuration for the job </td>
</tr>
<tr>
<td colspan="1" rowspan="1"> -jt host:port or local </td><td colspan="1" rowspan="1"> Optional </td><td colspan="1" rowspan="1"> Override the JobTracker configuration for the job </td>
</tr>
<tr>
<td colspan="1" rowspan="1"> -additionalconfspec specfile </td><td colspan="1" rowspan="1"> Optional </td><td colspan="1" rowspan="1"> Specifies a set of configuration variables in an XML file like hadoop-site.xml, instead of using multiple options of type "-jobconf name=value" </td>
</tr>
<tr>
<td colspan="1" rowspan="1"> -cmdenv name=value </td><td colspan="1" rowspan="1"> Optional </td><td colspan="1" rowspan="1"> Pass env var to streaming commands </td>
</tr>
<tr>
<td colspan="1" rowspan="1"> -cacheFile fileNameURI </td><td colspan="1" rowspan="1"> Optional </td><td colspan="1" rowspan="1"> Specify a file to be uploaded to the HDFS </td>
</tr>
<tr>
<td colspan="1" rowspan="1"> -cacheArchive fileNameURI </td><td colspan="1" rowspan="1"> Optional </td><td colspan="1" rowspan="1"> Specify a jar file to be uploaded to the HDFS. This jar file is unjarred automatically in the cwd of the task </td>
</tr>
<tr>
<td colspan="1" rowspan="1"> -inputreader JavaClassName </td><td colspan="1" rowspan="1"> Optional </td><td colspan="1" rowspan="1"> For backwards-compatibility: specifies a record reader class (instead of an input format class) </td>
</tr>
<tr>
<td colspan="1" rowspan="1"> -verbose </td><td colspan="1" rowspan="1"> Optional </td><td colspan="1" rowspan="1"> Verbose output </td>
</tr>
</table>
<p>
To switch between "local" Hadoop and one or more remote Hadoop clusters use -cluster &lt;name&gt;.
By default, hadoop-default.xml and hadoop-site.xml are used. The -cluster &lt;name&gt; option will cause $HADOOP_HOME/conf/hadoop-&lt;name&gt;.xml to be used instead.
</p>
<p>
To change the local temp directory use:
</p>
<pre class="code">
-jobconf dfs.data.dir=/tmp
</pre>
<p>
To specify additional local temp directories use:
</p>
<pre class="code">
-jobconf mapred.local.dir=/tmp/local
-jobconf mapred.system.dir=/tmp/system
-jobconf mapred.temp.dir=/tmp/temp
</pre>
<p>
For more details on jobconf parameters see: <a href="http://wiki.apache.org/hadoop/JobConfFile">http://wiki.apache.org/hadoop/JobConfFile</a>
</p>
<p>
To set an environment variable in a streaming command use:
</p>
<pre class="code">
-cmdenv EXAMPLE_DIR=/home/example/dictionaries/
</pre>
</div>
<a name="N1017C"></a><a name="More+usage+examples"></a>
<h2 class="h3">More usage examples </h2>
<div class="section">
<a name="N10182"></a><a name="Customizing+the+Way+to+Split+Lines+into+Key%2FValue+Pairs"></a>
<h3 class="h4">Customizing the Way to Split Lines into Key/Value Pairs </h3>
<p>
As noted earlier, when the map/reduce framework reads a line from the stdout of the mapper, it splits the line into a key/value pair. By default, the prefix of the line up to the first tab character is the key and the the rest of the line (excluding the tab character) is the value.
</p>
<p>
However, you can customize this default. You can specify a field separator other than the tab character (the default), and you can specify the nth (n &gt;= 1) character rather than the first character in a line (the default) as the separator between the key and value. For example:
</p>
<pre class="code">
$HADOOP_HOME/bin/hadoop jar $HADOOP_HOME/hadoop-streaming.jar \
-input myInputDirs \
-output myOutputDir \
-mapper org.apache.hadoop.mapred.lib.IdentityMapper \
-reducer org.apache.hadoop.mapred.lib.IdentityReducer \
-jobconf stream.map.output.field.separator=. \
-jobconf stream.num.map.output.key.fields=4
</pre>
<p>
In the above example, "-jobconf stream.map.output.field.separator=." specifies "." as the field separator for the map outputs, and the prefix up to the fourth "." in a line will be the key and the rest of the line (excluding the fourth ".") will be the value. If a line has less than four "."s, then the whole line will be the key and the value will be an empty Text object (like the one created by new Text("")).
</p>
<p>
Similarly, you can use "-jobconf stream.reduce.output.field.separator=SEP" and "-jobconf stream.num.reduce.output.fields=NUM" to specify the nth field separator in a line of the reduce outputs as the separator between the key and the value.
</p>
<a name="N10198"></a><a name="A+Useful+Partitioner+Class+%28secondary+sort%2C+the+-partitioner+org.apache.hadoop.mapred.lib.KeyFieldBasedPartitioner+option%29"></a>
<h3 class="h4">A Useful Partitioner Class (secondary sort, the -partitioner org.apache.hadoop.mapred.lib.KeyFieldBasedPartitioner option) </h3>
<p>
Hadoop has a library class, org.apache.hadoop.mapred.lib.KeyFieldBasedPartitioner, that is useful for many applications. This class allows the map/reduce framework to partition the map outputs based on prefixes of keys, not the whole keys. For example:
</p>
<pre class="code">
$HADOOP_HOME/bin/hadoop jar $HADOOP_HOME/hadoop-streaming.jar \
-input myInputDirs \
-output myOutputDir \
-mapper org.apache.hadoop.mapred.lib.IdentityMapper \
-reducer org.apache.hadoop.mapred.lib.IdentityReducer \
-partitioner org.apache.hadoop.mapred.lib.KeyFieldBasedPartitioner \
-jobconf stream.map.output.field.separator=. \
-jobconf stream.num.map.output.key.fields=4 \
-jobconf map.output.key.field.separator=. \
-jobconf num.key.fields.for.partition=2 \
-jobconf mapred.reduce.tasks=12
</pre>
<p>
Here, <em>-jobconf stream.map.output.field.separator=.</em> and <em>-jobconf stream.num.map.output.key.fields=4</em> are as explained in previous example. The two variables are used by streaming to identify the key/value pair of mapper.
</p>
<p>
The map output keys of the above map/reduce job normally have four fields separated by ".". However, the map/reduce framework will partition the map outputs by the first two fields of the keys using the <em>-jobconf num.key.fields.for.partition=2</em> option. Here, <em>-jobconf map.output.key.field.separator=.</em> specifies the separator for the partition. This guarantees that all the key/value pairs with the same first two fields in the keys will be partitioned into the same reducer.
</p>
<p>
<em>This is effectively equivalent to specifying the first two fields as the primary key and the next two fields as the secondary. The primary key is used for partitioning, and the combination of the primary and secondary keys is used for sorting.</em> A simple illustration is shown here:
</p>
<p>
Output of map (the keys)</p>
<pre class="code">
11.12.1.2
11.14.2.3
11.11.4.1
11.12.1.1
11.14.2.2
</pre>
<p>
Partition into 3 reducers (the first 2 fields are used as keys for partition)</p>
<pre class="code">
11.11.4.1
-----------
11.12.1.2
11.12.1.1
-----------
11.14.2.3
11.14.2.2
</pre>
<p>
Sorting within each partition for the reducer(all 4 fields used for sorting)</p>
<pre class="code">
11.11.4.1
-----------
11.12.1.1
11.12.1.2
-----------
11.14.2.2
11.14.2.3
</pre>
<a name="N101CE"></a><a name="Working+with+the+Hadoop+Aggregate+Package+%28the+-reduce+aggregate+option%29"></a>
<h3 class="h4">Working with the Hadoop Aggregate Package (the -reduce aggregate option) </h3>
<p>
Hadoop has a library package called "Aggregate" (<a href="https://svn.apache.org/repos/asf/hadoop/core/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate">https://svn.apache.org/repos/asf/hadoop/core/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate</a>). Aggregate provides a special reducer class and a special combiner class, and a list of simple aggregators that perform aggregations such as "sum", "max", "min" and so on over a sequence of values. Aggregate allows you to define a mapper plugin class that is expected to generate "aggregatable items" for each input key/value pair of the mappers. The combiner/reducer will aggregate those aggregatable items by invoking the appropriate aggregators.
</p>
<p>
To use Aggregate, simply specify "-reducer aggregate":
</p>
<pre class="code">
$HADOOP_HOME/bin/hadoop jar $HADOOP_HOME/hadoop-streaming.jar \
-input myInputDirs \
-output myOutputDir \
-mapper myAggregatorForKeyCount.py \
-reducer aggregate \
-file myAggregatorForKeyCount.py \
-jobconf mapred.reduce.tasks=12
</pre>
<p>
The python program myAggregatorForKeyCount.py looks like:
</p>
<pre class="code">
#!/usr/bin/python
import sys;
def generateLongCountToken(id):
return "LongValueSum:" + id + "\t" + "1"
def main(argv):
line = sys.stdin.readline();
try:
while line:
line = line[:-1];
fields = line.split("\t");
print generateLongCountToken(fields[0]);
line = sys.stdin.readline();
except "end of file":
return None
if __name__ == "__main__":
main(sys.argv)
</pre>
<a name="N101E9"></a><a name="Field+Selection+%28+similar+to+unix+%27cut%27+command%29"></a>
<h3 class="h4">Field Selection ( similar to unix 'cut' command) </h3>
<p>
Hadoop has a library class, org.apache.hadoop.mapred.lib.FieldSelectionMapReduce, that effectively allows you to process text data like the unix "cut" utility. The map function defined in the class treats each input key/value pair as a list of fields. You can specify the field separator (the default is the tab character). You can select an arbitrary list of fields as the map output key, and an arbitrary list of fields as the map output value. Similarly, the reduce function defined in the class treats each input key/value pair as a list of fields. You can select an arbitrary list of fields as the reduce output key, and an arbitrary list of fields as the reduce output value. For example:
</p>
<pre class="code">
$HADOOP_HOME/bin/hadoop jar $HADOOP_HOME/hadoop-streaming.jar \
-input myInputDirs \
-output myOutputDir \
-mapper org.apache.hadoop.mapred.lib.FieldSelectionMapReduce\
-reducer org.apache.hadoop.mapred.lib.FieldSelectionMapReduce\
-partitioner org.apache.hadoop.mapred.lib.KeyFieldBasedPartitioner \
-jobconf map.output.key.field.separa=. \
-jobconf num.key.fields.for.partition=2 \
-jobconf mapred.data.field.separator=. \
-jobconf map.output.key.value.fields.spec=6,5,1-3:0- \
-jobconf reduce.output.key.value.fields.spec=0-2:5- \
-jobconf mapred.reduce.tasks=12
</pre>
<p>
The option "-jobconf map.output.key.value.fields.spec=6,5,1-3:0-" specifies key/value selection for the map outputs. Key selection spec and value selection spec are separated by ":". In this case, the map output key will consist of fields 6, 5, 1, 2, and 3. The map output value will consist of all fields (0- means field 0 and all
the subsequent fields).
</p>
<p>
The option "-jobconf reduce.output.key.value.fields.spec=0-2:0-" specifies key/value selection for the reduce outputs. In this case, the reduce output key will consist of fields 0, 1, 2 (corresponding to the original fields 6, 5, 1). The reduce output value will consist of all fields starting from field 5 (corresponding to all the original fields).
</p>
</div>
<a name="N101FD"></a><a name="Frequently+Asked+Questions"></a>
<h2 class="h3">Frequently Asked Questions </h2>
<div class="section">
<a name="N10203"></a><a name="How+do+I+use+Hadoop+Streaming+to+run+an+arbitrary+set+of+%28semi-%29independent+tasks%3F"></a>
<h3 class="h4">How do I use Hadoop Streaming to run an arbitrary set of (semi-)independent tasks? </h3>
<p>
Often you do not need the full power of Map Reduce, but only need to run multiple instances of the same program - either on different parts of the data, or on the same data, but with different parameters. You can use Hadoop Streaming to do this.
</p>
<a name="N1020D"></a><a name="How+do+I+process+files%2C+one+per+map%3F"></a>
<h3 class="h4">How do I process files, one per map? </h3>
<p>
As an example, consider the problem of zipping (compressing) a set of files across the hadoop cluster. You can achieve this using either of these methods:
</p>
<ol>
<li> Hadoop Streaming and custom mapper script:<ul>
<li> Generate a file containing the full DFS path of the input files. Each map task would get one file name as input.</li>
<li> Create a mapper script which, given a filename, will get the file to local disk, gzip the file and put it back in the desired output directory</li>
</ul>
</li>
<li>The existing Hadoop Framework:<ul>
<li>Add these commands to your main function:
<pre class="code">
OutputFormatBase.setCompressOutput(conf, true);
OutputFormatBase.setOutputCompressorClass(conf, org.apache.hadoop.io.compress.GzipCodec.class);
conf.setOutputFormat(NonSplitableTextInputFormat.class);
conf.setNumReduceTasks(0);
</pre>
</li>
<li>Write your map function:
<pre class="code">
public void map(WritableComparable key, Writable value,
OutputCollector output,
Reporter reporter) throws IOException {
output.collect((Text)value, null);
}
</pre>
</li>
<li>Note that the output filename will not be the same as the original filename</li>
</ul>
</li>
</ol>
<a name="N10238"></a><a name="How+many+reducers+should+I+use%3F"></a>
<h3 class="h4">How many reducers should I use? </h3>
<p>
See the Hadoop Wiki for details: <a href="http://wiki.apache.org/hadoop/HowManyMapsAndReduces">http://wiki.apache.org/hadoop/HowManyMapsAndReduces</a>
</p>
<a name="N10246"></a><a name="If+I+set+up+an+alias+in+my+shell+script%2C+will+that+work+after+-mapper%2C+i.e.+say+I+do%3A+alias+c1%3D%27cut+-f1%27.+Will+-mapper+%22c1%22+work%3F"></a>
<h3 class="h4">If I set up an alias in my shell script, will that work after -mapper, i.e. say I do: alias c1='cut -f1'. Will -mapper "c1" work? </h3>
<p>
Using an alias will not work, but variable substitution is allowed as shown in this example:
</p>
<pre class="code">
$ hadoop dfs -cat samples/student_marks
alice 50
bruce 70
charlie 80
dan 75
$ c2='cut -f2'; $HADOOP_HOME/bin/hadoop jar $HADOOP_HOME/hadoop-streaming.jar \
-input /user/me/samples/student_marks
-mapper \"$c2\" -reducer 'cat'
-output /user/me/samples/student_out
-jobconf mapred.job.name='Experiment'
$ hadoop dfs -ls samples/student_out
Found 1 items/user/me/samples/student_out/part-00000 &lt;r 3&gt; 16
$ hadoop dfs -cat samples/student_out/part-00000
50
70
75
80
</pre>
<a name="N10254"></a><a name="Can+I+use+UNIX+pipes%3F+For+example%2C+will+-mapper+%22cut+-f1+%7C+sed+s%2Ffoo%2Fbar%2Fg%22+work%3F"></a>
<h3 class="h4">Can I use UNIX pipes? For example, will -mapper "cut -f1 | sed s/foo/bar/g" work?</h3>
<p>
Currently this does not work and gives an "java.io.IOException: Broken pipe" error. This is probably a bug that needs to be investigated.
</p>
<a name="N1025E"></a><a name="When+I+run+a+streaming+job+by"></a>
<h3 class="h4">When I run a streaming job by distributing large executables (for example, 3.6G) through the -file option, I get a "No space left on device" error. What do I do? </h3>
<p>
The jar packaging happens in a directory pointed to by the configuration variable stream.tmpdir. The default value of stream.tmpdir is /tmp. Set the value to a directory with more space:
</p>
<pre class="code">
-jobconf stream.tmpdir=/export/bigspace/...
</pre>
<a name="N1026F"></a><a name="How+do+I+specify+multiple+input+directories%3F"></a>
<h3 class="h4">How do I specify multiple input directories? </h3>
<p>
You can specify multiple input directories with multiple '-input' options:
</p>
<pre class="code">
hadoop jar hadoop-streaming.jar -input '/user/foo/dir1' -input '/user/foo/dir2'
</pre>
<a name="N1027C"></a><a name="How+do+I+generate+output+files+with+gzip+format%3F"></a>
<h3 class="h4">How do I generate output files with gzip format? </h3>
<p>
Instead of plain text files, you can generate gzip files as your generated output. Pass '-jobconf mapred.output.compress=true -jobconf mapred.output.compression.codec=org.apache.hadoop.io.compress.GzipCode' as option to your streaming job.
</p>
<a name="N10286"></a><a name="How+do+I+provide+my+own+input%2Foutput+format+with+streaming%3F"></a>
<h3 class="h4">How do I provide my own input/output format with streaming? </h3>
<p>
At least as late as version 0.14, Hadoop does not support multiple jar files. So, when specifying your own custom classes you will have to pack them along with the streaming jar and use the custom jar instead of the default hadoop streaming jar.
</p>
<a name="N10290"></a><a name="How+do+I+parse+XML+documents+using+streaming%3F"></a>
<h3 class="h4">How do I parse XML documents using streaming? </h3>
<p>
You can use the record reader StreamXmlRecordReader to process XML documents.
</p>
<pre class="code">
hadoop jar hadoop-streaming.jar -inputreader "StreamXmlRecord,begin=BEGIN_STRING,end=END_STRING" ..... (rest of the command)
</pre>
<p>
Anything found between BEGIN_STRING and END_STRING would be treated as one record for map tasks.
</p>
</div>
</div>
<!--+
|end content
+-->
<div class="clearboth">&nbsp;</div>
</div>
<div id="footer">
<!--+
|start bottomstrip
+-->
<div class="lastmodified">
<script type="text/javascript"><!--
document.write("Last Published: " + document.lastModified);
// --></script>
</div>
<div class="copyright">
Copyright &copy;
2007 <a href="http://www.apache.org/licenses/">The Apache Software Foundation.</a>
</div>
<!--+
|end bottomstrip
+-->
</div>
</body>
</html>