hadoop-common-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From acmur...@apache.org
Subject svn commit: r594460 [5/6] - in /lucene/hadoop/trunk: ./ docs/ src/docs/src/documentation/content/xdocs/
Date Tue, 13 Nov 2007 09:01:13 GMT
Added: lucene/hadoop/trunk/src/docs/src/documentation/content/xdocs/mapred_tutorial.xml
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/docs/src/documentation/content/xdocs/mapred_tutorial.xml?rev=594460&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/docs/src/documentation/content/xdocs/mapred_tutorial.xml (added)
+++ lucene/hadoop/trunk/src/docs/src/documentation/content/xdocs/mapred_tutorial.xml Tue Nov 13 01:01:11 2007
@@ -0,0 +1,2304 @@
+<?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 Map-Reduce Tutorial</title>
+  </header>
+  
+  <body>
+  
+    <section>
+      <title>Purpose</title>
+      
+      <p>This document comprehensively describes all user-facing facets of the 
+      Hadoop Map-Reduce framework and serve as a tutorial.
+      </p>
+    </section>
+    
+    <section>
+      <title>Pre-requisites</title>
+      
+      <p>Ensure that Hadoop is installed, configured and is running. More
+      details:</p> 
+      <ul>
+        <li>
+          Hadoop <a href="quickstart.html">Quickstart</a> for first-time users.
+        </li>
+        <li>
+          Hadoop <a href="cluster_setup.html">Cluster Setup</a> for large, 
+          distributed clusters.
+        </li>
+      </ul>
+    </section>
+    
+    <section>
+      <title>Overview</title>
+      
+      <p>Hadoop Map-Reduce is a software framework for easily writing 
+      applications which process vast amounts of data (multi-terabyte data-sets) 
+      in-parallel on large clusters (thousands of nodes) of commodity 
+      hardware in a reliable, fault-tolerant manner.</p>
+      
+      <p>A Map-Reduce <em>job</em> usually splits the input data-set into 
+      independent chunks which are processed by the <em>map tasks</em> in a
+      completely parallel manner. The framework sorts the outputs of the maps, 
+      which are then input to the <em>reduce tasks</em>. Typically both the 
+      input and the output of the job are stored in a file-system. The framework 
+      takes care of scheduling tasks, monitoring them and re-executes the failed
+      tasks.</p>
+      
+      <p>Typically the compute nodes and the storage nodes are the same, that is, 
+      the Map-Reduce framework and the <a href="hdfs_design.html">Distributed 
+      FileSystem</a> are running on the same set of nodes. This configuration
+      allows the framework to effectively schedule tasks on the nodes where data 
+      is already present, resulting in very high aggregate bandwidth across the 
+      cluster.</p>
+      
+      <p>The Map-Reduce framework consists of a single master 
+      <code>JobTracker</code> and one slave <code>TaskTracker</code> per 
+      cluster-node. The master is responsible for scheduling the jobs' component 
+      tasks on the slaves, monitoring them and re-executing the failed tasks. The 
+      slaves execute the tasks as directed by the master.</p>
+      
+      <p>Minimally, applications specify the input/output locations and supply
+      <em>map</em> and <em>reduce</em> functions via implementations of
+      appropriate interfaces and/or abstract-classes. These, and other job 
+      parameters, comprise the <em>job configuration</em>. The Hadoop 
+      <em>job client</em> then submits the job (jar/executable etc.) and 
+      configuration to the <code>JobTracker</code> which then assumes the 
+      responsibility of distributing the software/configuration to the slaves, 
+      scheduling tasks and monitoring them, providing status and diagnostic 
+      information to the job-client.</p>
+      
+      <p>Although the Hadoop framework is implemented in Java<sup>TM</sup>, 
+      Map-Reduce applications need not be written in Java.</p>
+      <ul>
+        <li>
+          <a href="ext:api/org/apache/hadoop/streaming/package-summary">
+          Hadoop Streaming</a> is a utility which allows users to create and run 
+          jobs with any executables (e.g. shell utilities) as the mapper and/or 
+          the reducer.
+        </li>
+        <li>
+          <a href="ext:api/org/apache/hadoop/mapred/pipes/package-summary">
+          Hadoop Pipes</a> is a <a href="http://www.swig.org/">SWIG</a>-
+          compatible <em>C++ API</em> to implement Map-Reduce applications (non 
+          JNI<sup>TM</sup> based).
+        </li>
+      </ul>
+    </section>
+    
+    <section>
+      <title>Inputs and Outputs</title>
+
+      <p>The Map-Reduce framework operates exclusively on 
+      <code>&lt;key, value&gt;</code> pairs, that is, the framework views the 
+      input to the job as a set of <code>&lt;key, value&gt;</code> pairs and 
+      produces a set of <code>&lt;key, value&gt;</code> pairs as the output of 
+      the job, conceivably of different types.</p> 
+      
+      <p>The <code>key</code> and <code>value</code> classes have to be 
+      serializable by the framework and hence need to implement the 
+      <a href="ext:api/org/apache/hadoop/io/writable">Writable</a> 
+      interface. Additionally, the <code>key</code> classes have to implement the
+      <a href="ext:api/org/apache/hadoop/io/writablecomparable">
+      WritableComparable</a> interface to facilitate sorting by the framework.
+      </p>
+
+      <p>Input and Output types of a Map-Reduce job:</p>
+      <p>
+        (input) <code>&lt;k1, v1&gt;</code> 
+        -&gt; 
+        <strong>map</strong> 
+        -&gt; 
+        <code>&lt;k2, v2&gt;</code> 
+        -&gt; 
+        <strong>combine</strong> 
+        -&gt; 
+        <code>&lt;k2, v2&gt;</code> 
+        -&gt; 
+        <strong>reduce</strong> 
+        -&gt; 
+        <code>&lt;k3, v3&gt;</code> (output)
+      </p>
+    </section>
+
+    <section>
+      <title>Example: WordCount v1.0</title>
+      
+      <p>Before we jump into the details, lets walk through an example Map-Reduce 
+      application to get a flavour for how they work.</p>
+      
+      <p><code>WordCount</code> is a simple application that counts the number of
+      occurences of each word in a given input set.</p>
+      
+      <section>
+        <title>Source Code</title>
+        
+        <table>
+          <tr>
+            <th></th>
+            <th>WordCount.java</th>
+          </tr>
+          <tr>
+            <td>1.</td>
+            <td>
+              <code>package org.myorg;</code>
+            </td>
+          </tr>
+          <tr>
+            <td>2.</td>
+            <td></td>
+          </tr>
+          <tr>
+            <td>3.</td>
+            <td>
+              <code>import java.io.Exception;</code>
+            </td>
+          </tr>
+          <tr>
+            <td>4.</td>
+            <td>
+              <code>import java.util.*;</code>
+            </td>
+          </tr>
+          <tr>
+            <td>5.</td>
+            <td></td>
+          </tr>
+          <tr>
+            <td>6.</td>
+            <td>
+              <code>import org.apache.hadoop.fs.Path;</code>
+            </td>
+          </tr>
+          <tr>
+            <td>7.</td>
+            <td>
+              <code>import org.apache.hadoop.conf.*;</code>
+            </td>
+          </tr>
+          <tr>
+            <td>8.</td>
+            <td>
+              <code>import org.apache.hadoop.io.*;</code>
+            </td>
+          </tr>
+          <tr>
+            <td>9.</td>
+            <td>
+              <code>import org.apache.hadoop.mapred.*;</code>
+            </td>
+          </tr>
+          <tr>
+            <td>10.</td>
+            <td>
+              <code>import org.apache.hadoop.util.*;</code>
+            </td>
+          </tr>
+          <tr>
+            <td>11.</td>
+            <td></td>
+          </tr>
+          <tr>
+            <td>12.</td>
+            <td>
+              <code>public class WordCount {</code>
+            </td>
+          </tr>
+          <tr>
+            <td>13.</td>
+            <td></td>
+          </tr>
+          <tr>
+            <td>14.</td>
+            <td>
+              &nbsp;&nbsp;
+              <code>
+                public static class MapClass extends MapReduceBase 
+                implements Mapper&lt;LongWritable, Text, Text, IntWritable&gt; {
+              </code>
+            </td>
+          </tr>
+          <tr>
+            <td>15.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>
+                private final static IntWritable one = new IntWritable(1);
+              </code>
+            </td>
+          </tr>
+          <tr>
+            <td>16.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>private Text word = new Text();</code>
+            </td>
+          </tr>
+          <tr>
+            <td>17.</td>
+            <td></td>
+          </tr>
+          <tr>
+            <td>18.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>
+                public void map(LongWritable key, Text value, 
+                OutputCollector&lt;Text, IntWritable&gt; output, 
+                Reporter reporter) throws IOException {
+              </code>
+            </td>
+          </tr>
+          <tr>
+            <td>19.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>String line = value.toString();</code>
+            </td>
+          </tr>
+          <tr>
+            <td>20.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>StringTokenizer tokenizer = new StringTokenizer(line);</code>
+            </td>
+          </tr>
+          <tr>
+            <td>21.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>while (tokenizer.hasMoreTokens()) {</code>
+            </td>
+          </tr>
+          <tr>
+            <td>22.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>word.set(tokenizer.nextToken());</code>
+            </td>
+          </tr>
+          <tr>
+            <td>23.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>output.collect(word, one);</code>
+            </td>
+          </tr>
+          <tr>
+            <td>24.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>}</code>
+            </td>
+          </tr>
+          <tr>
+            <td>25.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>}</code>
+            </td>
+          </tr>
+          <tr>
+            <td>26.</td>
+            <td>
+              &nbsp;&nbsp;
+              <code>}</code>
+            </td>
+          </tr>
+          <tr>
+            <td>27.</td>
+            <td></td>
+          </tr>
+          <tr>
+            <td>28.</td>
+            <td>
+              &nbsp;&nbsp;
+              <code>
+                public static class Reduce extends MapReduceBase implements 
+                Reducer&lt;Text, IntWritable, Text, IntWritable&gt; {
+              </code>
+            </td>
+          </tr>
+          <tr>
+            <td>29.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>
+                public void reduce(Text key, Iterator&lt;IntWritable&gt; values,
+                OutputCollector&lt;Text, IntWritable&gt; output, 
+                Reporter reporter) throws IOException {
+              </code>
+            </td>
+          </tr>
+          <tr>
+            <td>30.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>int sum = 0;</code>
+            </td>
+          </tr>
+          <tr>
+            <td>31.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>while (values.hasNext()) {</code>
+            </td>
+          </tr>
+          <tr>
+            <td>32.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>sum += values.next().get();</code>
+            </td>
+          </tr>
+          <tr>
+            <td>33.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>}</code>
+            </td>
+          </tr>
+          <tr>
+            <td>34.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>output.collect(key, new IntWritable(sum));</code>
+            </td>
+          </tr>
+          <tr>
+            <td>35.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>}</code>
+            </td>
+          </tr>
+          <tr>
+            <td>36.</td>
+            <td>
+              &nbsp;&nbsp;
+              <code>}</code>
+            </td>
+          </tr>
+          <tr>
+            <td>37.</td>
+            <td></td>
+          </tr>
+          <tr>
+            <td>38.</td>
+            <td>
+              &nbsp;&nbsp;
+              <code>
+                public static void main(String[] args) throws Exception {
+              </code>
+            </td>
+          </tr>
+          <tr>
+            <td>39.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>
+                JobConf conf = new JobConf(WordCount.class);
+              </code>
+            </td>
+          </tr>
+          <tr>
+            <td>40.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>conf.setJobName("wordcount");</code>
+            </td>
+          </tr>
+          <tr>
+            <td>41.</td>
+            <td></td>
+          </tr>
+          <tr>
+            <td>42.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>conf.setOutputKeyClass(Text.class);</code>
+            </td>
+          </tr>
+          <tr>
+            <td>43.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>conf.setOutputValueClass(IntWritable.class);</code>
+            </td>
+          </tr>
+          <tr>
+            <td>44.</td>
+            <td></td>
+          </tr>
+          <tr>
+            <td>45.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>conf.setMapperClass(MapClass.class);</code>
+            </td>
+          </tr>
+          <tr>
+            <td>46.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>conf.setCombinerClass(Reduce.class);</code>
+            </td>
+          </tr>
+          <tr>
+            <td>47.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>conf.setReducerClass(Reduce.class);</code>
+            </td>
+          </tr>
+          <tr>
+            <td>48.</td>
+            <td></td>
+          </tr>
+          <tr>
+            <td>49.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>conf.setInputFormat(TextInputFormat.class);</code>
+            </td>
+          </tr>
+          <tr>
+            <td>50.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>conf.setOutputFormat(TextOutputFormat.class);</code>
+            </td>
+          </tr>
+          <tr>
+            <td>51.</td>
+            <td></td>
+          </tr>
+          <tr>
+            <td>52.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>conf.setInputPath(new Path(args[1]));</code>
+            </td>
+          </tr>
+          <tr>
+            <td>53.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>conf.setOutputPath(new Path(args[2]));</code>
+            </td>
+          </tr>
+          <tr>
+            <td>54.</td>
+            <td></td>
+          </tr>
+          <tr>
+            <td>55.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>JobClient.runJob(conf);</code>
+            </td>
+          </tr>
+          <tr>
+            <td>57.</td>
+            <td>
+              &nbsp;&nbsp;
+              <code>}</code>
+            </td>
+          </tr>
+          <tr>
+            <td>58.</td>
+            <td>
+              <code>}</code>
+            </td>
+          </tr>
+          <tr>
+            <td>59.</td>
+            <td></td>
+          </tr>
+        </table>
+      </section>
+        
+      <section>
+        <title>Usage</title>
+        
+        <p>Assuming <code>HADOOP_HOME</code> is the root of the installation and 
+        <code>HADOOP_VERSION</code> is the Hadoop version installed, compile 
+        <code>WordCount.java</code> and create a jar:</p>
+        <p>
+          <code>
+            $ javac -classpath ${HADOOP_HOME}/hadoop-${HADOOP_VERSION}-core.jar 
+              WordCount.java
+          </code><br/>
+          <code>$ jar -cvf /usr/joe/wordcount.jar WordCount.class</code> 
+        </p>
+        
+        <p>Assuming that:</p>
+        <ul>
+          <li>
+            <code>/usr/joe/wordcount/input</code>  - input directory in HDFS
+          </li>
+          <li>
+            <code>/usr/joe/wordcount/output</code> - output directory in HDFS
+          </li>
+        </ul>
+        
+        <p>Sample text-files as input:</p>
+        <p>
+          <code>$ bin/hadoop dfs -ls /usr/joe/wordcount/input/</code><br/>
+          <code>/usr/joe/wordcount/input/file01</code><br/>
+          <code>/usr/joe/wordcount/input/file02</code><br/>
+          <br/>
+          <code>$ bin/hadoop dfs -cat /usr/joe/wordcount/input/file01</code><br/>
+          <code>Hello World Bye World</code><br/>
+          <br/>
+          <code>$ bin/hadoop dfs -cat /usr/joe/wordcount/input/file02</code><br/>
+          <code>Hello Hadoop Goodbye Hadoop</code>
+        </p>
+
+        <p>Run the application:</p>
+        <p>
+          <code>
+            $ bin/hadoop jar /usr/joe/wordcount.jar org.myorg.WordCount 
+              /usr/joe/wordcount/input /usr/joe/wordcount/output 
+          </code>
+        </p>
+
+        <p>Output:</p>
+        <p>
+          <code>
+            $ bin/hadoop dfs -cat /usr/joe/wordcount/output/part-00000
+          </code>
+          <br/>
+          <code>Bye    1</code><br/>
+          <code>Goodbye    1</code><br/>
+          <code>Hadoop    2</code><br/>
+          <code>Hello    2</code><br/>
+          <code>World    2</code><br/>
+        </p>
+      </section>
+      
+      <section>
+        <title>Walk-through</title>
+        
+        <p>The <code>WordCount</code> application is quite straight-forward.</p>
+        
+        <p>The <code>Mapper</code> implementation (lines 14-26), via the 
+        <code>map</code> method (lines 18-25), processes one line at a time,
+        as provided by the specified <code>TextInputFormat</code> (line 49). 
+        It then splits the line into tokens separated by whitespaces, via the 
+        <code>StringTokenizer</code>, and emits a key-value pair of 
+        <code>&lt; &lt;word&gt;, 1&gt;</code>.</p>
+        
+        <p>
+          For the given sample input the first map emits:<br/>
+          <code>&lt; Hello, 1&gt;</code><br/>
+          <code>&lt; World, 1&gt;</code><br/>
+          <code>&lt; Bye, 1&gt;</code><br/>
+          <code>&lt; World, 1&gt;</code><br/>
+        </p>
+        
+        <p>
+          The second map emits:<br/>
+          <code>&lt; Hello, 1&gt;</code><br/>
+          <code>&lt; Hadoop, 1&gt;</code><br/>
+          <code>&lt; Goodbye, 1&gt;</code><br/>
+          <code>&lt; Hadoop, 1&gt;</code><br/>
+        </p>
+        
+        <p>We'll learn more about the number of maps spawned for a given job, and
+        how to control them in a fine-grained manner, a bit later in the 
+        tutorial.</p>
+        
+        <p><code>WordCount</code> also specifies a <code>combiner</code> (line 
+        46). Hence, the output of each map is passed through the local combiner 
+        (which is same as the <code>Reducer</code> as per the job 
+        configuration) for local aggregation, after being sorted on the 
+        <em>key</em>s.</p>
+
+        <p>
+          The output of the first map:<br/>
+          <code>&lt; Bye, 1&gt;</code><br/>
+          <code>&lt; Hello, 1&gt;</code><br/>
+          <code>&lt; World, 2&gt;</code><br/>
+        </p>
+        
+        <p>
+          The output of the second map:<br/>
+          <code>&lt; Goodbye, 1&gt;</code><br/>
+          <code>&lt; Hadoop, 2&gt;</code><br/>
+          <code>&lt; Hello, 1&gt;</code><br/>
+        </p>
+
+        <p>The <code>Reducer</code> implementation (lines 28-36), via the
+        <code>reduce</code> method (lines 29-35) just sums up the values,
+        which are the occurence counts for each key (i.e. words in this example).
+        </p>
+        
+        <p>
+          Thus the output of the job is:<br/>
+          <code>&lt; Bye, 1&gt;</code><br/>
+          <code>&lt; Goodbye, 1&gt;</code><br/>
+          <code>&lt; Hadoop, 2&gt;</code><br/>
+          <code>&lt; Hello, 2&gt;</code><br/>
+          <code>&lt; World, 2&gt;</code><br/>
+        </p>
+        
+        <p>The <code>run</code> method specifies various facets of the job, such 
+        as the input/output paths (passed via the command line), key/value 
+        types, input/output formats etc., in the <code>JobConf</code>.
+        It then calls the <code>JobClient.runJob</code> (line  55) to submit the
+        and monitor its progress.</p>
+
+        <p>We'll learn more about <code>JobConf</code>, <code>JobClient</code>,
+        <code>Tool</code> and other interfaces and classes a bit later in the 
+        tutorial.</p>
+      </section>
+    </section>
+    
+    <section>
+      <title>Map-Reduce - User Interfaces</title>
+      
+      <p>This section provides a reasonable amount of detail on every user-facing 
+      aspect of the Map-Reduce framwork. This should help users implement, 
+      configure and tune their jobs in a fine-grained manner. However, please 
+      note that the javadoc for each class/interface remains the most 
+      comprehensive documentation available; this is only meant to be a tutorial.
+      </p>
+      
+      <p>Let us first take the <code>Mapper</code> and <code>Reducer</code> 
+      interfaces. Applications typically implement them to provide the 
+      <code>map</code> and <code>reduce</code> methods.</p>
+      
+      <p>We will then discuss other core interfaces including 
+      <code>JobConf</code>, <code>JobClient</code>, <code>Partitioner</code>, 
+      <code>OutputCollector</code>, <code>Reporter</code>, 
+      <code>InputFormat</code>, <code>OutputFormat</code> and others.</p>
+      
+      <p>Finally, we will wrap up by discussing some useful features of the
+      framework such as the <code>DistributedCache</code>, 
+      <code>IsolationRunner</code> etc.</p>
+
+      <section>
+        <title>Payload</title>
+        
+        <p>Applications typically implement the <code>Mapper</code> and 
+        <code>Reducer</code> interfaces to provide the <code>map</code> and 
+        <code>reduce</code> methods. These form the core of the job.</p>
+        
+        <section>
+          <title>Mapper</title>
+
+          <p><a href="ext:api/org/apache/hadoop/mapred/mapper">
+          Mapper</a> maps input key/value pairs to a set of intermediate 
+          key/value pairs.</p>
+ 
+          <p>Maps are the individual tasks that transform input records into 
+          intermediate records. The transformed intermediate records do not need
+          to be of the same type as the input records. A given input pair may 
+          map to zero or many output pairs.</p> 
+ 
+          <p>The Hadoop Map-Reduce framework spawns one map task for each 
+          <code>InputSplit</code> generated by the <code>InputFormat</code> for 
+          the job.</p>
+          
+          <p>Overall, <code>Mapper</code> implementations are passed the 
+          <code>JobConf</code> for the job via the 
+          <a href="ext:api/org/apache/hadoop/mapred/jobconfigurable/configure">
+          JobConfigurable.configure(JobConf)</a> method and override it to 
+          initialize themselves. The framework then calls 
+          <a href="ext:api/org/apache/hadoop/mapred/mapper/map">
+          map(WritableComparable, Writable, OutputCollector, Reporter)</a> for 
+          each key/value pair in the <code>InputSplit</code> for that task.        
+          Applications can then override the
+          <a href="ext:api/org/apache/hadoop/io/closeable/close">
+          Closeable.close()</a> method to perform any required cleanup.</p>
+ 
+
+          <p>Output pairs do not need to be of the same types as input pairs. A 
+          given input pair may map to zero or many output pairs.  Output pairs 
+          are collected with calls to 
+          <a href="ext:api/org/apache/hadoop/mapred/outputcollector/collect">
+          OutputCollector.collect(WritableComparable,Writable)</a>.</p>
+
+          <p>Applications can use the <code>Reporter</code> to report 
+          progress, set application-level status messages and update 
+          <code>Counters</code>, or just indicate that they are alive.</p>
+ 
+          <p>All intermediate values associated with a given output key are 
+          subsequently grouped by the framework, and passed to the
+          <code>Reducer</code>(s) to  determine the final output. Users can 
+          control the grouping by specifying a <code>Comparator</code> via 
+          <a href="ext:api/org/apache/hadoop/mapred/jobconf/setoutputkeycomparatorclass">
+          JobConf.setOutputKeyComparatorClass(Class)</a>.</p>
+
+          <p>The <code>Mapper</code> outputs are sorted and then 
+          partitioned per <code>Reducer</code>. The total number of partitions is 
+          the same as the number of reduce tasks for the job. Users can control 
+          which keys (and hence records) go to which <code>Reducer</code> by 
+          implementing a custom <code>Partitioner</code>.</p>
+ 
+          <p>Users can optionally specify a <code>combiner</code>, via 
+          <a href="ext:api/org/apache/hadoop/mapred/jobconf/setcombinerclass">
+          JobConf.setCombinerClass(Class)</a>, to perform local aggregation of 
+          the intermediate outputs, which helps to cut down the amount of data 
+          transferred from the <code>Mapper</code> to the <code>Reducer</code>.
+          </p>
+ 
+          <p>The intermediate, sorted outputs are always stored in files of 
+          <a href="ext:api/org/apache/hadoop/io/sequencefile">
+          SequenceFile</a> format. Applications can control if, and how, the 
+          intermediate outputs are to be compressed and the 
+          <a href="ext:api/org/apache/hadoop/io/compress/compressioncodec">
+          CompressionCodec</a> to be used via the <code>JobConf</code>.
+          </p>
+          
+          <section>
+            <title>How Many Maps?</title>
+             
+            <p>The number of maps is usually driven by the total size of the 
+            inputs, that is, the total number of blocks of the input files.</p>
+  
+            <p>The right level of parallelism for maps seems to be around 10-100 
+            maps per-node, although it has been set up to 300 maps for very 
+            cpu-light map tasks. Task setup takes awhile, so it is best if the 
+            maps take at least a minute to execute.</p>
+ 
+            <p>Thus, if you expect 10TB of input data and have a blocksize of 
+            <code>128MB</code>, you'll end up with 82,000 maps, unless 
+            <a href="ext:api/org/apache/hadoop/mapred/jobconf/setnummaptasks">
+            setNumMapTasks(int)</a> (which only provides a hint to the framework) 
+            is used to set it even higher.</p>
+          </section>
+        </section>
+        
+        <section>
+          <title>Reducer</title>
+          
+          <p><a href="ext:api/org/apache/hadoop/mapred/reducer">
+          Reducer</a> reduces a set of intermediate values which share a key to
+          a smaller set of values.</p>
+          
+          <p>The number of reduces for the job is set by the user 
+          via <a href="ext:api/org/apache/hadoop/mapred/jobconf/setnumreducetasks">
+          JobConf.setNumReduceTasks(int)</a>.</p>
+          
+          <p>Overall, <code>Reducer</code> implementations are passed the 
+          <code>JobConf</code> for the job via the 
+          <a href="ext:api/org/apache/hadoop/mapred/jobconfigurable/configure">
+          JobConfigurable.configure(JobConf)</a> method and can override it to 
+          initialize themselves. The framework then calls   
+          <a href="ext:api/org/apache/hadoop/mapred/reducer/reduce">
+          reduce(WritableComparable, Iterator, OutputCollector, Reporter)</a>
+          method for each <code>&lt;key, (list of values)&gt;</code> 
+          pair in the grouped inputs. Applications can then override the           
+          <a href="ext:api/org/apache/hadoop/io/closeable/close">
+          Closeable.close()</a> method to perform any required cleanup.</p>
+
+          <p><code>Reducer</code> has 3 primary phases: shuffle, sort and reduce.
+          </p>
+          
+          <section>
+            <title>Shuffle</title>
+   
+            <p>Input to the <code>Reducer</code> is the sorted output of the
+            mappers. In this phase the framework fetches the relevant partition 
+            of the output of all the mappers, via HTTP.</p>
+          </section>
+   
+          <section>
+            <title>Sort</title>
+   
+            <p>The framework groups <code>Reducer</code> inputs by keys (since 
+            different mappers may have output the same key) in this stage.</p>
+   
+            <p>The shuffle and sort phases occur simultaneously; while 
+            map-outputs are being fetched they are merged.</p>
+      
+            <section>
+              <title>Secondary Sort</title>
+   
+              <p>If equivalence rules for grouping the intermediate keys are 
+              required to be different from those for grouping keys before 
+              reduction, then one may specify a <code>Comparator</code> via 
+              <a href="ext:api/org/apache/hadoop/mapred/jobconf/setoutputvaluegroupingcomparator">
+              JobConf.setOutputValueGroupingComparator(Class)</a>. Since 
+              <a href="ext:api/org/apache/hadoop/mapred/jobconf/setoutputkeycomparatorclass">
+              JobConf.setOutputKeyComparatorClass(Class)</a> can be used to 
+              control how intermediate keys are grouped, these can be used in 
+              conjunction to simulate <em>secondary sort on values</em>.</p>
+            </section>
+          </section>
+   
+          <section>   
+            <title>Reduce</title>
+   
+            <p>In this phase the 
+            <a href="ext:api/org/apache/hadoop/mapred/reducer/reduce">
+            reduce(WritableComparable, Iterator, OutputCollector, Reporter)</a>
+            method is called for each <code>&lt;key, (list of values)&gt;</code> 
+            pair in the grouped inputs.</p>
+            
+            <p>The output of the reduce task is typically written to the 
+            <a href="ext:api/org/apache/hadoop/fs/filesystem">
+            FileSystem</a> via 
+            <a href="ext:api/org/apache/hadoop/mapred/outputcollector/collect">
+            OutputCollector.collect(WritableComparable, Writable)</a>.</p>
+   
+            <p>Applications can use the <code>Reporter</code> to report 
+            progress, set application-level status messages and update 
+            <code>Counters</code>, or just indicate that they are alive.</p>
+ 
+           <p>The output of the <code>Reducer</code> is <em>not sorted</em>.</p>
+          </section>
+          
+          <section>
+            <title>How Many Reduces?</title>
+ 
+            <p>The right number of reduces seems to be <code>0.95</code> or 
+            <code>1.75</code> multiplied by (&lt;<em>no. of nodes</em>&gt; * 
+            <code>mapred.tasktracker.tasks.maximum</code>).</p>
+ 
+            <p>With <code>0.95</code> all of the reduces can launch immediately 
+            and start transfering map outputs as the maps finish. With 
+            <code>1.75</code> the faster nodes will finish their first round of 
+            reduces and launch a second wave of reduces doing a much better job 
+            of load balancing.</p>
+ 
+            <p>Increasing the number of reduces increases the framework overhead, 
+            but increases load balancing and lowers the cost of failures.</p>
+ 
+            <p>The scaling factors above are slightly less than whole numbers to 
+            reserve a few reduce slots in the framework for speculative-tasks and
+            failed tasks.</p>
+          </section>
+          
+          <section>
+            <title>Reducer NONE</title>
+            
+            <p>It is legal to set the number of reduce-tasks to <em>zero</em> if 
+            no reduction is desired.</p>
+ 
+            <p>In this case the outputs of the map-tasks go directly to the
+            <code>FileSystem</code>, into the output path set by 
+            <a href="ext:api/org/apache/hadoop/mapred/jobconf/setoutputpath">
+            setOutputPath(Path)</a>. The framework does not sort the 
+            map-outputs before writing them out to the <code>FileSystem</code>.
+            </p>
+          </section>
+        </section>
+        
+        <section>
+          <title>Partitioner</title>
+          
+          <p><a href="ext:api/org/apache/hadoop/mapred/partitioner">
+          Partitioner</a> partitions the key space.</p>
+
+          <p>Partitioner controls the partitioning of the keys of the 
+          intermediate map-outputs. The key (or a subset of the key) is used to 
+          derive the partition, typically by a <em>hash function</em>. The total 
+          number of partitions is the same as the number of reduce tasks for the 
+          job. Hence this controls which of the <code>m</code> reduce tasks the 
+          intermediate key (and hence the record) is sent to for reduction.</p>
+          
+          <p><a href="ext:api/org/apache/hadoop/mapred/lib/hashpartitioner">
+          HashPartitioner</a> is the default <code>Partitioner</code>.</p>
+        </section>
+        
+        <section>
+          <title>Reporter</title>
+        
+          <p><a href="ext:api/org/apache/hadoop/mapred/reporter">
+          Reporter</a> is a facility for Map-Reduce applications to report 
+          progress, set application-level status messages and update 
+          <code>Counters</code>.</p>
+ 
+          <p><code>Mapper</code> and <code>Reducer</code> implementations can use 
+          the <code>Reporter</code> to report progress or just indicate 
+          that they are alive. In scenarios where the application takes a
+          significant amount of time to process individual key/value pairs, 
+          this is crucial since the framework might assume that the task has 
+          timed-out and kill that task. Another way to avoid this is to 
+          set the configuration parameter <code>mapred.task.timeout</code> to a
+          high-enough value (or even set it to <em>zero</em> for no time-outs).
+          </p>
+
+          <p>Applications can also update <code>Counters</code> using the 
+          <code>Reporter</code>.</p>
+        </section>
+      
+        <section>
+          <title>OutputCollector</title>
+        
+          <p><a href="ext:api/org/apache/hadoop/mapred/outputcollector">
+          OutputCollector</a> is a generalization of the facility provided by
+          the Map-Reduce framework to collect data output by the 
+          <code>Mapper</code> or the <code>Reducer</code> (either the 
+          intermediate outputs or the output of the job).</p>
+        </section>
+      
+        <p>Hadoop Map-Reduce comes bundled with a 
+        <a href="ext:api/org/apache/hadoop/mapred/lib/package-summary">
+        library</a> of generally useful mappers, reducers, and partitioners.</p>
+      </section>
+      
+      <section>
+        <title>Job Configuration</title>
+        
+        <p><a href="ext:api/org/apache/hadoop/mapred/jobconf">
+        JobConf</a> represents a Map-Reduce job configuration.</p>
+ 
+        <p><code>JobConf</code> is the primary interface for a user to describe
+        a map-reduce job to the Hadoop framework for execution. The framework 
+        tries to faithfully execute the job as described by <code>JobConf</code>, 
+        however:</p> 
+        <ul>
+          <li>f
+            Some configuration parameters may have been marked as 
+            <a href="ext:api/org/apache/hadoop/conf/configuration/final">
+            final</a> by administrators and hence cannot be altered.
+          </li>
+          <li>
+            While some job parameters are straight-forward to set (e.g. 
+            <a href="ext:api/org/apache/hadoop/mapred/jobconf/setnumreducetasks">
+            setNumReduceTasks(int)</a>), other parameters interact subtly with 
+            the rest of the framework and/or job configuration and are 
+            more complex to set (e.g. 
+            <a href="ext:api/org/apache/hadoop/mapred/jobconf/setnummaptasks">
+            setNumMapTasks(int)</a>).
+          </li>
+        </ul>
+ 
+        <p><code>JobConf</code> is typically used to specify the 
+        <code>Mapper</code>, combiner (if any), <code>Partitioner</code>, 
+        <code>Reducer</code>, <code>InputFormat</code> and 
+        <code>OutputFormat</code> implementations. <code>JobConf</code> also 
+        indicates the set of input files 
+        (<a href="ext:api/org/apache/hadoop/mapred/jobconf/setinputpath">setInputPath(Path)</a>/<a href="ext:api/org/apache/hadoop/mapred/jobconf/addinputpath">addInputPath(Path)</a>)
+        and where the output files should be written
+        (<a href="ext:api/org/apache/hadoop/mapred/jobconf/setoutputpath">setOutputPath(Path)</a>).</p>
+
+        <p>Optionally, <code>JobConf</code> is used to specify other advanced 
+        facets of the job such as the <code>Comparator</code> to be used, files 
+        to be put in the <code>DistributedCache</code>, whether intermediate 
+        and/or job outputs are to be compressed (and how), debugging via 
+        user-provided scripts
+        (<a href="ext:api/org/apache/hadoop/mapred/jobconf/setmapdebugscript">setMapDebugScript(String)</a>/<a href="ext:api/org/apache/hadoop/mapred/jobconf/setreducedebugscript">setReduceDebugScript(String)</a>) 
+        , whether job tasks can be executed in a <em>speculative</em> manner 
+        (<a href="ext:api/org/apache/hadoop/mapred/jobconf/setspeculativeexecution">setSpeculativeExecution(boolean)</a>)
+        , maximum number of attempts per task
+        (<a href="ext:api/org/apache/hadoop/mapred/jobconf/setmaxmapattempts">setMaxMapAttempts(int)</a>/<a href="ext:api/org/apache/hadoop/mapred/jobconf/setmaxreduceattempts">setMaxReduceAttempts(int)</a>) 
+        , percentage of tasks failure which can be tolerated by the job
+        (<a href="ext:api/org/apache/hadoop/mapred/jobconf/setmaxmaptaskfailurespercent">setMaxMapTaskFailuresPercent(int)</a>/<a href="ext:api/org/apache/hadoop/mapred/jobconf/setmaxreducetaskfailurespercent">setMaxReduceTaskFailuresPercent(int)</a>) 
+        etc.</p>
+        
+        <p>Of course, users can use 
+        <a href="ext:api/org/apache/hadoop/conf/configuration/set">set(String, String)</a>/<a href="ext:api/org/apache/hadoop/conf/configuration/get">get(String, String)</a>
+        to set/get arbitrary parameters needed by applications. However, use the 
+        <code>DistributedCache</code> for large amounts of (read-only) data.</p>
+      </section>
+
+      <section>
+        <title>Job Submission and Monitoring</title>
+        
+        <p><a href="ext:api/org/apache/hadoop/mapred/jobclient">
+        JobClient</a> is the primary interface by which user-job interacts
+        with the <code>JobTracker</code>.</p>
+ 
+        <p><code>JobClient</code> provides facilities to submit jobs, track their 
+        progress, access component-tasks' reports/logs, get the Map-Reduce 
+        cluster's status information and so on.</p>
+ 
+        <p>The job submission process involves:</p>
+        <ol>
+          <li>Checking the input and output specifications of the job.</li>
+          <li>Computing the <code>InputSplit</code> values for the job.</li>
+          <li>
+            Setting up the requisite accounting information for the 
+            <code>DistributedCache</code> of the job, if necessary.
+          </li>
+          <li>
+            Copying the job's jar and configuration to the map-reduce system 
+            directory on the <code>FileSystem</code>.
+          </li>
+          <li>
+            Submitting the job to the <code>JobTracker</code> and optionally 
+            monitoring it's status.
+          </li>
+        </ol>
+  
+        <p>Normally the user creates the application, describes various facets 
+        of the job via <code>JobConf</code>, and then uses the 
+        <code>JobClient</code> to submit the job and monitor its progress.</p>
+
+        <section>
+          <title>Job Control</title>
+ 
+          <p>Users may need to chain map-reduce jobs to accomplish complex
+          tasks which cannot be done via a single map-reduce job. This is fairly
+          easy since the output of the job typically goes to distributed 
+          file-system, and the output, in turn, can be used as the input for the 
+          next job.</p>
+ 
+          <p>However, this also means that the onus on ensuring jobs are 
+          complete (success/failure) lies squarely on the clients. In such 
+          cases, the various job-control options are:</p>
+          <ul>
+            <li>
+              <a href="ext:api/org/apache/hadoop/mapred/jobclient/runjob">
+              runJob(JobConf)</a> : Submits the job and returns only after the 
+              job has completed.
+            </li>
+            <li>
+              <a href="ext:api/org/apache/hadoop/mapred/jobclient/submitjob">
+              submitJob(JobConf)</a> : Only submits the job, then poll the 
+              returned handle to the 
+              <a href="ext:api/org/apache/hadoop/mapred/runningjob">
+              RunningJob</a> to query status and make scheduling decisions.
+            </li>
+            <li>
+              <a href="ext:api/org/apache/hadoop/mapred/jobconf/setjobendnotificationuri">
+              JobConf.setJobEndNotificationURI(String)</a> : Sets up a 
+              notification upon job-completion, thus avoiding polling.
+            </li>
+          </ul>
+        </section>
+      </section>
+
+      <section>
+        <title>Job Input</title>
+        
+        <p><a href="ext:api/org/apache/hadoop/mapred/inputformat">
+        InputFormat</a> describes the input-specification for a Map-Reduce job.
+        </p> 
+ 
+        <p>The Map-Reduce framework relies on the <code>InputFormat</code> of 
+        the job to:</p>
+        <ol>
+          <li>Validate the input-specification of the job.</li>
+          <li>
+            Split-up the input file(s) into logical <code>InputSplit</code> 
+            instances, each of which is then assigned to an individual 
+            <code>Mapper</code>.
+          </li>
+          <li>
+            Provide the <code>RecordReader</code> implementation used to
+            glean input records from the logical <code>InputSplit</code> for 
+            processing by the <code>Mapper</code>.
+          </li>
+        </ol>
+ 
+        <p>The default behavior of file-based <code>InputFormat</code>
+        implementations, typically sub-classes of 
+        <a href="ext:api/org/apache/hadoop/mapred/fileinputformat">
+        FileInputFormat</a>, is to split the input into <em>logical</em> 
+        <code>InputSplit</code> instances based on the total size, in bytes, of 
+        the input files. However, the <code>FileSystem</code> blocksize of the 
+        input files is treated as an upper bound for input splits. A lower bound
+        on the split size can be set via <code>mapred.min.split.size</code>.</p>
+ 
+        <p>Clearly, logical splits based on input-size is insufficient for many
+        applications since record boundaries must be respected. In such cases, 
+        the application should implement a <code>RecordReader</code>, who is 
+        responsible for respecting record-boundaries and presents a 
+        record-oriented view of the logical <code>InputSplit</code> to the 
+        individual task.</p>
+
+        <p><a href="ext:api/org/apache/hadoop/mapred/textinputformat">
+        TextInputFormat</a> is the default <code>InputFormat</code>.
+        </p>
+        
+        <section>
+          <title>InputSplit</title>
+          
+          <p><a href="ext:api/org/apache/hadoop/mapred/inputsplit">
+          InputSplit</a> represents the data to be processed by an individual 
+          <code>Mapper</code>.</p>
+
+          <p>Typically <code>InputSplit</code> presents a byte-oriented view of
+          the input, and it is the responsibility of <code>RecordReader</code>
+          to process and present a record-oriented view.</p>
+          
+          <p><a href="ext:api/org/apache/hadoop/mapred/filesplit">
+          FileSplit</a> is the default <code>InputSplit</code>. It sets 
+          <code>map.input.file</code> to the path of the input file for the
+          logical split.</p>
+        </section>
+        
+        <section>
+          <title>RecordReader</title>
+          
+          <p><a href="ext:api/org/apache/hadoop/mapred/recordreader">
+          RecordReader</a> reads <code>&lt;key, value&gt;</code> pairs from an 
+          <code>InputSplit</code>.</p>
+
+          <p>Typically the <code>RecordReader</code> converts the byte-oriented 
+          view of the input, provided by the <code>InputSplit</code>, and 
+          presents a record-oriented to the <code>Mapper</code> implementations 
+          for processing. <code>RecordReader</code> thus assumes the 
+          responsibility of processing record boundaries and presents the tasks 
+          with keys and values.</p>
+        </section>
+      </section>
+
+      <section>
+        <title>Job Output</title>
+        
+        <p><a href="ext:api/org/apache/hadoop/mapred/outputformat">
+        OutputFormat</a> describes the output-specification for a Map-Reduce 
+        job.</p>
+
+        <p>The Map-Reduce framework relies on the <code>OutputFormat</code> of 
+        the job to:</p>
+        <ol>
+          <li>
+            Validate the output-specification of the job; for example, check that 
+            the output directory doesn't already exist.
+          </li>
+          <li>
+            Provide the <code>RecordWriter</code> implementation used to 
+            write the output files of the job. Output files are stored in a 
+            <code>FileSystem</code>.
+          </li>
+        </ol>
+ 
+        <p><code>TextOutputFormat</code> is the default 
+        <code>OutputFormat</code>.</p>
+ 
+        <section>
+          <title>Task Side-Effect Files</title>
+ 
+          <p>In some applications, component tasks need to create and/or write to
+          side-files, which differ from the actual job-output files.</p>
+ 
+          <p>In such cases there could be issues with two instances of the same 
+          <code>Mapper</code> or <code>Reducer</code> running simultaneously (for
+          example, speculative tasks) trying to open and/or write to the same 
+          file (path) on the <code>FileSystem</code>. Hence the 
+          application-writer will have to pick unique names per task-attempt 
+          (using the taskid, say <code>task_200709221812_0001_m_000000_0</code>), 
+          not just per task.</p> 
+ 
+          <p>To avoid these issues the Map-Reduce framework maintains a special 
+          <code>${mapred.output.dir}/_${taskid}</code> sub-directory for each 
+          task-attempt on the <code>FileSystem</code> where the output of the 
+          task-attempt is stored. On successful completion of the task-attempt, 
+          the files in the <code>${mapred.output.dir}/_${taskid}</code> (only) 
+          are <em>promoted</em> to <code>${mapred.output.dir}</code>. Of course, 
+          the framework discards the sub-directory of unsuccessful task-attempts. 
+          This process is completely transparent to the application.</p>
+ 
+          <p>The application-writer can take advantage of this feature by 
+          creating any side-files required in <code>${mapred.output.dir}</code> 
+          during execution of a task via 
+          <a href="ext:api/org/apache/hadoop/mapred/jobconf/getoutputpath">
+          JobConf.getOutputPath()</a>, and the framework will promote them 
+          similarly for succesful task-attempts, thus eliminating the need to 
+          pick unique paths per task-attempt.</p>
+        </section>
+        
+        <section>
+          <title>RecordWriter</title>
+          
+          <p><a href="ext:api/org/apache/hadoop/mapred/recordwriter">
+          RecordWriter</a> writes the output <code>&lt;key, value&gt;</code> 
+          pairs to an output file.</p>
+
+          <p>RecordWriter implementations write the job outputs to the 
+          <code>FileSystem</code>.</p>
+        </section>
+      </section>
+      
+      <section>
+        <title>Other Useful Features</title>
+ 
+        <section>
+          <title>Counters</title>
+          
+          <p><code>Counters</code> represent global counters, defined either by 
+          the Map-Reduce framework or applications. Each <code>Counter</code> can 
+          be of any <code>Enum</code> type. Counters of a particular 
+          <code>Enum</code> are bunched into groups of type 
+          <code>Counters.Group</code>.</p>
+          
+          <p>Applications can define arbitrary <code>Counters</code> (of type 
+          <code>Enum</code>) and update them via 
+          <a href="ext:api/org/apache/hadoop/mapred/reporter/incrcounter">
+          Reporter.incrCounter(Enum, long)</a> in the <code>map</code> and/or 
+          <code>reduce</code> methods. These counters are then globally 
+          aggregated by the framework.</p>
+        </section>       
+        
+        <section>
+          <title>DistributedCache</title>
+          
+          <p><a href="ext:api/org/apache/hadoop/filecache/distributedcache">
+          DistributedCache</a> distributes application-specific, large, read-only 
+          files efficiently.</p>
+ 
+          <p><code>DistributedCache</code> is a facility provided by the 
+          Map-Reduce framework to cache files (text, archives, jars and so on) 
+          needed by applications.</p>
+ 
+          <p>Applications specify the files to be cached via urls (hdfs:// or 
+          http://) in the <code>JobConf</code>. The <code>DistributedCache</code> 
+          assumes that the files specified via hdfs:// urls are already present 
+          on the <code>FileSystem</code>.</p>
+
+          <p>The framework will copy the necessary files to the slave node 
+          before any tasks for the job are executed on that node. Its 
+          efficiency stems from the fact that the files are only copied once 
+          per job and the ability to cache archives which are un-archived on 
+          the slaves.</p> 
+
+          <p><code>DistributedCache</code> can be used to distribute simple, 
+          read-only data/text files and more complex types such as archives and
+          jars. Archives (zip files) are <em>un-archived</em> at the slave nodes.
+          Jars maybe be optionally added to the classpath of the tasks, a
+          rudimentary <em>software distribution</em> mechanism.  Files have 
+          <em>execution permissions</em> set. Optionally users can also direct the
+          <code>DistributedCache</code> to <em>symlink</em> the cached file(s) 
+          into the working directory of the task.</p>
+ 
+          <p><code>DistributedCache</code> tracks the modification timestamps of 
+          the cached files. Clearly the cache files should not be modified by 
+          the application or externally while the job is executing.</p>
+        </section>
+        
+        <section>
+          <title>Tool</title>
+          
+          <p>The <a href="ext:api/org/apache/hadoop/util/tool">Tool</a> 
+          interface supports the handling of generic Hadoop command-line options.
+          </p>
+          
+          <p><code>Tool</code> is the standard for any Map-Reduce tool or 
+          application. The application should delegate the handling of 
+          standard command-line options to 
+          <a href="ext:api/org/apache/hadoop/util/genericoptionsparser">
+          GenericOptionsParser</a> via          
+          <a href="ext:api/org/apache/hadoop/util/toolrunner/run">
+          ToolRunner.run(Tool, String[])</a> and only handle its custom 
+          arguments.</p>
+          
+          <p>
+            The generic Hadoop command-line options are:<br/>
+            <code>
+              -conf &lt;configuration file&gt;
+            </code>
+            <br/>
+            <code>
+              -D &lt;property=value&gt;
+            </code>
+            <br/>
+            <code>
+              -fs &lt;local|namenode:port&gt;
+            </code>
+            <br/>
+            <code>
+              -jt &lt;local|jobtracker:port&gt;
+            </code>
+          </p>
+        </section>
+        
+        <section>
+          <title>IsolationRunner</title>
+          
+          <p><a href="ext:api/org/apache/hadoop/mapred/isolationrunner">
+          IsolationRunner</a> is a utility to help debug Map-Reduce programs.</p>
+          
+          <p>To use the <code>IsolationRunner</code>, first set 
+          <code>keep.failed.tasks.files</code> to <code>true</code> 
+          (also see <code>keep.tasks.files.pattern</code>).</p>
+          
+          <p>
+            Next, go to the node on which the failed task ran and go to the 
+            <code>TaskTracker</code>'s local directory and run the 
+            <code>IsolationRunner</code>:<br/>
+            <code>$ cd &lt;local path&gt;/taskTracker/${taskid}/work</code><br/>
+            <code>
+              $ bin/hadoop org.apache.hadoop.mapred.IsolationRunner ../job.xml
+            </code>
+          </p>
+          
+          <p><code>IsolationRunner</code> will run the failed task in a single 
+          jvm, which can be in the debugger, over precisely the same input.</p>
+        </section>
+        
+        <section>
+          <title>JobControl</title>
+          
+          <p><a href="ext:api/org/apache/hadoop/mapred/jobcontrol/package-summary">
+          JobControl</a> is a utility which encapsulates a set of Map-Reduce jobs
+          and their dependencies.</p>
+        </section>
+      </section>
+    </section>
+
+    <section>
+      <title>Example: WordCount v2.0</title>
+      
+      <p>Here is a more complete <code>WordCount</code> which uses many of the
+      features provided by the Map-Reduce framework we discussed so far:</p>
+      
+      <section>
+        <title>Source Code</title>
+        
+        <table>
+          <tr>
+            <th></th>
+            <th>WordCount.java</th>
+          </tr>
+          <tr>
+            <td>1.</td>
+            <td>
+              <code>package org.myorg;</code>
+            </td>
+          </tr>
+          <tr>
+            <td>2.</td>
+            <td></td>
+          </tr>
+          <tr>
+            <td>3.</td>
+            <td>
+              <code>import java.io.*;</code>
+            </td>
+          </tr>
+          <tr>
+            <td>4.</td>
+            <td>
+              <code>import java.util.*;</code>
+            </td>
+          </tr>
+          <tr>
+            <td>5.</td>
+            <td></td>
+          </tr>
+          <tr>
+            <td>6.</td>
+            <td>
+              <code>import org.apache.hadoop.fs.Path;</code>
+            </td>
+          </tr>
+          <tr>
+            <td>7.</td>
+            <td>
+              <code>import org.apache.hadoop.filecache.DistributedCache;</code>
+            </td>
+          </tr>
+          <tr>
+            <td>8.</td>
+            <td>
+              <code>import org.apache.hadoop.conf.*;</code>
+            </td>
+          </tr>
+          <tr>
+            <td>9.</td>
+            <td>
+              <code>import org.apache.hadoop.io.*;</code>
+            </td>
+          </tr>
+          <tr>
+            <td>10.</td>
+            <td>
+              <code>import org.apache.hadoop.mapred.*;</code>
+            </td>
+          </tr>
+          <tr>
+            <td>11.</td>
+            <td>
+              <code>import org.apache.hadoop.util.*;</code>
+            </td>
+          </tr>
+          <tr>
+            <td>12.</td>
+            <td></td>
+          </tr>
+          <tr>
+            <td>13.</td>
+            <td>
+              <code>public class WordCount extends Configured implements Tool {</code>
+            </td>
+          </tr>
+          <tr>
+            <td>14.</td>
+            <td></td>
+          </tr>
+          <tr>
+            <td>15.</td>
+            <td>
+              &nbsp;&nbsp;
+              <code>
+                public static class MapClass extends MapReduceBase 
+                implements Mapper&lt;LongWritable, Text, Text, IntWritable&gt; {
+              </code>
+            </td>
+          </tr>
+          <tr>
+            <td>16.</td>
+            <td></td>
+          </tr>
+          <tr>
+            <td>17.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>
+                static enum Counters { INPUT_WORDS }
+              </code>
+            </td>
+          </tr>
+          <tr>
+            <td>18.</td>
+            <td></td>
+          </tr>
+          <tr>
+            <td>19.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>
+                private final static IntWritable one = new IntWritable(1);
+              </code>
+            </td>
+          </tr>
+          <tr>
+            <td>20.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>private Text word = new Text();</code>
+            </td>
+          </tr>
+          <tr>
+            <td>21.</td>
+            <td></td>
+          </tr>
+          <tr>
+            <td>22.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>private boolean caseSensitive = true;</code>
+            </td>
+          </tr>
+          <tr>
+            <td>23.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>private Set&lt;String&gt; patternsToSkip = new HashSet&lt;String&gt;();</code>
+            </td>
+          </tr>
+          <tr>
+            <td>24.</td>
+            <td></td>
+          </tr>
+          <tr>
+            <td>25.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>private long numRecords = 0;</code>
+            </td>
+          </tr>
+          <tr>
+            <td>26.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>private String inputFile;</code>
+            </td>
+          </tr>
+          <tr>
+            <td>27.</td>
+            <td></td>
+          </tr>
+          <tr>
+            <td>28.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>public void configure(JobConf job) {</code>
+            </td>
+          </tr>
+          <tr>
+            <td>29.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>
+                caseSensitive = job.getBoolean("wordcount.case.sensitive", true);
+              </code>
+            </td>
+          </tr>
+          <tr>
+            <td>30.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>inputFile = job.get("map.input.file");</code>
+            </td>
+          </tr>
+          <tr>
+            <td>31.</td>
+            <td></td>
+          </tr>
+          <tr>
+            <td>32.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>Path[] patternsFiles = new Path[0];</code>
+            </td>
+          </tr>
+          <tr>
+            <td>33.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>try {</code>
+            </td>
+          </tr>
+          <tr>
+            <td>34.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>
+                patternsFiles = DistributedCache.getLocalCacheFiles(job);
+              </code>
+            </td>
+          </tr>
+          <tr>
+            <td>35.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>} catch (IOException ioe) {</code>
+            </td>
+          </tr>
+          <tr>
+            <td>36.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>
+                System.err.println("Caught exception while getting cached files: " 
+                + StringUtils.stringifyException(ioe));
+              </code>
+            </td>
+          </tr>
+          <tr>
+            <td>37.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>}</code>
+            </td>
+          </tr>
+          <tr>
+            <td>38.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>for (Path patternsFile : patternsFiles) {</code>
+            </td>
+          </tr>
+          <tr>
+            <td>39.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>parseSkipFile(patternsFile);</code>
+            </td>
+          </tr>
+          <tr>
+            <td>40.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>}</code>
+            </td>
+          </tr>
+          <tr>
+            <td>41.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>}</code>
+            </td>
+          </tr>
+          <tr>
+            <td>42.</td>
+            <td></td>
+          </tr>
+          <tr>
+            <td>43.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>private void parseSkipFile(Path patternsFile) {</code>
+            </td>
+          </tr>
+          <tr>
+            <td>44.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>try {</code>
+            </td>
+          </tr>
+          <tr>
+            <td>45.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>
+                BufferedReader fis = 
+                  new BufferedReader(new FileReader(patternsFile.toString()));
+              </code>
+            </td>
+          </tr>
+          <tr>
+            <td>46.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>String pattern = null;</code>
+            </td>
+          </tr>
+          <tr>
+            <td>47.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>while ((pattern = fis.readLine()) != null) {</code>
+            </td>
+          </tr>
+          <tr>
+            <td>48.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>patternsToSkip.add(pattern);</code>
+            </td>
+          </tr>
+          <tr>
+            <td>49.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>}</code>
+            </td>
+          </tr>
+          <tr>
+            <td>50.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>} catch (IOException ioe) {</code>
+            </td>
+          </tr>
+          <tr>
+            <td>51.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>
+                System.err.println("Caught exception while parsing the cached file '" +
+                                   patternsFile + "' : " + 
+                                   StringUtils.stringifyException(ioe));
+                
+              </code>
+            </td>
+          </tr>
+          <tr>
+            <td>52.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>}</code>
+            </td>
+          </tr>
+          <tr>
+            <td>53.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>}</code>
+            </td>
+          </tr>
+          <tr>
+            <td>54.</td>
+            <td></td>
+          </tr>
+          <tr>
+            <td>55.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>
+                public void map(LongWritable key, Text value, 
+                OutputCollector&lt;Text, IntWritable&gt; output, 
+                Reporter reporter) throws IOException {
+              </code>
+            </td>
+          </tr>
+          <tr>
+            <td>56.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>
+                String line = 
+                  (caseSensitive) ? value.toString() : 
+                                    value.toString().toLowerCase();
+              </code>
+            </td>
+          </tr>
+          <tr>
+            <td>57.</td>
+            <td></td>
+          </tr>
+          <tr>
+            <td>58.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>for (String pattern : patternsToSkip) {</code>
+            </td>
+          </tr>
+          <tr>
+            <td>59.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>line = line.replaceAll(pattern, "");</code>
+            </td>
+          </tr>
+          <tr>
+            <td>60.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>}</code>
+            </td>
+          </tr>
+          <tr>
+            <td>61.</td>
+            <td></td>
+          </tr>
+          <tr>
+            <td>62.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>StringTokenizer tokenizer = new StringTokenizer(line);</code>
+            </td>
+          </tr>
+          <tr>
+            <td>63.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>while (tokenizer.hasMoreTokens()) {</code>
+            </td>
+          </tr>
+          <tr>
+            <td>64.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>word.set(tokenizer.nextToken());</code>
+            </td>
+          </tr>
+          <tr>
+            <td>65.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>output.collect(word, one);</code>
+            </td>
+          </tr>
+          <tr>
+            <td>66.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>reporter.incrCounter(Counters.INPUT_WORDS, 1);</code>
+            </td>
+          </tr>
+          <tr>
+            <td>67.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>}</code>
+            </td>
+          </tr>
+          <tr>
+            <td>68.</td>
+            <td></td>
+          </tr>
+          <tr>
+            <td>69.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>if ((++numRecords % 100) == 0) {</code>
+            </td>
+          </tr>
+          <tr>
+            <td>70.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>
+                reporter.setStatus("Finished processing " + numRecords + 
+                                   " records " + "from the input file: " + 
+                                   inputFile);
+              </code>
+            </td>
+          </tr>
+          <tr>
+            <td>71.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>}</code>
+            </td>
+          </tr>
+          <tr>
+            <td>72.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>}</code>
+            </td>
+          </tr>
+          <tr>
+            <td>73.</td>
+            <td>
+              &nbsp;&nbsp;
+              <code>}</code>
+            </td>
+          </tr>
+          <tr>
+            <td>74.</td>
+            <td></td>
+          </tr>
+          <tr>
+            <td>75.</td>
+            <td>
+              &nbsp;&nbsp;
+              <code>
+                public static class Reduce extends MapReduceBase implements 
+                Reducer&lt;Text, IntWritable, Text, IntWritable&gt; {
+              </code>
+            </td>
+          </tr>
+          <tr>
+            <td>76.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>
+                public void reduce(Text key, Iterator&lt;IntWritable&gt; values,
+                OutputCollector&lt;Text, IntWritable&gt; output, 
+                Reporter reporter) throws IOException {
+              </code>
+            </td>
+          </tr>
+          <tr>
+            <td>77.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>int sum = 0;</code>
+            </td>
+          </tr>
+          <tr>
+            <td>78.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>while (values.hasNext()) {</code>
+            </td>
+          </tr>
+          <tr>
+            <td>79.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>sum += values.next().get();</code>
+            </td>
+          </tr>
+          <tr>
+            <td>80.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>}</code>
+            </td>
+          </tr>
+          <tr>
+            <td>81.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>output.collect(key, new IntWritable(sum));</code>
+            </td>
+          </tr>
+          <tr>
+            <td>82.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>}</code>
+            </td>
+          </tr>
+          <tr>
+            <td>83.</td>
+            <td>
+              &nbsp;&nbsp;
+              <code>}</code>
+            </td>
+          </tr>
+          <tr>
+            <td>84.</td>
+            <td></td>
+          </tr>
+          <tr>
+            <td>85.</td>
+            <td>
+              &nbsp;&nbsp;
+              <code>public int run(String[] args) throws Exception {</code>
+            </td>
+          </tr>
+          <tr>
+            <td>86.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>
+                JobConf conf = new JobConf(getConf(), WordCount.class);
+              </code>
+            </td>
+          </tr>
+          <tr>
+            <td>87.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>conf.setJobName("wordcount");</code>
+            </td>
+          </tr>
+          <tr>
+            <td>88.</td>
+            <td></td>
+          </tr>
+          <tr>
+            <td>89.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>conf.setOutputKeyClass(Text.class);</code>
+            </td>
+          </tr>
+          <tr>
+            <td>90.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>conf.setOutputValueClass(IntWritable.class);</code>
+            </td>
+          </tr>
+          <tr>
+            <td>91.</td>
+            <td></td>
+          </tr>
+          <tr>
+            <td>92.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>conf.setMapperClass(MapClass.class);</code>
+            </td>
+          </tr>
+          <tr>
+            <td>93.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>conf.setCombinerClass(Reduce.class);</code>
+            </td>
+          </tr>
+          <tr>
+            <td>94.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>conf.setReducerClass(Reduce.class);</code>
+            </td>
+          </tr>
+          <tr>
+            <td>95.</td>
+            <td></td>
+          </tr>
+          <tr>
+            <td>96.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>conf.setInputFormat(TextInputFormat.class);</code>
+            </td>
+          </tr>
+          <tr>
+            <td>97.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>conf.setOutputFormat(TextOutputFormat.class);</code>
+            </td>
+          </tr>
+          <tr>
+            <td>98.</td>
+            <td></td>
+          </tr>
+          <tr>
+            <td>99.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>
+                List&lt;String&gt; other_args = new ArrayList&lt;String&gt;();
+              </code>
+            </td>
+          </tr>
+          <tr>
+            <td>100.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>for (int i=0; i &lt; args.length; ++i) {</code>
+            </td>
+          </tr>
+          <tr>
+            <td>101.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>if ("-skip".equals(args[i]) {</code>
+            </td>
+          </tr>
+          <tr>
+            <td>102.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>
+                DistributedCache.addCacheFile(new Path(args[++i]).toUri(), conf);
+              </code>
+            </td>
+          </tr>
+          <tr>
+            <td>103.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>} else {</code>
+            </td>
+          </tr>
+          <tr>
+            <td>104.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>other_args.add(args[i]);</code>
+            </td>
+          </tr>
+          <tr>
+            <td>105.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
+              <code>}</code>
+            </td>
+          </tr>
+          <tr>
+            <td>106.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>}</code>
+            </td>
+          </tr>
+          <tr>
+            <td>107.</td>
+            <td></td>
+          </tr>
+          <tr>
+            <td>108.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>conf.setInputPath(new Path(other_args[0]));</code>
+            </td>
+          </tr>
+          <tr>
+            <td>109.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>conf.setOutputPath(new Path(other_args[1]));</code>
+            </td>
+          </tr>
+          <tr>
+            <td>110.</td>
+            <td></td>
+          </tr>
+          <tr>
+            <td>111.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>JobClient.runJob(conf);</code>
+            </td>
+          </tr>
+          <tr>
+            <td>112.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>return 0;</code>
+            </td>
+          </tr>
+          <tr>
+            <td>113.</td>
+            <td>
+              &nbsp;&nbsp;
+              <code>}</code>
+            </td>
+          </tr>
+          <tr>
+            <td>114.</td>
+            <td></td>
+          </tr>
+          <tr>
+            <td>115.</td>
+            <td>
+              &nbsp;&nbsp;
+              <code>
+                public static void main(String[] args) throws Exception {
+              </code>
+            </td>
+          </tr>
+          <tr>
+            <td>116.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>
+                int res = ToolRunner.run(new Configuration(), new WordCount(), 
+                                         args);
+              </code>
+            </td>
+          </tr>
+          <tr>
+            <td>117.</td>
+            <td>
+              &nbsp;&nbsp;&nbsp;&nbsp;
+              <code>System.exit(res);</code>
+            </td>
+          </tr>
+          <tr>
+            <td>118.</td>
+            <td>
+              &nbsp;&nbsp;
+              <code>}</code>
+            </td>
+          </tr>
+          <tr>
+            <td>119.</td>
+            <td>
+              <code>}</code>
+            </td>
+          </tr>
+          <tr>
+            <td>120.</td>
+            <td></td>
+          </tr>
+        </table>
+      </section>
+        
+      <section>
+        <title>Sample Runs</title>
+        
+        <p>Sample text-files as input:</p>
+        <p>
+          <code>$ bin/hadoop dfs -ls /usr/joe/wordcount/input/</code><br/>
+          <code>/usr/joe/wordcount/input/file01</code><br/>
+          <code>/usr/joe/wordcount/input/file02</code><br/>
+          <br/>
+          <code>$ bin/hadoop dfs -cat /usr/joe/wordcount/input/file01</code><br/>
+          <code>Hello World, Bye World!</code><br/>
+          <br/>
+          <code>$ bin/hadoop dfs -cat /usr/joe/wordcount/input/file02</code><br/>
+          <code>Hello Hadoop, Goodbye the Hadoop.</code>
+        </p>
+        
+        <p>Run the application:</p>
+        <p>
+          <code>
+            $ bin/hadoop jar /usr/joe/wordcount.jar org.myorg.WordCount 
+              /usr/joe/wordcount/input /usr/joe/wordcount/output 
+          </code>
+        </p>
+
+        <p>Output:</p>
+        <p>
+          <code>
+            $ bin/hadoop dfs -cat /usr/joe/wordcount/output/part-00000
+          </code>
+          <br/>
+          <code>Bye    1</code><br/>
+          <code>Goodbye    1</code><br/>
+          <code>Hadoop,    1</code><br/>
+          <code>Hadoop.    1</code><br/>
+          <code>Hello    2</code><br/>
+          <code>World!    1</code><br/>
+          <code>World,    1</code><br/>
+          <code>the    1</code><br/>
+        </p>
+        
+        <p>Notice that the inputs differ from the first version we looked at, 
+        and how they affect the outputs.</p>
+
+        <p>Now, lets plug-in a pattern-file which lists the word-patterns to be 
+        ignored, via the <code>DistributedCache</code>.</p>
+        
+        <p>
+          <code>$ hadoop dfs -cat /user/joe/wordcount/patterns.txt</code><br/>
+          <code>\.</code><br/>
+          <code>\,</code><br/>
+          <code>\!</code><br/>
+          <code>the</code><br/>
+        </p>
+        
+        <p>Run it again, this time with more options:</p>
+        <p>
+          <code>
+            $ bin/hadoop jar /usr/joe/wordcount.jar org.myorg.WordCount 
+              -Dwordcount.case.sensitive=true /usr/joe/wordcount/input 
+              /usr/joe/wordcount/output -skip /user/joe/wordcount/patterns.txt
+          </code>
+        </p>
+        
+        <p>As expected, the output:</p>
+        <p>
+          <code>
+            $ bin/hadoop dfs -cat /usr/joe/wordcount/output/part-00000
+          </code>
+          <br/>
+          <code>Bye    1</code><br/>
+          <code>Goodbye    1</code><br/>
+          <code>Hadoop    2</code><br/>
+          <code>Hello    2</code><br/>
+          <code>World    2</code><br/>
+        </p>
+        
+        <p>Run it once more, this time switch-off case-sensitivity:</p>
+        <p>
+          <code>
+            $ bin/hadoop jar /usr/joe/wordcount.jar org.myorg.WordCount 
+              -Dwordcount.case.sensitive=false /usr/joe/wordcount/input 
+              /usr/joe/wordcount/output -skip /user/joe/wordcount/patterns.txt
+          </code>
+        </p>
+        
+        <p>Sure enough, the output:</p>
+        <p>
+          <code>
+            $ bin/hadoop dfs -cat /usr/joe/wordcount/output/part-00000
+          </code>
+          <br/>
+          <code>bye    1</code><br/>
+          <code>goodbye    1</code><br/>
+          <code>hadoop    2</code><br/>
+          <code>hello    2</code><br/>
+          <code>world    2</code><br/>
+        </p>
+      </section>
+      
+      <section>
+        <title>Salient Points</title>
+        
+        <p>The second version of <code>WordCount</code> improves upon the 
+        previous one by using some features offered by the Map-Reduce framework:
+        </p>
+        <ul>
+          <li>
+            Demonstrates how applications can access configuration parameters
+            in the <code>configure</code> method of the <code>Mapper</code> (and
+            <code>Reducer</code>) implementations (lines 28-41).
+          </li>
+          <li>
+            Demonstrates how the <code>DistributedCache</code> can be used to 
+            distribute read-only data needed by the jobs. Here it allows the user 
+            to specify word-patterns to skip while counting (line 102).
+          </li>
+          <li>
+            Demonstrates the utility of the <code>Tool</code> interface and the
+            <code>GenericOptionsParser</code> to handle generic Hadoop 
+            command-line options (lines 85-86, 116).
+          </li>
+          <li>
+            Demonstrates how applications can use <code>Counters</code> (line 66)
+            and how they can set application-specific status information via 
+            the <code>Reporter</code> instance passed to the <code>map</code> (and
+            <code>reduce</code>) method (line 70).
+          </li>
+        </ul>
+        
+      </section>
+    </section>
+
+    <p>
+      <em>Java and JNI are trademarks or registered trademarks of 
+      Sun Microsystems, Inc. in the United States and other countries.</em>
+    </p>
+    
+  </body>
+  
+</document>



Mime
View raw message