| <?xml version="1.0"?> |
| <!-- |
| Copyright 2002-2004 The Apache Software Foundation |
| |
| Licensed under the Apache License, Version 2.0 (the "License"); |
| you may not use this file except in compliance with the License. |
| You may obtain a copy of the License at |
| |
| http://www.apache.org/licenses/LICENSE-2.0 |
| |
| Unless required by applicable law or agreed to in writing, software |
| distributed under the License is distributed on an "AS IS" BASIS, |
| WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| See the License for the specific language governing permissions and |
| limitations under the License. |
| --> |
| |
| <!DOCTYPE document PUBLIC "-//APACHE//DTD Documentation V2.0//EN" "http://forrest.apache.org/dtd/document-v20.dtd"> |
| |
| <document> |
| |
| <header> |
| <title>MapReduce Tutorial</title> |
| </header> |
| |
| <body> |
| |
| <section> |
| <title>Purpose</title> |
| |
| <p>This document comprehensively describes all user-facing facets of the |
| Hadoop MapReduce framework and serves as a tutorial. |
| </p> |
| </section> |
| |
| <section> |
| <title>Prerequisites</title> |
| |
| <p>Ensure that Hadoop is installed, configured and is running. More |
| details:</p> |
| <ul> |
| <li> |
| <a href="single_node_setup.html">Single Node Setup</a> for first-time users. |
| </li> |
| <li> |
| <a href="cluster_setup.html">Cluster Setup</a> for large, |
| distributed clusters. |
| </li> |
| </ul> |
| </section> |
| |
| <section> |
| <title>Overview</title> |
| |
| <p>Hadoop MapReduce 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 MapReduce <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 MapReduce framework and the Hadoop Distributed File System (see <a href="hdfs_design.html">HDFS Architecture Guide</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 MapReduce 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>, |
| MapReduce 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 MapReduce applications (non |
| JNI<sup>TM</sup> based). |
| </li> |
| </ul> |
| </section> |
| |
| <section> |
| <title>Inputs and Outputs</title> |
| |
| <p>The MapReduce framework operates exclusively on |
| <code><key, value></code> pairs, that is, the framework views the |
| input to the job as a set of <code><key, value></code> pairs and |
| produces a set of <code><key, value></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 MapReduce job:</p> |
| <p> |
| (input) <code><k1, v1></code> |
| -> |
| <strong>map</strong> |
| -> |
| <code><k2, v2></code> |
| -> |
| <strong>combine</strong> |
| -> |
| <code><k2, v2></code> |
| -> |
| <strong>reduce</strong> |
| -> |
| <code><k3, v3></code> (output) |
| </p> |
| </section> |
| |
| <section> |
| <title>Example: WordCount v1.0</title> |
| |
| <p>Before we jump into the details, lets walk through an example MapReduce |
| 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> |
| |
| <p>This works with a local-standalone, pseudo-distributed or fully-distributed |
| Hadoop installation (<a href="single_node_setup.html">Single Node Setup</a>).</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.IOException;</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> |
| |
| <code> |
| public static class Map extends MapReduceBase |
| implements Mapper<LongWritable, Text, Text, IntWritable> { |
| </code> |
| </td> |
| </tr> |
| <tr> |
| <td>15.</td> |
| <td> |
| |
| <code> |
| private final static IntWritable one = new IntWritable(1); |
| </code> |
| </td> |
| </tr> |
| <tr> |
| <td>16.</td> |
| <td> |
| |
| <code>private Text word = new Text();</code> |
| </td> |
| </tr> |
| <tr> |
| <td>17.</td> |
| <td></td> |
| </tr> |
| <tr> |
| <td>18.</td> |
| <td> |
| |
| <code> |
| public void map(LongWritable key, Text value, |
| OutputCollector<Text, IntWritable> output, |
| Reporter reporter) throws IOException { |
| </code> |
| </td> |
| </tr> |
| <tr> |
| <td>19.</td> |
| <td> |
| |
| <code>String line = value.toString();</code> |
| </td> |
| </tr> |
| <tr> |
| <td>20.</td> |
| <td> |
| |
| <code>StringTokenizer tokenizer = new StringTokenizer(line);</code> |
| </td> |
| </tr> |
| <tr> |
| <td>21.</td> |
| <td> |
| |
| <code>while (tokenizer.hasMoreTokens()) {</code> |
| </td> |
| </tr> |
| <tr> |
| <td>22.</td> |
| <td> |
| |
| <code>word.set(tokenizer.nextToken());</code> |
| </td> |
| </tr> |
| <tr> |
| <td>23.</td> |
| <td> |
| |
| <code>output.collect(word, one);</code> |
| </td> |
| </tr> |
| <tr> |
| <td>24.</td> |
| <td> |
| |
| <code>}</code> |
| </td> |
| </tr> |
| <tr> |
| <td>25.</td> |
| <td> |
| |
| <code>}</code> |
| </td> |
| </tr> |
| <tr> |
| <td>26.</td> |
| <td> |
| |
| <code>}</code> |
| </td> |
| </tr> |
| <tr> |
| <td>27.</td> |
| <td></td> |
| </tr> |
| <tr> |
| <td>28.</td> |
| <td> |
| |
| <code> |
| public static class Reduce extends MapReduceBase implements |
| Reducer<Text, IntWritable, Text, IntWritable> { |
| </code> |
| </td> |
| </tr> |
| <tr> |
| <td>29.</td> |
| <td> |
| |
| <code> |
| public void reduce(Text key, Iterator<IntWritable> values, |
| OutputCollector<Text, IntWritable> output, |
| Reporter reporter) throws IOException { |
| </code> |
| </td> |
| </tr> |
| <tr> |
| <td>30.</td> |
| <td> |
| |
| <code>int sum = 0;</code> |
| </td> |
| </tr> |
| <tr> |
| <td>31.</td> |
| <td> |
| |
| <code>while (values.hasNext()) {</code> |
| </td> |
| </tr> |
| <tr> |
| <td>32.</td> |
| <td> |
| |
| <code>sum += values.next().get();</code> |
| </td> |
| </tr> |
| <tr> |
| <td>33.</td> |
| <td> |
| |
| <code>}</code> |
| </td> |
| </tr> |
| <tr> |
| <td>34.</td> |
| <td> |
| |
| <code>output.collect(key, new IntWritable(sum));</code> |
| </td> |
| </tr> |
| <tr> |
| <td>35.</td> |
| <td> |
| |
| <code>}</code> |
| </td> |
| </tr> |
| <tr> |
| <td>36.</td> |
| <td> |
| |
| <code>}</code> |
| </td> |
| </tr> |
| <tr> |
| <td>37.</td> |
| <td></td> |
| </tr> |
| <tr> |
| <td>38.</td> |
| <td> |
| |
| <code> |
| public static void main(String[] args) throws Exception { |
| </code> |
| </td> |
| </tr> |
| <tr> |
| <td>39.</td> |
| <td> |
| |
| <code> |
| JobConf conf = new JobConf(WordCount.class); |
| </code> |
| </td> |
| </tr> |
| <tr> |
| <td>40.</td> |
| <td> |
| |
| <code>conf.setJobName("wordcount");</code> |
| </td> |
| </tr> |
| <tr> |
| <td>41.</td> |
| <td></td> |
| </tr> |
| <tr> |
| <td>42.</td> |
| <td> |
| |
| <code>conf.setOutputKeyClass(Text.class);</code> |
| </td> |
| </tr> |
| <tr> |
| <td>43.</td> |
| <td> |
| |
| <code>conf.setOutputValueClass(IntWritable.class);</code> |
| </td> |
| </tr> |
| <tr> |
| <td>44.</td> |
| <td></td> |
| </tr> |
| <tr> |
| <td>45.</td> |
| <td> |
| |
| <code>conf.setMapperClass(Map.class);</code> |
| </td> |
| </tr> |
| <tr> |
| <td>46.</td> |
| <td> |
| |
| <code>conf.setCombinerClass(Reduce.class);</code> |
| </td> |
| </tr> |
| <tr> |
| <td>47.</td> |
| <td> |
| |
| <code>conf.setReducerClass(Reduce.class);</code> |
| </td> |
| </tr> |
| <tr> |
| <td>48.</td> |
| <td></td> |
| </tr> |
| <tr> |
| <td>49.</td> |
| <td> |
| |
| <code>conf.setInputFormat(TextInputFormat.class);</code> |
| </td> |
| </tr> |
| <tr> |
| <td>50.</td> |
| <td> |
| |
| <code>conf.setOutputFormat(TextOutputFormat.class);</code> |
| </td> |
| </tr> |
| <tr> |
| <td>51.</td> |
| <td></td> |
| </tr> |
| <tr> |
| <td>52.</td> |
| <td> |
| |
| <code>FileInputFormat.setInputPaths(conf, new Path(args[0]));</code> |
| </td> |
| </tr> |
| <tr> |
| <td>53.</td> |
| <td> |
| |
| <code>FileOutputFormat.setOutputPath(conf, new Path(args[1]));</code> |
| </td> |
| </tr> |
| <tr> |
| <td>54.</td> |
| <td></td> |
| </tr> |
| <tr> |
| <td>55.</td> |
| <td> |
| |
| <code>JobClient.runJob(conf);</code> |
| </td> |
| </tr> |
| <tr> |
| <td>57.</td> |
| <td> |
| |
| <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>$ mkdir wordcount_classes</code><br/> |
| <code> |
| $ javac -classpath ${HADOOP_HOME}/hadoop-${HADOOP_VERSION}-core.jar |
| -d wordcount_classes WordCount.java |
| </code><br/> |
| <code>$ jar -cvf /usr/joe/wordcount.jar -C wordcount_classes/ .</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> |
| |
| <p> Applications can specify a comma separated list of paths which |
| would be present in the current working directory of the task |
| using the option <code>-files</code>. The <code>-libjars</code> |
| option allows applications to add jars to the classpaths of the maps |
| and reduces. The option <code>-archives</code> allows them to pass |
| comma separated list of archives as arguments. These archives are |
| unarchived and a link with name of the archive is created in |
| the current working directory of tasks. More |
| details about the command line options are available at |
| <a href="commands_manual.html">Commands Guide.</a></p> |
| |
| <p>Running <code>wordcount</code> example with |
| <code>-libjars</code>, <code>-files</code> and <code>-archives</code>: |
| <br/> |
| <code> hadoop jar hadoop-examples.jar wordcount -files cachefile.txt |
| -libjars mylib.jar -archives myarchive.zip input output </code> |
| Here, myarchive.zip will be placed and unzipped into a directory |
| by the name "myarchive.zip". |
| </p> |
| |
| <p> Users can specify a different symbolic name for |
| files and archives passed through -files and -archives option, using #. |
| </p> |
| |
| <p> For example, |
| <code> hadoop jar hadoop-examples.jar wordcount |
| -files dir1/dict.txt#dict1,dir2/dict.txt#dict2 |
| -archives mytar.tgz#tgzdir input output </code> |
| Here, the files dir1/dict.txt and dir2/dict.txt can be accessed by |
| tasks using the symbolic names dict1 and dict2 respectively. |
| The archive mytar.tgz will be placed and unarchived into a |
| directory by the name "tgzdir". |
| </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>< <word>, 1></code>.</p> |
| |
| <p> |
| For the given sample input the first map emits:<br/> |
| <code>< Hello, 1></code><br/> |
| <code>< World, 1></code><br/> |
| <code>< Bye, 1></code><br/> |
| <code>< World, 1></code><br/> |
| </p> |
| |
| <p> |
| The second map emits:<br/> |
| <code>< Hello, 1></code><br/> |
| <code>< Hadoop, 1></code><br/> |
| <code>< Goodbye, 1></code><br/> |
| <code>< Hadoop, 1></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>< Bye, 1></code><br/> |
| <code>< Hello, 1></code><br/> |
| <code>< World, 2></code><br/> |
| </p> |
| |
| <p> |
| The output of the second map:<br/> |
| <code>< Goodbye, 1></code><br/> |
| <code>< Hadoop, 2></code><br/> |
| <code>< Hello, 1></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>< 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>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>MapReduce - User Interfaces</title> |
| |
| <p>This section provides a reasonable amount of detail on every user-facing |
| aspect of the MapReduce framework. 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>, |
| <code>OutputCommitter</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 MapReduce 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 a simple |
| (key-len, key, value-len, value) 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><key, (list of values)></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><key, (list of values)></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 (<<em>no. of nodes</em>> * |
| <code>mapred.tasktracker.reduce.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/fileoutputformat/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 MapReduce 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 MapReduce 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 MapReduce 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 MapReduce job configuration.</p> |
| |
| <p><code>JobConf</code> is the primary interface for a user to describe |
| a MapReduce 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_parameters"> |
| 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>, |
| <code>OutputFormat</code> and <code>OutputCommitter</code> |
| implementations. <code>JobConf</code> also |
| indicates the set of input files |
| (<a href="ext:api/org/apache/hadoop/mapred/fileinputformat/setinputpaths">setInputPaths(JobConf, Path...)</a> |
| /<a href="ext:api/org/apache/hadoop/mapred/fileinputformat/addinputpath">addInputPath(JobConf, Path)</a>) |
| and (<a href="ext:api/org/apache/hadoop/mapred/fileinputformat/setinputpathstring">setInputPaths(JobConf, String)</a> |
| /<a href="ext:api/org/apache/hadoop/mapred/fileinputformat/addinputpathstring">addInputPaths(JobConf, String)</a>) |
| and where the output files should be written |
| (<a href="ext:api/org/apache/hadoop/mapred/fileoutputformat/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/setmapspeculativeexecution">setMapSpeculativeExecution(boolean)</a>)/(<a href="ext:api/org/apache/hadoop/mapred/jobconf/setreducespeculativeexecution">setReduceSpeculativeExecution(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>Task Execution & Environment</title> |
| |
| <p>The <code>TaskTracker</code> executes the <code>Mapper</code>/ |
| <code>Reducer</code> <em>task</em> as a child process in a separate jvm. |
| </p> |
| |
| <p>The child-task inherits the environment of the parent |
| <code>TaskTracker</code>. The user can specify additional options to the |
| child-jvm via the <code>mapred.{map|reduce}.child.java.opts</code> |
| configuration parameter in the <code>JobConf</code> such as non-standard |
| paths for the run-time linker to search shared libraries via |
| <code>-Djava.library.path=<></code> etc. If the |
| <code>mapred.{map|reduce}.child.java.opts</code> parameters contains the |
| symbol <em>@taskid@</em> it is interpolated with value of |
| <code>taskid</code> of the MapReduce task.</p> |
| |
| <p>Here is an example with multiple arguments and substitutions, |
| showing jvm GC logging, and start of a passwordless JVM JMX agent so that |
| it can connect with jconsole and the likes to watch child memory, |
| threads and get thread dumps. It also sets the maximum heap-size of the |
| map and reduce child jvm to 512MB & 1024MB respectively. It also |
| adds an additional path to the <code>java.library.path</code> of the |
| child-jvm.</p> |
| |
| <p> |
| <code><property></code><br/> |
| <code><name>mapred.map.child.java.opts</name></code><br/> |
| <code><value></code><br/> |
| <code> |
| -Xmx512M -Djava.library.path=/home/mycompany/lib |
| -verbose:gc -Xloggc:/tmp/@taskid@.gc</code><br/> |
| <code> |
| -Dcom.sun.management.jmxremote.authenticate=false |
| -Dcom.sun.management.jmxremote.ssl=false</code><br/> |
| <code></value></code><br/> |
| <code></property></code> |
| </p> |
| |
| <p> |
| <code><property></code><br/> |
| <code><name>mapred.reduce.child.java.opts</name></code><br/> |
| <code><value></code><br/> |
| <code> |
| -Xmx1024M -Djava.library.path=/home/mycompany/lib |
| -verbose:gc -Xloggc:/tmp/@taskid@.gc</code><br/> |
| <code> |
| -Dcom.sun.management.jmxremote.authenticate=false |
| -Dcom.sun.management.jmxremote.ssl=false</code><br/> |
| <code></value></code><br/> |
| <code></property></code> |
| </p> |
| |
| <section> |
| <title> Memory Management</title> |
| <p>Users/admins can also specify the maximum virtual memory |
| of the launched child-task, and any sub-process it launches |
| recursively, using <code>mapred.{map|reduce}.child.ulimit</code>. Note |
| that the value set here is a per process limit. |
| The value for <code>mapred.{map|reduce}.child.ulimit</code> should be |
| specified in kilo bytes (KB). And also the value must be greater than |
| or equal to the -Xmx passed to JavaVM, else the VM might not start. |
| </p> |
| |
| <p>Note: <code>mapred.{map|reduce}.child.java.opts</code> are used only |
| for configuring the launched child tasks from task tracker. Configuring |
| the memory options for daemons is documented in |
| <a href="cluster_setup.html#Configuring+the+Environment+of+the+Hadoop+Daemons"> |
| Configuring the Environment of the Hadoop Daemons</a>.</p> |
| |
| <p>The memory available to some parts of the framework is also |
| configurable. In map and reduce tasks, performance may be influenced |
| by adjusting parameters influencing the concurrency of operations and |
| the frequency with which data will hit disk. Monitoring the filesystem |
| counters for a job- particularly relative to byte counts from the map |
| and into the reduce- is invaluable to the tuning of these |
| parameters.</p> |
| |
| <p>Users can choose to override default limits of Virtual Memory and RAM |
| enforced by the task tracker, if memory management is enabled. |
| Users can set the following parameter per job:</p> |
| |
| <table> |
| <tr><th>Name</th><th>Type</th><th>Description</th></tr> |
| <tr><td><code>mapred.task.maxvmem</code></td><td>int</td> |
| <td>A number, in bytes, that represents the maximum Virtual Memory |
| task-limit for each task of the job. A task will be killed if |
| it consumes more Virtual Memory than this number. |
| </td></tr> |
| <tr><td>mapred.task.maxpmem</td><td>int</td> |
| <td>A number, in bytes, that represents the maximum RAM task-limit |
| for each task of the job. This number can be optionally used by |
| Schedulers to prevent over-scheduling of tasks on a node based |
| on RAM needs. |
| </td></tr> |
| </table> |
| </section> |
| <section> |
| <title>Map Parameters</title> |
| |
| <p>A record emitted from a map will be serialized into a buffer and |
| metadata will be stored into accounting buffers. As described in the |
| following options, when either the serialization buffer or the |
| metadata exceed a threshold, the contents of the buffers will be |
| sorted and written to disk in the background while the map continues |
| to output records. If either buffer fills completely while the spill |
| is in progress, the map thread will block. When the map is finished, |
| any remaining records are written to disk and all on-disk segments |
| are merged into a single file. Minimizing the number of spills to |
| disk can decrease map time, but a larger buffer also decreases the |
| memory available to the mapper.</p> |
| |
| <table> |
| <tr><th>Name</th><th>Type</th><th>Description</th></tr> |
| <tr><td>io.sort.mb</td><td>int</td> |
| <td>The cumulative size of the serialization and accounting |
| buffers storing records emitted from the map, in megabytes. |
| </td></tr> |
| <tr><td>io.sort.record.percent</td><td>float</td> |
| <td>The ratio of serialization to accounting space can be |
| adjusted. Each serialized record requires 16 bytes of |
| accounting information in addition to its serialized size to |
| effect the sort. This percentage of space allocated from |
| <code>io.sort.mb</code> affects the probability of a spill to |
| disk being caused by either exhaustion of the serialization |
| buffer or the accounting space. Clearly, for a map outputting |
| small records, a higher value than the default will likely |
| decrease the number of spills to disk.</td></tr> |
| <tr><td>io.sort.spill.percent</td><td>float</td> |
| <td>This is the threshold for the accounting and serialization |
| buffers. When this percentage of either buffer has filled, |
| their contents will be spilled to disk in the background. Let |
| <code>io.sort.record.percent</code> be <em>r</em>, |
| <code>io.sort.mb</code> be <em>x</em>, and this value be |
| <em>q</em>. The maximum number of records collected before the |
| collection thread will spill is <code>r * x * q * 2^16</code>. |
| Note that a higher value may decrease the number of- or even |
| eliminate- merges, but will also increase the probability of |
| the map task getting blocked. The lowest average map times are |
| usually obtained by accurately estimating the size of the map |
| output and preventing multiple spills.</td></tr> |
| </table> |
| |
| <p>Other notes</p> |
| <ul> |
| <li>If either spill threshold is exceeded while a spill is in |
| progress, collection will continue until the spill is finished. |
| For example, if <code>io.sort.buffer.spill.percent</code> is set |
| to 0.33, and the remainder of the buffer is filled while the spill |
| runs, the next spill will include all the collected records, or |
| 0.66 of the buffer, and will not generate additional spills. In |
| other words, the thresholds are defining triggers, not |
| blocking.</li> |
| <li>A record larger than the serialization buffer will first |
| trigger a spill, then be spilled to a separate file. It is |
| undefined whether or not this record will first pass through the |
| combiner.</li> |
| </ul> |
| </section> |
| |
| <section> |
| <title>Shuffle/Reduce Parameters</title> |
| |
| <p>As described previously, each reduce fetches the output assigned |
| to it by the Partitioner via HTTP into memory and periodically |
| merges these outputs to disk. If intermediate compression of map |
| outputs is turned on, each output is decompressed into memory. The |
| following options affect the frequency of these merges to disk prior |
| to the reduce and the memory allocated to map output during the |
| reduce.</p> |
| |
| <table> |
| <tr><th>Name</th><th>Type</th><th>Description</th></tr> |
| <tr><td>io.sort.factor</td><td>int</td> |
| <td>Specifies the number of segments on disk to be merged at |
| the same time. It limits the number of open files and |
| compression codecs during the merge. If the number of files |
| exceeds this limit, the merge will proceed in several passes. |
| Though this limit also applies to the map, most jobs should be |
| configured so that hitting this limit is unlikely |
| there.</td></tr> |
| <tr><td>mapred.inmem.merge.threshold</td><td>int</td> |
| <td>The number of sorted map outputs fetched into memory |
| before being merged to disk. Like the spill thresholds in the |
| preceding note, this is not defining a unit of partition, but |
| a trigger. In practice, this is usually set very high (1000) |
| or disabled (0), since merging in-memory segments is often |
| less expensive than merging from disk (see notes following |
| this table). This threshold influences only the frequency of |
| in-memory merges during the shuffle.</td></tr> |
| <tr><td>mapred.job.shuffle.merge.percent</td><td>float</td> |
| <td>The memory threshold for fetched map outputs before an |
| in-memory merge is started, expressed as a percentage of |
| memory allocated to storing map outputs in memory. Since map |
| outputs that can't fit in memory can be stalled, setting this |
| high may decrease parallelism between the fetch and merge. |
| Conversely, values as high as 1.0 have been effective for |
| reduces whose input can fit entirely in memory. This parameter |
| influences only the frequency of in-memory merges during the |
| shuffle.</td></tr> |
| <tr><td>mapred.job.shuffle.input.buffer.percent</td><td>float</td> |
| <td>The percentage of memory- relative to the maximum heapsize |
| as typically specified in <code>mapred.reduce.child.java.opts</code>- |
| that can be allocated to storing map outputs during the |
| shuffle. Though some memory should be set aside for the |
| framework, in general it is advantageous to set this high |
| enough to store large and numerous map outputs.</td></tr> |
| <tr><td>mapred.job.reduce.input.buffer.percent</td><td>float</td> |
| <td>The percentage of memory relative to the maximum heapsize |
| in which map outputs may be retained during the reduce. When |
| the reduce begins, map outputs will be merged to disk until |
| those that remain are under the resource limit this defines. |
| By default, all map outputs are merged to disk before the |
| reduce begins to maximize the memory available to the reduce. |
| For less memory-intensive reduces, this should be increased to |
| avoid trips to disk.</td></tr> |
| </table> |
| |
| <p>Other notes</p> |
| <ul> |
| <li>If a map output is larger than 25 percent of the memory |
| allocated to copying map outputs, it will be written directly to |
| disk without first staging through memory.</li> |
| <li>When running with a combiner, the reasoning about high merge |
| thresholds and large buffers may not hold. For merges started |
| before all map outputs have been fetched, the combiner is run |
| while spilling to disk. In some cases, one can obtain better |
| reduce times by spending resources combining map outputs- making |
| disk spills small and parallelizing spilling and fetching- rather |
| than aggressively increasing buffer sizes.</li> |
| <li>When merging in-memory map outputs to disk to begin the |
| reduce, if an intermediate merge is necessary because there are |
| segments to spill and at least <code>io.sort.factor</code> |
| segments already on disk, the in-memory map outputs will be part |
| of the intermediate merge.</li> |
| </ul> |
| |
| </section> |
| |
| <section> |
| <title> Directory Structure </title> |
| <p>The task tracker has local directory, |
| <code> ${mapred.local.dir}/taskTracker/</code> to create localized |
| cache and localized job. It can define multiple local directories |
| (spanning multiple disks) and then each filename is assigned to a |
| semi-random local directory. When the job starts, task tracker |
| creates a localized job directory relative to the local directory |
| specified in the configuration. Thus the task tracker directory |
| structure looks as following: </p> |
| <ul> |
| <li><code>${mapred.local.dir}/taskTracker/distcache/</code> : |
| The public distributed cache for the jobs of all users. This directory |
| holds the localized public distributed cache. Thus localized public |
| distributed cache is shared among all the tasks and jobs of all users. |
| </li> |
| <li><code>${mapred.local.dir}/taskTracker/$user/distcache/</code> : |
| The private distributed cache for the jobs of the specific user. This |
| directory holds the localized private distributed cache. Thus localized |
| private distributed cache is shared among all the tasks and jobs of the |
| specific user only. It is not accessible to jobs of other users. |
| </li> |
| <li><code>${mapred.local.dir}/taskTracker/$user/jobcache/$jobid/ |
| </code> : The localized job directory |
| <ul> |
| <li><code>${mapred.local.dir}/taskTracker/$user/jobcache/$jobid/work/ |
| </code> |
| : The job-specific shared directory. The tasks can use this space as |
| scratch space and share files among them. This directory is exposed |
| to the users through the configuration property |
| <code>job.local.dir</code>. The directory can accessed through |
| the API <a href="ext:api/org/apache/hadoop/mapred/jobconf/getjoblocaldir"> |
| JobConf.getJobLocalDir()</a>. It is available as System property also. |
| So, users (streaming etc.) can call |
| <code>System.getProperty("job.local.dir")</code> to access the |
| directory.</li> |
| <li><code>${mapred.local.dir}/taskTracker/$user/jobcache/$jobid/jars/ |
| </code> |
| : The jars directory, which has the job jar file and expanded jar. |
| The <code>job.jar</code> is the application's jar file that is |
| automatically distributed to each machine. It is expanded in jars |
| directory before the tasks for the job start. The job.jar location |
| is accessible to the application through the api |
| <a href="ext:api/org/apache/hadoop/mapred/jobconf/getjar"> |
| JobConf.getJar() </a>. To access the unjarred directory, |
| JobConf.getJar().getParent() can be called.</li> |
| <li><code>${mapred.local.dir}/taskTracker/$user/jobcache/$jobid/job.xml |
| </code> |
| : The job.xml file, the generic job configuration, localized for |
| the job. </li> |
| <li><code>${mapred.local.dir}/taskTracker/$user/jobcache/$jobid/$taskid |
| </code> |
| : The task directory for each task attempt. Each task directory |
| again has the following structure : |
| <ul> |
| <li><code> |
| ${mapred.local.dir}/taskTracker/$user/jobcache/$jobid/$taskid/job.xml |
| </code> |
| : A job.xml file, task localized job configuration, Task localization |
| means that properties have been set that are specific to |
| this particular task within the job. The properties localized for |
| each task are described below.</li> |
| <li><code> |
| ${mapred.local.dir}/taskTracker/$user/jobcache/$jobid/$taskid/output |
| </code> |
| : A directory for intermediate output files. This contains the |
| temporary map reduce data generated by the framework |
| such as map output files etc. </li> |
| <li><code> |
| ${mapred.local.dir}/taskTracker/$user/jobcache/$jobid/$taskid/work |
| </code> |
| : The current working directory of the task. |
| With <a href="#Task+JVM+Reuse">jvm reuse</a> enabled for tasks, this |
| directory will be the directory on which the jvm has started</li> |
| <li><code> |
| ${mapred.local.dir}/taskTracker/$user/jobcache/$jobid/$taskid/work/tmp |
| </code> |
| : The temporary directory for the task. |
| (User can specify the property <code>mapred.child.tmp</code> to set |
| the value of temporary directory for map and reduce tasks. This |
| defaults to <code>./tmp</code>. If the value is not an absolute path, |
| it is prepended with task's working directory. Otherwise, it is |
| directly assigned. The directory will be created if it doesn't exist. |
| Then, the child java tasks are executed with option |
| <code>-Djava.io.tmpdir='the absolute path of the tmp dir'</code>. |
| Pipes and streaming are set with environment variable, |
| <code>TMPDIR='the absolute path of the tmp dir'</code>). This |
| directory is created, if <code>mapred.child.tmp</code> has the value |
| <code>./tmp</code> </li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| </ul> |
| </section> |
| |
| <section> |
| <title>Task JVM Reuse</title> |
| <p>Jobs can enable task JVMs to be reused by specifying the job |
| configuration <code>mapred.job.reuse.jvm.num.tasks</code>. If the |
| value is 1 (the default), then JVMs are not reused |
| (i.e. 1 task per JVM). If it is -1, there is no limit to the number |
| of tasks a JVM can run (of the same job). One can also specify some |
| value greater than 1 using the api |
| <a href="ext:api/org/apache/hadoop/mapred/jobconf/setnumtaskstoexecuteperjvm"> |
| JobConf.setNumTasksToExecutePerJvm(int)</a></p> |
| </section> |
| |
| <section> |
| <title>Configured Parameters</title> |
| <p>The following properties are localized in the job configuration |
| for each task's execution: </p> |
| <table> |
| <tr><th>Name</th><th>Type</th><th>Description</th></tr> |
| <tr><td>mapred.job.id</td><td>String</td><td>The job id</td></tr> |
| <tr><td>mapred.jar</td><td>String</td> |
| <td>job.jar location in job directory</td></tr> |
| <tr><td>job.local.dir</td><td> String</td> |
| <td> The job specific shared scratch space</td></tr> |
| <tr><td>mapred.tip.id</td><td> String</td> |
| <td> The task id</td></tr> |
| <tr><td>mapred.task.id</td><td> String</td> |
| <td> The task attempt id</td></tr> |
| <tr><td>mapred.task.is.map</td><td> boolean </td> |
| <td>Is this a map task</td></tr> |
| <tr><td>mapred.task.partition</td><td> int </td> |
| <td>The id of the task within the job</td></tr> |
| <tr><td>map.input.file</td><td> String</td> |
| <td> The filename that the map is reading from</td></tr> |
| <tr><td>map.input.start</td><td> long</td> |
| <td> The offset of the start of the map input split</td></tr> |
| <tr><td>map.input.length </td><td>long </td> |
| <td>The number of bytes in the map input split</td></tr> |
| <tr><td>mapred.work.output.dir</td><td> String </td> |
| <td>The task's temporary output directory</td></tr> |
| </table> |
| |
| <p> |
| <strong>Note:</strong> |
| During the execution of a streaming job, the names of the "mapred" parameters are transformed. |
| The dots ( . ) become underscores ( _ ). |
| For example, mapred.job.id becomes mapred_job_id and mapred.jar becomes mapred_jar. |
| To get the values in a streaming job's mapper/reducer use the parameter names with the underscores. |
| </p> |
| </section> |
| |
| <section> |
| <title>Task Logs</title> |
| <p>The standard output (stdout) and error (stderr) streams of the task |
| are read by the TaskTracker and logged to |
| <code>${HADOOP_LOG_DIR}/userlogs</code></p> |
| </section> |
| |
| <section> |
| <title>Distributing Libraries</title> |
| <p>The <a href="#DistributedCache">DistributedCache</a> can also be used |
| to distribute both jars and native libraries for use in the map |
| and/or reduce tasks. The child-jvm always has its |
| <em>current working directory</em> added to the |
| <code>java.library.path</code> and <code>LD_LIBRARY_PATH</code>. |
| And hence the cached libraries can be loaded via |
| <a href="http://java.sun.com/javase/6/docs/api/java/lang/System.html#loadLibrary(java.lang.String)"> |
| System.loadLibrary</a> or |
| <a href="http://java.sun.com/javase/6/docs/api/java/lang/System.html#load(java.lang.String)"> |
| System.load</a>. More details on how to load shared libraries through |
| distributed cache are documented at |
| <a href="native_libraries.html#Loading+native+libraries+through+DistributedCache"> |
| native_libraries.html</a></p> |
| </section> |
| </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 and logs, get the MapReduce |
| 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 MapReduce 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> Job history files are also logged to user specified directory |
| <code>hadoop.job.history.user.location</code> |
| which defaults to job output directory. The files are stored in |
| "_logs/history/" in the specified directory. Hence, by default they |
| will be in mapred.output.dir/_logs/history. User can stop |
| logging by giving the value <code>none</code> for |
| <code>hadoop.job.history.user.location</code></p> |
| |
| <p> User can view the history logs summary in specified directory |
| using the following command <br/> |
| <code>$ bin/hadoop job -history output-dir</code><br/> |
| This command will print job details, failed and killed tip |
| details. <br/> |
| More details about the job such as successful tasks and |
| task attempts made for each task can be viewed using the |
| following command <br/> |
| <code>$ bin/hadoop job -history all output-dir</code><br/></p> |
| |
| <p> User can use |
| <a href="ext:api/org/apache/hadoop/mapred/outputlogfilter">OutputLogFilter</a> |
| to filter log files from the output directory listing. </p> |
| |
| <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 Authorization</title> |
| <p>Job level authorization and queue level authorization are enabled |
| on the cluster, if the configuration |
| <code>mapred.acls.enabled</code> is set to |
| true. When enabled, access control checks are done by (a) the |
| JobTracker before allowing users to submit jobs to queues and |
| administering these jobs and (b) by the JobTracker and the TaskTracker |
| before allowing users to view job details or to modify a job using |
| MapReduce APIs, CLI or web user interfaces.</p> |
| |
| <p>A job submitter can specify access control lists for viewing or |
| modifying a job via the configuration properties |
| <code>mapreduce.job.acl-view-job</code> and |
| <code>mapreduce.job.acl-modify-job</code> respectively. By default, |
| nobody is given access in these properties.</p> |
| |
| <p>However, irrespective of the job ACLs configured, a job's owner, |
| the superuser and cluster administrators |
| (<code>mapreduce.cluster.administrators</code>) and queue |
| administrators of the queue to which the job was submitted to |
| (<code>mapred.queue.queue-name.acl-administer-jobs</code>) always |
| have access to view and modify a job.</p> |
| |
| <p> A job view ACL authorizes users against the configured |
| <code>mapreduce.job.acl-view-job</code> before returning possibly |
| sensitive information about a job, like: </p> |
| <ul> |
| <li> job level counters </li> |
| <li> task level counters </li> |
| <li> tasks's diagnostic information </li> |
| <li> task logs displayed on the TaskTracker web UI </li> |
| <li> job.xml showed by the JobTracker's web UI </li> |
| </ul> |
| <p>Other information about a job, like its status and its profile, |
| is accessible to all users, without requiring authorization.</p> |
| |
| <p> A job modification ACL authorizes users against the configured |
| <code>mapreduce.job.acl-modify-job</code> before allowing |
| modifications to jobs, like: </p> |
| <ul> |
| <li> killing a job </li> |
| <li> killing/failing a task of a job </li> |
| <li> setting the priority of a job </li> |
| </ul> |
| <p>These operations are also permitted by the queue level ACL, |
| "mapred.queue.queue-name.acl-administer-jobs", configured via |
| mapred-queue-acls.xml. The caller will be able to do the operation |
| if he/she is part of either queue admins ACL or job modification ACL. |
| </p> |
| |
| <p>The format of a job level ACL is the same as the format for a |
| queue level ACL as defined in the |
| <a href ="cluster_setup.html#Configuring+the+Hadoop+Daemons"> |
| Cluster Setup</a> documentation. |
| </p> |
| |
| </section> |
| |
| <section> |
| <title>Job Control</title> |
| |
| <p>Users may need to chain MapReduce jobs to accomplish complex |
| tasks which cannot be done via a single MapReduce 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> |
| <title>Job Credentials</title> |
| <p>In a secure cluster, the user is authenticated via Kerberos' |
| kinit command. Because of scalability concerns, we don't push |
| the client's Kerberos' tickets in MapReduce jobs. Instead, we |
| acquire delegation tokens from each HDFS NameNode that the job |
| will use and store them in the job as part of job submission. |
| The delegation tokens are automatically obtained |
| for the HDFS that holds the staging directories, where the job |
| job files are written, and any HDFS systems referenced by |
| FileInputFormats, FileOutputFormats, DistCp, and the |
| distributed cache. |
| Other applications require to set the configuration |
| "mapreduce.job.hdfs-servers" for all NameNodes that tasks might |
| need to talk during the job execution. This is a comma separated |
| list of file system names, such as "hdfs://nn1/,hdfs://nn2/". |
| These tokens are passed to the JobTracker |
| as part of the job submission as <a href="ext:api/org/apache/hadoop/ |
| security/credentials">Credentials</a>. </p> |
| |
| <p>Similar to HDFS delegation tokens, we also have MapReduce delegation tokens. The |
| MapReduce tokens are provided so that tasks can spawn jobs if they wish to. The tasks authenticate |
| to the JobTracker via the MapReduce delegation tokens. The delegation token can |
| be obtained via the API in <a href="api/org/apache/hadoop/mapred/jobclient/getdelegationtoken"> |
| JobClient.getDelegationToken</a>. The obtained token must then be pushed onto the |
| credentials that is there in the JobConf used for job submission. The API |
| <a href="ext:api/org/apache/hadoop/security/credentials/addtoken">Credentials.addToken</a> |
| can be used for this. </p> |
| |
| <p>The credentials are sent to the JobTracker as part of the job submission process. |
| The JobTracker persists the tokens and secrets in its filesystem (typically HDFS) |
| in a file within mapred.system.dir/JOBID. The TaskTracker localizes the file as part |
| job localization. Tasks see an environment variable called |
| HADOOP_TOKEN_FILE_LOCATION and the framework sets this to point to the |
| localized file. In order to launch jobs from tasks or for doing any HDFS operation, |
| tasks must set the configuration "mapreduce.job.credentials.binary" to point to |
| this token file.</p> |
| |
| <p>The HDFS delegation tokens passed to the JobTracker during job submission are |
| are cancelled by the JobTracker when the job completes. This is the default behavior |
| unless mapreduce.job.complete.cancel.delegation.tokens is set to false in the |
| JobConf. For jobs whose tasks in turn spawns jobs, this should be set to false. |
| Applications sharing JobConf objects between multiple jobs on the JobClient side |
| should look at setting mapreduce.job.complete.cancel.delegation.tokens to false. |
| This is because the Credentials object within the JobConf will then be shared. |
| All jobs will end up sharing the same tokens, and hence the tokens should not be |
| canceled when the jobs in the sequence finish.</p> |
| |
| <p>Apart from the HDFS delegation tokens, arbitrary secrets can also be |
| passed during the job submission for tasks to access other third party services. |
| The APIs |
| <a href="ext:api/org/apache/hadoop/mapred/jobconf/getcredentials"> |
| JobConf.getCredentials</a> or <a href="ext:api/org/apache/ |
| hadoop/mapreduce/jobcontext/getcredentials">JobContext.getCredentials()</a> |
| should be used to get the credentials object and then |
| <a href="ext:api/org/apache/hadoop/security/credentials/addsecretkey"> |
| Credentials.addSecretKey</a> should be used to add secrets.</p> |
| |
| <p>For applications written using the old MapReduce API, the Mapper/Reducer classes |
| need to implement <a href="api/org/apache/hadoop/mapred/jobconfigurable"> |
| JobConfigurable</a> in order to get access to the credentials in the tasks. |
| A reference to the JobConf passed in the |
| <a href="api/org/apache/hadoop/mapred/jobconfigurable/configure"> |
| JobConfigurable.configure</a> should be stored. In the new MapReduce API, |
| a similar thing can be done in the |
| <a href="api/org/apache/hadoop/mapreduce/mapper/setup">Mapper.setup</a> |
| method. |
| The api <a href="ext:api/org/apache/hadoop/mapred/jobconf/getcredentials"> |
| JobConf.getCredentials()</a> or the api <a href="ext:api/org/apache/ |
| hadoop/mapreduce/jobcontext/getcredentials">JobContext.getCredentials()</a> |
| should be used to get the credentials reference (depending |
| on whether the new MapReduce API or the old MapReduce API is used). |
| Tasks can access the secrets using the APIs in <a href="ext:api/ |
| org/apache/hadoop/security/credentials">Credentials</a> </p> |
| |
| |
| </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 MapReduce job. |
| </p> |
| |
| <p>The MapReduce 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> |
| |
| <p>If <code>TextInputFormat</code> is the <code>InputFormat</code> for a |
| given job, the framework detects input-files with the <em>.gz</em> |
| extensions and automatically decompresses them using the |
| appropriate <code>CompressionCodec</code>. However, it must be noted that |
| compressed files with the above extensions cannot be <em>split</em> and |
| each compressed file is processed in its entirety by a single mapper.</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><key, value></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 MapReduce |
| job.</p> |
| |
| <p>The MapReduce 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>OutputCommitter</title> |
| |
| <p><a href="ext:api/org/apache/hadoop/mapred/outputcommitter"> |
| OutputCommitter</a> describes the commit of task output for a |
| MapReduce job.</p> |
| |
| <p>The MapReduce framework relies on the <code>OutputCommitter</code> |
| of the job to:</p> |
| <ol> |
| <li> |
| Setup the job during initialization. For example, create |
| the temporary output directory for the job during the |
| initialization of the job. |
| Job setup is done by a separate task when the job is |
| in PREP state and after initializing tasks. Once the setup task |
| completes, the job will be moved to RUNNING state. |
| </li> |
| <li> |
| Cleanup the job after the job completion. For example, remove the |
| temporary output directory after the job completion. |
| Job cleanup is done by a separate task at the end of the job. |
| Job is declared SUCCEDED/FAILED/KILLED after the cleanup |
| task completes. |
| </li> |
| <li> |
| Setup the task temporary output. |
| Task setup is done as part of the same task, during task initialization. |
| </li> |
| <li> |
| Check whether a task needs a commit. This is to avoid the commit |
| procedure if a task does not need commit. |
| </li> |
| <li> |
| Commit of the task output. |
| Once task is done, the task will commit it's output if required. |
| </li> |
| <li> |
| Discard the task commit. |
| If the task has been failed/killed, the output will be cleaned-up. |
| If task could not cleanup (in exception block), a separate task |
| will be launched with same attempt-id to do the cleanup. |
| </li> |
| </ol> |
| <p><code>FileOutputCommitter</code> is the default |
| <code>OutputCommitter</code>. Job setup/cleanup tasks occupy |
| map or reduce slots, whichever is free on the TaskTracker. And |
| JobCleanup task, TaskCleanup tasks and JobSetup task have the highest |
| priority, and in that order.</p> |
| </section> |
| |
| <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 attemptid, say <code>attempt_200709221812_0001_m_000000_0</code>), |
| not just per task.</p> |
| |
| <p>To avoid these issues the MapReduce framework, when the |
| <code>OutputCommitter</code> is <code>FileOutputCommitter</code>, |
| maintains a special |
| <code>${mapred.output.dir}/_temporary/_${taskid}</code> sub-directory |
| accessible via <code>${mapred.work.output.dir}</code> |
| 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}/_temporary/_${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.work.output.dir}</code> |
| during execution of a task via |
| <a href="ext:api/org/apache/hadoop/mapred/fileoutputformat/getworkoutputpath"> |
| FileOutputFormat.getWorkOutputPath()</a>, and the framework will promote them |
| similarly for succesful task-attempts, thus eliminating the need to |
| pick unique paths per task-attempt.</p> |
| |
| <p>Note: The value of <code>${mapred.work.output.dir}</code> during |
| execution of a particular task-attempt is actually |
| <code>${mapred.output.dir}/_temporary/_{$taskid}</code>, and this value is |
| set by the MapReduce framework. So, just create any side-files in the |
| path returned by |
| <a href="ext:api/org/apache/hadoop/mapred/fileoutputformat/getworkoutputpath"> |
| FileOutputFormat.getWorkOutputPath() </a>from MapReduce |
| task to take advantage of this feature.</p> |
| |
| <p>The entire discussion holds true for maps of jobs with |
| reducer=NONE (i.e. 0 reduces) since output of the map, in that case, |
| goes directly to HDFS.</p> |
| </section> |
| |
| <section> |
| <title>RecordWriter</title> |
| |
| <p><a href="ext:api/org/apache/hadoop/mapred/recordwriter"> |
| RecordWriter</a> writes the output <code><key, value></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>Submitting Jobs to Queues</title> |
| <p>Users submit jobs to Queues. Queues, as collection of jobs, |
| allow the system to provide specific functionality. For example, |
| queues use ACLs to control which users |
| who can submit jobs to them. Queues are expected to be primarily |
| used by Hadoop Schedulers. </p> |
| |
| <p>Hadoop comes configured with a single mandatory queue, called |
| 'default'. Queue names are defined in the |
| <code>mapred.queue.names</code> property of the Hadoop site |
| configuration. Some job schedulers, such as the |
| <a href="capacity_scheduler.html">Capacity Scheduler</a>, |
| support multiple queues.</p> |
| |
| <p>A job defines the queue it needs to be submitted to through the |
| <code>mapred.job.queue.name</code> property, or through the |
| <a href="ext:api/org/apache/hadoop/mapred/jobconf/setqueuename">setQueueName(String)</a> |
| API. Setting the queue name is optional. If a job is submitted |
| without an associated queue name, it is submitted to the 'default' |
| queue.</p> |
| </section> |
| <section> |
| <title>Counters</title> |
| |
| <p><code>Counters</code> represent global counters, defined either by |
| the MapReduce 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/incrcounterEnum"> |
| Reporter.incrCounter(Enum, long)</a> or |
| <a href="ext:api/org/apache/hadoop/mapred/reporter/incrcounterString"> |
| Reporter.incrCounter(String, String, 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 |
| MapReduce 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://) |
| 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> 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> |
| |
| <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, tar, tgz and tar.gz files) are |
| <em>un-archived</em> at the slave nodes. Files |
| have <em>execution permissions</em> set. </p> |
| |
| <p>The files/archives can be distributed by setting the property |
| <code>mapred.cache.{files|archives}</code>. If more than one |
| file/archive has to be distributed, they can be added as comma |
| separated paths. The properties can also be set by APIs |
| <a href="ext:api/org/apache/hadoop/filecache/distributedcache/addcachefile"> |
| DistributedCache.addCacheFile(URI,conf)</a>/ |
| <a href="ext:api/org/apache/hadoop/filecache/distributedcache/addcachearchive"> |
| DistributedCache.addCacheArchive(URI,conf)</a> and |
| <a href="ext:api/org/apache/hadoop/filecache/distributedcache/setcachefiles"> |
| DistributedCache.setCacheFiles(URIs,conf)</a>/ |
| <a href="ext:api/org/apache/hadoop/filecache/distributedcache/setcachearchives"> |
| DistributedCache.setCacheArchives(URIs,conf)</a> |
| where URI is of the form |
| <code>hdfs://host:port/absolute-path#link-name</code>. |
| In Streaming, the files can be distributed through command line |
| option <code>-cacheFile/-cacheArchive</code>.</p> |
| |
| <p>Optionally users can also direct the <code>DistributedCache</code> |
| to <em>symlink</em> the cached file(s) into the <code>current working |
| directory</code> of the task via the |
| <a href="ext:api/org/apache/hadoop/filecache/distributedcache/createsymlink"> |
| DistributedCache.createSymlink(Configuration)</a> api. Or by setting |
| the configuration property <code>mapred.create.symlink</code> |
| as <code>yes</code>. The DistributedCache will use the |
| <code>fragment</code> of the URI as the name of the symlink. |
| For example, the URI |
| <code>hdfs://namenode:port/lib.so.1#lib.so</code> |
| will have the symlink name as <code>lib.so</code> in task's cwd |
| for the file <code>lib.so.1</code> in distributed cache.</p> |
| |
| <p>The <code>DistributedCache</code> can also be used as a |
| rudimentary software distribution mechanism for use in the |
| map and/or reduce tasks. It can be used to distribute both |
| jars and native libraries. The |
| <a href="ext:api/org/apache/hadoop/filecache/distributedcache/addarchivetoclasspath"> |
| DistributedCache.addArchiveToClassPath(Path, Configuration)</a> or |
| <a href="ext:api/org/apache/hadoop/filecache/distributedcache/addfiletoclasspath"> |
| DistributedCache.addFileToClassPath(Path, Configuration)</a> api |
| can be used to cache files/jars and also add them to the |
| <em>classpath</em> of child-jvm. The same can be done by setting |
| the configuration properties |
| <code>mapred.job.classpath.{files|archives}</code>. Similarly the |
| cached files that are symlinked into the working directory of the |
| task can be used to distribute native libraries and load them.</p> |
| <section> |
| <title>Private and Public DistributedCache Files</title> |
| <p>DistributedCache files can be private or public, that |
| determines how they can be shared on the slave nodes.</p> |
| <ul> |
| <li>"Private" DistributedCache files are cached in a local |
| directory private to the user whose jobs need these |
| files. These files are shared by all |
| tasks and jobs of the specific user only and cannot be accessed by |
| jobs of other users on the slaves. A DistributedCache file becomes private by |
| virtue of its permissions on the file system where the files |
| are uploaded, typically HDFS. If the file has no world readable |
| access, or if the directory path leading to the file has no |
| world executable access for lookup, then the file becomes private. |
| </li> |
| <li>"Public" DistributedCache files are cached in a global |
| directory and the file access is setup such that they are |
| publicly visible to all users. These files can be shared by |
| tasks and jobs of all users on the slaves. |
| A DistributedCache file becomes public by virtue of its permissions |
| on the file system where the files are uploaded, typically HDFS. |
| If the file has world readable access, AND if the directory |
| path leading to the file has world executable access for lookup, |
| then the file becomes public. In other words, if the user intends |
| to make a file publicly available to all users, the file permissions |
| must be set to be world readable, and the directory permissions |
| on the path leading to the file must be world executable. |
| </li> |
| </ul> |
| </section> |
| </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 MapReduce 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 <configuration file> |
| </code> |
| <br/> |
| <code> |
| -D <property=value> |
| </code> |
| <br/> |
| <code> |
| -fs <local|namenode:port> |
| </code> |
| <br/> |
| <code> |
| -jt <local|jobtracker:port> |
| </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 MapReduce programs.</p> |
| |
| <p>To use the <code>IsolationRunner</code>, first set |
| <code>keep.failed.task.files</code> to <code>true</code> |
| (also see <code>keep.task.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 <local path>/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> |
| |
| <p>Note that currently IsolationRunner will only re-run map tasks.</p> |
| </section> |
| |
| <section> |
| <title>Profiling</title> |
| <p>Profiling is a utility to get a representative (2 or 3) sample |
| of built-in java profiler for a sample of maps and reduces. </p> |
| |
| <p>User can specify whether the system should collect profiler |
| information for some of the tasks in the job by setting the |
| configuration property <code>mapred.task.profile</code>. The |
| value can be set using the api |
| <a href="ext:api/org/apache/hadoop/mapred/jobconf/setprofileenabled"> |
| JobConf.setProfileEnabled(boolean)</a>. If the value is set |
| <code>true</code>, the task profiling is enabled. The profiler |
| information is stored in the user log directory. By default, |
| profiling is not enabled for the job. </p> |
| |
| <p>Once user configures that profiling is needed, she/he can use |
| the configuration property |
| <code>mapred.task.profile.{maps|reduces}</code> to set the ranges |
| of MapReduce tasks to profile. The value can be set using the api |
| <a href="ext:api/org/apache/hadoop/mapred/jobconf/setprofiletaskrange"> |
| JobConf.setProfileTaskRange(boolean,String)</a>. |
| By default, the specified range is <code>0-2</code>.</p> |
| |
| <p>User can also specify the profiler configuration arguments by |
| setting the configuration property |
| <code>mapred.task.profile.params</code>. The value can be specified |
| using the api |
| <a href="ext:api/org/apache/hadoop/mapred/jobconf/setprofileparams"> |
| JobConf.setProfileParams(String)</a>. If the string contains a |
| <code>%s</code>, it will be replaced with the name of the profiling |
| output file when the task runs. These parameters are passed to the |
| task child JVM on the command line. The default value for |
| the profiling parameters is |
| <code>-agentlib:hprof=cpu=samples,heap=sites,force=n,thread=y,verbose=n,file=%s</code> |
| </p> |
| </section> |
| |
| <section> |
| <title>Debugging</title> |
| <p>The MapReduce framework provides a facility to run user-provided |
| scripts for debugging. When a MapReduce task fails, a user can run |
| a debug script, to process task logs for example. The script is |
| given access to the task's stdout and stderr outputs, syslog and |
| jobconf. The output from the debug script's stdout and stderr is |
| displayed on the console diagnostics and also as part of the |
| job UI. </p> |
| |
| <p> In the following sections we discuss how to submit a debug script |
| with a job. The script file needs to be distributed and submitted to |
| the framework.</p> |
| <section> |
| <title> How to distribute the script file: </title> |
| <p> |
| The user needs to use |
| <a href="mapred_tutorial.html#DistributedCache">DistributedCache</a> |
| to <em>distribute</em> and <em>symlink</em> the script file.</p> |
| </section> |
| <section> |
| <title> How to submit the script: </title> |
| <p> A quick way to submit the debug script is to set values for the |
| properties <code>mapred.map.task.debug.script</code> and |
| <code>mapred.reduce.task.debug.script</code>, for debugging map and |
| reduce tasks respectively. These properties can also be set by using APIs |
| <a href="ext:api/org/apache/hadoop/mapred/jobconf/setmapdebugscript"> |
| JobConf.setMapDebugScript(String) </a> and |
| <a href="ext:api/org/apache/hadoop/mapred/jobconf/setreducedebugscript"> |
| JobConf.setReduceDebugScript(String) </a>. In streaming mode, a debug |
| script can be submitted with the command-line options |
| <code>-mapdebug</code> and <code>-reducedebug</code>, for debugging |
| map and reduce tasks respectively.</p> |
| |
| <p>The arguments to the script are the task's stdout, stderr, |
| syslog and jobconf files. The debug command, run on the node where |
| the MapReduce task failed, is: <br/> |
| <code> $script $stdout $stderr $syslog $jobconf </code> </p> |
| |
| <p> Pipes programs have the c++ program name as a fifth argument |
| for the command. Thus for the pipes programs the command is <br/> |
| <code>$script $stdout $stderr $syslog $jobconf $program </code> |
| </p> |
| </section> |
| |
| <section> |
| <title> Default Behavior: </title> |
| <p> For pipes, a default script is run to process core dumps under |
| gdb, prints stack trace and gives info about running threads. </p> |
| </section> |
| </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 MapReduce jobs |
| and their dependencies.</p> |
| </section> |
| |
| <section> |
| <title>Data Compression</title> |
| |
| <p>Hadoop MapReduce provides facilities for the application-writer to |
| specify compression for both intermediate map-outputs and the |
| job-outputs i.e. output of the reduces. It also comes bundled with |
| <a href="ext:api/org/apache/hadoop/io/compress/compressioncodec"> |
| CompressionCodec</a> implementation for the |
| <a href="ext:zlib">zlib</a> compression |
| algorithm. The <a href="ext:gzip">gzip</a> file format is also |
| supported.</p> |
| |
| <p>Hadoop also provides native implementations of the above compression |
| codecs for reasons of both performance (zlib) and non-availability of |
| Java libraries. More details on their usage and availability are |
| available <a href="native_libraries.html">here</a>.</p> |
| |
| <section> |
| <title>Intermediate Outputs</title> |
| |
| <p>Applications can control compression of intermediate map-outputs |
| via the |
| <a href="ext:api/org/apache/hadoop/mapred/jobconf/setcompressmapoutput"> |
| JobConf.setCompressMapOutput(boolean)</a> api and the |
| <code>CompressionCodec</code> to be used via the |
| <a href="ext:api/org/apache/hadoop/mapred/jobconf/setmapoutputcompressorclass"> |
| JobConf.setMapOutputCompressorClass(Class)</a> api.</p> |
| </section> |
| |
| <section> |
| <title>Job Outputs</title> |
| |
| <p>Applications can control compression of job-outputs via the |
| <a href="ext:api/org/apache/hadoop/mapred/fileoutputformat/setcompressoutput"> |
| FileOutputFormat.setCompressOutput(JobConf, boolean)</a> api and the |
| <code>CompressionCodec</code> to be used can be specified via the |
| <a href="ext:api/org/apache/hadoop/mapred/fileoutputformat/setoutputcompressorclass"> |
| FileOutputFormat.setOutputCompressorClass(JobConf, Class)</a> api.</p> |
| |
| <p>If the job outputs are to be stored in the |
| <a href="ext:api/org/apache/hadoop/mapred/sequencefileoutputformat"> |
| SequenceFileOutputFormat</a>, the required |
| <code>SequenceFile.CompressionType</code> (i.e. <code>RECORD</code> / |
| <code>BLOCK</code> - defaults to <code>RECORD</code>) can be |
| specified via the |
| <a href="ext:api/org/apache/hadoop/mapred/sequencefileoutputformat/setoutputcompressiontype"> |
| SequenceFileOutputFormat.setOutputCompressionType(JobConf, |
| SequenceFile.CompressionType)</a> api.</p> |
| </section> |
| </section> |
| |
| <section> |
| <title>Skipping Bad Records</title> |
| <p>Hadoop provides an option where a certain set of bad input |
| records can be skipped when processing map inputs. Applications |
| can control this feature through the |
| <a href="ext:api/org/apache/hadoop/mapred/skipbadrecords"> |
| SkipBadRecords</a> class.</p> |
| |
| <p>This feature can be used when map tasks crash deterministically |
| on certain input. This usually happens due to bugs in the |
| map function. Usually, the user would have to fix these bugs. |
| This is, however, not possible sometimes. The bug may be in third |
| party libraries, for example, for which the source code is not |
| available. In such cases, the task never completes successfully even |
| after multiple attempts, and the job fails. With this feature, only |
| a small portion of data surrounding the |
| bad records is lost, which may be acceptable for some applications |
| (those performing statistical analysis on very large data, for |
| example). </p> |
| |
| <p>By default this feature is disabled. For enabling it, |
| refer to <a href="ext:api/org/apache/hadoop/mapred/skipbadrecords/setmappermaxskiprecords"> |
| SkipBadRecords.setMapperMaxSkipRecords(Configuration, long)</a> and |
| <a href="ext:api/org/apache/hadoop/mapred/skipbadrecords/setreducermaxskipgroups"> |
| SkipBadRecords.setReducerMaxSkipGroups(Configuration, long)</a>. |
| </p> |
| |
| <p>With this feature enabled, the framework gets into 'skipping |
| mode' after a certain number of map failures. For more details, |
| see <a href="ext:api/org/apache/hadoop/mapred/skipbadrecords/setattemptsTostartskipping"> |
| SkipBadRecords.setAttemptsToStartSkipping(Configuration, int)</a>. |
| In 'skipping mode', map tasks maintain the range of records being |
| processed. To do this, the framework relies on the processed record |
| counter. See <a href="ext:api/org/apache/hadoop/mapred/skipbadrecords/counter_map_processed_records"> |
| SkipBadRecords.COUNTER_MAP_PROCESSED_RECORDS</a> and |
| <a href="ext:api/org/apache/hadoop/mapred/skipbadrecords/counter_reduce_processed_groups"> |
| SkipBadRecords.COUNTER_REDUCE_PROCESSED_GROUPS</a>. |
| This counter enables the framework to know how many records have |
| been processed successfully, and hence, what record range caused |
| a task to crash. On further attempts, this range of records is |
| skipped.</p> |
| |
| <p>The number of records skipped depends on how frequently the |
| processed record counter is incremented by the application. |
| It is recommended that this counter be incremented after every |
| record is processed. This may not be possible in some applications |
| that typically batch their processing. In such cases, the framework |
| may skip additional records surrounding the bad record. Users can |
| control the number of skipped records through |
| <a href="ext:api/org/apache/hadoop/mapred/skipbadrecords/setmappermaxskiprecords"> |
| SkipBadRecords.setMapperMaxSkipRecords(Configuration, long)</a> and |
| <a href="ext:api/org/apache/hadoop/mapred/skipbadrecords/setreducermaxskipgroups"> |
| SkipBadRecords.setReducerMaxSkipGroups(Configuration, long)</a>. |
| The framework tries to narrow the range of skipped records using a |
| binary search-like approach. The skipped range is divided into two |
| halves and only one half gets executed. On subsequent |
| failures, the framework figures out which half contains |
| bad records. A task will be re-executed till the |
| acceptable skipped value is met or all task attempts are exhausted. |
| To increase the number of task attempts, use |
| <a href="ext:api/org/apache/hadoop/mapred/jobconf/setmaxmapattempts"> |
| JobConf.setMaxMapAttempts(int)</a> and |
| <a href="ext:api/org/apache/hadoop/mapred/jobconf/setmaxreduceattempts"> |
| JobConf.setMaxReduceAttempts(int)</a>. |
| </p> |
| |
| <p>Skipped records are written to HDFS in the sequence file |
| format, for later analysis. The location can be changed through |
| <a href="ext:api/org/apache/hadoop/mapred/skipbadrecords/setskipoutputpath"> |
| SkipBadRecords.setSkipOutputPath(JobConf, Path)</a>. |
| </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 MapReduce framework we discussed so far.</p> |
| |
| <p>This needs the HDFS to be up and running, especially for the |
| <code>DistributedCache</code>-related features. Hence it only works with a |
| <a href="single_node_setup.html#SingleNodeSetup">pseudo-distributed</a> or |
| <a href="single_node_setup.html#Fully-Distributed+Operation">fully-distributed</a> |
| Hadoop installation.</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> |
| |
| <code> |
| public static class Map extends MapReduceBase |
| implements Mapper<LongWritable, Text, Text, IntWritable> { |
| </code> |
| </td> |
| </tr> |
| <tr> |
| <td>16.</td> |
| <td></td> |
| </tr> |
| <tr> |
| <td>17.</td> |
| <td> |
| |
| <code> |
| static enum Counters { INPUT_WORDS } |
| </code> |
| </td> |
| </tr> |
| <tr> |
| <td>18.</td> |
| <td></td> |
| </tr> |
| <tr> |
| <td>19.</td> |
| <td> |
| |
| <code> |
| private final static IntWritable one = new IntWritable(1); |
| </code> |
| </td> |
| </tr> |
| <tr> |
| <td>20.</td> |
| <td> |
| |
| <code>private Text word = new Text();</code> |
| </td> |
| </tr> |
| <tr> |
| <td>21.</td> |
| <td></td> |
| </tr> |
| <tr> |
| <td>22.</td> |
| <td> |
| |
| <code>private boolean caseSensitive = true;</code> |
| </td> |
| </tr> |
| <tr> |
| <td>23.</td> |
| <td> |
| |
| <code>private Set<String> patternsToSkip = new HashSet<String>();</code> |
| </td> |
| </tr> |
| <tr> |
| <td>24.</td> |
| <td></td> |
| </tr> |
| <tr> |
| <td>25.</td> |
| <td> |
| |
| <code>private long numRecords = 0;</code> |
| </td> |
| </tr> |
| <tr> |
| <td>26.</td> |
| <td> |
| |
| <code>private String inputFile;</code> |
| </td> |
| </tr> |
| <tr> |
| <td>27.</td> |
| <td></td> |
| </tr> |
| <tr> |
| <td>28.</td> |
| <td> |
| |
| <code>public void configure(JobConf job) {</code> |
| </td> |
| </tr> |
| <tr> |
| <td>29.</td> |
| <td> |
| |
| <code> |
| caseSensitive = job.getBoolean("wordcount.case.sensitive", true); |
| </code> |
| </td> |
| </tr> |
| <tr> |
| <td>30.</td> |
| <td> |
| |
| <code>inputFile = job.get("map.input.file");</code> |
| </td> |
| </tr> |
| <tr> |
| <td>31.</td> |
| <td></td> |
| </tr> |
| <tr> |
| <td>32.</td> |
| <td> |
| |
| <code>if (job.getBoolean("wordcount.skip.patterns", false)) {</code> |
| </td> |
| </tr> |
| <tr> |
| <td>33.</td> |
| <td> |
| |
| <code>Path[] patternsFiles = new Path[0];</code> |
| </td> |
| </tr> |
| <tr> |
| <td>34.</td> |
| <td> |
| |
| <code>try {</code> |
| </td> |
| </tr> |
| <tr> |
| <td>35.</td> |
| <td> |
| |
| <code> |
| patternsFiles = DistributedCache.getLocalCacheFiles(job); |
| </code> |
| </td> |
| </tr> |
| <tr> |
| <td>36.</td> |
| <td> |
| |
| <code>} catch (IOException ioe) {</code> |
| </td> |
| </tr> |
| <tr> |
| <td>37.</td> |
| <td> |
| |
| <code> |
| System.err.println("Caught exception while getting cached files: " |
| + StringUtils.stringifyException(ioe)); |
| </code> |
| </td> |
| </tr> |
| <tr> |
| <td>38.</td> |
| <td> |
| |
| <code>}</code> |
| </td> |
| </tr> |
| <tr> |
| <td>39.</td> |
| <td> |
| |
| <code>for (Path patternsFile : patternsFiles) {</code> |
| </td> |
| </tr> |
| <tr> |
| <td>40.</td> |
| <td> |
| |
| <code>parseSkipFile(patternsFile);</code> |
| </td> |
| </tr> |
| <tr> |
| <td>41.</td> |
| <td> |
| |
| <code>}</code> |
| </td> |
| </tr> |
| <tr> |
| <td>42.</td> |
| <td> |
| |
| <code>}</code> |
| </td> |
| </tr> |
| <tr> |
| <td>43.</td> |
| <td> |
| |
| <code>}</code> |
| </td> |
| </tr> |
| <tr> |
| <td>44.</td> |
| <td></td> |
| </tr> |
| <tr> |
| <td>45.</td> |
| <td> |
| |
| <code>private void parseSkipFile(Path patternsFile) {</code> |
| </td> |
| </tr> |
| <tr> |
| <td>46.</td> |
| <td> |
| |
| <code>try {</code> |
| </td> |
| </tr> |
| <tr> |
| <td>47.</td> |
| <td> |
| |
| <code> |
| BufferedReader fis = |
| new BufferedReader(new FileReader(patternsFile.toString())); |
| </code> |
| </td> |
| </tr> |
| <tr> |
| <td>48.</td> |
| <td> |
| |
| <code>String pattern = null;</code> |
| </td> |
| </tr> |
| <tr> |
| <td>49.</td> |
| <td> |
| |
| <code>while ((pattern = fis.readLine()) != null) {</code> |
| </td> |
| </tr> |
| <tr> |
| <td>50.</td> |
| <td> |
| |
| <code>patternsToSkip.add(pattern);</code> |
| </td> |
| </tr> |
| <tr> |
| <td>51.</td> |
| <td> |
| |
| <code>}</code> |
| </td> |
| </tr> |
| <tr> |
| <td>52.</td> |
| <td> |
| |
| <code>} catch (IOException ioe) {</code> |
| </td> |
| </tr> |
| <tr> |
| <td>53.</td> |
| <td> |
| |
| <code> |
| System.err.println("Caught exception while parsing the cached file '" + |
| patternsFile + "' : " + |
| StringUtils.stringifyException(ioe)); |
| |
| </code> |
| </td> |
| </tr> |
| <tr> |
| <td>54.</td> |
| <td> |
| |
| <code>}</code> |
| </td> |
| </tr> |
| <tr> |
| <td>55.</td> |
| <td> |
| |
| <code>}</code> |
| </td> |
| </tr> |
| <tr> |
| <td>56.</td> |
| <td></td> |
| </tr> |
| <tr> |
| <td>57.</td> |
| <td> |
| |
| <code> |
| public void map(LongWritable key, Text value, |
| OutputCollector<Text, IntWritable> output, |
| Reporter reporter) throws IOException { |
| </code> |
| </td> |
| </tr> |
| <tr> |
| <td>58.</td> |
| <td> |
| |
| <code> |
| String line = |
| (caseSensitive) ? value.toString() : |
| value.toString().toLowerCase(); |
| </code> |
| </td> |
| </tr> |
| <tr> |
| <td>59.</td> |
| <td></td> |
| </tr> |
| <tr> |
| <td>60.</td> |
| <td> |
| |
| <code>for (String pattern : patternsToSkip) {</code> |
| </td> |
| </tr> |
| <tr> |
| <td>61.</td> |
| <td> |
| |
| <code>line = line.replaceAll(pattern, "");</code> |
| </td> |
| </tr> |
| <tr> |
| <td>62.</td> |
| <td> |
| |
| <code>}</code> |
| </td> |
| </tr> |
| <tr> |
| <td>63.</td> |
| <td></td> |
| </tr> |
| <tr> |
| <td>64.</td> |
| <td> |
| |
| <code>StringTokenizer tokenizer = new StringTokenizer(line);</code> |
| </td> |
| </tr> |
| <tr> |
| <td>65.</td> |
| <td> |
| |
| <code>while (tokenizer.hasMoreTokens()) {</code> |
| </td> |
| </tr> |
| <tr> |
| <td>66.</td> |
| <td> |
| |
| <code>word.set(tokenizer.nextToken());</code> |
| </td> |
| </tr> |
| <tr> |
| <td>67.</td> |
| <td> |
| |
| <code>output.collect(word, one);</code> |
| </td> |
| </tr> |
| <tr> |
| <td>68.</td> |
| <td> |
| |
| <code>reporter.incrCounter(Counters.INPUT_WORDS, 1);</code> |
| </td> |
| </tr> |
| <tr> |
| <td>69.</td> |
| <td> |
| |
| <code>}</code> |
| </td> |
| </tr> |
| <tr> |
| <td>70.</td> |
| <td></td> |
| </tr> |
| <tr> |
| <td>71.</td> |
| <td> |
| |
| <code>if ((++numRecords % 100) == 0) {</code> |
| </td> |
| </tr> |
| <tr> |
| <td>72.</td> |
| <td> |
| |
| <code> |
| reporter.setStatus("Finished processing " + numRecords + |
| " records " + "from the input file: " + |
| inputFile); |
| </code> |
| </td> |
| </tr> |
| <tr> |
| <td>73.</td> |
| <td> |
| |
| <code>}</code> |
| </td> |
| </tr> |
| <tr> |
| <td>74.</td> |
| <td> |
| |
| <code>}</code> |
| </td> |
| </tr> |
| <tr> |
| <td>75.</td> |
| <td> |
| |
| <code>}</code> |
| </td> |
| </tr> |
| <tr> |
| <td>76.</td> |
| <td></td> |
| </tr> |
| <tr> |
| <td>77.</td> |
| <td> |
| |
| <code> |
| public static class Reduce extends MapReduceBase implements |
| Reducer<Text, IntWritable, Text, IntWritable> { |
| </code> |
| </td> |
| </tr> |
| <tr> |
| <td>78.</td> |
| <td> |
| |
| <code> |
| public void reduce(Text key, Iterator<IntWritable> values, |
| OutputCollector<Text, IntWritable> output, |
| Reporter reporter) throws IOException { |
| </code> |
| </td> |
| </tr> |
| <tr> |
| <td>79.</td> |
| <td> |
| |
| <code>int sum = 0;</code> |
| </td> |
| </tr> |
| <tr> |
| <td>80.</td> |
| <td> |
| |
| <code>while (values.hasNext()) {</code> |
| </td> |
| </tr> |
| <tr> |
| <td>81.</td> |
| <td> |
| |
| <code>sum += values.next().get();</code> |
| </td> |
| </tr> |
| <tr> |
| <td>82.</td> |
| <td> |
| |
| <code>}</code> |
| </td> |
| </tr> |
| <tr> |
| <td>83.</td> |
| <td> |
| |
| <code>output.collect(key, new IntWritable(sum));</code> |
| </td> |
| </tr> |
| <tr> |
| <td>84.</td> |
| <td> |
| |
| <code>}</code> |
| </td> |
| </tr> |
| <tr> |
| <td>85.</td> |
| <td> |
| |
| <code>}</code> |
| </td> |
| </tr> |
| <tr> |
| <td>86.</td> |
| <td></td> |
| </tr> |
| <tr> |
| <td>87.</td> |
| <td> |
| |
| <code>public int run(String[] args) throws Exception {</code> |
| </td> |
| </tr> |
| <tr> |
| <td>88.</td> |
| <td> |
| |
| <code> |
| JobConf conf = new JobConf(getConf(), WordCount.class); |
| </code> |
| </td> |
| </tr> |
| <tr> |
| <td>89.</td> |
| <td> |
| |
| <code>conf.setJobName("wordcount");</code> |
| </td> |
| </tr> |
| <tr> |
| <td>90.</td> |
| <td></td> |
| </tr> |
| <tr> |
| <td>91.</td> |
| <td> |
| |
| <code>conf.setOutputKeyClass(Text.class);</code> |
| </td> |
| </tr> |
| <tr> |
| <td>92.</td> |
| <td> |
| |
| <code>conf.setOutputValueClass(IntWritable.class);</code> |
| </td> |
| </tr> |
| <tr> |
| <td>93.</td> |
| <td></td> |
| </tr> |
| <tr> |
| <td>94.</td> |
| <td> |
| |
| <code>conf.setMapperClass(Map.class);</code> |
| </td> |
| </tr> |
| <tr> |
| <td>95.</td> |
| <td> |
| |
| <code>conf.setCombinerClass(Reduce.class);</code> |
| </td> |
| </tr> |
| <tr> |
| <td>96.</td> |
| <td> |
| |
| <code>conf.setReducerClass(Reduce.class);</code> |
| </td> |
| </tr> |
| <tr> |
| <td>97.</td> |
| <td></td> |
| </tr> |
| <tr> |
| <td>98.</td> |
| <td> |
| |
| <code>conf.setInputFormat(TextInputFormat.class);</code> |
| </td> |
| </tr> |
| <tr> |
| <td>99.</td> |
| <td> |
| |
| <code>conf.setOutputFormat(TextOutputFormat.class);</code> |
| </td> |
| </tr> |
| <tr> |
| <td>100.</td> |
| <td></td> |
| </tr> |
| <tr> |
| <td>101.</td> |
| <td> |
| |
| <code> |
| List<String> other_args = new ArrayList<String>(); |
| </code> |
| </td> |
| </tr> |
| <tr> |
| <td>102.</td> |
| <td> |
| |
| <code>for (int i=0; i < args.length; ++i) {</code> |
| </td> |
| </tr> |
| <tr> |
| <td>103.</td> |
| <td> |
| |
| <code>if ("-skip".equals(args[i])) {</code> |
| </td> |
| </tr> |
| <tr> |
| <td>104.</td> |
| <td> |
| |
| <code> |
| DistributedCache.addCacheFile(new Path(args[++i]).toUri(), conf); |
| </code> |
| </td> |
| </tr> |
| <tr> |
| <td>105.</td> |
| <td> |
| |
| <code> |
| conf.setBoolean("wordcount.skip.patterns", true); |
| </code> |
| </td> |
| </tr> |
| <tr> |
| <td>106.</td> |
| <td> |
| |
| <code>} else {</code> |
| </td> |
| </tr> |
| <tr> |
| <td>107.</td> |
| <td> |
| |
| <code>other_args.add(args[i]);</code> |
| </td> |
| </tr> |
| <tr> |
| <td>108.</td> |
| <td> |
| |
| <code>}</code> |
| </td> |
| </tr> |
| <tr> |
| <td>109.</td> |
| <td> |
| |
| <code>}</code> |
| </td> |
| </tr> |
| <tr> |
| <td>110.</td> |
| <td></td> |
| </tr> |
| <tr> |
| <td>111.</td> |
| <td> |
| |
| <code>FileInputFormat.setInputPaths(conf, new Path(other_args.get(0)));</code> |
| </td> |
| </tr> |
| <tr> |
| <td>112.</td> |
| <td> |
| |
| <code>FileOutputFormat.setOutputPath(conf, new Path(other_args.get(1)));</code> |
| </td> |
| </tr> |
| <tr> |
| <td>113.</td> |
| <td></td> |
| </tr> |
| <tr> |
| <td>114.</td> |
| <td> |
| |
| <code>JobClient.runJob(conf);</code> |
| </td> |
| </tr> |
| <tr> |
| <td>115.</td> |
| <td> |
| |
| <code>return 0;</code> |
| </td> |
| </tr> |
| <tr> |
| <td>116.</td> |
| <td> |
| |
| <code>}</code> |
| </td> |
| </tr> |
| <tr> |
| <td>117.</td> |
| <td></td> |
| </tr> |
| <tr> |
| <td>118.</td> |
| <td> |
| |
| <code> |
| public static void main(String[] args) throws Exception { |
| </code> |
| </td> |
| </tr> |
| <tr> |
| <td>119.</td> |
| <td> |
| |
| <code> |
| int res = ToolRunner.run(new Configuration(), new WordCount(), |
| args); |
| </code> |
| </td> |
| </tr> |
| <tr> |
| <td>120.</td> |
| <td> |
| |
| <code>System.exit(res);</code> |
| </td> |
| </tr> |
| <tr> |
| <td>121.</td> |
| <td> |
| |
| <code>}</code> |
| </td> |
| </tr> |
| <tr> |
| <td>122.</td> |
| <td> |
| <code>}</code> |
| </td> |
| </tr> |
| <tr> |
| <td>123.</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 to 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>Hello 2</code><br/> |
| <code>World! 1</code><br/> |
| <code>World, 1</code><br/> |
| <code>hadoop. 1</code><br/> |
| <code>to 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>to</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 1</code><br/> |
| <code>Hello 2</code><br/> |
| <code>World 2</code><br/> |
| <code>hadoop 1</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>Highlights</title> |
| |
| <p>The second version of <code>WordCount</code> improves upon the |
| previous one by using some features offered by the MapReduce 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-43). |
| </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 104). |
| </li> |
| <li> |
| Demonstrates the utility of the <code>Tool</code> interface and the |
| <code>GenericOptionsParser</code> to handle generic Hadoop |
| command-line options (lines 87-116, 119). |
| </li> |
| <li> |
| Demonstrates how applications can use <code>Counters</code> (line 68) |
| 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 72). |
| </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> |