hadoop-common-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From omal...@apache.org
Subject svn commit: r601817 - in /lucene/hadoop/trunk: CHANGES.txt src/docs/src/documentation/content/xdocs/site.xml src/docs/src/documentation/content/xdocs/streaming.xml
Date Thu, 06 Dec 2007 19:04:05 GMT
Author: omalley
Date: Thu Dec  6 11:04:03 2007
New Revision: 601817

URL: http://svn.apache.org/viewvc?rev=601817&view=rev
Log:
HADOOP-1327. Adding user documentation for streaming.

Added:
    lucene/hadoop/trunk/src/docs/src/documentation/content/xdocs/streaming.xml
Modified:
    lucene/hadoop/trunk/CHANGES.txt
    lucene/hadoop/trunk/src/docs/src/documentation/content/xdocs/site.xml

Modified: lucene/hadoop/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/CHANGES.txt?rev=601817&r1=601816&r2=601817&view=diff
==============================================================================
--- lucene/hadoop/trunk/CHANGES.txt (original)
+++ lucene/hadoop/trunk/CHANGES.txt Thu Dec  6 11:04:03 2007
@@ -103,6 +103,9 @@
     Earlier, it was a random backoff from an interval. 
     (Amar Kamat via ddas)
 
+    HADOOP-1327.  Include website documentation for streaming. (Rob Weltman
+    via omalley)
+
   OPTIMIZATIONS
 
     HADOOP-1898.  Release the lock protecting the last time of the last stack

Modified: lucene/hadoop/trunk/src/docs/src/documentation/content/xdocs/site.xml
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/docs/src/documentation/content/xdocs/site.xml?rev=601817&r1=601816&r2=601817&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/docs/src/documentation/content/xdocs/site.xml (original)
+++ lucene/hadoop/trunk/src/docs/src/documentation/content/xdocs/site.xml Thu Dec  6 11:04:03
2007
@@ -22,6 +22,7 @@
     <setup     label="Cluster Setup"      href="cluster_setup.html" />
     <hdfs      label="HDFS Architecture"  href="hdfs_design.html" />
     <mapred    label="Map-Reduce Tutorial" href="mapred_tutorial.html" />
+    <streaming label="Streaming"          href="streaming.html" />
     <api       label="API Docs"           href="ext:api/index" />
     <wiki      label="Wiki"               href="ext:wiki" />
     <faq       label="FAQ"                href="ext:faq" />

