Browse Source

Merge -c 1195764 from trunk to branch-0.23 to fix MAPREDUCE-3322.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.23@1195765 13f79535-47bb-0310-9956-ffa450edef68
Arun Murthy 13 years ago
parent
commit
76224deb1a

+ 3 - 0
hadoop-mapreduce-project/CHANGES.txt

@@ -410,6 +410,9 @@ Release 0.23.0 - Unreleased
     MAPREDUCE-3275. Added documentation for AM WebApp Proxy. (Robert Evans via
     MAPREDUCE-3275. Added documentation for AM WebApp Proxy. (Robert Evans via
     acmurthy)
     acmurthy)
 
 
+    MAPREDUCE-3322. Added a better index.html and an brief overview of YARN
+    architecture. (acmurthy) 
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     MAPREDUCE-2026. Make JobTracker.getJobCounters() and
     MAPREDUCE-2026. Make JobTracker.getJobCounters() and

+ 0 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/CapacityScheduler.apt.vm

@@ -22,7 +22,6 @@ Hadoop MapReduce Next Generation - Capacity Scheduler
 
 
 %{toc|section=1|fromDepth=0}
 %{toc|section=1|fromDepth=0}
 
 
-
 * {Purpose} 
 * {Purpose} 
 
 
   This document describes the <<<CapacityScheduler>>>, a pluggable scheduler 
   This document describes the <<<CapacityScheduler>>>, a pluggable scheduler 

+ 2 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/SingleCluster.apt.vm

@@ -20,6 +20,8 @@ Hadoop MapReduce Next Generation - Setting up a Single Node Cluster.
 
 
   \[ {{{./index.html}Go Back}} \]
   \[ {{{./index.html}Go Back}} \]
 
 
+%{toc|section=1|fromDepth=0}
+
 * Mapreduce Tarball
 * Mapreduce Tarball
 
 
   You should be able to obtain the MapReduce tarball from the release.
   You should be able to obtain the MapReduce tarball from the release.

+ 1 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/WritingYarnApplications.apt.vm

@@ -21,7 +21,7 @@ Hadoop MapReduce Next Generation - Writing YARN Applications
 
 
   \[ {{{./index.html}Go Back}} \]
   \[ {{{./index.html}Go Back}} \]
 
 
-%{toc|section=1|fromDepth=1}
+%{toc|section=1|fromDepth=0}
 
 
 * Purpose 
 * Purpose 
 
 

+ 77 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/YARN.apt.vm

@@ -0,0 +1,77 @@
+~~ 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.
+
+  ---
+  YARN
+  ---
+  ---
+  ${maven.build.timestamp}
+
+Apache Hadoop NextGen MapReduce (YARN)
+
+  MapReduce has undergone a complete overhaul in hadoop-0.23 and we now have, 
+  what we call, MapReduce 2.0 (MRv2) or YARN.
+
+  The fundamental idea of MRv2 is to split up the two major functionalities of 
+  the JobTracker, resource management and job scheduling/monitoring, into 
+  separate daemons. The idea is to have a global ResourceManager (<RM>) and 
+  per-application ApplicationMaster (<AM>).  An application is either a single 
+  job in the classical sense of Map-Reduce jobs or a DAG of jobs. 
+
+  The ResourceManager and per-node slave, the NodeManager (<NM>), form the 
+  data-computation framework. The ResourceManager is the ultimate authority that 
+  arbitrates resources among all the applications in the system. 
+
+  The per-application ApplicationMaster is, in effect, a framework specific 
+  library and is tasked with negotiating resources from the ResourceManager and 
+  working with the NodeManager(s) to execute and monitor the tasks.
+
+[./yarn_architecture.gif] MapReduce NextGen Architecture
+
+  The ResourceManager has two main components: Scheduler and 
+  ApplicationsManager.
+
+  The Scheduler is responsible for allocating resources to the various running 
+  applications subject to familiar constraints of capacities, queues etc. The 
+  Scheduler is pure scheduler in the sense that it performs no monitoring or 
+  tracking of status for the application. Also, it offers no guarantees about 
+  restarting failed tasks either due to application failure or hardware 
+  failures. The Scheduler performs its scheduling function based the resource 
+  requirements of the applications; it does so based on the abstract notion of 
+  a resource <Container> which incorporates elements such as memory, cpu, disk, 
+  network etc. In the first version, only <<<memory>>> is supported. 
+
+  The Scheduler has a pluggable policy plug-in, which is responsible for 
+  partitioning the cluster resources among the various queues, applications etc. 
+  The current Map-Reduce schedulers such as the CapacityScheduler and the 
+  FairScheduler would be some examples of the plug-in. 
+
+  The CapacityScheduler supports <<<hierarchical queues>>> to allow for more 
+  predictable sharing of cluster resources
+
+  The ApplicationsManager is responsible for accepting job-submissions, 
+  negotiating the first container for executing the application specific 
+  ApplicationMaster and provides the service for restarting the
+  ApplicationMaster container on failure.
+
+  The NodeManager is the per-machine framework agent who is responsible for 
+  containers, monitoring their resource usage (cpu, memory, disk, network) and 
+  reporting the same to the ResourceManager/Scheduler.
+
+  The per-application ApplicationMaster has the responsibility of negotiating 
+  appropriate resource containers from the Scheduler, tracking their status and 
+  monitoring for progress.
+
+  MRV2 maintains <<API compatibility>> with previous stable release 
+  (hadoop-0.20.205).  This means that all Map-Reduce jobs should still run 
+  unchanged on top of MRv2 with just a recompile.
+

