You are viewing a plain text version of this content. The canonical link for it is here.
Posted to yarn-commits@hadoop.apache.org by su...@apache.org on 2012/10/22 22:43:35 UTC

svn commit: r1401071 [2/3] - in /hadoop/common/branches/branch-trunk-win/hadoop-yarn-project: ./ hadoop-yarn/conf/ hadoop-yarn/hadoop-yarn-client/ hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/factory/providers/ hadoop-yarn/hadoop...

Modified: hadoop/common/branches/branch-trunk-win/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ClusterSetup.apt.vm
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-trunk-win/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ClusterSetup.apt.vm?rev=1401071&r1=1401070&r2=1401071&view=diff
==============================================================================
--- hadoop/common/branches/branch-trunk-win/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ClusterSetup.apt.vm (original)
+++ hadoop/common/branches/branch-trunk-win/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ClusterSetup.apt.vm Mon Oct 22 20:43:16 2012
@@ -1,1126 +1,1126 @@
-~~ 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. See accompanying LICENSE file.
-
-  ---
-  Hadoop Map Reduce Next Generation-${project.version} - Cluster Setup
-  ---
-  ---
-  ${maven.build.timestamp}
-
-Hadoop MapReduce Next Generation - Cluster Setup
-
-  \[ {{{./index.html}Go Back}} \]
-
-%{toc|section=1|fromDepth=0}
-
-* {Purpose}
-
-  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.
-
-  To play with Hadoop, you may first want to install it on a single 
-  machine (see {{{SingleCluster}Single Node Setup}}).
-  
-* {Prerequisites}
-
-  Download a stable version of Hadoop from Apache mirrors.
-  
-* {Installation}
-
-  Installing a Hadoop cluster typically involves unpacking the software on all 
-  the machines in the cluster or installing RPMs.
-
-  Typically one machine in the cluster is designated as the NameNode and 
-  another machine the as ResourceManager, exclusively. These are the masters. 
-  
-  The rest of the machines in the cluster act as both DataNode and NodeManager. 
-  These are the slaves.
-
-* {Running Hadoop in Non-Secure Mode}
-
-  The following sections describe how to configure a Hadoop cluster.
-
-  * {Configuration Files}
-  
-    Hadoop configuration is driven by two types of important configuration files:
-
-      * Read-only default configuration - <<<core-default.xml>>>, 
-        <<<hdfs-default.xml>>>, <<<yarn-default.xml>>> and 
-        <<<mapred-default.xml>>>.
-        
-      * Site-specific configuration - <<conf/core-site.xml>>, 
-        <<conf/hdfs-site.xml>>, <<conf/yarn-site.xml>> and 
-        <<conf/mapred-site.xml>>.
-
-
-    Additionally, you can control the Hadoop scripts found in the bin/ 
-    directory of the distribution, by setting site-specific values via the 
-    <<conf/hadoop-env.sh>> and <<yarn-env.sh>>.
-
-  * {Site Configuration}
-  
-  To configure the Hadoop cluster you will need to configure the 
-  <<<environment>>> in which the Hadoop daemons execute as well as the 
-  <<<configuration parameters>>> for the Hadoop daemons.
-
-  The Hadoop daemons are NameNode/DataNode and ResourceManager/NodeManager.
-
-
-    * {Configuring Environment of Hadoop Daemons}
-    
-    Administrators should use the <<conf/hadoop-env.sh>> and 
-    <<conf/yarn-env.sh>> script to do site-specific customization of the 
-    Hadoop daemons' process environment.
-
-    At the very least you should specify the <<<JAVA_HOME>>> so that it is 
-    correctly defined on each remote node.
-
-    In most cases you should also specify <<<HADOOP_PID_DIR>>> and 
-    <<<HADOOP_SECURE_DN_PID_DIR>>> to point to directories that can only be
-    written to by the users that are going to run the hadoop daemons.  
-    Otherwise there is the potential for a symlink attack.
-
-    Administrators can configure individual daemons using the configuration 
-    options shown below in the table:
-
-*--------------------------------------+--------------------------------------+
-|| Daemon                              || Environment Variable                |
-*--------------------------------------+--------------------------------------+
-| NameNode                             | HADOOP_NAMENODE_OPTS                 |
-*--------------------------------------+--------------------------------------+
-| DataNode                             | HADOOP_DATANODE_OPTS                 |
-*--------------------------------------+--------------------------------------+
-| Secondary NameNode                   | HADOOP_SECONDARYNAMENODE_OPTS        |
-*--------------------------------------+--------------------------------------+
-| ResourceManager                      | YARN_RESOURCEMANAGER_OPTS            |
-*--------------------------------------+--------------------------------------+
-| NodeManager                          | YARN_NODEMANAGER_OPTS                |
-*--------------------------------------+--------------------------------------+
-| WebAppProxy                          | YARN_PROXYSERVER_OPTS                |
-*--------------------------------------+--------------------------------------+
-| Map Reduce Job History Server        | HADOOP_JOB_HISTORYSERVER_OPTS        |
-*--------------------------------------+--------------------------------------+
-
-
-    For example, To configure Namenode to use parallelGC, the following 
-    statement should be added in hadoop-env.sh :
-     
-----
-    export HADOOP_NAMENODE_OPTS="-XX:+UseParallelGC ${HADOOP_NAMENODE_OPTS}" 
-----
-    
-    Other useful configuration parameters that you can customize include:
-
-      * <<<HADOOP_LOG_DIR>>> / <<<YARN_LOG_DIR>>> - The directory where the 
-        daemons' log files are stored. They are automatically created if they 
-        don't exist.
-        
-      * <<<HADOOP_HEAPSIZE>>> / <<<YARN_HEAPSIZE>>> - The maximum amount of 
-        heapsize to use, in MB e.g. if the varibale is set to 1000 the heap
-        will be set to 1000MB.  This is used to configure the heap 
-        size for the daemon. By default, the value is 1000.  If you want to
-        configure the values separately for each deamon you can use.
-*--------------------------------------+--------------------------------------+
-|| Daemon                              || Environment Variable                |
-*--------------------------------------+--------------------------------------+
-| ResourceManager                      | YARN_RESOURCEMANAGER_HEAPSIZE        |
-*--------------------------------------+--------------------------------------+
-| NodeManager                          | YARN_NODEMANAGER_HEAPSIZE            |
-*--------------------------------------+--------------------------------------+
-| WebAppProxy                          | YARN_PROXYSERVER_HEAPSIZE            |
-*--------------------------------------+--------------------------------------+
-| Map Reduce Job History Server        | HADOOP_JOB_HISTORYSERVER_HEAPSIZE    |
-*--------------------------------------+--------------------------------------+
- 
-    * {Configuring the Hadoop Daemons in Non-Secure Mode}
-
-      This section deals with important parameters to be specified in 
-      the given configuration files:
-       
-      * <<<conf/core-site.xml>>>
-      
-*-------------------------+-------------------------+------------------------+
-|| Parameter              || Value                  || Notes                 |
-*-------------------------+-------------------------+------------------------+
-| <<<fs.defaultFS>>>      | NameNode URI            | <hdfs://host:port/>    |
-*-------------------------+-------------------------+------------------------+
-| <<<io.file.buffer.size>>> | 131072 |  |
-| | | Size of read/write buffer used in SequenceFiles. |
-*-------------------------+-------------------------+------------------------+
-
-      * <<<conf/hdfs-site.xml>>>
-      
-        * Configurations for NameNode:
-
-*-------------------------+-------------------------+------------------------+
-|| Parameter              || Value                  || Notes                 |
-*-------------------------+-------------------------+------------------------+
-| <<<dfs.namenode.name.dir>>> | | | 
-| | Path on the local filesystem where the NameNode stores the namespace | | 
-| | and transactions logs persistently. | | 
-| | | If this is a comma-delimited list of directories then the name table is  |
-| | | replicated in all of the directories, for redundancy. |
-*-------------------------+-------------------------+------------------------+
-| <<<dfs.namenode.hosts>>> / <<<dfs.namenode.hosts.exclude>>> | | |
-| | List of permitted/excluded DataNodes. | |
-| | | If necessary, use these files to control the list of allowable |
-| | | datanodes. |
-*-------------------------+-------------------------+------------------------+
-| <<<dfs.blocksize>>> | 268435456 | |
-| | | HDFS blocksize of 256MB for large file-systems. |      
-*-------------------------+-------------------------+------------------------+
-| <<<dfs.namenode.handler.count>>> | 100 | |
-| | | More NameNode server threads to handle RPCs from large number of |
-| | | DataNodes. |
-*-------------------------+-------------------------+------------------------+
-        
-        * Configurations for DataNode:
-      
-*-------------------------+-------------------------+------------------------+
-|| Parameter              || Value                  || Notes                 |
-*-------------------------+-------------------------+------------------------+
-| <<<dfs.datanode.data.dir>>> | | |
-| | Comma separated list of paths on the local filesystem of a | | 
-| | <<<DataNode>>> where it should store its blocks. | |
-| | | If this is a comma-delimited list of directories, then data will be | 
-| | | stored in all named directories, typically on different devices. |
-*-------------------------+-------------------------+------------------------+
-      
-      * <<<conf/yarn-site.xml>>>
-
-        * Configurations for ResourceManager and NodeManager:
-
-*-------------------------+-------------------------+------------------------+
-|| Parameter              || Value                  || Notes                 |
-*-------------------------+-------------------------+------------------------+
-| <<<yarn.acl.enable>>> | | |
-| | <<<true>>> / <<<false>>> | |
-| | | Enable ACLs? Defaults to <false>. |
-*-------------------------+-------------------------+------------------------+
-| <<<yarn.admin.acl>>> | | |
-| | Admin ACL | |
-| | | ACL to set admins on the cluster. |
-| | | ACLs are of for <comma-separated-users><space><comma-separated-groups>. |
-| | | Defaults to special value of <<*>> which means <anyone>. |
-| | | Special value of just <space> means no one has access. |
-*-------------------------+-------------------------+------------------------+
-| <<<yarn.log-aggregation-enable>>> | | |
-| | <false> | |
-| | | Configuration to enable or disable log aggregation |
-*-------------------------+-------------------------+------------------------+
-
-
-        * Configurations for ResourceManager:
-
-*-------------------------+-------------------------+------------------------+
-|| Parameter              || Value                  || Notes                 |
-*-------------------------+-------------------------+------------------------+
-| <<<yarn.resourcemanager.address>>> | | | 
-| | <<<ResourceManager>>> host:port for clients to submit jobs. | |
-| | | <host:port> |
-*-------------------------+-------------------------+------------------------+
-| <<<yarn.resourcemanager.scheduler.address>>> | | | 
-| | <<<ResourceManager>>> host:port for ApplicationMasters to talk to | |
-| | Scheduler to obtain resources. | |
-| | | <host:port> |
-*-------------------------+-------------------------+------------------------+
-| <<<yarn.resourcemanager.resource-tracker.address>>> | | | 
-| | <<<ResourceManager>>> host:port for NodeManagers. | |
-| | | <host:port> |
-*-------------------------+-------------------------+------------------------+
-| <<<yarn.resourcemanager.admin.address>>> | | | 
-| | <<<ResourceManager>>> host:port for administrative commands. | |
-| | | <host:port> |
-*-------------------------+-------------------------+------------------------+
-| <<<yarn.resourcemanager.webapp.address>>> | | | 
-| | <<<ResourceManager>>> web-ui host:port. | |
-| | | <host:port> |
-*-------------------------+-------------------------+------------------------+
-| <<<yarn.resourcemanager.scheduler.class>>> | | |
-| | <<<ResourceManager>>> Scheduler class. | |
-| | | <<<CapacityScheduler>>> (recommended) or <<<FifoScheduler>>> |
-*-------------------------+-------------------------+------------------------+
-| <<<yarn.scheduler.minimum-allocation-mb>>> | | |
-| | Minimum limit of memory to allocate to each container request at the <<<Resource Manager>>>. | |
-| | | In MBs |
-*-------------------------+-------------------------+------------------------+
-| <<<yarn.scheduler.maximum-allocation-mb>>> | | |
-| | Maximum limit of memory to allocate to each container request at the <<<Resource Manager>>>. | |
-| | | In MBs |
-*-------------------------+-------------------------+------------------------+
-| <<<yarn.resourcemanager.nodes.include-path>>> / | | | 
-| <<<yarn.resourcemanager.nodes.exclude-path>>> | | |  
-| | List of permitted/excluded NodeManagers. | |
-| | | If necessary, use these files to control the list of allowable | 
-| | | NodeManagers. |
-*-------------------------+-------------------------+------------------------+
- 
-        * Configurations for NodeManager:
-
-*-------------------------+-------------------------+------------------------+
-|| Parameter              || Value                  || Notes                 |
-*-------------------------+-------------------------+------------------------+
-| <<<yarn.nodemanager.resource.memory-mb>>> | | |
-| | Resource i.e. available physical memory, in MB, for given <<<NodeManager>>> | |
-| | | Defines total available resources on the <<<NodeManager>>> to be made |
-| | | available to running containers |
-*-------------------------+-------------------------+------------------------+
-| <<<yarn.nodemanager.vmem-pmem-ratio>>> | | |
-| | Maximum ratio by which virtual memory usage of tasks may exceed |
-| | physical memory | |
-| | | The virtual memory usage of each task may exceed its physical memory |
-| | | limit by this ratio. The total amount of virtual memory used by tasks |
-| | | on the NodeManager may exceed its physical memory usage by this ratio. |
-*-------------------------+-------------------------+------------------------+
-| <<<yarn.nodemanager.local-dirs>>> | | |
-| | Comma-separated list of paths on the local filesystem where | |
-| | intermediate data is written. ||
-| | | Multiple paths help spread disk i/o. |
-*-------------------------+-------------------------+------------------------+
-| <<<yarn.nodemanager.log-dirs>>> | | |
-| | Comma-separated list of paths on the local filesystem where logs  | |
-| | are written. | |
-| | | Multiple paths help spread disk i/o. |
-*-------------------------+-------------------------+------------------------+
-| <<<yarn.nodemanager.log.retain-seconds>>> | | |
-| | <10800> | |
-| | | Default time (in seconds) to retain log files on the NodeManager |
-| | | Only applicable if log-aggregation is disabled. |
-*-------------------------+-------------------------+------------------------+
-| <<<yarn.nodemanager.remote-app-log-dir>>> | | |
-| | </logs> | |
-| | | HDFS directory where the application logs are moved on application |
-| | | completion. Need to set appropriate permissions. |
-| | | Only applicable if log-aggregation is enabled. |
-*-------------------------+-------------------------+------------------------+
-| <<<yarn.nodemanager.remote-app-log-dir-suffix>>> | | |
-| | <logs> | |
-| | | Suffix appended to the remote log dir. Logs will be aggregated to  |
-| | | $\{yarn.nodemanager.remote-app-log-dir\}/$\{user\}/$\{thisParam\} |
-| | | Only applicable if log-aggregation is enabled. |
-*-------------------------+-------------------------+------------------------+
-| <<<yarn.nodemanager.aux-services>>> | | |
-| | mapreduce.shuffle  | |
-| | | Shuffle service that needs to be set for Map Reduce applications. |
-*-------------------------+-------------------------+------------------------+
-
-        * Configurations for History Server (Needs to be moved elsewhere):
-
-*-------------------------+-------------------------+------------------------+
-|| Parameter              || Value                  || Notes                 |
-*-------------------------+-------------------------+------------------------+
-| <<<yarn.log-aggregation.retain-seconds>>> | | |
-| | <-1> | |
-| | | How long to keep aggregation logs before deleting them. -1 disables. |
-| | | Be careful, set this too small and you will spam the name node. |
-*-------------------------+-------------------------+------------------------+
-
-
-
-      * <<<conf/mapred-site.xml>>>
-
-        * Configurations for MapReduce Applications:
-
-*-------------------------+-------------------------+------------------------+
-|| Parameter              || Value                  || Notes                 |
-*-------------------------+-------------------------+------------------------+
-| <<<mapreduce.framework.name>>> | | |
-| | yarn | |
-| | | Execution framework set to Hadoop YARN. |
-*-------------------------+-------------------------+------------------------+
-| <<<mapreduce.map.memory.mb>>> | 1536 | |
-| | | Larger resource limit for maps. |
-*-------------------------+-------------------------+------------------------+
-| <<<mapreduce.map.java.opts>>> | -Xmx1024M | |
-| | | Larger heap-size for child jvms of maps. |
-*-------------------------+-------------------------+------------------------+
-| <<<mapreduce.reduce.memory.mb>>> | 3072 | |
-| | | Larger resource limit for reduces. |
-*-------------------------+-------------------------+------------------------+
-| <<<mapreduce.reduce.java.opts>>> | -Xmx2560M | |
-| | | Larger heap-size for child jvms of reduces. |
-*-------------------------+-------------------------+------------------------+
-| <<<mapreduce.task.io.sort.mb>>> | 512 | |
-| | | Higher memory-limit while sorting data for efficiency. |
-*-------------------------+-------------------------+------------------------+
-| <<<mapreduce.task.io.sort.factor>>> | 100 | |
-| | | More streams merged at once while sorting files. |
-*-------------------------+-------------------------+------------------------+
-| <<<mapreduce.reduce.shuffle.parallelcopies>>> | 50 | |
-| | | Higher number of parallel copies run by reduces to fetch outputs |
-| | | from very large number of maps. |
-*-------------------------+-------------------------+------------------------+
-
-        * Configurations for MapReduce JobHistory Server:
-
-*-------------------------+-------------------------+------------------------+
-|| Parameter              || Value                  || Notes                 |
-*-------------------------+-------------------------+------------------------+
-| <<<mapreduce.jobhistory.address>>> | | |
-| | MapReduce JobHistory Server <host:port> | Default port is 10020. |
-*-------------------------+-------------------------+------------------------+
-| <<<mapreduce.jobhistory.webapp.address>>> | | |
-| | MapReduce JobHistory Server Web UI <host:port> | Default port is 19888. |
-*-------------------------+-------------------------+------------------------+
-| <<<mapreduce.jobhistory.intermediate-done-dir>>> | /mr-history/tmp | |
-|  | | Directory where history files are written by MapReduce jobs. | 
-*-------------------------+-------------------------+------------------------+
-| <<<mapreduce.jobhistory.done-dir>>> | /mr-history/done| |
-| | | Directory where history files are managed by the MR JobHistory Server. | 
-*-------------------------+-------------------------+------------------------+
-
-      * Hadoop Rack Awareness
-      
-      The HDFS and the YARN components are rack-aware.
-
-      The NameNode and the ResourceManager obtains the rack information of the 
-      slaves in the cluster by invoking an API <resolve> in an administrator 
-      configured module. 
-      
-      The API resolves the DNS name (also IP address) to a rack id. 
-      
-      The site-specific module to use can be configured using the configuration 
-      item <<<topology.node.switch.mapping.impl>>>. The default implementation 
-      of the same runs a script/command configured using 
-      <<<topology.script.file.name>>>. If <<<topology.script.file.name>>> is 
-      not set, the rack id </default-rack> is returned for any passed IP address. 
-
-      * Monitoring Health of NodeManagers
-      
-      Hadoop provides a mechanism by which administrators can configure the 
-      NodeManager 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 ERROR. The NodeManager spawns 
-      the script periodically and checks its output. If the script's output 
-      contains the string ERROR, as described above, the node's status is 
-      reported as <<<unhealthy>>> and the node is black-listed by the 
-      ResourceManager. No further tasks will be assigned to this node. 
-      However, the NodeManager continues to run the script, so that if the 
-      node becomes healthy again, it will be removed from the blacklisted nodes
-      on the ResourceManager automatically. The node's health along with the 
-      output of the script, if it is unhealthy, is available to the 
-      administrator in the ResourceManager web interface. The time since the 
-      node was healthy is also displayed on the web interface.
-
-      The following parameters can be used to control the node health 
-      monitoring script in <<<conf/yarn-site.xml>>>.
-
-*-------------------------+-------------------------+------------------------+
-|| Parameter              || Value                  || Notes                 |
-*-------------------------+-------------------------+------------------------+
-| <<<yarn.nodemanager.health-checker.script.path>>> | | |
-| | Node health script  | |
-| | | Script to check for node's health status. |
-*-------------------------+-------------------------+------------------------+
-| <<<yarn.nodemanager.health-checker.script.opts>>> | | |
-| | Node health script options  | |
-| | | Options for script to check for node's health status. |
-*-------------------------+-------------------------+------------------------+
-| <<<yarn.nodemanager.health-checker.script.interval-ms>>> | | |
-| | Node health script interval  | |
-| | | Time interval for running health script. |
-*-------------------------+-------------------------+------------------------+
-| <<<yarn.nodemanager.health-checker.script.timeout-ms>>> | | |
-| | Node health script timeout interval  | |
-| | | Timeout for health script execution. |
-*-------------------------+-------------------------+------------------------+
-
-    The health checker script is not supposed to give ERROR if only some of the
-    local disks become bad. NodeManager has the ability to periodically check
-    the health of the local disks (specifically checks nodemanager-local-dirs
-    and nodemanager-log-dirs) and after reaching the threshold of number of
-    bad directories based on the value set for the config property
-    yarn.nodemanager.disk-health-checker.min-healthy-disks, the whole node is
-    marked unhealthy and this info is sent to resource manager also. The boot
-    disk is either raided or a failure in the boot disk is identified by the
-    health checker script.
-
-    * {Slaves file}
-      
-    Typically you choose one machine in the cluster to act as the NameNode and 
-    one machine as to act as the ResourceManager, exclusively. The rest of the 
-    machines act as both a DataNode and NodeManager and are referred to as 
-    <slaves>.
-
-    List all slave hostnames or IP addresses in your <<<conf/slaves>>> file, 
-    one per line.
-
-    * {Logging}
-    
-    Hadoop uses the Apache log4j via the Apache Commons Logging framework for 
-    logging. Edit the <<<conf/log4j.properties>>> file to customize the 
-    Hadoop daemons' logging configuration (log-formats and so on).
-    
-  * {Operating the Hadoop Cluster}
-
-  Once all the necessary configuration is complete, distribute the files to the 
-  <<<HADOOP_CONF_DIR>>> directory on all the machines.
-
-    * Hadoop Startup
-  
-    To start a Hadoop cluster you will need to start both the HDFS and YARN 
-    cluster.
-
-    Format a new distributed filesystem:
-  
-----
-  $ $HADOOP_PREFIX/bin/hdfs namenode -format <cluster_name>
-----
-
-    Start the HDFS with the following command, run on the designated NameNode:
-  
-----
-  $ $HADOOP_PREFIX/sbin/hadoop-daemon.sh --config $HADOOP_CONF_DIR --script hdfs start namenode
-----    	  
-
-    Run a script to start DataNodes on all slaves:
-
-----
-  $ $HADOOP_PREFIX/sbin/hadoop-daemon.sh --config $HADOOP_CONF_DIR --script hdfs start datanode
-----    	  
-  
-    Start the YARN with the following command, run on the designated 
-    ResourceManager:
-  
-----
-  $ $HADOOP_YARN_HOME/sbin/yarn-daemon.sh --config $HADOOP_CONF_DIR start resourcemanager 
-----    	  
-
-    Run a script to start NodeManagers on all slaves:
-
-----
-  $ $HADOOP_YARN_HOME/sbin/yarn-daemon.sh --config $HADOOP_CONF_DIR start nodemanager 
-----    	  
-
-    Start a standalone WebAppProxy server.  If multiple servers
-    are used with load balancing it should be run on each of them:
-
-----
-  $ $HADOOP_YARN_HOME/bin/yarn start proxyserver --config $HADOOP_CONF_DIR  
-----
-
-    Start the MapReduce JobHistory Server with the following command, run on the  
-    designated server:
-  
-----
-  $ $HADOOP_PREFIX/sbin/mr-jobhistory-daemon.sh start historyserver --config $HADOOP_CONF_DIR  
-----    	  
-
-    * Hadoop Shutdown      
-
-    Stop the NameNode with the following command, run on the designated 
-    NameNode:
-  
-----
-  $ $HADOOP_PREFIX/sbin/hadoop-daemon.sh --config $HADOOP_CONF_DIR --script hdfs stop namenode
-----    	  
-
-    Run a script to stop DataNodes on all slaves:
-
-----
-  $ $HADOOP_PREFIX/sbin/hadoop-daemon.sh --config $HADOOP_CONF_DIR --script hdfs stop datanode
-----    	  
-  
-    Stop the ResourceManager with the following command, run on the designated 
-    ResourceManager:
-  
-----
-  $ $HADOOP_YARN_HOME/sbin/yarn-daemon.sh --config $HADOOP_CONF_DIR stop resourcemanager 
-----    	  
-
-    Run a script to stop NodeManagers on all slaves:
-
-----
-  $ $HADOOP_YARN_HOME/sbin/yarn-daemon.sh --config $HADOOP_CONF_DIR stop nodemanager 
-----    	  
-
-    Stop the WebAppProxy server. If multiple servers are used with load
-    balancing it should be run on each of them:
-
-----
-  $ $HADOOP_YARN_HOME/bin/yarn stop proxyserver --config $HADOOP_CONF_DIR  
-----
-
-
-    Stop the MapReduce JobHistory Server with the following command, run on the  
-    designated server:
-  
-----
-  $ $HADOOP_PREFIX/sbin/mr-jobhistory-daemon.sh stop historyserver --config $HADOOP_CONF_DIR  
-----    	  
-
-    
-* {Running Hadoop in Secure Mode}
-
-  This section deals with important parameters to be specified in 
-  to run Hadoop in <<secure mode>> with strong, Kerberos-based
-  authentication.
-      
-  * <<<User Accounts for Hadoop Daemons>>>
-      
-  Ensure that HDFS and YARN daemons run as different Unix users, for e.g.
-  <<<hdfs>>> and <<<yarn>>>. Also, ensure that the MapReduce JobHistory
-  server runs as user <<<mapred>>>. 
-      
-  It's recommended to have them share a Unix group, for e.g. <<<hadoop>>>.
-      
-*--------------------------------------+----------------------------------------------------------------------+
-|| User:Group                          || Daemons                                                             |
-*--------------------------------------+----------------------------------------------------------------------+
-| hdfs:hadoop                          | NameNode, Secondary NameNode, Checkpoint Node, Backup Node, DataNode |
-*--------------------------------------+----------------------------------------------------------------------+
-| yarn:hadoop                          | ResourceManager, NodeManager                                         |
-*--------------------------------------+----------------------------------------------------------------------+
-| mapred:hadoop                        | MapReduce JobHistory Server                                          |
-*--------------------------------------+----------------------------------------------------------------------+
-      
-  * <<<Permissions for both HDFS and local fileSystem paths>>>
-     
-  The following table lists various paths on HDFS and local filesystems (on
-  all nodes) and recommended permissions:
-   
-*-------------------+-------------------+------------------+------------------+
-|| Filesystem       || Path             || User:Group      || Permissions     |
-*-------------------+-------------------+------------------+------------------+
-| local | <<<dfs.namenode.name.dir>>> | hdfs:hadoop | drwx------ | 
-*-------------------+-------------------+------------------+------------------+
-| local | <<<dfs.datanode.data.dir>>> | hdfs:hadoop | drwx------ |
-*-------------------+-------------------+------------------+------------------+
-| local | $HADOOP_LOG_DIR | hdfs:hadoop | drwxrwxr-x |
-*-------------------+-------------------+------------------+------------------+
-| local | $YARN_LOG_DIR | yarn:hadoop | drwxrwxr-x |
-*-------------------+-------------------+------------------+------------------+
-| local | <<<yarn.nodemanager.local-dirs>>> | yarn:hadoop | drwxr-xr-x |
-*-------------------+-------------------+------------------+------------------+
-| local | <<<yarn.nodemanager.log-dirs>>> | yarn:hadoop | drwxr-xr-x |
-*-------------------+-------------------+------------------+------------------+
-| local | container-executor | root:hadoop | --Sr-s--- |
-*-------------------+-------------------+------------------+------------------+
-| local | <<<conf/container-executor.cfg>>> | root:hadoop | r-------- |
-*-------------------+-------------------+------------------+------------------+
-| hdfs | / | hdfs:hadoop | drwxr-xr-x |
-*-------------------+-------------------+------------------+------------------+
-| hdfs | /tmp | hdfs:hadoop | drwxrwxrwxt |
-*-------------------+-------------------+------------------+------------------+
-| hdfs | /user | hdfs:hadoop | drwxr-xr-x |
-*-------------------+-------------------+------------------+------------------+
-| hdfs | <<<yarn.nodemanager.remote-app-log-dir>>> | yarn:hadoop | drwxrwxrwxt |
-*-------------------+-------------------+------------------+------------------+
-| hdfs | <<<mapreduce.jobhistory.intermediate-done-dir>>> | mapred:hadoop | |
-| | | | drwxrwxrwxt |      
-*-------------------+-------------------+------------------+------------------+
-| hdfs | <<<mapreduce.jobhistory.done-dir>>> | mapred:hadoop | |
-| | | | drwxr-x--- |      
-*-------------------+-------------------+------------------+------------------+
-
-  * Kerberos Keytab files
-  
-    * HDFS
-    
-    The NameNode keytab file, on the NameNode host, should look like the 
-    following:
-    
-----
-
-$ /usr/kerberos/bin/klist -e -k -t /etc/security/keytab/nn.service.keytab 
-Keytab name: FILE:/etc/security/keytab/nn.service.keytab
-KVNO Timestamp         Principal
-   4 07/18/11 21:08:09 nn/full.qualified.domain.name@REALM.TLD (AES-256 CTS mode with 96-bit SHA-1 HMAC) 
-   4 07/18/11 21:08:09 nn/full.qualified.domain.name@REALM.TLD (AES-128 CTS mode with 96-bit SHA-1 HMAC) 
-   4 07/18/11 21:08:09 nn/full.qualified.domain.name@REALM.TLD (ArcFour with HMAC/md5) 
-   4 07/18/11 21:08:09 host/full.qualified.domain.name@REALM.TLD (AES-256 CTS mode with 96-bit SHA-1 HMAC) 
-   4 07/18/11 21:08:09 host/full.qualified.domain.name@REALM.TLD (AES-128 CTS mode with 96-bit SHA-1 HMAC) 
-   4 07/18/11 21:08:09 host/full.qualified.domain.name@REALM.TLD (ArcFour with HMAC/md5) 
-
-----
-
-    The Secondary NameNode keytab file, on that host, should look like the 
-    following:
-    
-----
-
-$ /usr/kerberos/bin/klist -e -k -t /etc/security/keytab/sn.service.keytab 
-Keytab name: FILE:/etc/security/keytab/sn.service.keytab
-KVNO Timestamp         Principal
-   4 07/18/11 21:08:09 sn/full.qualified.domain.name@REALM.TLD (AES-256 CTS mode with 96-bit SHA-1 HMAC) 
-   4 07/18/11 21:08:09 sn/full.qualified.domain.name@REALM.TLD (AES-128 CTS mode with 96-bit SHA-1 HMAC) 
-   4 07/18/11 21:08:09 sn/full.qualified.domain.name@REALM.TLD (ArcFour with HMAC/md5) 
-   4 07/18/11 21:08:09 host/full.qualified.domain.name@REALM.TLD (AES-256 CTS mode with 96-bit SHA-1 HMAC) 
-   4 07/18/11 21:08:09 host/full.qualified.domain.name@REALM.TLD (AES-128 CTS mode with 96-bit SHA-1 HMAC) 
-   4 07/18/11 21:08:09 host/full.qualified.domain.name@REALM.TLD (ArcFour with HMAC/md5) 
-
-----
-
-    The DataNode keytab file, on each host, should look like the following:
-    
-----
-
-$ /usr/kerberos/bin/klist -e -k -t /etc/security/keytab/dn.service.keytab 
-Keytab name: FILE:/etc/security/keytab/dn.service.keytab
-KVNO Timestamp         Principal
-   4 07/18/11 21:08:09 dn/full.qualified.domain.name@REALM.TLD (AES-256 CTS mode with 96-bit SHA-1 HMAC) 
-   4 07/18/11 21:08:09 dn/full.qualified.domain.name@REALM.TLD (AES-128 CTS mode with 96-bit SHA-1 HMAC) 
-   4 07/18/11 21:08:09 dn/full.qualified.domain.name@REALM.TLD (ArcFour with HMAC/md5) 
-   4 07/18/11 21:08:09 host/full.qualified.domain.name@REALM.TLD (AES-256 CTS mode with 96-bit SHA-1 HMAC) 
-   4 07/18/11 21:08:09 host/full.qualified.domain.name@REALM.TLD (AES-128 CTS mode with 96-bit SHA-1 HMAC) 
-   4 07/18/11 21:08:09 host/full.qualified.domain.name@REALM.TLD (ArcFour with HMAC/md5) 
-
-----
-    
-    * YARN
-    
-    The ResourceManager keytab file, on the ResourceManager host, should look  
-    like the following:
-    
-----
-
-$ /usr/kerberos/bin/klist -e -k -t /etc/security/keytab/rm.service.keytab 
-Keytab name: FILE:/etc/security/keytab/rm.service.keytab
-KVNO Timestamp         Principal
-   4 07/18/11 21:08:09 rm/full.qualified.domain.name@REALM.TLD (AES-256 CTS mode with 96-bit SHA-1 HMAC) 
-   4 07/18/11 21:08:09 rm/full.qualified.domain.name@REALM.TLD (AES-128 CTS mode with 96-bit SHA-1 HMAC) 
-   4 07/18/11 21:08:09 rm/full.qualified.domain.name@REALM.TLD (ArcFour with HMAC/md5) 
-   4 07/18/11 21:08:09 host/full.qualified.domain.name@REALM.TLD (AES-256 CTS mode with 96-bit SHA-1 HMAC) 
-   4 07/18/11 21:08:09 host/full.qualified.domain.name@REALM.TLD (AES-128 CTS mode with 96-bit SHA-1 HMAC) 
-   4 07/18/11 21:08:09 host/full.qualified.domain.name@REALM.TLD (ArcFour with HMAC/md5) 
-
-----
-
-    The NodeManager keytab file, on each host, should look like the following:
-    
-----
-
-$ /usr/kerberos/bin/klist -e -k -t /etc/security/keytab/nm.service.keytab 
-Keytab name: FILE:/etc/security/keytab/nm.service.keytab
-KVNO Timestamp         Principal
-   4 07/18/11 21:08:09 nm/full.qualified.domain.name@REALM.TLD (AES-256 CTS mode with 96-bit SHA-1 HMAC) 
-   4 07/18/11 21:08:09 nm/full.qualified.domain.name@REALM.TLD (AES-128 CTS mode with 96-bit SHA-1 HMAC) 
-   4 07/18/11 21:08:09 nm/full.qualified.domain.name@REALM.TLD (ArcFour with HMAC/md5) 
-   4 07/18/11 21:08:09 host/full.qualified.domain.name@REALM.TLD (AES-256 CTS mode with 96-bit SHA-1 HMAC) 
-   4 07/18/11 21:08:09 host/full.qualified.domain.name@REALM.TLD (AES-128 CTS mode with 96-bit SHA-1 HMAC) 
-   4 07/18/11 21:08:09 host/full.qualified.domain.name@REALM.TLD (ArcFour with HMAC/md5) 
-
-----
-    
-    * MapReduce JobHistory Server
-
-    The MapReduce JobHistory Server keytab file, on that host, should look  
-    like the following:
-    
-----
-
-$ /usr/kerberos/bin/klist -e -k -t /etc/security/keytab/jhs.service.keytab 
-Keytab name: FILE:/etc/security/keytab/jhs.service.keytab
-KVNO Timestamp         Principal
-   4 07/18/11 21:08:09 jhs/full.qualified.domain.name@REALM.TLD (AES-256 CTS mode with 96-bit SHA-1 HMAC) 
-   4 07/18/11 21:08:09 jhs/full.qualified.domain.name@REALM.TLD (AES-128 CTS mode with 96-bit SHA-1 HMAC) 
-   4 07/18/11 21:08:09 jhs/full.qualified.domain.name@REALM.TLD (ArcFour with HMAC/md5) 
-   4 07/18/11 21:08:09 host/full.qualified.domain.name@REALM.TLD (AES-256 CTS mode with 96-bit SHA-1 HMAC) 
-   4 07/18/11 21:08:09 host/full.qualified.domain.name@REALM.TLD (AES-128 CTS mode with 96-bit SHA-1 HMAC) 
-   4 07/18/11 21:08:09 host/full.qualified.domain.name@REALM.TLD (ArcFour with HMAC/md5) 
-
-----
-    
-  * Configuration in Secure Mode
-  
-    * <<<conf/core-site.xml>>>
-
-*-------------------------+-------------------------+------------------------+
-|| Parameter              || Value                  || Notes                 |
-*-------------------------+-------------------------+------------------------+
-| <<<hadoop.security.authentication>>> | <kerberos> | <simple> is non-secure. |
-*-------------------------+-------------------------+------------------------+
-| <<<hadoop.security.authorization>>> | <true> | |
-| | | Enable RPC service-level authorization. |
-*-------------------------+-------------------------+------------------------+
-
-    * <<<conf/hdfs-site.xml>>>
-    
-      * Configurations for NameNode:
-    
-*-------------------------+-------------------------+------------------------+
-|| Parameter              || Value                  || Notes                 |
-*-------------------------+-------------------------+------------------------+
-| <<<dfs.block.access.token.enable>>> | <true> |  |
-| | | Enable HDFS block access tokens for secure operations. |
-*-------------------------+-------------------------+------------------------+
-| <<<dfs.https.enable>>> | <true> | |
-*-------------------------+-------------------------+------------------------+
-| <<<dfs.namenode.https-address>>> | <nn_host_fqdn:50470> | |
-*-------------------------+-------------------------+------------------------+
-| <<<dfs.https.port>>> | <50470> | |
-*-------------------------+-------------------------+------------------------+
-| <<<dfs.namenode.keytab.file>>> | </etc/security/keytab/nn.service.keytab> | |
-| | | Kerberos keytab file for the NameNode. |
-*-------------------------+-------------------------+------------------------+
-| <<<dfs.namenode.kerberos.principal>>> | nn/_HOST@REALM.TLD | |
-| | | Kerberos principal name for the NameNode. |
-*-------------------------+-------------------------+------------------------+
-| <<<dfs.namenode.kerberos.https.principal>>> | host/_HOST@REALM.TLD | |
-| | | HTTPS Kerberos principal name for the NameNode. |
-*-------------------------+-------------------------+------------------------+
-
-      * Configurations for Secondary NameNode:
-    
-*-------------------------+-------------------------+------------------------+
-|| Parameter              || Value                  || Notes                 |
-*-------------------------+-------------------------+------------------------+
-| <<<dfs.namenode.secondary.http-address>>> | <c_nn_host_fqdn:50090> | |
-*-------------------------+-------------------------+------------------------+
-| <<<dfs.namenode.secondary.https-port>>> | <50470> | |
-*-------------------------+-------------------------+------------------------+
-| <<<dfs.namenode.secondary.keytab.file>>> | | | 
-| | </etc/security/keytab/sn.service.keytab> | |
-| | | Kerberos keytab file for the NameNode. |
-*-------------------------+-------------------------+------------------------+
-| <<<dfs.namenode.secondary.kerberos.principal>>> | sn/_HOST@REALM.TLD | |
-| | | Kerberos principal name for the Secondary NameNode. |
-*-------------------------+-------------------------+------------------------+
-| <<<dfs.namenode.secondary.kerberos.https.principal>>> | | |
-| | host/_HOST@REALM.TLD | |
-| | | HTTPS Kerberos principal name for the Secondary NameNode. |
-*-------------------------+-------------------------+------------------------+
-
-      * Configurations for DataNode:
-
-*-------------------------+-------------------------+------------------------+
-|| Parameter              || Value                  || Notes                 |
-*-------------------------+-------------------------+------------------------+
-| <<<dfs.datanode.data.dir.perm>>> | 700 | |
-*-------------------------+-------------------------+------------------------+
-| <<<dfs.datanode.address>>> | <0.0.0.0:2003> | |
-*-------------------------+-------------------------+------------------------+
-| <<<dfs.datanode.https.address>>> | <0.0.0.0:2005> | |
-*-------------------------+-------------------------+------------------------+
-| <<<dfs.datanode.keytab.file>>> | </etc/security/keytab/dn.service.keytab> | |
-| | | Kerberos keytab file for the DataNode. |
-*-------------------------+-------------------------+------------------------+
-| <<<dfs.datanode.kerberos.principal>>> | dn/_HOST@REALM.TLD | |
-| | | Kerberos principal name for the DataNode. |
-*-------------------------+-------------------------+------------------------+
-| <<<dfs.datanode.kerberos.https.principal>>> | | |
-| | host/_HOST@REALM.TLD | |
-| | | HTTPS Kerberos principal name for the DataNode. |
-*-------------------------+-------------------------+------------------------+
-
-    * <<<conf/yarn-site.xml>>>
-    
-      * WebAppProxy
-
-      The <<<WebAppProxy>>> provides a proxy between the web applications
-      exported by an application and an end user.  If security is enabled
-      it will warn users before accessing a potentially unsafe web application.
-      Authentication and authorization using the proxy is handled just like
-      any other privileged web application.
-
-*-------------------------+-------------------------+------------------------+
-|| Parameter              || Value                  || Notes                 |
-*-------------------------+-------------------------+------------------------+
-| <<<yarn.web-proxy.address>>> | | |
-| | <<<WebAppProxy>>> host:port for proxy to AM web apps. | |
-| | | <host:port> if this is the same as <<<yarn.resourcemanager.webapp.address>>>|
-| | | or it is not defined then the <<<ResourceManager>>> will run the proxy|
-| | | otherwise a standalone proxy server will need to be launched.|
-*-------------------------+-------------------------+------------------------+
-| <<<yarn.web-proxy.keytab>>> | | |
-| | </etc/security/keytab/web-app.service.keytab> | |
-| | | Kerberos keytab file for the WebAppProxy. |
-*-------------------------+-------------------------+------------------------+
-| <<<yarn.web-proxy.principal>>> | wap/_HOST@REALM.TLD | |
-| | | Kerberos principal name for the WebAppProxy. |
-*-------------------------+-------------------------+------------------------+
-
-      * LinuxContainerExecutor
-      
-      A <<<ContainerExecutor>>> used by YARN framework which define how any
-      <container> launched and controlled. 
-      
-      The following are the available in Hadoop YARN:
-
-*--------------------------------------+--------------------------------------+
-|| ContainerExecutor                   || Description                         |
-*--------------------------------------+--------------------------------------+
-| <<<DefaultContainerExecutor>>>             | |
-| | The default executor which YARN uses to manage container execution. |
-| | The container process has the same Unix user as the NodeManager.  |
-*--------------------------------------+--------------------------------------+
-| <<<LinuxContainerExecutor>>>               | |
-| | Supported only on GNU/Linux, this executor runs the containers as the | 
-| | user who submitted the application. It requires all user accounts to be |
-| | created on the cluster nodes where the containers are launched. It uses |
-| | a <setuid> executable that is included in the Hadoop distribution. |
-| | The NodeManager uses this executable to launch and kill containers. |
-| | The setuid executable switches to the user who has submitted the |
-| | application and launches or kills the containers. For maximum security, |
-| | this executor sets up restricted permissions and user/group ownership of |
-| | local files and directories used by the containers such as the shared |
-| | objects, jars, intermediate files, log files etc. Particularly note that, |
-| | because of this, except the application owner and NodeManager, no other |
-| | user can access any of the local files/directories including those |
-| | localized as part of the distributed cache. |
-*--------------------------------------+--------------------------------------+
-
-      To build the LinuxContainerExecutor executable run:
-        
-----
- $ mvn package -Dcontainer-executor.conf.dir=/etc/hadoop/
-----
-        
-      The path passed in <<<-Dcontainer-executor.conf.dir>>> should be the 
-      path on the cluster nodes where a configuration file for the setuid 
-      executable should be located. The executable should be installed in
-      $HADOOP_YARN_HOME/bin.
-
-      The executable must have specific permissions: 6050 or --Sr-s--- 
-      permissions user-owned by <root> (super-user) and group-owned by a 
-      special group (e.g. <<<hadoop>>>) of which the NodeManager Unix user is 
-      the group member and no ordinary application user is. If any application 
-      user belongs to this special group, security will be compromised. This 
-      special group name should be specified for the configuration property 
-      <<<yarn.nodemanager.linux-container-executor.group>>> in both 
-      <<<conf/yarn-site.xml>>> and <<<conf/container-executor.cfg>>>. 
-      
-      For example, let's say that the NodeManager is run as user <yarn> who is 
-      part of the groups users and <hadoop>, any of them being the primary group.
-      Let also be that <users> has both <yarn> and another user 
-      (application submitter) <alice> as its members, and <alice> does not 
-      belong to <hadoop>. Going by the above description, the setuid/setgid 
-      executable should be set 6050 or --Sr-s--- with user-owner as <yarn> and 
-      group-owner as <hadoop> which has <yarn> as its member (and not <users> 
-      which has <alice> also as its member besides <yarn>).
-
-      The LinuxTaskController requires that paths including and leading up to 
-      the directories specified in <<<yarn.nodemanager.local-dirs>>> and 
-      <<<yarn.nodemanager.log-dirs>>> to be set 755 permissions as described 
-      above in the table on permissions on directories.
-
-        * <<<conf/container-executor.cfg>>>
-        
-        The executable requires a configuration file called 
-        <<<container-executor.cfg>>> to be present in the configuration 
-        directory passed to the mvn target mentioned above. 
-
-        The configuration file must be owned by the user running NodeManager 
-        (user <<<yarn>>> in the above example), group-owned by anyone and 
-        should have the permissions 0400 or r--------.
-
-        The executable requires following configuration items to be present 
-        in the <<<conf/container-executor.cfg>>> file. The items should be 
-        mentioned as simple key=value pairs, one per-line:
-
-*-------------------------+-------------------------+------------------------+
-|| Parameter              || Value                  || Notes                 |
-*-------------------------+-------------------------+------------------------+
-| <<<yarn.nodemanager.linux-container-executor.group>>> | <hadoop> | |
-| | | Unix group of the NodeManager. The group owner of the |
-| | |<container-executor> binary should be this group. Should be same as the |
-| | | value with which the NodeManager is configured. This configuration is |
-| | | required for validating the secure access of the <container-executor> |
-| | | binary. |        
-*-------------------------+-------------------------+------------------------+
-| <<<banned.users>>> | hfds,yarn,mapred,bin | Banned users. |
-*-------------------------+-------------------------+------------------------+
-| <<<min.user.id>>> | 1000 | Prevent other super-users. |      
-*-------------------------+-------------------------+------------------------+
-
-      To re-cap, here are the local file-ssytem permissions required for the 
-      various paths related to the <<<LinuxContainerExecutor>>>:
-      
-*-------------------+-------------------+------------------+------------------+
-|| Filesystem       || Path             || User:Group      || Permissions     |
-*-------------------+-------------------+------------------+------------------+
-| local | container-executor | root:hadoop | --Sr-s--- |
-*-------------------+-------------------+------------------+------------------+
-| local | <<<conf/container-executor.cfg>>> | root:hadoop | r-------- |
-*-------------------+-------------------+------------------+------------------+
-| local | <<<yarn.nodemanager.local-dirs>>> | yarn:hadoop | drwxr-xr-x |
-*-------------------+-------------------+------------------+------------------+
-| local | <<<yarn.nodemanager.log-dirs>>> | yarn:hadoop | drwxr-xr-x |
-*-------------------+-------------------+------------------+------------------+
-      
-      * Configurations for ResourceManager:
-    
-*-------------------------+-------------------------+------------------------+
-|| Parameter              || Value                  || Notes                 |
-*-------------------------+-------------------------+------------------------+
-| <<<yarn.resourcemanager.keytab>>> | | |
-| | </etc/security/keytab/rm.service.keytab> | |
-| | | Kerberos keytab file for the ResourceManager. |
-*-------------------------+-------------------------+------------------------+
-| <<<yarn.resourcemanager.principal>>> | rm/_HOST@REALM.TLD | |
-| | | Kerberos principal name for the ResourceManager. |
-*-------------------------+-------------------------+------------------------+
-      
-      * Configurations for NodeManager:
-      
-*-------------------------+-------------------------+------------------------+
-|| Parameter              || Value                  || Notes                 |
-*-------------------------+-------------------------+------------------------+
-| <<<yarn.nodemanager.keytab>>> | </etc/security/keytab/nm.service.keytab> | |
-| | | Kerberos keytab file for the NodeManager. |
-*-------------------------+-------------------------+------------------------+
-| <<<yarn.nodemanager.principal>>> | nm/_HOST@REALM.TLD | |
-| | | Kerberos principal name for the NodeManager. |
-*-------------------------+-------------------------+------------------------+
-| <<<yarn.nodemanager.container-executor.class>>> | | |
-| | <<<org.apache.hadoop.yarn.server.nodemanager.LinuxContainerExecutor>>> | 
-| | | Use LinuxContainerExecutor. | 
-*-------------------------+-------------------------+------------------------+
-| <<<yarn.nodemanager.linux-container-executor.group>>> | <hadoop> | |
-| | | Unix group of the NodeManager. |
-*-------------------------+-------------------------+------------------------+
-
-    * <<<conf/mapred-site.xml>>>
-    
-      * Configurations for MapReduce JobHistory Server:
-
-*-------------------------+-------------------------+------------------------+
-|| Parameter              || Value                  || Notes                 |
-*-------------------------+-------------------------+------------------------+
-| <<<mapreduce.jobhistory.address>>> | | |
-| | MapReduce JobHistory Server <host:port> | Default port is 10020. |
-*-------------------------+-------------------------+------------------------+
-| <<<mapreduce.jobhistory.keytab>>> | |
-| | </etc/security/keytab/jhs.service.keytab> | |
-| | | Kerberos keytab file for the MapReduce JobHistory Server. |
-*-------------------------+-------------------------+------------------------+
-| <<<mapreduce.jobhistory.principal>>> | jhs/_HOST@REALM.TLD | |
-| | | Kerberos principal name for the MapReduce JobHistory Server. |
-*-------------------------+-------------------------+------------------------+
-
-        
-  * {Operating the Hadoop Cluster}
-
-  Once all the necessary configuration is complete, distribute the files to the 
-  <<<HADOOP_CONF_DIR>>> directory on all the machines.
-
-  This section also describes the various Unix users who should be starting the
-  various components and uses the same Unix accounts and groups used previously:
-  
-    * Hadoop Startup
-  
-    To start a Hadoop cluster you will need to start both the HDFS and YARN 
-    cluster.
-
-    Format a new distributed filesystem as <hdfs>:
-  
-----
-[hdfs]$ $HADOOP_PREFIX/bin/hdfs namenode -format <cluster_name>
-----
-
-    Start the HDFS with the following command, run on the designated NameNode
-    as <hdfs>:
-  
-----
-[hdfs]$ $HADOOP_PREFIX/sbin/hadoop-daemon.sh --config $HADOOP_CONF_DIR --script hdfs start namenode
-----    	  
-
-    Run a script to start DataNodes on all slaves as <root> with a special
-    environment variable <<<HADOOP_SECURE_DN_USER>>> set to <hdfs>:
-
-----
-[root]$ HADOOP_SECURE_DN_USER=hdfs $HADOOP_PREFIX/sbin/hadoop-daemon.sh --config $HADOOP_CONF_DIR --script hdfs start datanode
-----    	  
-  
-    Start the YARN with the following command, run on the designated 
-    ResourceManager as <yarn>:
-  
-----
-[yarn]$ $HADOOP_YARN_HOME/sbin/yarn-daemon.sh --config $HADOOP_CONF_DIR start resourcemanager 
-----    	  
-
-    Run a script to start NodeManagers on all slaves as <yarn>:
-
-----
-[yarn]$ $HADOOP_YARN_HOME/sbin/yarn-daemon.sh --config $HADOOP_CONF_DIR start nodemanager 
-----    	  
-
-    Start a standalone WebAppProxy server. Run on the WebAppProxy 
-    server as <yarn>.  If multiple servers are used with load balancing
-    it should be run on each of them:
-
-----
-[yarn]$ $HADOOP_YARN_HOME/bin/yarn start proxyserver --config $HADOOP_CONF_DIR  
-----    	  
-
-    Start the MapReduce JobHistory Server with the following command, run on the  
-    designated server as <mapred>:
-  
-----
-[mapred]$ $HADOOP_PREFIX/sbin/mr-jobhistory-daemon.sh start historyserver --config $HADOOP_CONF_DIR  
-----    	  
-
-    * Hadoop Shutdown      
-
-    Stop the NameNode with the following command, run on the designated NameNode
-    as <hdfs>:
-  
-----
-[hdfs]$ $HADOOP_PREFIX/sbin/hadoop-daemon.sh --config $HADOOP_CONF_DIR --script hdfs stop namenode
-----    	  
-
-    Run a script to stop DataNodes on all slaves as <root>:
-
-----
-[root]$ $HADOOP_PREFIX/sbin/hadoop-daemon.sh --config $HADOOP_CONF_DIR --script hdfs stop datanode
-----    	  
-  
-    Stop the ResourceManager with the following command, run on the designated 
-    ResourceManager as <yarn>:
-  
-----
-[yarn]$ $HADOOP_YARN_HOME/sbin/yarn-daemon.sh --config $HADOOP_CONF_DIR stop resourcemanager 
-----    	  
-
-    Run a script to stop NodeManagers on all slaves as <yarn>:
-
-----
-[yarn]$ $HADOOP_YARN_HOME/sbin/yarn-daemon.sh --config $HADOOP_CONF_DIR stop nodemanager 
-----    	  
-
-    Stop the WebAppProxy server. Run on the WebAppProxy  server as
-    <yarn>.  If multiple servers are used with load balancing it
-    should be run on each of them:
-
-----
-[yarn]$ $HADOOP_YARN_HOME/bin/yarn stop proxyserver --config $HADOOP_CONF_DIR  
-----
-
-    Stop the MapReduce JobHistory Server with the following command, run on the  
-    designated server as <mapred>:
-
-----
-[mapred]$ $HADOOP_PREFIX/sbin/mr-jobhistory-daemon.sh stop historyserver --config $HADOOP_CONF_DIR  
-----    	  
-    
-* {Web Interfaces}      
-
-    Once the Hadoop cluster is up and running check the web-ui of the 
-    components as described below:
-    
-*-------------------------+-------------------------+------------------------+
-|| Daemon                 || Web Interface          || Notes                 |
-*-------------------------+-------------------------+------------------------+
-| NameNode | http://<nn_host:port>/ | Default HTTP port is 50070. |
-*-------------------------+-------------------------+------------------------+
-| ResourceManager | http://<rm_host:port>/ | Default HTTP port is 8088. |
-*-------------------------+-------------------------+------------------------+
-| MapReduce JobHistory Server | http://<jhs_host:port>/ | |
-| | | Default HTTP port is 19888. |
-*-------------------------+-------------------------+------------------------+
-    
-    
+~~ 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. See accompanying LICENSE file.
+
+  ---
+  Hadoop Map Reduce Next Generation-${project.version} - Cluster Setup
+  ---
+  ---
+  ${maven.build.timestamp}
+
+Hadoop MapReduce Next Generation - Cluster Setup
+
+  \[ {{{./index.html}Go Back}} \]
+
+%{toc|section=1|fromDepth=0}
+
+* {Purpose}
+
+  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.
+
+  To play with Hadoop, you may first want to install it on a single 
+  machine (see {{{SingleCluster}Single Node Setup}}).
+  
+* {Prerequisites}
+
+  Download a stable version of Hadoop from Apache mirrors.
+  
+* {Installation}
+
+  Installing a Hadoop cluster typically involves unpacking the software on all 
+  the machines in the cluster or installing RPMs.
+
+  Typically one machine in the cluster is designated as the NameNode and 
+  another machine the as ResourceManager, exclusively. These are the masters. 
+  
+  The rest of the machines in the cluster act as both DataNode and NodeManager. 
+  These are the slaves.
+
+* {Running Hadoop in Non-Secure Mode}
+
+  The following sections describe how to configure a Hadoop cluster.
+
+  * {Configuration Files}
+  
+    Hadoop configuration is driven by two types of important configuration files:
+
+      * Read-only default configuration - <<<core-default.xml>>>, 
+        <<<hdfs-default.xml>>>, <<<yarn-default.xml>>> and 
+        <<<mapred-default.xml>>>.
+        
+      * Site-specific configuration - <<conf/core-site.xml>>, 
+        <<conf/hdfs-site.xml>>, <<conf/yarn-site.xml>> and 
+        <<conf/mapred-site.xml>>.
+
+
+    Additionally, you can control the Hadoop scripts found in the bin/ 
+    directory of the distribution, by setting site-specific values via the 
+    <<conf/hadoop-env.sh>> and <<yarn-env.sh>>.
+
+  * {Site Configuration}
+  
+  To configure the Hadoop cluster you will need to configure the 
+  <<<environment>>> in which the Hadoop daemons execute as well as the 
+  <<<configuration parameters>>> for the Hadoop daemons.
+
+  The Hadoop daemons are NameNode/DataNode and ResourceManager/NodeManager.
+
+
+    * {Configuring Environment of Hadoop Daemons}
+    
+    Administrators should use the <<conf/hadoop-env.sh>> and 
+    <<conf/yarn-env.sh>> script to do site-specific customization of the 
+    Hadoop daemons' process environment.
+
+    At the very least you should specify the <<<JAVA_HOME>>> so that it is 
+    correctly defined on each remote node.
+
+    In most cases you should also specify <<<HADOOP_PID_DIR>>> and 
+    <<<HADOOP_SECURE_DN_PID_DIR>>> to point to directories that can only be
+    written to by the users that are going to run the hadoop daemons.  
+    Otherwise there is the potential for a symlink attack.
+
+    Administrators can configure individual daemons using the configuration 
+    options shown below in the table:
+
+*--------------------------------------+--------------------------------------+
+|| Daemon                              || Environment Variable                |
+*--------------------------------------+--------------------------------------+
+| NameNode                             | HADOOP_NAMENODE_OPTS                 |
+*--------------------------------------+--------------------------------------+
+| DataNode                             | HADOOP_DATANODE_OPTS                 |
+*--------------------------------------+--------------------------------------+
+| Secondary NameNode                   | HADOOP_SECONDARYNAMENODE_OPTS        |
+*--------------------------------------+--------------------------------------+
+| ResourceManager                      | YARN_RESOURCEMANAGER_OPTS            |
+*--------------------------------------+--------------------------------------+
+| NodeManager                          | YARN_NODEMANAGER_OPTS                |
+*--------------------------------------+--------------------------------------+
+| WebAppProxy                          | YARN_PROXYSERVER_OPTS                |
+*--------------------------------------+--------------------------------------+
+| Map Reduce Job History Server        | HADOOP_JOB_HISTORYSERVER_OPTS        |
+*--------------------------------------+--------------------------------------+
+
+
+    For example, To configure Namenode to use parallelGC, the following 
+    statement should be added in hadoop-env.sh :
+     
+----
+    export HADOOP_NAMENODE_OPTS="-XX:+UseParallelGC ${HADOOP_NAMENODE_OPTS}" 
+----
+    
+    Other useful configuration parameters that you can customize include:
+
+      * <<<HADOOP_LOG_DIR>>> / <<<YARN_LOG_DIR>>> - The directory where the 
+        daemons' log files are stored. They are automatically created if they 
+        don't exist.
+        
+      * <<<HADOOP_HEAPSIZE>>> / <<<YARN_HEAPSIZE>>> - The maximum amount of 
+        heapsize to use, in MB e.g. if the varibale is set to 1000 the heap
+        will be set to 1000MB.  This is used to configure the heap 
+        size for the daemon. By default, the value is 1000.  If you want to
+        configure the values separately for each deamon you can use.
+*--------------------------------------+--------------------------------------+
+|| Daemon                              || Environment Variable                |
+*--------------------------------------+--------------------------------------+
+| ResourceManager                      | YARN_RESOURCEMANAGER_HEAPSIZE        |
+*--------------------------------------+--------------------------------------+
+| NodeManager                          | YARN_NODEMANAGER_HEAPSIZE            |
+*--------------------------------------+--------------------------------------+
+| WebAppProxy                          | YARN_PROXYSERVER_HEAPSIZE            |
+*--------------------------------------+--------------------------------------+
+| Map Reduce Job History Server        | HADOOP_JOB_HISTORYSERVER_HEAPSIZE    |
+*--------------------------------------+--------------------------------------+
+ 
+    * {Configuring the Hadoop Daemons in Non-Secure Mode}
+
+      This section deals with important parameters to be specified in 
+      the given configuration files:
+       
+      * <<<conf/core-site.xml>>>
+      
+*-------------------------+-------------------------+------------------------+
+|| Parameter              || Value                  || Notes                 |
+*-------------------------+-------------------------+------------------------+
+| <<<fs.defaultFS>>>      | NameNode URI            | <hdfs://host:port/>    |
+*-------------------------+-------------------------+------------------------+
+| <<<io.file.buffer.size>>> | 131072 |  |
+| | | Size of read/write buffer used in SequenceFiles. |
+*-------------------------+-------------------------+------------------------+
+
+      * <<<conf/hdfs-site.xml>>>
+      
+        * Configurations for NameNode:
+
+*-------------------------+-------------------------+------------------------+
+|| Parameter              || Value                  || Notes                 |
+*-------------------------+-------------------------+------------------------+
+| <<<dfs.namenode.name.dir>>> | | | 
+| | Path on the local filesystem where the NameNode stores the namespace | | 
+| | and transactions logs persistently. | | 
+| | | If this is a comma-delimited list of directories then the name table is  |
+| | | replicated in all of the directories, for redundancy. |
+*-------------------------+-------------------------+------------------------+
+| <<<dfs.namenode.hosts>>> / <<<dfs.namenode.hosts.exclude>>> | | |
+| | List of permitted/excluded DataNodes. | |
+| | | If necessary, use these files to control the list of allowable |
+| | | datanodes. |
+*-------------------------+-------------------------+------------------------+
+| <<<dfs.blocksize>>> | 268435456 | |
+| | | HDFS blocksize of 256MB for large file-systems. |      
+*-------------------------+-------------------------+------------------------+
+| <<<dfs.namenode.handler.count>>> | 100 | |
+| | | More NameNode server threads to handle RPCs from large number of |
+| | | DataNodes. |
+*-------------------------+-------------------------+------------------------+
+        
+        * Configurations for DataNode:
+      
+*-------------------------+-------------------------+------------------------+
+|| Parameter              || Value                  || Notes                 |
+*-------------------------+-------------------------+------------------------+
+| <<<dfs.datanode.data.dir>>> | | |
+| | Comma separated list of paths on the local filesystem of a | | 
+| | <<<DataNode>>> where it should store its blocks. | |
+| | | If this is a comma-delimited list of directories, then data will be | 
+| | | stored in all named directories, typically on different devices. |
+*-------------------------+-------------------------+------------------------+
+      
+      * <<<conf/yarn-site.xml>>>
+
+        * Configurations for ResourceManager and NodeManager:
+
+*-------------------------+-------------------------+------------------------+
+|| Parameter              || Value                  || Notes                 |
+*-------------------------+-------------------------+------------------------+
+| <<<yarn.acl.enable>>> | | |
+| | <<<true>>> / <<<false>>> | |
+| | | Enable ACLs? Defaults to <false>. |
+*-------------------------+-------------------------+------------------------+
+| <<<yarn.admin.acl>>> | | |
+| | Admin ACL | |
+| | | ACL to set admins on the cluster. |
+| | | ACLs are of for <comma-separated-users><space><comma-separated-groups>. |
+| | | Defaults to special value of <<*>> which means <anyone>. |
+| | | Special value of just <space> means no one has access. |
+*-------------------------+-------------------------+------------------------+
+| <<<yarn.log-aggregation-enable>>> | | |
+| | <false> | |
+| | | Configuration to enable or disable log aggregation |
+*-------------------------+-------------------------+------------------------+
+
+
+        * Configurations for ResourceManager:
+
+*-------------------------+-------------------------+------------------------+
+|| Parameter              || Value                  || Notes                 |
+*-------------------------+-------------------------+------------------------+
+| <<<yarn.resourcemanager.address>>> | | | 
+| | <<<ResourceManager>>> host:port for clients to submit jobs. | |
+| | | <host:port> |
+*-------------------------+-------------------------+------------------------+
+| <<<yarn.resourcemanager.scheduler.address>>> | | | 
+| | <<<ResourceManager>>> host:port for ApplicationMasters to talk to | |
+| | Scheduler to obtain resources. | |
+| | | <host:port> |
+*-------------------------+-------------------------+------------------------+
+| <<<yarn.resourcemanager.resource-tracker.address>>> | | | 
+| | <<<ResourceManager>>> host:port for NodeManagers. | |
+| | | <host:port> |
+*-------------------------+-------------------------+------------------------+
+| <<<yarn.resourcemanager.admin.address>>> | | | 
+| | <<<ResourceManager>>> host:port for administrative commands. | |
+| | | <host:port> |
+*-------------------------+-------------------------+------------------------+
+| <<<yarn.resourcemanager.webapp.address>>> | | | 
+| | <<<ResourceManager>>> web-ui host:port. | |
+| | | <host:port> |
+*-------------------------+-------------------------+------------------------+
+| <<<yarn.resourcemanager.scheduler.class>>> | | |
+| | <<<ResourceManager>>> Scheduler class. | |
+| | | <<<CapacityScheduler>>> (recommended) or <<<FifoScheduler>>> |
+*-------------------------+-------------------------+------------------------+
+| <<<yarn.scheduler.minimum-allocation-mb>>> | | |
+| | Minimum limit of memory to allocate to each container request at the <<<Resource Manager>>>. | |
+| | | In MBs |
+*-------------------------+-------------------------+------------------------+
+| <<<yarn.scheduler.maximum-allocation-mb>>> | | |
+| | Maximum limit of memory to allocate to each container request at the <<<Resource Manager>>>. | |
+| | | In MBs |
+*-------------------------+-------------------------+------------------------+
+| <<<yarn.resourcemanager.nodes.include-path>>> / | | | 
+| <<<yarn.resourcemanager.nodes.exclude-path>>> | | |  
+| | List of permitted/excluded NodeManagers. | |
+| | | If necessary, use these files to control the list of allowable | 
+| | | NodeManagers. |
+*-------------------------+-------------------------+------------------------+
+ 
+        * Configurations for NodeManager:
+
+*-------------------------+-------------------------+------------------------+
+|| Parameter              || Value                  || Notes                 |
+*-------------------------+-------------------------+------------------------+
+| <<<yarn.nodemanager.resource.memory-mb>>> | | |
+| | Resource i.e. available physical memory, in MB, for given <<<NodeManager>>> | |
+| | | Defines total available resources on the <<<NodeManager>>> to be made |
+| | | available to running containers |
+*-------------------------+-------------------------+------------------------+
+| <<<yarn.nodemanager.vmem-pmem-ratio>>> | | |
+| | Maximum ratio by which virtual memory usage of tasks may exceed |
+| | physical memory | |
+| | | The virtual memory usage of each task may exceed its physical memory |
+| | | limit by this ratio. The total amount of virtual memory used by tasks |
+| | | on the NodeManager may exceed its physical memory usage by this ratio. |
+*-------------------------+-------------------------+------------------------+
+| <<<yarn.nodemanager.local-dirs>>> | | |
+| | Comma-separated list of paths on the local filesystem where | |
+| | intermediate data is written. ||
+| | | Multiple paths help spread disk i/o. |
+*-------------------------+-------------------------+------------------------+
+| <<<yarn.nodemanager.log-dirs>>> | | |
+| | Comma-separated list of paths on the local filesystem where logs  | |
+| | are written. | |
+| | | Multiple paths help spread disk i/o. |
+*-------------------------+-------------------------+------------------------+
+| <<<yarn.nodemanager.log.retain-seconds>>> | | |
+| | <10800> | |
+| | | Default time (in seconds) to retain log files on the NodeManager |
+| | | Only applicable if log-aggregation is disabled. |
+*-------------------------+-------------------------+------------------------+
+| <<<yarn.nodemanager.remote-app-log-dir>>> | | |
+| | </logs> | |
+| | | HDFS directory where the application logs are moved on application |
+| | | completion. Need to set appropriate permissions. |
+| | | Only applicable if log-aggregation is enabled. |
+*-------------------------+-------------------------+------------------------+
+| <<<yarn.nodemanager.remote-app-log-dir-suffix>>> | | |
+| | <logs> | |
+| | | Suffix appended to the remote log dir. Logs will be aggregated to  |
+| | | $\{yarn.nodemanager.remote-app-log-dir\}/$\{user\}/$\{thisParam\} |
+| | | Only applicable if log-aggregation is enabled. |
+*-------------------------+-------------------------+------------------------+
+| <<<yarn.nodemanager.aux-services>>> | | |
+| | mapreduce.shuffle  | |
+| | | Shuffle service that needs to be set for Map Reduce applications. |
+*-------------------------+-------------------------+------------------------+
+
+        * Configurations for History Server (Needs to be moved elsewhere):
+
+*-------------------------+-------------------------+------------------------+
+|| Parameter              || Value                  || Notes                 |
+*-------------------------+-------------------------+------------------------+
+| <<<yarn.log-aggregation.retain-seconds>>> | | |
+| | <-1> | |
+| | | How long to keep aggregation logs before deleting them. -1 disables. |
+| | | Be careful, set this too small and you will spam the name node. |
+*-------------------------+-------------------------+------------------------+
+
+
+
+      * <<<conf/mapred-site.xml>>>
+
+        * Configurations for MapReduce Applications:
+
+*-------------------------+-------------------------+------------------------+
+|| Parameter              || Value                  || Notes                 |
+*-------------------------+-------------------------+------------------------+
+| <<<mapreduce.framework.name>>> | | |
+| | yarn | |
+| | | Execution framework set to Hadoop YARN. |
+*-------------------------+-------------------------+------------------------+
+| <<<mapreduce.map.memory.mb>>> | 1536 | |
+| | | Larger resource limit for maps. |
+*-------------------------+-------------------------+------------------------+
+| <<<mapreduce.map.java.opts>>> | -Xmx1024M | |
+| | | Larger heap-size for child jvms of maps. |
+*-------------------------+-------------------------+------------------------+
+| <<<mapreduce.reduce.memory.mb>>> | 3072 | |
+| | | Larger resource limit for reduces. |
+*-------------------------+-------------------------+------------------------+
+| <<<mapreduce.reduce.java.opts>>> | -Xmx2560M | |
+| | | Larger heap-size for child jvms of reduces. |
+*-------------------------+-------------------------+------------------------+
+| <<<mapreduce.task.io.sort.mb>>> | 512 | |
+| | | Higher memory-limit while sorting data for efficiency. |
+*-------------------------+-------------------------+------------------------+
+| <<<mapreduce.task.io.sort.factor>>> | 100 | |
+| | | More streams merged at once while sorting files. |
+*-------------------------+-------------------------+------------------------+
+| <<<mapreduce.reduce.shuffle.parallelcopies>>> | 50 | |
+| | | Higher number of parallel copies run by reduces to fetch outputs |
+| | | from very large number of maps. |
+*-------------------------+-------------------------+------------------------+
+
+        * Configurations for MapReduce JobHistory Server:
+
+*-------------------------+-------------------------+------------------------+
+|| Parameter              || Value                  || Notes                 |
+*-------------------------+-------------------------+------------------------+
+| <<<mapreduce.jobhistory.address>>> | | |
+| | MapReduce JobHistory Server <host:port> | Default port is 10020. |
+*-------------------------+-------------------------+------------------------+
+| <<<mapreduce.jobhistory.webapp.address>>> | | |
+| | MapReduce JobHistory Server Web UI <host:port> | Default port is 19888. |
+*-------------------------+-------------------------+------------------------+
+| <<<mapreduce.jobhistory.intermediate-done-dir>>> | /mr-history/tmp | |
+|  | | Directory where history files are written by MapReduce jobs. | 
+*-------------------------+-------------------------+------------------------+
+| <<<mapreduce.jobhistory.done-dir>>> | /mr-history/done| |
+| | | Directory where history files are managed by the MR JobHistory Server. | 
+*-------------------------+-------------------------+------------------------+
+
+      * Hadoop Rack Awareness
+      
+      The HDFS and the YARN components are rack-aware.
+
+      The NameNode and the ResourceManager obtains the rack information of the 
+      slaves in the cluster by invoking an API <resolve> in an administrator 
+      configured module. 
+      
+      The API resolves the DNS name (also IP address) to a rack id. 
+      
+      The site-specific module to use can be configured using the configuration 
+      item <<<topology.node.switch.mapping.impl>>>. The default implementation 
+      of the same runs a script/command configured using 
+      <<<topology.script.file.name>>>. If <<<topology.script.file.name>>> is 
+      not set, the rack id </default-rack> is returned for any passed IP address. 
+
+      * Monitoring Health of NodeManagers
+      
+      Hadoop provides a mechanism by which administrators can configure the 
+      NodeManager 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 ERROR. The NodeManager spawns 
+      the script periodically and checks its output. If the script's output 
+      contains the string ERROR, as described above, the node's status is 
+      reported as <<<unhealthy>>> and the node is black-listed by the 
+      ResourceManager. No further tasks will be assigned to this node. 
+      However, the NodeManager continues to run the script, so that if the 
+      node becomes healthy again, it will be removed from the blacklisted nodes
+      on the ResourceManager automatically. The node's health along with the 
+      output of the script, if it is unhealthy, is available to the 
+      administrator in the ResourceManager web interface. The time since the 
+      node was healthy is also displayed on the web interface.
+
+      The following parameters can be used to control the node health 
+      monitoring script in <<<conf/yarn-site.xml>>>.
+
+*-------------------------+-------------------------+------------------------+
+|| Parameter              || Value                  || Notes                 |
+*-------------------------+-------------------------+------------------------+
+| <<<yarn.nodemanager.health-checker.script.path>>> | | |
+| | Node health script  | |
+| | | Script to check for node's health status. |
+*-------------------------+-------------------------+------------------------+
+| <<<yarn.nodemanager.health-checker.script.opts>>> | | |
+| | Node health script options  | |
+| | | Options for script to check for node's health status. |
+*-------------------------+-------------------------+------------------------+
+| <<<yarn.nodemanager.health-checker.script.interval-ms>>> | | |
+| | Node health script interval  | |
+| | | Time interval for running health script. |
+*-------------------------+-------------------------+------------------------+
+| <<<yarn.nodemanager.health-checker.script.timeout-ms>>> | | |
+| | Node health script timeout interval  | |
+| | | Timeout for health script execution. |
+*-------------------------+-------------------------+------------------------+
+
+    The health checker script is not supposed to give ERROR if only some of the
+    local disks become bad. NodeManager has the ability to periodically check
+    the health of the local disks (specifically checks nodemanager-local-dirs
+    and nodemanager-log-dirs) and after reaching the threshold of number of
+    bad directories based on the value set for the config property
+    yarn.nodemanager.disk-health-checker.min-healthy-disks, the whole node is
+    marked unhealthy and this info is sent to resource manager also. The boot
+    disk is either raided or a failure in the boot disk is identified by the
+    health checker script.
+
+    * {Slaves file}
+      
+    Typically you choose one machine in the cluster to act as the NameNode and 
+    one machine as to act as the ResourceManager, exclusively. The rest of the 
+    machines act as both a DataNode and NodeManager and are referred to as 
+    <slaves>.
+
+    List all slave hostnames or IP addresses in your <<<conf/slaves>>> file, 
+    one per line.
+
+    * {Logging}
+    
+    Hadoop uses the Apache log4j via the Apache Commons Logging framework for 
+    logging. Edit the <<<conf/log4j.properties>>> file to customize the 
+    Hadoop daemons' logging configuration (log-formats and so on).
+    
+  * {Operating the Hadoop Cluster}
+
+  Once all the necessary configuration is complete, distribute the files to the 
+  <<<HADOOP_CONF_DIR>>> directory on all the machines.
+
+    * Hadoop Startup
+  
+    To start a Hadoop cluster you will need to start both the HDFS and YARN 
+    cluster.
+
+    Format a new distributed filesystem:
+  
+----
+  $ $HADOOP_PREFIX/bin/hdfs namenode -format <cluster_name>
+----
+
+    Start the HDFS with the following command, run on the designated NameNode:
+  
+----
+  $ $HADOOP_PREFIX/sbin/hadoop-daemon.sh --config $HADOOP_CONF_DIR --script hdfs start namenode
+----    	  
+
+    Run a script to start DataNodes on all slaves:
+
+----
+  $ $HADOOP_PREFIX/sbin/hadoop-daemon.sh --config $HADOOP_CONF_DIR --script hdfs start datanode
+----    	  
+  
+    Start the YARN with the following command, run on the designated 
+    ResourceManager:
+  
+----
+  $ $HADOOP_YARN_HOME/sbin/yarn-daemon.sh --config $HADOOP_CONF_DIR start resourcemanager 
+----    	  
+
+    Run a script to start NodeManagers on all slaves:
+
+----
+  $ $HADOOP_YARN_HOME/sbin/yarn-daemon.sh --config $HADOOP_CONF_DIR start nodemanager 
+----    	  
+
+    Start a standalone WebAppProxy server.  If multiple servers
+    are used with load balancing it should be run on each of them:
+
+----
+  $ $HADOOP_YARN_HOME/bin/yarn start proxyserver --config $HADOOP_CONF_DIR  
+----
+
+    Start the MapReduce JobHistory Server with the following command, run on the  
+    designated server:
+  
+----
+  $ $HADOOP_PREFIX/sbin/mr-jobhistory-daemon.sh start historyserver --config $HADOOP_CONF_DIR  
+----    	  
+
+    * Hadoop Shutdown      
+
+    Stop the NameNode with the following command, run on the designated 
+    NameNode:
+  
+----
+  $ $HADOOP_PREFIX/sbin/hadoop-daemon.sh --config $HADOOP_CONF_DIR --script hdfs stop namenode
+----    	  
+
+    Run a script to stop DataNodes on all slaves:
+
+----
+  $ $HADOOP_PREFIX/sbin/hadoop-daemon.sh --config $HADOOP_CONF_DIR --script hdfs stop datanode
+----    	  
+  
+    Stop the ResourceManager with the following command, run on the designated 
+    ResourceManager:
+  
+----
+  $ $HADOOP_YARN_HOME/sbin/yarn-daemon.sh --config $HADOOP_CONF_DIR stop resourcemanager 
+----    	  
+
+    Run a script to stop NodeManagers on all slaves:
+
+----
+  $ $HADOOP_YARN_HOME/sbin/yarn-daemon.sh --config $HADOOP_CONF_DIR stop nodemanager 
+----    	  
+
+    Stop the WebAppProxy server. If multiple servers are used with load
+    balancing it should be run on each of them:
+
+----
+  $ $HADOOP_YARN_HOME/bin/yarn stop proxyserver --config $HADOOP_CONF_DIR  
+----
+
+
+    Stop the MapReduce JobHistory Server with the following command, run on the  
+    designated server:
+  
+----
+  $ $HADOOP_PREFIX/sbin/mr-jobhistory-daemon.sh stop historyserver --config $HADOOP_CONF_DIR  
+----    	  
+
+    
+* {Running Hadoop in Secure Mode}
+
+  This section deals with important parameters to be specified in 
+  to run Hadoop in <<secure mode>> with strong, Kerberos-based
+  authentication.
+      
+  * <<<User Accounts for Hadoop Daemons>>>
+      
+  Ensure that HDFS and YARN daemons run as different Unix users, for e.g.
+  <<<hdfs>>> and <<<yarn>>>. Also, ensure that the MapReduce JobHistory
+  server runs as user <<<mapred>>>. 
+      
+  It's recommended to have them share a Unix group, for e.g. <<<hadoop>>>.
+      
+*--------------------------------------+----------------------------------------------------------------------+
+|| User:Group                          || Daemons                                                             |
+*--------------------------------------+----------------------------------------------------------------------+
+| hdfs:hadoop                          | NameNode, Secondary NameNode, Checkpoint Node, Backup Node, DataNode |
+*--------------------------------------+----------------------------------------------------------------------+
+| yarn:hadoop                          | ResourceManager, NodeManager                                         |
+*--------------------------------------+----------------------------------------------------------------------+
+| mapred:hadoop                        | MapReduce JobHistory Server                                          |
+*--------------------------------------+----------------------------------------------------------------------+
+      
+  * <<<Permissions for both HDFS and local fileSystem paths>>>
+     
+  The following table lists various paths on HDFS and local filesystems (on
+  all nodes) and recommended permissions:
+   
+*-------------------+-------------------+------------------+------------------+
+|| Filesystem       || Path             || User:Group      || Permissions     |
+*-------------------+-------------------+------------------+------------------+
+| local | <<<dfs.namenode.name.dir>>> | hdfs:hadoop | drwx------ | 
+*-------------------+-------------------+------------------+------------------+
+| local | <<<dfs.datanode.data.dir>>> | hdfs:hadoop | drwx------ |
+*-------------------+-------------------+------------------+------------------+
+| local | $HADOOP_LOG_DIR | hdfs:hadoop | drwxrwxr-x |
+*-------------------+-------------------+------------------+------------------+
+| local | $YARN_LOG_DIR | yarn:hadoop | drwxrwxr-x |
+*-------------------+-------------------+------------------+------------------+
+| local | <<<yarn.nodemanager.local-dirs>>> | yarn:hadoop | drwxr-xr-x |
+*-------------------+-------------------+------------------+------------------+
+| local | <<<yarn.nodemanager.log-dirs>>> | yarn:hadoop | drwxr-xr-x |
+*-------------------+-------------------+------------------+------------------+
+| local | container-executor | root:hadoop | --Sr-s--- |
+*-------------------+-------------------+------------------+------------------+
+| local | <<<conf/container-executor.cfg>>> | root:hadoop | r-------- |
+*-------------------+-------------------+------------------+------------------+
+| hdfs | / | hdfs:hadoop | drwxr-xr-x |
+*-------------------+-------------------+------------------+------------------+
+| hdfs | /tmp | hdfs:hadoop | drwxrwxrwxt |
+*-------------------+-------------------+------------------+------------------+
+| hdfs | /user | hdfs:hadoop | drwxr-xr-x |
+*-------------------+-------------------+------------------+------------------+
+| hdfs | <<<yarn.nodemanager.remote-app-log-dir>>> | yarn:hadoop | drwxrwxrwxt |
+*-------------------+-------------------+------------------+------------------+
+| hdfs | <<<mapreduce.jobhistory.intermediate-done-dir>>> | mapred:hadoop | |
+| | | | drwxrwxrwxt |      
+*-------------------+-------------------+------------------+------------------+
+| hdfs | <<<mapreduce.jobhistory.done-dir>>> | mapred:hadoop | |
+| | | | drwxr-x--- |      
+*-------------------+-------------------+------------------+------------------+
+
+  * Kerberos Keytab files
+  
+    * HDFS
+    
+    The NameNode keytab file, on the NameNode host, should look like the 
+    following:
+    
+----
+
+$ /usr/kerberos/bin/klist -e -k -t /etc/security/keytab/nn.service.keytab 

[... 488 lines stripped ...]