123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913 |
- <?xml version="1.0"?>
- <!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements. See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You 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>Cluster Setup</title>
- </header>
-
- <body>
-
- <section>
- <title>Purpose</title>
-
- <p>This document describes how to install, configure and manage non-trivial
- Hadoop clusters ranging from a few nodes to extremely large clusters with
- thousands of nodes.</p>
- <p>
- To play with Hadoop, you may first want to install Hadoop on a single machine (see <a href="quickstart.html"> Hadoop Quick Start</a>).
- </p>
- </section>
-
- <section>
- <title>Pre-requisites</title>
-
- <ol>
- <li>
- Make sure all <a href="quickstart.html#PreReqs">requisite</a> software
- is installed on all nodes in your cluster.
- </li>
- <li>
- <a href="quickstart.html#Download">Get</a> the Hadoop software.
- </li>
- </ol>
- </section>
-
- <section>
- <title>Installation</title>
-
- <p>Installing a Hadoop cluster typically involves unpacking the software
- on all the machines in the cluster.</p>
-
- <p>Typically one machine in the cluster is designated as the
- <code>NameNode</code> and another machine the as <code>JobTracker</code>,
- exclusively. These are the <em>masters</em>. The rest of the machines in
- the cluster act as both <code>DataNode</code> <em>and</em>
- <code>TaskTracker</code>. These are the <em>slaves</em>.</p>
-
- <p>The root of the distribution is referred to as
- <code>HADOOP_HOME</code>. All machines in the cluster usually have the same
- <code>HADOOP_HOME</code> path.</p>
- </section>
-
- <section>
- <title>Configuration</title>
-
- <p>The following sections describe how to configure a Hadoop cluster.</p>
-
- <section>
- <title>Configuration Files</title>
-
- <p>Hadoop configuration is driven by two types of important
- configuration files:</p>
- <ol>
- <li>
- Read-only default configuration -
- <a href="ext:core-default">src/core/core-default.xml</a>,
- <a href="ext:hdfs-default">src/hdfs/hdfs-default.xml</a> and
- <a href="ext:mapred-default">src/mapred/mapred-default.xml</a>.
- </li>
- <li>
- Site-specific configuration -
- <em>conf/core-site.xml</em>,
- <em>conf/hdfs-site.xml</em> and
- <em>conf/mapred-site.xml</em>.
- </li>
- </ol>
-
- <p>To learn more about how the Hadoop framework is controlled by these
- configuration files, look
- <a href="ext:api/org/apache/hadoop/conf/configuration">here</a>.</p>
-
- <p>Additionally, you can control the Hadoop scripts found in the
- <code>bin/</code> directory of the distribution, by setting site-specific
- values via the <code>conf/hadoop-env.sh</code>.</p>
- </section>
-
- <section>
- <title>Site Configuration</title>
-
- <p>To configure the Hadoop cluster you will need to configure the
- <em>environment</em> in which the Hadoop daemons execute as well as
- the <em>configuration parameters</em> for the Hadoop daemons.</p>
-
- <p>The Hadoop daemons are <code>NameNode</code>/<code>DataNode</code>
- and <code>JobTracker</code>/<code>TaskTracker</code>.</p>
-
- <section>
- <title>Configuring the Environment of the Hadoop Daemons</title>
- <p>Administrators should use the <code>conf/hadoop-env.sh</code> script
- to do site-specific customization of the Hadoop daemons' process
- environment.</p>
-
- <p>At the very least you should specify the
- <code>JAVA_HOME</code> so that it is correctly defined on each
- remote node.</p>
-
- <p>Administrators can configure individual daemons using the
- configuration options <code>HADOOP_*_OPTS</code>. Various options
- available are shown below in the table. </p>
- <table>
- <tr><th>Daemon</th><th>Configure Options</th></tr>
- <tr><td>NameNode</td><td>HADOOP_NAMENODE_OPTS</td></tr>
- <tr><td>DataNode</td><td>HADOOP_DATANODE_OPTS</td></tr>
- <tr><td>SecondaryNamenode</td>
- <td>HADOOP_SECONDARYNAMENODE_OPTS</td></tr>
- <tr><td>JobTracker</td><td>HADOOP_JOBTRACKER_OPTS</td></tr>
- <tr><td>TaskTracker</td><td>HADOOP_TASKTRACKER_OPTS</td></tr>
- </table>
-
- <p> For example, To configure Namenode to use parallelGC, the
- following statement should be added in <code>hadoop-env.sh</code> :
- <br/><code>
- export HADOOP_NAMENODE_OPTS="-XX:+UseParallelGC ${HADOOP_NAMENODE_OPTS}"
- </code><br/></p>
-
- <p>Other useful configuration parameters that you can customize
- include:</p>
- <ul>
- <li>
- <code>HADOOP_LOG_DIR</code> - The directory where the daemons'
- log files are stored. They are automatically created if they don't
- exist.
- </li>
- <li>
- <code>HADOOP_HEAPSIZE</code> - The maximum amount of heapsize
- to use, in MB e.g. <code>1000MB</code>. This is used to
- configure the heap size for the hadoop daemon. By default,
- the value is <code>1000MB</code>.
- </li>
- </ul>
- </section>
-
- <section>
- <title>Configuring the Hadoop Daemons</title>
-
- <p>This section deals with important parameters to be specified in the
- following:
- <br/>
- <code>conf/core-site.xml</code>:</p>
- <table>
- <tr>
- <th>Parameter</th>
- <th>Value</th>
- <th>Notes</th>
- </tr>
- <tr>
- <td>fs.default.name</td>
- <td>URI of <code>NameNode</code>.</td>
- <td><em>hdfs://hostname/</em></td>
- </tr>
- </table>
- <p><br/><code>conf/hdfs-site.xml</code>:</p>
-
- <table>
- <tr>
- <th>Parameter</th>
- <th>Value</th>
- <th>Notes</th>
- </tr>
- <tr>
- <td>dfs.name.dir</td>
- <td>
- Path on the local filesystem where the <code>NameNode</code>
- stores the namespace and transactions logs persistently.</td>
- <td>
- If this is a comma-delimited list of directories then the name
- table is replicated in all of the directories, for redundancy.
- </td>
- </tr>
- <tr>
- <td>dfs.data.dir</td>
- <td>
- Comma separated list of paths on the local filesystem of a
- <code>DataNode</code> where it should store its blocks.
- </td>
- <td>
- If this is a comma-delimited list of directories, then data will
- be stored in all named directories, typically on different
- devices.
- </td>
- </tr>
- </table>
- <p><br/><code>conf/mapred-site.xml</code>:</p>
- <table>
- <tr>
- <th>Parameter</th>
- <th>Value</th>
- <th>Notes</th>
- </tr>
- <tr>
- <td>mapred.job.tracker</td>
- <td>Host or IP and port of <code>JobTracker</code>.</td>
- <td><em>host:port</em> pair.</td>
- </tr>
- <tr>
- <td>mapred.system.dir</td>
- <td>
- Path on the HDFS where where the Map/Reduce framework stores
- system files e.g. <code>/hadoop/mapred/system/</code>.
- </td>
- <td>
- This is in the default filesystem (HDFS) and must be accessible
- from both the server and client machines.
- </td>
- </tr>
- <tr>
- <td>mapred.local.dir</td>
- <td>
- Comma-separated list of paths on the local filesystem where
- temporary Map/Reduce data is written.
- </td>
- <td>Multiple paths help spread disk i/o.</td>
- </tr>
- <tr>
- <td>mapred.tasktracker.{map|reduce}.tasks.maximum</td>
- <td>
- The maximum number of Map/Reduce tasks, which are run
- simultaneously on a given <code>TaskTracker</code>, individually.
- </td>
- <td>
- Defaults to 2 (2 maps and 2 reduces), but vary it depending on
- your hardware.
- </td>
- </tr>
- <tr>
- <td>dfs.hosts/dfs.hosts.exclude</td>
- <td>List of permitted/excluded DataNodes.</td>
- <td>
- If necessary, use these files to control the list of allowable
- datanodes.
- </td>
- </tr>
- <tr>
- <td>mapred.hosts/mapred.hosts.exclude</td>
- <td>List of permitted/excluded TaskTrackers.</td>
- <td>
- If necessary, use these files to control the list of allowable
- TaskTrackers.
- </td>
- </tr>
- <tr>
- <td>mapred.queue.names</td>
- <td>Comma separated list of queues to which jobs can be submitted.</td>
- <td>
- The Map/Reduce system always supports atleast one queue
- with the name as <em>default</em>. Hence, this parameter's
- value should always contain the string <em>default</em>.
- Some job schedulers supported in Hadoop, like the
- <a href="capacity_scheduler.html">Capacity
- Scheduler</a>, support multiple queues. If such a scheduler is
- being used, the list of configured queue names must be
- specified here. Once queues are defined, users can submit
- jobs to a queue using the property name
- <em>mapred.job.queue.name</em> in the job configuration.
- There could be a separate
- configuration file for configuring properties of these
- queues that is managed by the scheduler.
- Refer to the documentation of the scheduler for information on
- the same.
- </td>
- </tr>
- <tr>
- <td>mapred.acls.enabled</td>
- <td>Specifies whether ACLs are supported for controlling job
- submission and administration</td>
- <td>
- If <em>true</em>, ACLs would be checked while submitting
- and administering jobs. ACLs can be specified using the
- configuration parameters of the form
- <em>mapred.queue.queue-name.acl-name</em>, defined below.
- </td>
- </tr>
- </table>
-
- <p><br/><code> conf/mapred-queue-acls.xml</code></p>
-
- <table>
- <tr>
- <th>Parameter</th>
- <th>Value</th>
- <th>Notes</th>
- </tr>
- <tr>
- <td>mapred.queue.<em>queue-name</em>.acl-submit-job</td>
- <td>List of users and groups that can submit jobs to the
- specified <em>queue-name</em>.</td>
- <td>
- The list of users and groups are both comma separated
- list of names. The two lists are separated by a blank.
- Example: <em>user1,user2 group1,group2</em>.
- If you wish to define only a list of groups, provide
- a blank at the beginning of the value.
- </td>
- </tr>
- <tr>
- <td>mapred.queue.<em>queue-name</em>.acl-administer-job</td>
- <td>List of users and groups that can change the priority
- or kill jobs that have been submitted to the
- specified <em>queue-name</em>.</td>
- <td>
- The list of users and groups are both comma separated
- list of names. The two lists are separated by a blank.
- Example: <em>user1,user2 group1,group2</em>.
- If you wish to define only a list of groups, provide
- a blank at the beginning of the value. Note that an
- owner of a job can always change the priority or kill
- his/her own job, irrespective of the ACLs.
- </td>
- </tr>
- </table>
-
- <p>Typically all the above parameters are marked as
- <a href="ext:api/org/apache/hadoop/conf/configuration/final_parameters">
- final</a> to ensure that they cannot be overriden by user-applications.
- </p>
- <section>
- <title>Real-World Cluster Configurations</title>
-
- <p>This section lists some non-default configuration parameters which
- have been used to run the <em>sort</em> benchmark on very large
- clusters.</p>
-
- <ul>
- <li>
- <p>Some non-default configuration values used to run sort900,
- that is 9TB of data sorted on a cluster with 900 nodes:</p>
- <table>
- <tr>
- <th>Configuration File</th>
- <th>Parameter</th>
- <th>Value</th>
- <th>Notes</th>
- </tr>
- <tr>
- <td>conf/hdfs-site.xml</td>
- <td>dfs.block.size</td>
- <td>134217728</td>
- <td>HDFS blocksize of 128MB for large file-systems.</td>
- </tr>
- <tr>
- <td>conf/hdfs-site.xml</td>
- <td>dfs.namenode.handler.count</td>
- <td>40</td>
- <td>
- More NameNode server threads to handle RPCs from large
- number of DataNodes.
- </td>
- </tr>
- <tr>
- <td>conf/mapred-site.xml</td>
- <td>mapred.reduce.parallel.copies</td>
- <td>20</td>
- <td>
- Higher number of parallel copies run by reduces to fetch
- outputs from very large number of maps.
- </td>
- </tr>
- <tr>
- <td>conf/mapred-site.xml</td>
- <td>mapred.child.java.opts</td>
- <td>-Xmx512M</td>
- <td>
- Larger heap-size for child jvms of maps/reduces.
- </td>
- </tr>
- <tr>
- <td>conf/core-site.xml</td>
- <td>fs.inmemory.size.mb</td>
- <td>200</td>
- <td>
- Larger amount of memory allocated for the in-memory
- file-system used to merge map-outputs at the reduces.
- </td>
- </tr>
- <tr>
- <td>conf/core-site.xml</td>
- <td>io.sort.factor</td>
- <td>100</td>
- <td>More streams merged at once while sorting files.</td>
- </tr>
- <tr>
- <td>conf/core-site.xml</td>
- <td>io.sort.mb</td>
- <td>200</td>
- <td>Higher memory-limit while sorting data.</td>
- </tr>
- <tr>
- <td>conf/core-site.xml</td>
- <td>io.file.buffer.size</td>
- <td>131072</td>
- <td>Size of read/write buffer used in SequenceFiles.</td>
- </tr>
- </table>
- </li>
- <li>
- <p>Updates to some configuration values to run sort1400 and
- sort2000, that is 14TB of data sorted on 1400 nodes and 20TB of
- data sorted on 2000 nodes:</p>
- <table>
- <tr>
- <th>Configuration File</th>
- <th>Parameter</th>
- <th>Value</th>
- <th>Notes</th>
- </tr>
- <tr>
- <td>conf/mapred-site.xml</td>
- <td>mapred.job.tracker.handler.count</td>
- <td>60</td>
- <td>
- More JobTracker server threads to handle RPCs from large
- number of TaskTrackers.
- </td>
- </tr>
- <tr>
- <td>conf/mapred-site.xml</td>
- <td>mapred.reduce.parallel.copies</td>
- <td>50</td>
- <td></td>
- </tr>
- <tr>
- <td>conf/mapred-site.xml</td>
- <td>tasktracker.http.threads</td>
- <td>50</td>
- <td>
- More worker threads for the TaskTracker's http server. The
- http server is used by reduces to fetch intermediate
- map-outputs.
- </td>
- </tr>
- <tr>
- <td>conf/mapred-site.xml</td>
- <td>mapred.child.java.opts</td>
- <td>-Xmx1024M</td>
- <td>Larger heap-size for child jvms of maps/reduces.</td>
- </tr>
- </table>
- </li>
- </ul>
- </section>
-
- <section>
- <title>Task Controllers</title>
- <p>Task controllers are classes in the Hadoop Map/Reduce
- framework that define how user's map and reduce tasks
- are launched and controlled. They can
- be used in clusters that require some customization in
- the process of launching or controlling the user tasks.
- For example, in some
- clusters, there may be a requirement to run tasks as
- the user who submitted the job, instead of as the task
- tracker user, which is how tasks are launched by default.
- This section describes how to configure and use
- task controllers.</p>
- <p>The following task controllers are the available in
- Hadoop.
- </p>
- <table>
- <tr><th>Name</th><th>Class Name</th><th>Description</th></tr>
- <tr>
- <td>DefaultTaskController</td>
- <td>org.apache.hadoop.mapred.DefaultTaskController</td>
- <td> The default task controller which Hadoop uses to manage task
- execution. The tasks run as the task tracker user.</td>
- </tr>
- <tr>
- <td>LinuxTaskController</td>
- <td>org.apache.hadoop.mapred.LinuxTaskController</td>
- <td>This task controller, which is supported only on Linux,
- runs the tasks as the user who submitted the job. It requires
- these user accounts to be created on the cluster nodes
- where the tasks are launched. It
- uses a setuid executable that is included in the Hadoop
- distribution. The task tracker uses this executable to
- launch and kill tasks. The setuid executable switches to
- the user who has submitted the job and launches or kills
- the tasks. Currently, this task controller
- opens up permissions to local files and directories used
- by the tasks such as the job jar files, distributed archive
- files, intermediate files and task log files. In future,
- it is expected that stricter file permissions are used.
- </td>
- </tr>
- </table>
- <section>
- <title>Configuring Task Controllers</title>
- <p>The task controller to be used can be configured by setting the
- value of the following key in mapred-site.xml</p>
- <table>
- <tr>
- <th>Property</th><th>Value</th><th>Notes</th>
- </tr>
- <tr>
- <td>mapred.task.tracker.task-controller</td>
- <td>Fully qualified class name of the task controller class</td>
- <td>Currently there are two implementations of task controller
- in the Hadoop system, DefaultTaskController and LinuxTaskController.
- Refer to the class names mentioned above to determine the value
- to set for the class of choice.
- </td>
- </tr>
- </table>
- </section>
- <section>
- <title>Using the LinuxTaskController</title>
- <p>This section of the document describes the steps required to
- use the LinuxTaskController.</p>
-
- <p>In order to use the LinuxTaskController, a setuid executable
- should be built and deployed on the compute nodes. The
- executable is named task-controller. To build the executable,
- execute
- <em>ant task-controller -Dhadoop.conf.dir=/path/to/conf/dir.
- </em>
- The path passed in <em>-Dhadoop.conf.dir</em> should be the path
- on the cluster nodes where a configuration file for the setuid
- executable would be located. The executable would be built to
- <em>build.dir/dist.dir/bin</em> and should be installed to
- <em>$HADOOP_HOME/bin</em>.
- </p>
-
- <p>
- The executable must be deployed as a setuid executable, by changing
- the ownership to <em>root</em>, group ownership to that of tasktracker
- and giving it permissions <em>4510</em>.Please take a note that,
- group which owns task-controller should contain only tasktracker
- as its memeber and not users who submit jobs.
- </p>
-
- <p>The executable requires a configuration file called
- <em>taskcontroller.cfg</em> to be
- present in the configuration directory passed to the ant target
- mentioned above. If the binary was not built with a specific
- conf directory, the path defaults to <em>/path-to-binary/../conf</em>.
- </p>
-
- <p>The executable requires following configuration items to be
- present in the <em>taskcontroller.cfg</em> file. The items should
- be mentioned as simple <em>key=value</em> pairs.
- </p>
- <table><tr><th>Name</th><th>Description</th></tr>
- <tr>
- <td>mapred.local.dir</td>
- <td>Path to mapred local directories. Should be same as the value
- which was provided to key in mapred-site.xml. This is required to
- validate paths passed to the setuid executable in order to prevent
- arbitrary paths being passed to it.</td>
- </tr>
- </table>
- <p>
- The LinuxTaskController requires that paths leading up to
- the directories specified in
- <em>mapred.local.dir</em> and <em>hadoop.log.dir</em> to be 755
- and directories themselves having 777 permissions.
- </p>
- </section>
-
- </section>
- <section>
- <title>Monitoring Health of TaskTracker Nodes</title>
- <p>Hadoop Map/Reduce provides a mechanism by which administrators
- can configure the TaskTracker to run an administrator supplied
- script periodically to determine if a node is healthy or not.
- Administrators can determine if the node is in a healthy state
- by performing any checks of their choice in the script. If the
- script detects the node to be in an unhealthy state, it must print
- a line to standard output beginning with the string <em>ERROR</em>.
- The TaskTracker spawns the script periodically and checks its
- output. If the script's output contains the string <em>ERROR</em>,
- as described above, the node's status is reported as 'unhealthy'
- and the node is black-listed on the JobTracker. No further tasks
- will be assigned to this node. However, the
- TaskTracker continues to run the script, so that if the node
- becomes healthy again, it will be removed from the blacklisted
- nodes on the JobTracker automatically. The node's health
- along with the output of the script, if it is unhealthy, is
- available to the administrator in the JobTracker's web interface.
- The time since the node was healthy is also displayed on the
- web interface.
- </p>
-
- <section>
- <title>Configuring the Node Health Check Script</title>
- <p>The following parameters can be used to control the node health
- monitoring script in <em>mapred-site.xml</em>.</p>
- <table>
- <tr><th>Name</th><th>Description</th></tr>
- <tr><td><code>mapred.healthChecker.script.path</code></td>
- <td>Absolute path to the script which is periodically run by the
- TaskTracker to determine if the node is
- healthy or not. The file should be executable by the TaskTracker.
- If the value of this key is empty or the file does
- not exist or is not executable, node health monitoring
- is not started.</td>
- </tr>
- <tr>
- <td><code>mapred.healthChecker.interval</code></td>
- <td>Frequency at which the node health script is run,
- in milliseconds</td>
- </tr>
- <tr>
- <td><code>mapred.healthChecker.script.timeout</code></td>
- <td>Time after which the node health script will be killed by
- the TaskTracker if unresponsive.
- The node is marked unhealthy. if node health script times out.</td>
- </tr>
- <tr>
- <td><code>mapred.healthChecker.script.args</code></td>
- <td>Extra arguments that can be passed to the node health script
- when launched.
- These should be comma separated list of arguments. </td>
- </tr>
- </table>
- </section>
- </section>
-
- </section>
- <section>
- <title> Memory monitoring</title>
- <p>A <code>TaskTracker</code>(TT) can be configured to monitor memory
- usage of tasks it spawns, so that badly-behaved jobs do not bring
- down a machine due to excess memory consumption. With monitoring
- enabled, every task is assigned a task-limit for virtual memory (VMEM).
- In addition, every node is assigned a node-limit for VMEM usage.
- A TT ensures that a task is killed if it, and
- its descendants, use VMEM over the task's per-task limit. It also
- ensures that one or more tasks are killed if the sum total of VMEM
- usage by all tasks, and their descendents, cross the node-limit.</p>
-
- <p>Users can, optionally, specify the VMEM task-limit per job. If no
- such limit is provided, a default limit is used. A node-limit can be
- set per node.</p>
- <p>Currently the memory monitoring and management is only supported
- in Linux platform.</p>
- <p>To enable monitoring for a TT, the
- following parameters all need to be set:</p>
- <table>
- <tr><th>Name</th><th>Type</th><th>Description</th></tr>
- <tr><td>mapred.tasktracker.vmem.reserved</td><td>long</td>
- <td>A number, in bytes, that represents an offset. The total VMEM on
- the machine, minus this offset, is the VMEM node-limit for all
- tasks, and their descendants, spawned by the TT.
- </td></tr>
- <tr><td>mapred.task.default.maxvmem</td><td>long</td>
- <td>A number, in bytes, that represents the default VMEM task-limit
- associated with a task. Unless overridden by a job's setting,
- this number defines the VMEM task-limit.
- </td></tr>
- <tr><td>mapred.task.limit.maxvmem</td><td>long</td>
- <td>A number, in bytes, that represents the upper VMEM task-limit
- associated with a task. Users, when specifying a VMEM task-limit
- for their tasks, should not specify a limit which exceeds this amount.
- </td></tr>
- </table>
-
- <p>In addition, the following parameters can also be configured.</p>
- <table>
- <tr><th>Name</th><th>Type</th><th>Description</th></tr>
- <tr><td>mapred.tasktracker.taskmemorymanager.monitoring-interval</td>
- <td>long</td>
- <td>The time interval, in milliseconds, between which the TT
- checks for any memory violation. The default value is 5000 msec
- (5 seconds).
- </td></tr>
- </table>
-
- <p>Here's how the memory monitoring works for a TT.</p>
- <ol>
- <li>If one or more of the configuration parameters described
- above are missing or -1 is specified , memory monitoring is
- disabled for the TT.
- </li>
- <li>In addition, monitoring is disabled if
- <code>mapred.task.default.maxvmem</code> is greater than
- <code>mapred.task.limit.maxvmem</code>.
- </li>
- <li>If a TT receives a task whose task-limit is set by the user
- to a value larger than <code>mapred.task.limit.maxvmem</code>, it
- logs a warning but executes the task.
- </li>
- <li>Periodically, the TT checks the following:
- <ul>
- <li>If any task's current VMEM usage is greater than that task's
- VMEM task-limit, the task is killed and reason for killing
- the task is logged in task diagonistics . Such a task is considered
- failed, i.e., the killing counts towards the task's failure count.
- </li>
- <li>If the sum total of VMEM used by all tasks and descendants is
- greater than the node-limit, the TT kills enough tasks, in the
- order of least progress made, till the overall VMEM usage falls
- below the node-limt. Such killed tasks are not considered failed
- and their killing does not count towards the tasks' failure counts.
- </li>
- </ul>
- </li>
- </ol>
-
- <p>Schedulers can choose to ease the monitoring pressure on the TT by
- preventing too many tasks from running on a node and by scheduling
- tasks only if the TT has enough VMEM free. In addition, Schedulers may
- choose to consider the physical memory (RAM) available on the node
- as well. To enable Scheduler support, TTs report their memory settings
- to the JobTracker in every heartbeat. Before getting into details,
- consider the following additional memory-related parameters than can be
- configured to enable better scheduling:</p>
- <table>
- <tr><th>Name</th><th>Type</th><th>Description</th></tr>
- <tr><td>mapred.tasktracker.pmem.reserved</td><td>int</td>
- <td>A number, in bytes, that represents an offset. The total
- physical memory (RAM) on the machine, minus this offset, is the
- recommended RAM node-limit. The RAM node-limit is a hint to a
- Scheduler to scheduler only so many tasks such that the sum
- total of their RAM requirements does not exceed this limit.
- RAM usage is not monitored by a TT.
- </td></tr>
- </table>
-
- <p>A TT reports the following memory-related numbers in every
- heartbeat:</p>
- <ul>
- <li>The total VMEM available on the node.</li>
- <li>The value of <code>mapred.tasktracker.vmem.reserved</code>,
- if set.</li>
- <li>The total RAM available on the node.</li>
- <li>The value of <code>mapred.tasktracker.pmem.reserved</code>,
- if set.</li>
- </ul>
- </section>
-
- <section>
- <title>Slaves</title>
-
- <p>Typically you choose one machine in the cluster to act as the
- <code>NameNode</code> and one machine as to act as the
- <code>JobTracker</code>, exclusively. The rest of the machines act as
- both a <code>DataNode</code> and <code>TaskTracker</code> and are
- referred to as <em>slaves</em>.</p>
-
- <p>List all slave hostnames or IP addresses in your
- <code>conf/slaves</code> file, one per line.</p>
- </section>
-
- <section>
- <title>Logging</title>
-
- <p>Hadoop uses the <a href="http://logging.apache.org/log4j/">Apache
- log4j</a> via the <a href="http://commons.apache.org/logging/">Apache
- Commons Logging</a> framework for logging. Edit the
- <code>conf/log4j.properties</code> file to customize the Hadoop
- daemons' logging configuration (log-formats and so on).</p>
-
- <section>
- <title>History Logging</title>
-
- <p> The job history files are stored in central location
- <code> hadoop.job.history.location </code> which can be on DFS also,
- whose default value is <code>${HADOOP_LOG_DIR}/history</code>.
- The history web UI is accessible from job tracker web UI.</p>
-
- <p> The 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>
- </section>
- </section>
- </section>
-
- <p>Once all the necessary configuration is complete, distribute the files
- to the <code>HADOOP_CONF_DIR</code> directory on all the machines,
- typically <code>${HADOOP_HOME}/conf</code>.</p>
- </section>
- <section>
- <title>Cluster Restartability</title>
- <section>
- <title>Map/Reduce</title>
- <p>The job tracker restart can recover running jobs if
- <code>mapred.jobtracker.restart.recover</code> is set true and
- <a href="#Logging">JobHistory logging</a> is enabled. Also
- <code>mapred.jobtracker.job.history.block.size</code> value should be
- set to an optimal value to dump job history to disk as soon as
- possible, the typical value is 3145728(3MB).</p>
- </section>
- </section>
-
- <section>
- <title>Hadoop Rack Awareness</title>
- <p>The HDFS and the Map/Reduce components are rack-aware.</p>
- <p>The <code>NameNode</code> and the <code>JobTracker</code> obtains the
- <code>rack id</code> of the slaves in the cluster by invoking an API
- <a href="ext:api/org/apache/hadoop/net/dnstoswitchmapping/resolve
- ">resolve</a> in an administrator configured
- module. The API resolves the slave's DNS name (also IP address) to a
- rack id. What module to use can be configured using the configuration
- item <code>topology.node.switch.mapping.impl</code>. The default
- implementation of the same runs a script/command configured using
- <code>topology.script.file.name</code>. If topology.script.file.name is
- not set, the rack id <code>/default-rack</code> is returned for any
- passed IP address. The additional configuration in the Map/Reduce
- part is <code>mapred.cache.task.levels</code> which determines the number
- of levels (in the network topology) of caches. So, for example, if it is
- the default value of 2, two levels of caches will be constructed -
- one for hosts (host -> task mapping) and another for racks
- (rack -> task mapping).
- </p>
- </section>
-
- <section>
- <title>Hadoop Startup</title>
-
- <p>To start a Hadoop cluster you will need to start both the HDFS and
- Map/Reduce cluster.</p>
- <p>
- Format a new distributed filesystem:<br/>
- <code>$ bin/hadoop namenode -format</code>
- </p>
-
- <p>
- Start the HDFS with the following command, run on the designated
- <code>NameNode</code>:<br/>
- <code>$ bin/start-dfs.sh</code>
- </p>
- <p>The <code>bin/start-dfs.sh</code> script also consults the
- <code>${HADOOP_CONF_DIR}/slaves</code> file on the <code>NameNode</code>
- and starts the <code>DataNode</code> daemon on all the listed slaves.</p>
-
- <p>
- Start Map-Reduce with the following command, run on the designated
- <code>JobTracker</code>:<br/>
- <code>$ bin/start-mapred.sh</code>
- </p>
- <p>The <code>bin/start-mapred.sh</code> script also consults the
- <code>${HADOOP_CONF_DIR}/slaves</code> file on the <code>JobTracker</code>
- and starts the <code>TaskTracker</code> daemon on all the listed slaves.
- </p>
- </section>
-
- <section>
- <title>Hadoop Shutdown</title>
-
- <p>
- Stop HDFS with the following command, run on the designated
- <code>NameNode</code>:<br/>
- <code>$ bin/stop-dfs.sh</code>
- </p>
- <p>The <code>bin/stop-dfs.sh</code> script also consults the
- <code>${HADOOP_CONF_DIR}/slaves</code> file on the <code>NameNode</code>
- and stops the <code>DataNode</code> daemon on all the listed slaves.</p>
-
- <p>
- Stop Map/Reduce with the following command, run on the designated
- the designated <code>JobTracker</code>:<br/>
- <code>$ bin/stop-mapred.sh</code><br/>
- </p>
- <p>The <code>bin/stop-mapred.sh</code> script also consults the
- <code>${HADOOP_CONF_DIR}/slaves</code> file on the <code>JobTracker</code>
- and stops the <code>TaskTracker</code> daemon on all the listed slaves.</p>
- </section>
- </body>
-
- </document>
|