+ 39 - 11
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/index.apt.vm

@@ -11,14 +11,32 @@
 ~~ limitations under the License. See accompanying LICENSE file.
 ~~ limitations under the License. See accompanying LICENSE file.
 
 
   ---
   ---
-  Hadoop MapReduce Next Generation  ${project.version}
+  Apache Hadoop 0.23
   ---
   ---
   ---
   ---
   ${maven.build.timestamp}
   ${maven.build.timestamp}
 
 
-Hadoop MapReduce Next Generation
+Apache Hadoop 0.23
 
 
-* Architecture
+  Apache Hadoop 0.23 consists of significant improvements over the previous 
+  stable release (hadoop-0.20.205).
+
+  Here is a short overview of the improvments to both HDFS and MapReduce.
+
+* {HDFS Federation}
+
+  In order to scale the name service horizontally, <federation> uses multiple 
+  independent Namenodes/Namespaces. The Namenodes are federated, that is, the 
+  Namenodes are independent and don't require coordination with each other. 
+  The datanodes are used as common storage for blocks by all the Namenodes. 
+  Each datanode registers with all the Namenodes in the cluster. Datanodes 
+  send periodic heartbeats and block reports and handles commands from the 
+  Namenodes.
+
+  More details are available in the {{{./Federation.html}HDFS Federation}}
+  document.
+  
+* {MapReduce NextGen aka YARN aka MRv2}
 
 
   The new architecture introduced in hadoop-0.23, divides the two major 
   The new architecture introduced in hadoop-0.23, divides the two major 
   functions of the JobTracker: resource management and job life-cycle management 
   functions of the JobTracker: resource management and job life-cycle management 
@@ -32,22 +50,32 @@ Hadoop MapReduce Next Generation
   or a DAG of such jobs. 
   or a DAG of such jobs. 
   
   
   The ResourceManager and per-machine NodeManager daemon, which manages the 
   The ResourceManager and per-machine NodeManager daemon, which manages the 
-  user processes on that machine, form the computation fabric. The 
-  per-application ApplicationMaster is, in effect, a framework specific library 
-  and is tasked with negotiating resources from the ResourceManager and working 
-  with the NodeManager(s) to execute and monitor the tasks.
+  user processes on that machine, form the computation fabric. 
+  
+  The per-application ApplicationMaster is, in effect, a framework specific 
+  library and is tasked with negotiating resources from the ResourceManager and 
+  working with the NodeManager(s) to execute and monitor the tasks.
 
 
-* User Documentation
+  More details are available in the {{{./YARN.html}YARN}}
+  document.
+
+* Release Documentation
 
 
   * {{{./SingleCluster.html}Setting up a Single Node Cluster}}
   * {{{./SingleCluster.html}Setting up a Single Node Cluster}}
 
 
   * {{{./ClusterSetup.html}Setting up a full-fledged Hadoop Cluster}}
   * {{{./ClusterSetup.html}Setting up a full-fledged Hadoop Cluster}}
   
   
-  * {{{./apidocs/index.html}JavaDocs}}
+  * {{{./CapacityScheduler.html}Capacity Scheduler}}
+
+  * {{{./Federation.html}HDFS Federation feature description, configuration and 
+    management}}
+
+  * {{{./YARN.html}NextGen MapReduce}}
+
+* User Documentation
 
 
   * {{{./WritingYarnApplications.html}Writing Yarn Applications}}
   * {{{./WritingYarnApplications.html}Writing Yarn Applications}}
 
 
-  * {{{./CapacityScheduler.html}Capacity Scheduler}}
+  * {{{./apidocs/index.html}JavaDocs}}
 
 
-  * {{{./Federation.html}HDFS Federation feature description, configuration and management}}
 
 

BIN
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/resources/yarn_architecture.gif


+ 1 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/site.xml

@@ -11,7 +11,7 @@
  See the License for the specific language governing permissions and
  See the License for the specific language governing permissions and
  limitations under the License. See accompanying LICENSE file.
  limitations under the License. See accompanying LICENSE file.
 -->
 -->
-<project name="Hadoop MapReduce Next Gen">
+<project name="Apache Hadoop 0.23">
 
 
   <skin>
   <skin>
     <groupId>org.apache.maven.skins</groupId>
     <groupId>org.apache.maven.skins</groupId>