Added: lucene/hadoop/trunk/src/docs/src/documentation/content/xdocs/streaming.xml
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/docs/src/documentation/content/xdocs/streaming.xml?rev=601817&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/docs/src/documentation/content/xdocs/streaming.xml (added)
+++ lucene/hadoop/trunk/src/docs/src/documentation/content/xdocs/streaming.xml Thu Dec  6
11:04:03 2007
@@ -0,0 +1,542 @@
+<?xml version="1.0"?>
+
+<!DOCTYPE document PUBLIC "-//APACHE//DTD Documentation V2.0//EN"
+          "http://forrest.apache.org/dtd/document-v20.dtd">
+
+
+<document>
+<header>
+<title>Hadoop Streaming</title>
+<meta name="http-equiv">Content-Type</meta>
+<meta name="content">text/html;</meta>
+<meta name="charset">utf-8</meta>
+</header>
+<body>
+<section>
+<title>Hadoop Streaming</title>
+
+<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>
+<source>
+$HADOOP_HOME/bin/hadoop  jar $HADOOP_HOME/hadoop-streaming.jar \
+    -input myInputDirs \
+    -output myOutputDir \
+    -mapper /bin/cat \
+    -reducer /bin/wc
+</source>
+</section>
+
+<section>
+<title>How Does Streaming Work </title>
+<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>.
However, this can be customized, as <a href="#Customizing_the_Way_to_Split_Lin">discussed
later</a>.
+</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
<a href="#Customizing_the_Way_to_Split_Lin">discussed later</a>.
+</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>
+<source>
+$HADOOP_HOME/bin/hadoop  jar $HADOOP_HOME/hadoop-streaming.jar \
+    -input myInputDirs \
+    -output myOutputDir \
+    -mapper org.apache.hadoop.mapred.lib.IdentityMapper \
+    -reducer /bin/wc
+</source>
+</section>
+
+<section>
+<title>Package Files With Job Submissions</title>
+<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>
+<source>
+$HADOOP_HOME/bin/hadoop  jar $HADOOP_HOME/hadoop-streaming.jar \
+    -input myInputDirs \
+    -output myOutputDir \
+    -mapper myPythonScript.py \
+    -reducer /bin/wc \
+    -file myPythonScript.py 
+</source>
+<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>
+<source>
+$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
+</source>
+</section>
+
+<section>
+<title>Streaming Options and Usage </title>
+
+<section>
+<title>Mapper-Only Jobs </title>
+<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>
+</section>
+
+<section>
+<title>Specifying Other Plugins for Jobs </title>
+<p>
+Just as with a normal map/reduce job, you can specify other plugins for a streaming job:
+</p>
+<source>
+   -inputformat JavaClassName
+   -outputformat JavaClassName
+   -partitioner JavaClassName
+   -combiner JavaClassName
+</source>
+<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>
+</section>
+
+<section>
+<title>Large files and archives in Hadoop Streaming </title>
+
+<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> 
+<source>
+-cacheFile hdfs://host:fs_port/user/testfile.txt#testlink
+</source>
+<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>
+<source>
+-cacheFile hdfs://host:fs_port/user/testfile1.txt#testlink1 -cacheFile hdfs://host:fs_port/user/testfile2.txt#testlink2
+</source>
+<p>
+The -cacheArchive option allows you to copy jars locally to the cwd of tasks and automatically
unjar the files. For example:
+</p>
+<source>
+-cacheArchive hdfs://host:fs_port/user/testfile.jar#testlink3
+</source>
+<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>
+<source>
+$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
+
+</source>
+</section>
+
+<section>
+<title>Specifying Additional Configuration Variables for Jobs </title>
+<p>
+You can specify additional configuration variables by using "-jobconf  &lt;n&gt;=&lt;v&gt;".
For example: 
+</p>
+<source>
+$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
+</source>
+<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/lucene-hadoop/JobConfFile">http://wiki.apache.org/lucene-hadoop/JobConfFile</a></p>
+</section>
+
+<section>
+<title>Other Supported Options </title>
+<p>
+Other options you may specify for a streaming job are described here:
+</p>
+<table>
+<tr><th>Parameter</th><th>Optional/Required </th><th>Description
</th></tr>
+<tr><td> -cluster name </td><td> Optional </td><td> Switch
between local Hadoop and one or more remote clusters </td></tr>
+
+<tr><td> -dfs  host:port or local </td><td> Optional </td><td>
Override the DFS configuration for the job </td></tr>
+<tr><td> -jt host:port or local </td><td> Optional </td><td>
Override the JobTracker configuration for the job </td></tr>
+<tr><td> -additionalconfspec specfile </td><td> Optional </td><td>
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> -cmdenv   name=value </td><td> Optional </td><td>
Pass env var to streaming commands </td></tr>
+<tr><td> -cacheFile fileNameURI </td><td> Optional </td><td>
Specify a file to be uploaded to the HDFS </td></tr>
+<tr><td> -cacheArchive fileNameURI </td><td> Optional </td><td>
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> -inputreader JavaClassName </td><td> Optional </td><td>
For backwards-compatibility: specifies a record reader class (instead of an input format class)
</td></tr>
+<tr><td> -verbose </td><td> Optional </td><td> 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>
+<source>
+  -jobconf dfs.data.dir=/tmp
+</source>
+<p>
+To specify additional local temp directories use:
+</p>
+<source>
+   -jobconf mapred.local.dir=/tmp/local
+   -jobconf mapred.system.dir=/tmp/system
+   -jobconf mapred.temp.dir=/tmp/temp
+</source>
+<p>
+For more details on jobconf parameters see: <a href="http://wiki.apache.org/lucene-hadoop/JobConfFile">http://wiki.apache.org/lucene-hadoop/JobConfFile</a>
+</p><p>
+To set an environment variable in a streaming command use:
+</p>
+<source>
+-cmdenv EXAMPLE_DIR=/home/example/dictionaries/
+</source>
+</section>
+</section>
+
+<section>
+<title>More usage examples </title>
+
+<section>
+<title>Customizing the Way to Split Lines into Key/Value Pairs </title>
+<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 >= 1) character rather than
the first character in a line (the default) as the separator between the key and value. For
example:
+</p>
+
+<source>
+$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 
+</source>
+<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>
+</section>
+
+
+<section>
+<title>A Useful Partitioner Class (secondary sort, the -partitioner org.apache.hadoop.mapred.lib.KeyFieldBasedPartitioner
option) </title>
+<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>
+<source>
+$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
+</source>
+<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><source>
+11.12.1.2
+11.14.2.3
+11.11.4.1
+11.12.1.1
+11.14.2.2
+
+</source>
+<p>
+Partition into 3 reducers (the first 2 fields are used as keys for partition)</p><source>
+11.11.4.1
+-----------
+11.12.1.2
+11.12.1.1
+-----------
+11.14.2.3
+11.14.2.2
+</source>
+<p>
+Sorting within each partition for the reducer(all 4 fields used for sorting)</p><source>
+11.11.4.1
+-----------
+11.12.1.1
+11.12.1.2
+-----------
+11.14.2.2
+11.14.2.3
+</source>
+</section>
+
+<section>
+<title>Working with the Hadoop Aggregate Package (the -reduce aggregate option) </title>
+<p>
+Hadoop has a library package called "Aggregate" (<a href="https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/lib/aggregate">https://svn.apache.org/repos/asf/lucene/hadoop/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>
+<source>
+$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
+</source>
+<p>
+The python program myAggregatorForKeyCount.py looks like:
+</p>
+<source>
+#!/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&#91;:-1];
+            fields = line.split("\t");
+            print generateLongCountToken(fields&#91;0]);
+            line = sys.stdin.readline();
+    except "end of file":
+        return None
+if __name__ == "__main__":
+     main(sys.argv)
+</source>
+</section>
+
+<section>
+<title>Field Selection ( similar to unix 'cut' command) </title>
+<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>
+<source>
+$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
+</source>
+<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>
+</section>
+</section>
+
+<section>
+<title>Frequently Asked Questions </title>
+
+<section>
+<title>How do I use Hadoop Streaming to run an arbitrary set of (semi-)independent
tasks? </title>
+<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>
+
+</section>
+
+<section>
+<title>How do I process files, one per map? </title>
+<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:
+<source>
+       OutputFormatBase.setCompressOutput(conf, true);
+       OutputFormatBase.setOutputCompressorClass(conf, org.apache.hadoop.io.compress.GzipCodec.class);
+       conf.setOutputFormat(NonSplitableTextInputFormat.class);
+       conf.setNumReduceTasks(0);
+</source></li>
+   <li>Write your map function:
+<source>
+
+       public void map(WritableComparable key, Writable value, 
+                               OutputCollector output, 
+                               Reporter reporter) throws IOException {
+            output.collect((Text)value, null);
+       }
+</source></li>
+  <li>Note that the output filename will not be the same as the original filename</li>
+</ul></li>
+</ol>
+</section>
+
+<section>
+<title>How many reducers should I use? </title>
+<p>
+See the Hadoop Wiki for details: <a href="http://wiki.apache.org/lucene-hadoop/HowManyMapsAndReduces">http://wiki.apache.org/lucene-hadoop/HowManyMapsAndReduces</a>
+</p>
+</section>
+
+<section>
+<title>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? </title>
+<p>
+Using an alias will not work, but variable substitution is allowed as shown in this example:
+</p>
+<source>
+$ 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
+</source>
+</section>
+
+<section>
+<title>Can I use UNIX pipes? For example, will -mapper "cut -f1 | sed s/foo/bar/g"
work?</title>
+<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>
+</section>
+
+<section>
+<title>When I run a streaming job by <strong>distributing large executables</strong>
(for example, 3.6G) through the -file option, I get a "No space left on device" error. What
do I do? </title>
+<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>
+<source>
+-jobconf stream.tmpdir=/export/bigspace/...
+</source>
+</section>
+
+<section>
+<title>How do I specify multiple input directories? </title>
+<p>
+You can specify multiple input directories with multiple '-input' options:
+</p><source>
+ hadoop jar hadoop-streaming.jar -input '/user/foo/dir1' -input '/user/foo/dir2' 
+</source>
+</section>
+
+<section>
+<title>How do I generate output files with gzip format? </title>
+<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>
+</section>
+
+<section>
+<title>How do I provide my own input/output format with streaming? </title>
+<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>
+</section>
+
+<section>
+<title>How do I parse XML documents using streaming? </title>
+<p>
+You can use the record reader StreamXmlRecordReader to process XML documents. 
+</p>
+<source>
+hadoop jar hadoop-streaming.jar -inputreader "StreamXmlRecord,begin=BEGIN_STRING,end=END_STRING"
..... (rest of the command)
+</source>
+<p>
+Anything found between BEGIN_STRING and END_STRING would be treated as one record for map
tasks.
+</p>
+</section>
+</section>
+</body>
+</document>



Mime
View raw message