Browse Source

svn merge -r 1097011:1099333 from branch-0.20-security-203.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.20-security-204@1101640 13f79535-47bb-0310-9956-ffa450edef68
Owen O'Malley 14 years ago
parent
commit
f923f55563
35 changed files with 2771 additions and 324 deletions
  1. 34 3
      CHANGES.txt
  2. 0 0
      conf/hadoop-metrics2.properties
  3. 4 3
      conf/log4j.properties
  4. 1 1
      ivy/libraries.properties
  5. 1 1
      src/ant/org/apache/hadoop/ant/condition/DfsBaseConditional.java
  6. 7 1
      src/c++/libhdfs/hdfs.c
  7. 0 2
      src/c++/libhdfs/hdfsJniHelper.h
  8. 2 1
      src/contrib/build-contrib.xml
  9. 2 2
      src/contrib/eclipse-plugin/src/java/org/apache/hadoop/eclipse/launch/HadoopApplicationLaunchShortcut.java
  10. 2 2
      src/contrib/eclipse-plugin/src/java/org/apache/hadoop/eclipse/servers/RunOnHadoopWizard.java
  11. 59 45
      src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/FairSchedulerServlet.java
  12. 7 1
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/PipeMapRed.java
  13. 11 2
      src/contrib/streaming/src/test/org/apache/hadoop/streaming/StderrApp.java
  14. 101 0
      src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingStatus.java
  15. 4 3
      src/core/org/apache/hadoop/conf/Configuration.java
  16. 882 0
      src/core/org/apache/hadoop/fs/HarFileSystem.java
  17. 8 0
      src/core/org/apache/hadoop/io/compress/CodecPool.java
  18. 35 0
      src/core/org/apache/hadoop/io/compress/DoNotPool.java
  19. 2 0
      src/core/org/apache/hadoop/io/compress/zlib/BuiltInGzipDecompressor.java
  20. 2 2
      src/core/overview.html
  21. 456 198
      src/docs/src/documentation/content/xdocs/capacity_scheduler.xml
  22. 2 2
      src/docs/src/documentation/content/xdocs/fair_scheduler.xml
  23. 2 2
      src/docs/src/documentation/content/xdocs/single_node_setup.xml
  24. 1 1
      src/examples/org/apache/hadoop/examples/AggregateWordCount.java
  25. 1 1
      src/examples/org/apache/hadoop/examples/AggregateWordHistogram.java
  26. 2 2
      src/examples/org/apache/hadoop/examples/dancing/package.html
  27. 1 1
      src/examples/org/apache/hadoop/examples/terasort/TeraGen.java
  28. 1 1
      src/examples/org/apache/hadoop/examples/terasort/TeraSort.java
  29. 1 1
      src/examples/org/apache/hadoop/examples/terasort/TeraValidate.java
  30. 32 9
      src/mapred/org/apache/hadoop/mapred/lib/CombineFileInputFormat.java
  31. 240 0
      src/test/org/apache/hadoop/fs/TestHarFileSystem.java
  32. 164 0
      src/test/org/apache/hadoop/io/compress/TestCodec.java
  33. 12 22
      src/test/org/apache/hadoop/mapred/lib/TestCombineFileInputFormat.java
  34. 0 15
      src/test/org/apache/hadoop/metrics2/impl/TestMetricsConfig.java
  35. 692 0
      src/tools/org/apache/hadoop/tools/HadoopArchives.java

+ 34 - 3
CHANGES.txt

@@ -40,9 +40,6 @@ Release 0.20.204.0 - unreleased
     HDFS-1258. Clearing namespace quota on "/" corrupts fs image.  
     (Aaron T. Myers via szetszwo)
 
-    HADOOP-7215. RPC clients must use network interface corresponding to 
-    the host in the client's kerberos principal key. (suresh)
-
     HDFS-1189. Quota counts missed between clear quota and set quota.
     (John George via szetszwo)
 
@@ -75,6 +72,26 @@ Release 0.20.204.0 - unreleased
 
 Release 0.20.203.0 - unreleased
 
+    MAPREDUCE-1280. Update Eclipse plugin to the new eclipse.jdt API.
+    (Alex Kozlov via szetszwo)
+
+    HADOOP-7259. Contrib modules should include the build.properties from
+    the enclosing hadoop directory. (omalley)
+
+    HADOOP-7253. Update the default configuration to fix security audit log
+    and metrics2 property configuration warnings. (omalley)
+
+    HADOOP-7247. Update documentation to match current jar names. (omalley)
+
+    HADOOP-7246. Update the log4j configuration to match the EventCounter
+    package. (Luke Lu via omalley)
+
+    HADOOP-7143. Restore HadoopArchives. (Joep Rottinghuis via omalley)
+
+    MAPREDUCE-2316. Updated CapacityScheduler documentation. (acmurthy) 
+
+    HADOOP-7243. Fix contrib unit tests missing dependencies. (omalley)
+
     HADOOP-7190. Add metrics v1 back for backwards compatibility. (omalley)
 
     MAPREDUCE-2360. Remove stripping of scheme, authority from submit dir in 
@@ -125,8 +142,13 @@ Release 0.20.203.0 - unreleased
     HADOOP-6879. Provide SSH based (Jsch) remote execution API for system
     tests. (cos)
 
+    HADOOP-7215. RPC clients must use network interface corresponding to 
+    the host in the client's kerberos principal key. (suresh)
+
     HADOOP-7232. Fix Javadoc warnings. (omalley)
 
+    HADOOP-7258. The Gzip codec should not return null decompressors. (omalley)
+
 Release 0.20.202.0 - unreleased
 
     MAPREDUCE-2355. Add a configuration knob 
@@ -1764,6 +1786,15 @@ Release 0.20.2 - Unreleased
     MAPREDUCE-1163. Remove unused, hard-coded paths from libhdfs. (Allen
     Wittenauer via cdouglas)
 
+    HADOOP-6315. Avoid incorrect use of BuiltInflater/BuiltInDeflater in
+    GzipCodec. (Aaron Kimball via cdouglas)
+
+    HADOOP-6269. Fix threading issue with defaultResource in Configuration.
+    (Sreekanth Ramakrishnan via cdouglas)
+
+    HADOOP-5759. Fix for  IllegalArgumentException when CombineFileInputFormat
+    is used as job InputFormat. (Amareshwari Sriramadasu via zshao)
+
 Release 0.20.1 - 2009-09-01
 
   INCOMPATIBLE CHANGES

+ 0 - 0
conf/hadoop-metrics2.properties.example → conf/hadoop-metrics2.properties


+ 4 - 3
conf/log4j.properties

@@ -79,7 +79,8 @@ log4j.appender.DRFAS.File=${hadoop.log.dir}/${hadoop.security.log.file}
 log4j.appender.DRFAS.layout=org.apache.log4j.PatternLayout
 log4j.appender.DRFAS.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
 #new logger
-log4j.category.SecurityLogger=INFO,DRFAS
+log4j.logger.SecurityLogger=OFF,console
+log4j.logger.SecurityLogger.additivity=false
 
 #
 # Rolling File Appender
@@ -100,7 +101,7 @@ log4j.category.SecurityLogger=INFO,DRFAS
 # FSNamesystem Audit logging
 # All audit events are logged at INFO level
 #
-log4j.logger.org.apache.hadoop.fs.FSNamesystem.audit=WARN
+log4j.logger.org.apache.hadoop.hdfs.server.namenode.FSNamesystem.audit=WARN
 
 # Custom Logging levels
 
@@ -117,7 +118,7 @@ log4j.logger.org.jets3t.service.impl.rest.httpclient.RestS3Service=ERROR
 # Event Counter Appender
 # Sends counts of logging messages at different severity levels to Hadoop Metrics.
 #
-log4j.appender.EventCounter=org.apache.hadoop.log.EventCounter
+log4j.appender.EventCounter=org.apache.hadoop.log.metrics.EventCounter
 
 #
 # Job Summary Appender

+ 1 - 1
ivy/libraries.properties

@@ -14,7 +14,7 @@
 #It drives ivy and the generation of a maven POM
 
 # This is the version of hadoop we are generating
-hadoop.version=0.20.0
+hadoop.version=0.20.203.0
 hadoop-gpl-compression.version=0.1.0
 
 #These are the versions of our dependencies (in alphabetical order)

+ 1 - 1
src/ant/org/apache/hadoop/ant/condition/DfsBaseConditional.java

@@ -56,7 +56,7 @@ public abstract class DfsBaseConditional extends org.apache.hadoop.ant.DfsTask
 
   protected int postCmd(int exit_code) {
     exit_code = super.postCmd(exit_code);
-    result = exit_code == 1;
+    result = exit_code == 0;
     return exit_code;
   }
 

+ 7 - 1
src/c++/libhdfs/hdfs.c

@@ -2022,12 +2022,18 @@ hdfsFileInfo *hdfsGetPathInfo(hdfsFS fs, const char* path)
 
 void hdfsFreeFileInfo(hdfsFileInfo *hdfsFileInfo, int numEntries)
 {
-    //Free the mName
+    //Free the mName, mOwner, and mGroup
     int i;
     for (i=0; i < numEntries; ++i) {
         if (hdfsFileInfo[i].mName) {
             free(hdfsFileInfo[i].mName);
         }
+        if (hdfsFileInfo[i].mOwner) {
+            free(hdfsFileInfo[i].mOwner);
+        }
+        if (hdfsFileInfo[i].mGroup) {
+            free(hdfsFileInfo[i].mGroup);
+        }
     }
 
     //Free entire block

+ 0 - 2
src/c++/libhdfs/hdfsJniHelper.h

@@ -30,8 +30,6 @@
 
 #define PATH_SEPARATOR ':'
 
-#define USER_CLASSPATH "/home/y/libexec/hadoop/conf:/home/y/libexec/hadoop/lib/hadoop-0.1.0.jar"
-
 
 /** Denote the method we want to invoke as STATIC or INSTANCE */
 typedef enum {

+ 2 - 1
src/contrib/build-contrib.xml

@@ -23,13 +23,14 @@
 
   <property name="name" value="${ant.project.name}"/>
   <property name="root" value="${basedir}"/>
+  <property name="hadoop.root" location="${root}/../../../"/>
 
   <!-- Load all the default properties, and any the user wants    -->
   <!-- to contribute (without having to type -D or edit this file -->
   <property file="${user.home}/${name}.build.properties" />
   <property file="${root}/build.properties" />
+  <property file="${hadoop.root}/build.properties" />
 
-  <property name="hadoop.root" location="${root}/../../../"/>
   <property name="src.dir"  location="${root}/src/java"/>
   <property name="src.test" location="${root}/src/test"/>
   <!-- Property added for contrib system tests -->

+ 2 - 2
src/contrib/eclipse-plugin/src/java/org/apache/hadoop/eclipse/launch/HadoopApplicationLaunchShortcut.java

@@ -32,7 +32,7 @@ import org.eclipse.debug.core.ILaunchConfigurationWorkingCopy;
 import org.eclipse.jdt.core.IJavaProject;
 import org.eclipse.jdt.core.IType;
 import org.eclipse.jdt.core.JavaCore;
-import org.eclipse.jdt.internal.debug.ui.launcher.JavaApplicationLaunchShortcut;
+import org.eclipse.jdt.debug.ui.launchConfigurations.JavaApplicationLaunchShortcut;
 import org.eclipse.jdt.launching.IJavaLaunchConfigurationConstants;
 import org.eclipse.jdt.launching.IRuntimeClasspathEntry;
 import org.eclipse.jdt.launching.JavaRuntime;
@@ -64,7 +64,7 @@ public class HadoopApplicationLaunchShortcut extends
     // Find an existing or create a launch configuration (Standard way)
     ILaunchConfiguration iConf =
         super.findLaunchConfiguration(type, configType);
-
+    if (iConf == null) iConf = super.createConfiguration(type);
     ILaunchConfigurationWorkingCopy iConfWC;
     try {
       /*

+ 2 - 2
src/contrib/eclipse-plugin/src/java/org/apache/hadoop/eclipse/servers/RunOnHadoopWizard.java

@@ -159,9 +159,9 @@ public class RunOnHadoopWizard extends Wizard {
 
     // Write it to the disk file
     try {
-      // File confFile = File.createTempFile("hadoop-site-", ".xml",
+      // File confFile = File.createTempFile("core-site-", ".xml",
       // confDir);
-      File confFile = new File(confDir, "hadoop-site.xml");
+      File confFile = new File(confDir, "core-site.xml");
       FileOutputStream fos = new FileOutputStream(confFile);
       conf.writeXml(fos);
       fos.close();

+ 59 - 45
src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/FairSchedulerServlet.java

@@ -18,7 +18,9 @@
 
 package org.apache.hadoop.mapred;
 
+import java.io.ByteArrayOutputStream;
 import java.io.IOException;
+import java.io.OutputStream;
 import java.io.PrintWriter;
 import java.text.DateFormat;
 import java.text.SimpleDateFormat;
@@ -122,7 +124,12 @@ public class FairSchedulerServlet extends HttpServlet {
     }
     // Print out the normal response
     response.setContentType("text/html");
-    PrintWriter out = new PrintWriter(response.getOutputStream());
+
+    // Because the client may read arbitrarily slow, and we hold locks while
+    // the servlet output, we want to write to our own buffer which we know
+    // won't block.
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    PrintWriter out = new PrintWriter(baos);
     String hostname = StringUtils.simpleHostname(
         jobTracker.getJobTrackerMachine());
     out.print("<html><head>");
@@ -137,6 +144,11 @@ public class FairSchedulerServlet extends HttpServlet {
     showAdminForm(out, advancedView);
     out.print("</body></html>\n");
     out.close();
+
+    // Flush our buffer to the real servlet output
+    OutputStream servletOut = response.getOutputStream();
+    baos.writeTo(servletOut);
+    servletOut.close();
   }
 
   /**
@@ -206,51 +218,53 @@ public class FairSchedulerServlet extends HttpServlet {
     out.print("<th>Finished</th><th>Running</th><th>Fair Share</th>" +
         (advancedView ? "<th>Weight</th><th>Deficit</th><th>minReduces</th>" : ""));
     out.print("</tr>\n");
-    Collection<JobInProgress> runningJobs = jobTracker.getRunningJobs();
-    synchronized (scheduler) {
-      for (JobInProgress job: runningJobs) {
-        JobProfile profile = job.getProfile();
-        JobInfo info = scheduler.infos.get(job);
-        if (info == null) { // Job finished, but let's show 0's for info
-          info = new JobInfo();
-        }
-        out.print("<tr>\n");
-        out.printf("<td>%s</td>\n", DATE_FORMAT.format(
-            new Date(job.getStartTime())));
-        out.printf("<td><a href=\"jobdetails.jsp?jobid=%s\">%s</a></td>",
-            profile.getJobID(), profile.getJobID());
-        out.printf("<td>%s</td>\n", profile.getUser());
-        out.printf("<td>%s</td>\n", profile.getJobName());
-        out.printf("<td>%s</td>\n", generateSelect(
-            scheduler.getPoolManager().getPoolNames(),
-            scheduler.getPoolManager().getPoolName(job),
-            "/scheduler?setPool=<CHOICE>&jobid=" + profile.getJobID() +
-            (advancedView ? "&advanced" : "")));
-        out.printf("<td>%s</td>\n", generateSelect(
-            Arrays.asList(new String[]
-                {"VERY_LOW", "LOW", "NORMAL", "HIGH", "VERY_HIGH"}),
-            job.getPriority().toString(),
-            "/scheduler?setPriority=<CHOICE>&jobid=" + profile.getJobID() +
-            (advancedView ? "&advanced" : "")));
-        out.printf("<td>%d / %d</td><td>%d</td><td>%8.1f</td>\n",
-            job.finishedMaps(), job.desiredMaps(), info.runningMaps,
-            info.mapFairShare);
-        if (advancedView) {
-          out.printf("<td>%8.1f</td>\n", info.mapWeight);
-          out.printf("<td>%s</td>\n", info.neededMaps > 0 ?
-              (info.mapDeficit / 1000) + "s" : "--");
-          out.printf("<td>%d</td>\n", info.minMaps);
-        }
-        out.printf("<td>%d / %d</td><td>%d</td><td>%8.1f</td>\n",
-            job.finishedReduces(), job.desiredReduces(), info.runningReduces,
-            info.reduceFairShare);
-        if (advancedView) {
-          out.printf("<td>%8.1f</td>\n", info.reduceWeight);
-          out.printf("<td>%s</td>\n", info.neededReduces > 0 ?
-              (info.reduceDeficit / 1000) + "s" : "--");
-          out.printf("<td>%d</td>\n", info.minReduces);
+    synchronized (jobTracker) {
+      Collection<JobInProgress> runningJobs = jobTracker.getRunningJobs();
+      synchronized (scheduler) {
+        for (JobInProgress job: runningJobs) {
+          JobProfile profile = job.getProfile();
+          JobInfo info = scheduler.infos.get(job);
+          if (info == null) { // Job finished, but let's show 0's for info
+            info = new JobInfo();
+          }
+          out.print("<tr>\n");
+          out.printf("<td>%s</td>\n", DATE_FORMAT.format(
+                       new Date(job.getStartTime())));
+          out.printf("<td><a href=\"jobdetails.jsp?jobid=%s\">%s</a></td>",
+                     profile.getJobID(), profile.getJobID());
+          out.printf("<td>%s</td>\n", profile.getUser());
+          out.printf("<td>%s</td>\n", profile.getJobName());
+          out.printf("<td>%s</td>\n", generateSelect(
+                       scheduler.getPoolManager().getPoolNames(),
+                       scheduler.getPoolManager().getPoolName(job),
+                       "/scheduler?setPool=<CHOICE>&jobid=" + profile.getJobID() +
+                       (advancedView ? "&advanced" : "")));
+          out.printf("<td>%s</td>\n", generateSelect(
+                       Arrays.asList(new String[]
+                         {"VERY_LOW", "LOW", "NORMAL", "HIGH", "VERY_HIGH"}),
+                       job.getPriority().toString(),
+                       "/scheduler?setPriority=<CHOICE>&jobid=" + profile.getJobID() +
+                       (advancedView ? "&advanced" : "")));
+          out.printf("<td>%d / %d</td><td>%d</td><td>%8.1f</td>\n",
+                     job.finishedMaps(), job.desiredMaps(), info.runningMaps,
+                     info.mapFairShare);
+          if (advancedView) {
+            out.printf("<td>%8.1f</td>\n", info.mapWeight);
+            out.printf("<td>%s</td>\n", info.neededMaps > 0 ?
+                       (info.mapDeficit / 1000) + "s" : "--");
+            out.printf("<td>%d</td>\n", info.minMaps);
+          }
+          out.printf("<td>%d / %d</td><td>%d</td><td>%8.1f</td>\n",
+                     job.finishedReduces(), job.desiredReduces(), info.runningReduces,
+                     info.reduceFairShare);
+          if (advancedView) {
+            out.printf("<td>%8.1f</td>\n", info.reduceWeight);
+            out.printf("<td>%s</td>\n", info.neededReduces > 0 ?
+                       (info.reduceDeficit / 1000) + "s" : "--");
+            out.printf("<td>%d</td>\n", info.minReduces);
+          }
+          out.print("</tr>\n");
         }
-        out.print("</tr>\n");
       }
     }
     out.print("</table>\n");

+ 7 - 1
src/contrib/streaming/src/java/org/apache/hadoop/streaming/PipeMapRed.java

@@ -385,7 +385,11 @@ public abstract class PipeMapRed {
           if (now-lastStdoutReport > reporterOutDelay_) {
             lastStdoutReport = now;
             String hline = "Records R/W=" + numRecRead_ + "/" + numRecWritten_;
-            reporter.setStatus(hline);
+            if (!processProvidedStatus_) {
+              reporter.setStatus(hline);
+            } else {
+              reporter.progress();
+            }
             logprintln(hline);
             logflush();
           }
@@ -446,6 +450,7 @@ public abstract class PipeMapRed {
             if (matchesCounter(lineStr)) {
               incrCounter(lineStr);
             } else if (matchesStatus(lineStr)) {
+              processProvidedStatus_ = true;
               setStatus(lineStr);
             } else {
               LOG.warn("Cannot parse reporter line: " + lineStr);
@@ -671,4 +676,5 @@ public abstract class PipeMapRed {
   String LOGNAME;
   PrintStream log_;
 
+  volatile boolean processProvidedStatus_ = false;
 }

+ 11 - 2
src/contrib/streaming/src/test/org/apache/hadoop/streaming/StderrApp.java

@@ -32,8 +32,16 @@ public class StderrApp
    * postWriteLines to stderr.
    */
   public static void go(int preWriteLines, int sleep, int postWriteLines) throws IOException {
+    go(preWriteLines, sleep, postWriteLines, false);
+  }
+  
+  public static void go(int preWriteLines, int sleep, int postWriteLines, boolean status) throws IOException {
     BufferedReader in = new BufferedReader(new InputStreamReader(System.in));
     String line;
+    
+    if (status) {
+      System.err.println("reporter:status:starting echo");
+    }      
        
     while (preWriteLines > 0) {
       --preWriteLines;
@@ -57,13 +65,14 @@ public class StderrApp
 
   public static void main(String[] args) throws IOException {
     if (args.length < 3) {
-      System.err.println("Usage: StderrApp PREWRITE SLEEP POSTWRITE");
+      System.err.println("Usage: StderrApp PREWRITE SLEEP POSTWRITE [STATUS]");
       return;
     }
     int preWriteLines = Integer.parseInt(args[0]);
     int sleep = Integer.parseInt(args[1]);
     int postWriteLines = Integer.parseInt(args[2]);
+    boolean status = args.length > 3 ? Boolean.parseBoolean(args[3]) : false;
     
-    go(preWriteLines, sleep, postWriteLines);
+    go(preWriteLines, sleep, postWriteLines, status);
   }
 }

+ 101 - 0
src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingStatus.java

@@ -0,0 +1,101 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.streaming;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.File;
+
+import junit.framework.TestCase;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.MiniMRCluster;
+import org.apache.hadoop.mapred.TaskReport;
+
+/**
+ * Tests for the ability of a streaming task to set the status
+ * by writing "reporter:status:" lines to stderr. Uses MiniMR
+ * since the local jobtracker doesn't track status.
+ */
+public class TestStreamingStatus extends TestCase {
+  private static String TEST_ROOT_DIR =
+    new File(System.getProperty("test.build.data","/tmp"))
+    .toURI().toString().replace(' ', '+');
+  protected String INPUT_FILE = TEST_ROOT_DIR + "/input.txt";
+  protected String OUTPUT_DIR = TEST_ROOT_DIR + "/out";
+  protected String input = "roses.are.red\nviolets.are.blue\nbunnies.are.pink\n";
+  protected String map = StreamUtil.makeJavaCommand(StderrApp.class, new String[]{"3", "0", "0", "true"});
+
+  protected String[] genArgs(int jobtrackerPort) {
+    return new String[] {
+      "-input", INPUT_FILE,
+      "-output", OUTPUT_DIR,
+      "-mapper", map,
+      "-jobconf", "mapred.map.tasks=1",
+      "-jobconf", "mapred.reduce.tasks=0",      
+      "-jobconf", "keep.failed.task.files=true",
+      "-jobconf", "stream.tmpdir="+System.getProperty("test.build.data","/tmp"),
+      "-jobconf", "mapred.job.tracker=localhost:"+jobtrackerPort,
+      "-jobconf", "fs.default.name=file:///"
+    };
+  }
+  
+  public void makeInput(FileSystem fs) throws IOException {
+    Path inFile = new Path(INPUT_FILE);
+    DataOutputStream file = fs.create(inFile);
+    file.writeBytes(input);
+    file.close();
+  }
+
+  public void clean(FileSystem fs) {
+    try {
+      Path outDir = new Path(OUTPUT_DIR);
+      fs.delete(outDir, true);
+    } catch (Exception e) {}
+    try {
+      Path inFile = new Path(INPUT_FILE);    
+      fs.delete(inFile, false);
+    } catch (Exception e) {}
+  }
+  
+  public void testStreamingStatus() throws Exception {
+    MiniMRCluster mr = null;
+    FileSystem fs = null;
+    try {
+      mr = new MiniMRCluster(1, "file:///", 3);
+
+      Path inFile = new Path(INPUT_FILE);
+      fs = inFile.getFileSystem(mr.createJobConf());
+      clean(fs);
+      makeInput(fs);
+      
+      StreamJob job = new StreamJob();
+      int failed = job.run(genArgs(mr.getJobTrackerPort()));
+      assertEquals(0, failed);
+
+      TaskReport[] reports = job.jc_.getMapTaskReports(job.jobId_);
+      assertEquals(1, reports.length);
+      assertEquals("starting echo", reports[0].getState());
+    } finally {
+      if (fs != null) { clean(fs); }
+      if (mr != null) { mr.shutdown(); }
+    }
+  }
+}

+ 4 - 3
src/core/org/apache/hadoop/conf/Configuration.java

@@ -43,6 +43,7 @@ import java.util.Properties;
 import java.util.Set;
 import java.util.StringTokenizer;
 import java.util.WeakHashMap;
+import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
@@ -165,8 +166,8 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
    * List of default Resources. Resources are loaded in the order of the list 
    * entries
    */
-  private static final ArrayList<String> defaultResources = 
-    new ArrayList<String>();
+  private static final CopyOnWriteArrayList<String> defaultResources =
+    new CopyOnWriteArrayList<String>();
   
   /**
    * Flag to indicate if the storage of resource which updates a key needs 
@@ -1356,7 +1357,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     return sb.toString();
   }
 
-  private void toString(ArrayList resources, StringBuffer sb) {
+  private void toString(List resources, StringBuffer sb) {
     ListIterator i = resources.listIterator();
     while (i.hasNext()) {
       if (i.nextIndex() != 0) {

+ 882 - 0
src/core/org/apache/hadoop/fs/HarFileSystem.java

@@ -0,0 +1,882 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.util.LineReader;
+import org.apache.hadoop.util.Progressable;
+
+/**
+ * This is an implementation of the Hadoop Archive 
+ * Filesystem. This archive Filesystem has index files
+ * of the form _index* and has contents of the form
+ * part-*. The index files store the indexes of the 
+ * real files. The index files are of the form _masterindex
+ * and _index. The master index is a level of indirection 
+ * in to the index file to make the look ups faster. the index
+ * file is sorted with hash code of the paths that it contains 
+ * and the master index contains pointers to the positions in 
+ * index for ranges of hashcodes.
+ */
+
+public class HarFileSystem extends FilterFileSystem {
+  public static final int VERSION = 1;
+  // uri representation of this Har filesystem
+  private URI uri;
+  // the version of this har filesystem
+  private int version;
+  // underlying uri 
+  private URI underLyingURI;
+  // the top level path of the archive
+  // in the underlying file system
+  private Path archivePath;
+  // the masterIndex of the archive
+  private Path masterIndex;
+  // the index file 
+  private Path archiveIndex;
+  // the har auth
+  private String harAuth;
+  
+  /**
+   * public construction of harfilesystem
+   *
+   */
+  public HarFileSystem() {
+  }
+  
+  /**
+   * Constructor to create a HarFileSystem with an
+   * underlying filesystem.
+   * @param fs
+   */
+  public HarFileSystem(FileSystem fs) {
+    super(fs);
+  }
+  
+  /**
+   * Initialize a Har filesystem per har archive. The 
+   * archive home directory is the top level directory
+   * in the filesystem that contains the HAR archive.
+   * Be careful with this method, you do not want to go 
+   * on creating new Filesystem instances per call to 
+   * path.getFileSystem().
+   * the uri of Har is 
+   * har://underlyingfsscheme-host:port/archivepath.
+   * or 
+   * har:///archivepath. This assumes the underlying filesystem
+   * to be used in case not specified.
+   */
+  public void initialize(URI name, Configuration conf) throws IOException {
+    //decode the name
+    underLyingURI = decodeHarURI(name, conf);
+    //  we got the right har Path- now check if this is 
+    //truly a har filesystem
+    Path harPath = archivePath(new Path(name.toString()));
+    if (harPath == null) { 
+      throw new IOException("Invalid path for the Har Filesystem. " + 
+                           name.toString());
+    }
+    if (fs == null) {
+      fs = FileSystem.get(underLyingURI, conf);
+    }
+    this.uri = harPath.toUri();
+    this.archivePath = new Path(this.uri.getPath());
+    this.harAuth = getHarAuth(this.underLyingURI);
+    //check for the underlying fs containing
+    // the index file
+    this.masterIndex = new Path(archivePath, "_masterindex");
+    this.archiveIndex = new Path(archivePath, "_index");
+    if (!fs.exists(masterIndex) || !fs.exists(archiveIndex)) {
+      throw new IOException("Invalid path for the Har Filesystem. " +
+          "No index file in " + harPath);
+    }
+    try{ 
+      this.version = getHarVersion();
+    } catch(IOException io) {
+      throw new IOException("Unable to " +
+          "read the version of the Har file system: " + this.archivePath);
+    }
+    if (this.version != HarFileSystem.VERSION) {
+      throw new IOException("Invalid version " + 
+          this.version + " expected " + HarFileSystem.VERSION);
+    }
+  }
+  
+  // get the version of the filesystem from the masterindex file
+  // the version is currently not useful since its the first version 
+  // of archives
+  public int getHarVersion() throws IOException { 
+    FSDataInputStream masterIn = fs.open(masterIndex);
+    LineReader lmaster = new LineReader(masterIn, getConf());
+    Text line = new Text();
+    lmaster.readLine(line);
+    try {
+      masterIn.close();
+    } catch(IOException e){
+      //disregard it.
+      // its a read.
+    }
+    String versionLine = line.toString();
+    String[] arr = versionLine.split(" ");
+    int version = Integer.parseInt(arr[0]);
+    return version;
+  }
+  
+  /*
+   * find the parent path that is the 
+   * archive path in the path. The last
+   * path segment that ends with .har is 
+   * the path that will be returned.
+   */
+  private Path archivePath(Path p) {
+    Path retPath = null;
+    Path tmp = p;
+    for (int i=0; i< p.depth(); i++) {
+      if (tmp.toString().endsWith(".har")) {
+        retPath = tmp;
+        break;
+      }
+      tmp = tmp.getParent();
+    }
+    return retPath;
+  }
+
+  /**
+   * decode the raw URI to get the underlying URI
+   * @param rawURI raw Har URI
+   * @return filtered URI of the underlying fileSystem
+   */
+  private URI decodeHarURI(URI rawURI, Configuration conf) throws IOException {
+    String tmpAuth = rawURI.getAuthority();
+    //we are using the default file
+    //system in the config 
+    //so create a underlying uri and 
+    //return it
+    if (tmpAuth == null) {
+      //create a path 
+      return FileSystem.getDefaultUri(conf);
+    }
+    String host = rawURI.getHost();
+    String[] str = host.split("-", 2);
+    if (str[0] == null) {
+      throw new IOException("URI: " + rawURI + " is an invalid Har URI.");
+    }
+    String underLyingScheme = str[0];
+    String underLyingHost = (str.length > 1)? str[1]:null;
+    int underLyingPort = rawURI.getPort();
+    String auth = (underLyingHost == null && underLyingPort == -1)?
+                  null:(underLyingHost+":"+underLyingPort);
+    URI tmp = null;
+    if (rawURI.getQuery() != null) {
+      // query component not allowed
+      throw new IOException("query component in Path not supported  " + rawURI);
+    }
+    try {
+      tmp = new URI(underLyingScheme, auth, rawURI.getPath(), 
+            rawURI.getQuery(), rawURI.getFragment());
+    } catch (URISyntaxException e) {
+        // do nothing should not happen
+    }
+    return tmp;
+  }
+  
+  /**
+   * return the top level archive.
+   */
+  public Path getWorkingDirectory() {
+    return new Path(uri.toString());
+  }
+  
+  /**
+   * Create a har specific auth 
+   * har-underlyingfs:port
+   * @param underLyingURI the uri of underlying
+   * filesystem
+   * @return har specific auth
+   */
+  private String getHarAuth(URI underLyingUri) {
+    String auth = underLyingUri.getScheme() + "-";
+    if (underLyingUri.getHost() != null) {
+      auth += underLyingUri.getHost() + ":";
+      if (underLyingUri.getPort() != -1) {
+        auth +=  underLyingUri.getPort();
+      }
+    }
+    else {
+      auth += ":";
+    }
+    return auth;
+  }
+  
+  /**
+   * Returns the uri of this filesystem.
+   * The uri is of the form 
+   * har://underlyingfsschema-host:port/pathintheunderlyingfs
+   */
+  @Override
+  public URI getUri() {
+    return this.uri;
+  }
+  
+  /**
+   * this method returns the path 
+   * inside the har filesystem.
+   * this is relative path inside 
+   * the har filesystem.
+   * @param path the fully qualified path in the har filesystem.
+   * @return relative path in the filesystem.
+   */
+  private Path getPathInHar(Path path) {
+    Path harPath = new Path(path.toUri().getPath());
+    if (archivePath.compareTo(harPath) == 0)
+      return new Path(Path.SEPARATOR);
+    Path tmp = new Path(harPath.getName());
+    Path parent = harPath.getParent();
+    while (!(parent.compareTo(archivePath) == 0)) {
+      if (parent.toString().equals(Path.SEPARATOR)) {
+        tmp = null;
+        break;
+      }
+      tmp = new Path(parent.getName(), tmp);
+      parent = parent.getParent();
+    }
+    if (tmp != null) 
+      tmp = new Path(Path.SEPARATOR, tmp);
+    return tmp;
+  }
+  
+  //the relative path of p. basically 
+  // getting rid of /. Parsing and doing 
+  // string manipulation is not good - so
+  // just use the path api to do it.
+  private Path makeRelative(String initial, Path p) {
+    Path root = new Path(Path.SEPARATOR);
+    if (root.compareTo(p) == 0)
+      return new Path(initial);
+    Path retPath = new Path(p.getName());
+    Path parent = p.getParent();
+    for (int i=0; i < p.depth()-1; i++) {
+      retPath = new Path(parent.getName(), retPath);
+      parent = parent.getParent();
+    }
+    return new Path(initial, retPath.toString());
+  }
+  
+  /* this makes a path qualified in the har filesystem
+   * (non-Javadoc)
+   * @see org.apache.hadoop.fs.FilterFileSystem#makeQualified(
+   * org.apache.hadoop.fs.Path)
+   */
+  @Override
+  public Path makeQualified(Path path) {
+    // make sure that we just get the 
+    // path component 
+    Path fsPath = path;
+    if (!path.isAbsolute()) {
+      fsPath = new Path(archivePath, path);
+    }
+
+    URI tmpURI = fsPath.toUri();
+    //change this to Har uri 
+    return new Path(uri.getScheme(), harAuth, tmpURI.getPath());
+  }
+  
+  /**
+   * get block locations from the underlying fs
+   * @param file the input filestatus to get block locations
+   * @param start the start in the file
+   * @param len the length in the file
+   * @return block locations for this segment of file
+   * @throws IOException
+   */
+  @Override
+  public BlockLocation[] getFileBlockLocations(FileStatus file, long start,
+      long len) throws IOException {
+    // need to look up the file in the underlying fs
+    // look up the index 
+    
+    // make sure this is a prt of this har filesystem
+    Path p = makeQualified(file.getPath());
+    Path harPath = getPathInHar(p);
+    String line = fileStatusInIndex(harPath);
+    if (line == null)  {
+      throw new FileNotFoundException("File " + file.getPath() + " not found");
+    }
+    HarStatus harStatus = new HarStatus(line);
+    if (harStatus.isDir()) {
+      return new BlockLocation[0];
+    }
+    FileStatus fsFile = fs.getFileStatus(new Path(archivePath,
+        harStatus.getPartName()));
+    BlockLocation[] rawBlocks = fs.getFileBlockLocations(fsFile, 
+        harStatus.getStartIndex() + start, len);
+    return fakeBlockLocations(rawBlocks, harStatus.getStartIndex());
+  }
+  
+  /**
+   * fake the rawblocks since map reduce uses the block offsets to 
+   * fo some computations regarding the blocks
+   * @param rawBlocks the raw blocks returned by the filesystem
+   * @return faked blocks with changed offsets.
+   */
+  private BlockLocation[] fakeBlockLocations(BlockLocation[] rawBlocks, 
+		  long startIndex) {
+	for (BlockLocation block : rawBlocks) {
+		long rawOffset = block.getOffset();
+		block.setOffset(rawOffset - startIndex);
+	}
+	return rawBlocks;
+  }
+  
+  /**
+   * the hash of the path p inside iniside
+   * the filesystem
+   * @param p the path in the harfilesystem
+   * @return the hash code of the path.
+   */
+  public static int getHarHash(Path p) {
+    return (p.toString().hashCode() & 0x7fffffff);
+  }
+  
+  static class Store {
+    public Store() {
+      begin = end = startHash = endHash = 0;
+    }
+    public Store(long begin, long end, int startHash, int endHash) {
+      this.begin = begin;
+      this.end = end;
+      this.startHash = startHash;
+      this.endHash = endHash;
+    }
+    public long begin;
+    public long end;
+    public int startHash;
+    public int endHash;
+  }
+  
+  // make sure that this harPath is relative to the har filesystem
+  // this only works for relative paths. This returns the line matching
+  // the file in the index. Returns a null if there is not matching 
+  // filename in the index file.
+  private String fileStatusInIndex(Path harPath) throws IOException {
+    // read the index file 
+    int hashCode = getHarHash(harPath);
+    // get the master index to find the pos 
+    // in the index file
+    FSDataInputStream in = fs.open(masterIndex);
+    FileStatus masterStat = fs.getFileStatus(masterIndex);
+    LineReader lin = new LineReader(in, getConf());
+    Text line = new Text();
+    long read = lin.readLine(line);
+   //ignore the first line. this is the header of the index files
+    String[] readStr = null;
+    List<Store> stores = new ArrayList<Store>();
+    while(read < masterStat.getLen()) {
+      int b = lin.readLine(line);
+      read += b;
+      readStr = line.toString().split(" ");
+      int startHash = Integer.parseInt(readStr[0]);
+      int endHash  = Integer.parseInt(readStr[1]);
+      if (startHash <= hashCode && hashCode <= endHash) {
+        stores.add(new Store(Long.parseLong(readStr[2]), 
+            Long.parseLong(readStr[3]), startHash,
+            endHash));
+      }
+      line.clear();
+    }
+    try {
+      lin.close();
+    } catch(IOException io){
+      // do nothing just a read.
+    }
+    FSDataInputStream aIn = fs.open(archiveIndex);
+    LineReader aLin;
+    String retStr = null;
+    // now start reading the real index file
+    for (Store s: stores) {
+      read = 0;
+      aIn.seek(s.begin);
+      aLin = new LineReader(aIn, getConf());
+      while (read + s.begin < s.end) {
+        int tmp = aLin.readLine(line);
+        read += tmp;
+        String lineFeed = line.toString();
+        String[] parsed = lineFeed.split(" ");
+        if (harPath.compareTo(new Path(parsed[0])) == 0) {
+          // bingo!
+          retStr = lineFeed;
+          break;
+        }
+        line.clear();
+      }
+      if (retStr != null)
+        break;
+    }
+    try {
+      aIn.close();
+    } catch(IOException io) {
+      //do nothing
+    }
+    return retStr;
+  }
+  
+  // a single line parser for hadoop archives status 
+  // stored in a single line in the index files 
+  // the format is of the form 
+  // filename "dir"/"file" partFileName startIndex length 
+  // <space seperated children>
+  private static class HarStatus {
+    boolean isDir;
+    String name;
+    List<String> children;
+    String partName;
+    long startIndex;
+    long length;
+    public HarStatus(String harString) {
+      String[] splits = harString.split(" ");
+      this.name = splits[0];
+      this.isDir = "dir".equals(splits[1]) ? true: false;
+      // this is equal to "none" if its a directory
+      this.partName = splits[2];
+      this.startIndex = Long.parseLong(splits[3]);
+      this.length = Long.parseLong(splits[4]);
+      if (isDir) {
+        children = new ArrayList<String>();
+        for (int i = 5; i < splits.length; i++) {
+          children.add(splits[i]);
+        }
+      }
+    }
+    public boolean isDir() {
+      return isDir;
+    }
+    
+    public String getName() {
+      return name;
+    }
+    
+    public List<String> getChildren() {
+      return children;
+    }
+    public String getFileName() {
+      return name;
+    }
+    public String getPartName() {
+      return partName;
+    }
+    public long getStartIndex() {
+      return startIndex;
+    }
+    public long getLength() {
+      return length;
+    }
+  }
+  
+  /**
+   * return the filestatus of files in har archive.
+   * The permission returned are that of the archive
+   * index files. The permissions are not persisted 
+   * while creating a hadoop archive.
+   * @param f the path in har filesystem
+   * @return filestatus.
+   * @throws IOException
+   */
+  @Override
+  public FileStatus getFileStatus(Path f) throws IOException {
+    FileStatus archiveStatus = fs.getFileStatus(archiveIndex);
+    // get the fs DataInputStream for the underlying file
+    // look up the index.
+    Path p = makeQualified(f);
+    Path harPath = getPathInHar(p);
+    if (harPath == null) {
+      throw new IOException("Invalid file name: " + f + " in " + uri);
+    }
+    String readStr = fileStatusInIndex(harPath);
+    if (readStr == null) {
+      throw new FileNotFoundException("File: " +  f + " does not exist in " + uri);
+    }
+    HarStatus hstatus = null;
+    hstatus = new HarStatus(readStr);
+    return new FileStatus(hstatus.isDir()?0:hstatus.getLength(), hstatus.isDir(),
+        (int)archiveStatus.getReplication(), archiveStatus.getBlockSize(),
+        archiveStatus.getModificationTime(), archiveStatus.getAccessTime(),
+        new FsPermission(
+        archiveStatus.getPermission()), archiveStatus.getOwner(), 
+        archiveStatus.getGroup(), 
+            makeRelative(this.uri.toString(), new Path(hstatus.name)));
+  }
+
+  /**
+   * Returns a har input stream which fakes end of 
+   * file. It reads the index files to get the part 
+   * file name and the size and start of the file.
+   */
+  @Override
+  public FSDataInputStream open(Path f, int bufferSize) throws IOException {
+    // get the fs DataInputStream for the underlying file
+    // look up the index.
+    Path p = makeQualified(f);
+    Path harPath = getPathInHar(p);
+    if (harPath == null) {
+      throw new IOException("Invalid file name: " + f + " in " + uri);
+    }
+    String readStr = fileStatusInIndex(harPath);
+    if (readStr == null) {
+      throw new FileNotFoundException(f + ": not found in " + archivePath);
+    }
+    HarStatus hstatus = new HarStatus(readStr); 
+    // we got it.. woo hooo!!! 
+    if (hstatus.isDir()) {
+      throw new FileNotFoundException(f + " : not a file in " +
+                archivePath);
+    }
+    return new HarFSDataInputStream(fs, new Path(archivePath, 
+        hstatus.getPartName()),
+        hstatus.getStartIndex(), hstatus.getLength(), bufferSize);
+  }
+ 
+  /*
+   * create throws an exception in Har filesystem.
+   * The archive once created cannot be changed.
+   */
+  public FSDataOutputStream create(Path f, int bufferSize) 
+                                    throws IOException {
+    throw new IOException("Har: Create not allowed");
+  }
+  
+  public FSDataOutputStream create(Path f,
+      FsPermission permission,
+      boolean overwrite,
+      int bufferSize,
+      short replication,
+      long blockSize,
+      Progressable progress) throws IOException {
+    throw new IOException("Har: create not allowed.");
+  }
+  
+  @Override
+  public void close() throws IOException {
+    if (fs != null) {
+      try {
+        fs.close();
+      } catch(IOException ie) {
+        //this might already be closed
+        // ignore
+      }
+    }
+  }
+  
+  /**
+   * Not implemented.
+   */
+  @Override
+  public boolean setReplication(Path src, short replication) throws IOException{
+    throw new IOException("Har: setreplication not allowed");
+  }
+  
+  /**
+   * Not implemented.
+   */
+  @Override
+  public boolean delete(Path f, boolean recursive) throws IOException { 
+    throw new IOException("Har: delete not allowed");
+  }
+  
+  /**
+   * liststatus returns the children of a directory 
+   * after looking up the index files.
+   */
+  @Override
+  public FileStatus[] listStatus(Path f) throws IOException {
+    //need to see if the file is an index in file
+    //get the filestatus of the archive directory
+    // we will create fake filestatuses to return
+    // to the client
+    List<FileStatus> statuses = new ArrayList<FileStatus>();
+    FileStatus archiveStatus = fs.getFileStatus(archiveIndex);
+    Path tmpPath = makeQualified(f);
+    Path harPath = getPathInHar(tmpPath);
+    String readStr = fileStatusInIndex(harPath);
+    if (readStr == null) {
+      throw new FileNotFoundException("File " + f + " not found in " + archivePath);
+    }
+    HarStatus hstatus = new HarStatus(readStr);
+    if (!hstatus.isDir()) 
+        statuses.add(new FileStatus(hstatus.getLength(), 
+            hstatus.isDir(),
+            archiveStatus.getReplication(), archiveStatus.getBlockSize(),
+            archiveStatus.getModificationTime(), archiveStatus.getAccessTime(),
+            new FsPermission(archiveStatus.getPermission()),
+            archiveStatus.getOwner(), archiveStatus.getGroup(), 
+            makeRelative(this.uri.toString(), new Path(hstatus.name))));
+    else 
+      for (String child: hstatus.children) {
+        FileStatus tmp = getFileStatus(new Path(tmpPath, child));
+        statuses.add(tmp);
+      }
+    return statuses.toArray(new FileStatus[statuses.size()]);
+  }
+  
+  /**
+   * return the top level archive path.
+   */
+  public Path getHomeDirectory() {
+    return new Path(uri.toString());
+  }
+  
+  public void setWorkingDirectory(Path newDir) {
+    //does nothing.
+  }
+  
+  /**
+   * not implemented.
+   */
+  public boolean mkdirs(Path f, FsPermission permission) throws IOException {
+    throw new IOException("Har: mkdirs not allowed");
+  }
+  
+  /**
+   * not implemented.
+   */
+  public void copyFromLocalFile(boolean delSrc, Path src, Path dst) throws 
+        IOException {
+    throw new IOException("Har: copyfromlocalfile not allowed");
+  }
+  
+  /**
+   * copies the file in the har filesystem to a local file.
+   */
+  public void copyToLocalFile(boolean delSrc, Path src, Path dst) 
+    throws IOException {
+    FileUtil.copy(this, src, getLocal(getConf()), dst, false, getConf());
+  }
+  
+  /**
+   * not implemented.
+   */
+  public Path startLocalOutput(Path fsOutputFile, Path tmpLocalFile) 
+    throws IOException {
+    throw new IOException("Har: startLocalOutput not allowed");
+  }
+  
+  /**
+   * not implemented.
+   */
+  public void completeLocalOutput(Path fsOutputFile, Path tmpLocalFile) 
+    throws IOException {
+    throw new IOException("Har: completeLocalOutput not allowed");
+  }
+  
+  /**
+   * not implemented.
+   */
+  public void setOwner(Path p, String username, String groupname)
+    throws IOException {
+    throw new IOException("Har: setowner not allowed");
+  }
+
+  /**
+   * Not implemented.
+   */
+  public void setPermission(Path p, FsPermission permisssion) 
+    throws IOException {
+    throw new IOException("Har: setPermission not allowed");
+  }
+  
+  /**
+   * Hadoop archives input stream. This input stream fakes EOF 
+   * since archive files are part of bigger part files.
+   */
+  private static class HarFSDataInputStream extends FSDataInputStream {
+    /**
+     * Create an input stream that fakes all the reads/positions/seeking.
+     */
+    private static class HarFsInputStream extends FSInputStream {
+      private long position, start, end;
+      //The underlying data input stream that the
+      // underlying filesystem will return.
+      private FSDataInputStream underLyingStream;
+      //one byte buffer
+      private byte[] oneBytebuff = new byte[1];
+      HarFsInputStream(FileSystem fs, Path path, long start,
+          long length, int bufferSize) throws IOException {
+        underLyingStream = fs.open(path, bufferSize);
+        underLyingStream.seek(start);
+        // the start of this file in the part file
+        this.start = start;
+        // the position pointer in the part file
+        this.position = start;
+        // the end pointer in the part file
+        this.end = start + length;
+      }
+      
+      public synchronized int available() throws IOException {
+        long remaining = end - underLyingStream.getPos();
+        if (remaining > (long)Integer.MAX_VALUE) {
+          return Integer.MAX_VALUE;
+        }
+        return (int) remaining;
+      }
+      
+      public synchronized  void close() throws IOException {
+        underLyingStream.close();
+        super.close();
+      }
+      
+      //not implemented
+      @Override
+      public void mark(int readLimit) {
+        // do nothing 
+      }
+      
+      /**
+       * reset is not implemented
+       */
+      public void reset() throws IOException {
+        throw new IOException("reset not implemented.");
+      }
+      
+      public synchronized int read() throws IOException {
+        int ret = read(oneBytebuff, 0, 1);
+        return (ret <= 0) ? -1: (oneBytebuff[0] & 0xff);
+      }
+      
+      public synchronized int read(byte[] b) throws IOException {
+        int ret = read(b, 0, b.length);
+        if (ret != -1) {
+          position += ret;
+        }
+        return ret;
+      }
+      
+      /**
+       * 
+       */
+      public synchronized int read(byte[] b, int offset, int len) 
+        throws IOException {
+        int newlen = len;
+        int ret = -1;
+        if (position + len > end) {
+          newlen = (int) (end - position);
+        }
+        // end case
+        if (newlen == 0) 
+          return ret;
+        ret = underLyingStream.read(b, offset, newlen);
+        position += ret;
+        return ret;
+      }
+      
+      public synchronized long skip(long n) throws IOException {
+        long tmpN = n;
+        if (tmpN > 0) {
+          if (position + tmpN > end) {
+            tmpN = end - position;
+          }
+          underLyingStream.seek(tmpN + position);
+          position += tmpN;
+          return tmpN;
+        }
+        return (tmpN < 0)? -1 : 0;
+      }
+      
+      public synchronized long getPos() throws IOException {
+        return (position - start);
+      }
+      
+      public synchronized void seek(long pos) throws IOException {
+        if (pos < 0 || (start + pos > end)) {
+          throw new IOException("Failed to seek: EOF");
+        }
+        position = start + pos;
+        underLyingStream.seek(position);
+      }
+
+      public boolean seekToNewSource(long targetPos) throws IOException {
+        //do not need to implement this
+        // hdfs in itself does seektonewsource 
+        // while reading.
+        return false;
+      }
+      
+      /**
+       * implementing position readable. 
+       */
+      public int read(long pos, byte[] b, int offset, int length) 
+      throws IOException {
+        int nlength = length;
+        if (start + nlength + pos > end) {
+          nlength = (int) (end - (start + pos));
+        }
+        return underLyingStream.read(pos + start , b, offset, nlength);
+      }
+      
+      /**
+       * position readable again.
+       */
+      public void readFully(long pos, byte[] b, int offset, int length) 
+      throws IOException {
+        if (start + length + pos > end) {
+          throw new IOException("Not enough bytes to read.");
+        }
+        underLyingStream.readFully(pos + start, b, offset, length);
+      }
+      
+      public void readFully(long pos, byte[] b) throws IOException {
+          readFully(pos, b, 0, b.length);
+      }
+      
+    }
+  
+    /**
+     * constructors for har input stream.
+     * @param fs the underlying filesystem
+     * @param p The path in the underlying filesystem
+     * @param start the start position in the part file
+     * @param length the length of valid data in the part file
+     * @param bufsize the buffer size
+     * @throws IOException
+     */
+    public HarFSDataInputStream(FileSystem fs, Path  p, long start, 
+        long length, int bufsize) throws IOException {
+        super(new HarFsInputStream(fs, p, start, length, bufsize));
+    }
+
+    /**
+     * constructor for har input stream.
+     * @param fs the underlying filesystem
+     * @param p the path in the underlying file system
+     * @param start the start position in the part file
+     * @param length the length of valid data in the part file.
+     * @throws IOException
+     */
+    public HarFSDataInputStream(FileSystem fs, Path  p, long start, long length)
+      throws IOException {
+        super(new HarFsInputStream(fs, p, start, length, 0));
+    }
+  }
+}

+ 8 - 0
src/core/org/apache/hadoop/io/compress/CodecPool.java

@@ -141,6 +141,10 @@ public class CodecPool {
     if (compressor == null) {
       return;
     }
+    // if the compressor can't be reused, don't pool it.
+    if (compressor.getClass().isAnnotationPresent(DoNotPool.class)) {
+      return;
+    }
     compressor.reset();
     payback(compressorPool, compressor);
   }
@@ -155,6 +159,10 @@ public class CodecPool {
     if (decompressor == null) {
       return;
     }
+    // if the decompressor can't be reused, don't pool it.
+    if (decompressor.getClass().isAnnotationPresent(DoNotPool.class)) {
+      return;
+    }
     decompressor.reset();
     payback(decompressorPool, decompressor);
   }

+ 35 - 0
src/core/org/apache/hadoop/io/compress/DoNotPool.java

@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.io.compress;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * This is a marker annotation that marks a compressor or decompressor 
+ * type as not to be pooled.
+ */
+@Retention(RetentionPolicy.RUNTIME)
+@Target(ElementType.TYPE)
+@Documented
+public @interface DoNotPool {
+
+}

+ 2 - 0
src/core/org/apache/hadoop/io/compress/zlib/BuiltInGzipDecompressor.java

@@ -24,12 +24,14 @@ import java.util.zip.Inflater;
 import java.util.zip.CRC32;
 
 import org.apache.hadoop.io.compress.Decompressor;
+import org.apache.hadoop.io.compress.DoNotPool;
 
 /**
  * A {@link Decompressor} based on the popular gzip compressed file format.
  * http://www.gzip.org/
  *
  */
+@DoNotPool
 public class BuiltInGzipDecompressor implements Decompressor {
   private static final int GZIP_MAGIC_ID = 0x8b1f;  // if read as LE short int
   private static final int GZIP_DEFLATE_METHOD = 8;

+ 2 - 2
src/core/overview.html

@@ -114,7 +114,7 @@ be demonstrated as follows:</p>
 <tt>
 mkdir input<br>
 cp conf/*.xml input<br>
-bin/hadoop jar hadoop-*-examples.jar grep input output 'dfs[a-z.]+'<br>
+bin/hadoop jar hadoop-examples-*.jar grep input output 'dfs[a-z.]+'<br>
 cat output/*
 </tt>
 <p>This will display counts for each match of the <a
@@ -217,7 +217,7 @@ command, run on the master node:</p>
 examine it:</p>
 
 <tt>
-bin/hadoop jar hadoop-*-examples.jar grep input output 'dfs[a-z.]+'<br>
+bin/hadoop jar hadoop-examples-*.jar grep input output 'dfs[a-z.]+'<br>
 bin/hadoop fs -get output output
 cat output/*
 </tt>

+ 456 - 198
src/docs/src/documentation/content/xdocs/capacity_scheduler.xml

@@ -20,7 +20,7 @@
 <document>
   
   <header>
-    <title>Capacity Scheduler Guide</title>
+    <title>CapacityScheduler Guide</title>
   </header>
   
   <body>
@@ -28,93 +28,127 @@
     <section>
       <title>Purpose</title>
       
-      <p>This document describes the Capacity Scheduler, a pluggable 
-      MapReduce scheduler for Hadoop which provides a way to share 
-      large clusters.</p>
+      <p>This document describes the CapacityScheduler, a pluggable 
+      MapReduce scheduler for Hadoop which allows for multiple-tenants to 
+      securely share a large cluster such that their applications are allocated
+      resources in a timely manner under constraints of allocated capacities.
+      </p>
+    </section>
+    
+    <section>
+      <title>Overview</title>
+     
+      <p>The CapacityScheduler is designed to run Hadoop Map-Reduce as a 
+      shared, multi-tenant cluster in an operator-friendly manner while 
+      maximizing the throughput and the utilization of the cluster while
+      running Map-Reduce applications. </p>
+     
+      <p>Traditionally each organization has it own private set of compute 
+      resources that have sufficient capacity to meet the organization's SLA 
+      under peak or near peak conditions. This generally leads to poor average 
+      utilization and the overhead of managing multiple independent clusters, 
+      one per each organization. Sharing clusters between organizations is a 
+      cost-effective manner of running large Hadoop installations since this 
+      allows them to reap benefits of economies of scale without creating 
+      private clusters.  However, organizations are concerned about sharing a 
+      cluster because they are worried about others using the resources that 
+      are critical for their SLAs.</p> 
+
+      <p>The CapacityScheduler is designed to allow sharing a large cluster 
+      while giving  each organization a minimum capacity guarantee. The central 
+      idea is that the available resources in the Hadoop Map-Reduce cluster are 
+      partitioned among multiple organizations who collectively fund the 
+      cluster based on computing needs. There is an added benefit that an 
+      organization can access any excess capacity no being used by others. This 
+      provides elasticity for the organizations in a cost-effective manner.</p> 
+
+      <p>Sharing clusters across organizations necessitates strong support for
+      multi-tenancy since each organization must be guaranteed capacity and 
+      safe-guards to ensure the shared cluster is impervious to single rouge 
+      job or user. The CapacityScheduler provides a stringent set of limits to 
+      ensure that a single job or user or queue cannot consume dispropotionate 
+      amount of resources in the cluster. Also, the JobTracker of the cluster,  
+      in particular, is a precious resource and the CapacityScheduler provides 
+      limits on initialized/pending tasks and jobs from a single user and queue 
+      to ensure fairness and stability of the cluster.</p> 
+
+      <p>The primary abstraction provided by the CapacityScheduler is the 
+      concept of <em>queues</em>. These queues are typically setup by administrators
+      to reflect the economics of the shared cluster.</p>
     </section>
     
     <section>
       <title>Features</title>
       
-      <p>The Capacity Scheduler supports the following features:</p> 
+      <p>The CapacityScheduler supports the following features:</p> 
       <ul>
         <li>
-          Support for multiple queues, where a job is submitted to a queue.
+          Capacity Guarantees - Support for multiple queues, where a job is 
+          submitted to a queue.Queues are allocated a fraction of the capacity 
+          of the grid in the sense that a certain capacity of resources will be 
+          at their disposal. All jobs submitted to a queue will have access to 
+          the capacity allocated to the queue. Adminstrators can configure soft 
+          limits and optional hard limits on the capacity allocated to each queue. 
         </li>
         <li>
-          Queues are allocated a fraction of the capacity of the grid in the 
-          sense that a certain capacity of resources will be at their 
-          disposal. All jobs submitted to a queue will have access to the 
-          capacity allocated to the queue.
+          Security - Each queue has strict ACLs which controls which users can 
+          submit jobs to individual queues. Also, there are safe-guards to 
+          ensure that users cannot view and/or modify jobs from other users if
+          so desired. Also, per-queue and system administrator roles are 
+          supported.
         </li>
         <li>
-          Free resources can be allocated to any queue beyond it's capacity. 
-          When there is demand for these resources from queues running below 
-          capacity at a future point in time, as tasks scheduled on these 
+          Elasticity - Free resources can be allocated to any queue beyond it's 
+          capacity. When there is demand for these resources from queues running 
+          below capacity at a future point in time, as tasks scheduled on these 
           resources complete, they will be assigned to jobs on queues 
-          running below the capacity.
+          running below the capacity. This ensures that resources are available 
+          in a predictable and elastic manner to queues, thus preventing 
+          artifical silos of resources in the cluster which helps utilization.
         </li>
         <li>
-          Queues optionally support job priorities (disabled by default).
+          Multi-tenancy - Comprehensive set of limits are provided to prevent 
+          a single job, user and queue from monpolizing resources of the queue 
+          or the cluster as a whole to ensure that the system, particularly the 
+          JobTracker, isn't overwhelmed by too many tasks or jobs. 
         </li>
         <li>
-          Within a queue, jobs with higher priority will have access to the 
-          queue's resources before jobs with lower priority. However, once a 
-          job is running, it will not be preempted for a higher priority job,
-          though new tasks from the higher priority job will be 
-          preferentially scheduled.
+          Operability - The queue definitions and properties can be changed, 
+          at runtime, by administrators in a secure manner to minimize 
+          disruption to users. Also, a console is provided for users and 
+          administrators to view current allocation of resources to various 
+          queues in the system.
         </li>
         <li>
-          In order to prevent one or more users from monopolizing its 
-          resources, each queue enforces a limit on the percentage of 
-          resources allocated to a user at any given time, if there is 
-          competition for them.  
+          Resource-based Scheduling - Support for resource-intensive jobs, 
+          wherein a job can optionally specify higher resource-requirements than 
+          the default, there-by accomodating applications with differing resource
+          requirements. Currently, memory is the the resource requirement 
+          supported.
         </li>
         <li>
-          Support for memory-intensive jobs, wherein a job can optionally 
-          specify higher memory-requirements than the default, and the tasks 
-          of the job will only be run on TaskTrackers that have enough memory 
-          to spare.
+          Job Priorities - Queues optionally support job priorities (disabled 
+          by default). Within a queue, jobs with higher priority will have 
+          access to the queue's resources before jobs with lower priority. 
+          However, once a job is running, it will not be preempted for a higher 
+          priority job, <em>premption</em> is on the roadmap is currently not 
+          supported.
         </li>
       </ul>
     </section>
     
-    <section>
-      <title>Picking a task to run</title>
-      
-      <p>Note that many of these steps can be, and will be, enhanced over time
-      to provide better algorithms.</p>
-      
-      <p>Whenever a TaskTracker is free, the Capacity Scheduler picks 
-      a queue which has most free space (whose ratio of # of running slots to 
-      capacity is the lowest).</p>
-      
-      <p>Once a queue is selected, the Scheduler picks a job in the queue. Jobs
-      are sorted based on when they're submitted and their priorities (if the 
-      queue supports priorities). Jobs are considered in order, and a job is 
-      selected if its user is within the user-quota for the queue, i.e., the 
-      user is not already using queue resources above his/her limit. The 
-      Scheduler also makes sure that there is enough free memory in the 
-      TaskTracker to tun the job's task, in case the job has special memory
-      requirements.</p>
-      
-      <p>Once a job is selected, the Scheduler picks a task to run. This logic 
-      to pick a task remains unchanged from earlier versions.</p> 
-      
-    </section>
-    
     <section>
       <title>Installation</title>
       
-        <p>The Capacity Scheduler is available as a JAR file in the Hadoop
+        <p>The CapacityScheduler is available as a JAR file in the Hadoop
         tarball under the <em>contrib/capacity-scheduler</em> directory. The name of 
-        the JAR file would be on the lines of hadoop-*-capacity-scheduler.jar.</p>
+        the JAR file would be on the lines of hadoop-capacity-scheduler-*.jar.</p>
         <p>You can also build the Scheduler from source by executing
         <em>ant package</em>, in which case it would be available under
         <em>build/contrib/capacity-scheduler</em>.</p>
-        <p>To run the Capacity Scheduler in your Hadoop installation, you need 
+        <p>To run the CapacityScheduler in your Hadoop installation, you need 
         to put it on the <em>CLASSPATH</em>. The easiest way is to copy the 
-        <code>hadoop-*-capacity-scheduler.jar</code> from 
+        <code>hadoop-capacity-scheduler-*.jar</code> from 
         to <code>HADOOP_HOME/lib</code>. Alternatively, you can modify 
         <em>HADOOP_CLASSPATH</em> to include this jar, in 
         <code>conf/hadoop-env.sh</code>.</p>
@@ -124,9 +158,9 @@
       <title>Configuration</title>
 
       <section>
-        <title>Using the Capacity Scheduler</title>
+        <title>Using the CapacityScheduler</title>
         <p>
-          To make the Hadoop framework use the Capacity Scheduler, set up
+          To make the Hadoop framework use the CapacityScheduler, set up
           the following property in the site configuration:</p>
           <table>
             <tr>
@@ -144,14 +178,22 @@
         <title>Setting up queues</title>
         <p>
           You can define multiple queues to which users can submit jobs with
-          the Capacity Scheduler. To define multiple queues, you should edit
-          the site configuration for Hadoop and modify the
-          <em>mapred.queue.names</em> property.
+          the CapacityScheduler. To define multiple queues, you should use the  
+          <em>mapred.queue.names</em> property in 
+          <code>conf/hadoop-site.xml</code>.
         </p>
+        
+        <p>
+          The CapacityScheduler can be configured with several properties
+          for each queue that control the behavior of the Scheduler. This
+          configuration is in the <em>conf/capacity-scheduler.xml</em>.
+        </p>
+        
         <p>
           You can also configure ACLs for controlling which users or groups
-          have access to the queues.
+          have access to the queues in <code>conf/mapred-queue-acls.xml</code>.
         </p>
+        
         <p>
           For more details, refer to
           <a href="cluster_setup.html#Configuring+the+Hadoop+Daemons">Cluster 
@@ -160,25 +202,12 @@
       </section>
   
       <section>
-        <title>Configuring properties for queues</title>
+        <title>Queue properties</title>
 
-        <p>The Capacity Scheduler can be configured with several properties
-        for each queue that control the behavior of the Scheduler. This
-        configuration is in the <em>conf/capacity-scheduler.xml</em>. By
-        default, the configuration is set up for one queue, named 
-        <em>default</em>.</p>
-        <p>To specify a property for a queue that is defined in the site
-        configuration, you should use the property name as
-        <em>mapred.capacity-scheduler.queue.&lt;queue-name&gt;.&lt;property-name&gt;</em>.
-        </p>
-        <p>For example, to define the property <em>capacity</em>
-        for queue named <em>research</em>, you should specify the property
-        name as 
-        <em>mapred.capacity-scheduler.queue.research.capacity</em>.
-        </p>
-
-        <p>The properties defined for queues and their descriptions are
-        listed in the table below:</p>
+        <section>
+        <title>Resource allocation</title>
+        <p>The properties defined for resource allocations to queues and their 
+        descriptions are listed in below:</p>
 
         <table>
           <tr><th>Name</th><th>Description</th></tr>
@@ -187,25 +216,8 @@
             to be available for jobs in this queue. The sum of capacities 
             for all queues should be less than or equal 100.</td>
           </tr>
-          <tr><td>mapred.capacity-scheduler.queue.&lt;queue-name&gt;.supports-priority</td>
-          	<td>If true, priorities of jobs will be taken into account in scheduling 
-          	decisions.</td>
-          </tr>
-          <tr><td>mapred.capacity-scheduler.queue.&lt;queue-name&gt;.minimum-user-limit-percent</td>
-          	<td>Each queue enforces a limit on the percentage of resources 
-          	allocated to a user at any given time, if there is competition 
-          	for them. This user limit can vary between a minimum and maximum 
-          	value. The former depends on the number of users who have submitted
-          	jobs, and the latter is set to this property value. For example, 
-          	suppose the value of this property is 25. If two users have 
-          	submitted jobs to a queue, no single user can use more than 50% 
-          	of the queue resources. If a third user submits a job, no single 
-          	user can use more than 33% of the queue resources. With 4 or more 
-          	users, no user can use more than 25% of the queue's resources. A 
-          	value of 100 implies no user limits are imposed.</td>
-          </tr>
           <tr><td>mapred.capacity-scheduler.queue.&lt;queue-name&gt;.maximum-capacity</td>
-          	<td>
+            <td>
                   maximum-capacity defines a limit beyond which a queue cannot
                   use the capacity of the cluster.This provides a means to limit
                   how much excess capacity a queue can use. By default, there
@@ -228,137 +240,175 @@
                   absolute terms would increase accordingly.
                 </td>
           </tr>
-        </table>
-      </section>
-      
-      <section>
-        <title>Memory management</title>
-      
-        <p>The Capacity Scheduler supports scheduling of tasks on a
-        <code>TaskTracker</code>(TT) based on a job's memory requirements
-        and the availability of RAM and Virtual Memory (VMEM) on the TT node.
-        See the <a href="mapred_tutorial.html#Memory+monitoring"> 
-        MapReduce Tutorial</a> for details on how the TT monitors
-        memory usage.</p>
-        <p>Currently the memory based scheduling is only supported
-        in Linux platform.</p>
-        <p>Memory-based scheduling works as follows:</p>
-        <ol>
-          <li>The absence of any one or more of three config parameters 
-          or -1 being set as value of any of the parameters, 
-          <code>mapred.tasktracker.vmem.reserved</code>, 
-          <code>mapred.task.default.maxvmem</code>, or
-          <code>mapred.task.limit.maxvmem</code>, disables memory-based
-          scheduling, just as it disables memory monitoring for a TT. These
-          config parameters are described in the 
-          <a href="mapred_tutorial.html#Memory+monitoring">MapReduce 
-          Tutorial</a>. The value of  
-          <code>mapred.tasktracker.vmem.reserved</code> is 
-          obtained from the TT via its heartbeat. 
-          </li>
-          <li>If all the three mandatory parameters are set, the Scheduler 
-          enables VMEM-based scheduling. First, the Scheduler computes the free
-          VMEM on the TT. This is the difference between the available VMEM on the
-          TT (the node's total VMEM minus the offset, both of which are sent by 
-          the TT on each heartbeat)and the sum of VMs already allocated to 
-          running tasks (i.e., sum of the VMEM task-limits). Next, the Scheduler
-          looks at the VMEM requirements for the job that's first in line to 
-          run. If the job's VMEM requirements are less than the available VMEM on 
-          the node, the job's task can be scheduled. If not, the Scheduler 
-          ensures that the TT does not get a task to run (provided the job 
-          has tasks to run). This way, the Scheduler ensures that jobs with 
-          high memory requirements are not starved, as eventually, the TT 
-          will have enough VMEM available. If the high-mem job does not have 
-          any task to run, the Scheduler moves on to the next job. 
-          </li>
-          <li>In addition to VMEM, the Capacity Scheduler can also consider 
-          RAM on the TT node. RAM is considered the same way as VMEM. TTs report
-          the total RAM available on their node, and an offset. If both are
-          set, the Scheduler computes the available RAM on the node. Next, 
-          the Scheduler figures out the RAM requirements of the job, if any. 
-          As with VMEM, users can optionally specify a RAM limit for their job
-          (<code>mapred.task.maxpmem</code>, described in the MapReduce 
-          Tutorial). The Scheduler also maintains a limit for this value 
-          (<code>mapred.capacity-scheduler.task.default-pmem-percentage-in-vmem</code>, 
-          described below). All these three values must be set for the 
-          Scheduler to schedule tasks based on RAM constraints.
-          </li>
-          <li>The Scheduler ensures that jobs cannot ask for RAM or VMEM higher
-          than configured limits. If this happens, the job is failed when it
-          is submitted. 
-          </li>
-        </ol>
-        
-        <p>As described above, the additional scheduler-based config 
-        parameters are as follows:</p>
-
-        <table>
-          <tr><th>Name</th><th>Description</th></tr>
-          <tr><td>mapred.capacity-scheduler.task.default-pmem-percentage-in-vmem</td>
-          	<td>A percentage of the default VMEM limit for jobs
-          	(<code>mapred.task.default.maxvmem</code>). This is the default 
-          	RAM task-limit associated with a task. Unless overridden by a 
-          	job's setting, this number defines the RAM task-limit.</td>
+          <tr><td>mapred.capacity-scheduler.queue.&lt;queue-name&gt;.minimum-user-limit-percent</td>
+          	<td>Each queue enforces a limit on the percentage of resources 
+          	allocated to a user at any given time, if there is competition 
+          	for them. This user limit can vary between a minimum and maximum 
+          	value. The former depends on the number of users who have submitted
+          	jobs, and the latter is set to this property value. For example, 
+          	suppose the value of this property is 25. If two users have 
+          	submitted jobs to a queue, no single user can use more than 50% 
+          	of the queue resources. If a third user submits a job, no single 
+          	user can use more than 33% of the queue resources. With 4 or more 
+          	users, no user can use more than 25% of the queue's resources. A 
+          	value of 100 implies no user limits are imposed.</td>
+          </tr>
+          <tr><td>mapred.capacity-scheduler.queue.&lt;queue-name&gt;.user-limit-factor</td>
+            <td>The multiple of the queue capacity which can be configured to 
+              allow a single user to acquire more slots. By default this is set 
+              to 1 which ensure that a single user can never take more than the 
+              queue's configured capacity irrespective of how idle th cluster 
+              is.</td>
           </tr>
-          <tr><td>mapred.capacity-scheduler.task.limit.maxpmem</td>
-          <td>Configuration which provides an upper limit to maximum physical
-           memory which can be specified by a job. If a job requires more 
-           physical memory than what is specified in this limit then the same
-           is rejected.</td>
+          <tr><td>mapred.capacity-scheduler.queue.&lt;queue-name&gt;.supports-priority</td>
+            <td>If true, priorities of jobs will be taken into account in scheduling 
+            decisions.</td>
           </tr>
         </table>
-      </section>
+   </section>
    <section>
-        <title>Job Initialization Parameters</title>
+        <title>Job initialization</title>
         <p>Capacity scheduler lazily initializes the jobs before they are
         scheduled, for reducing the memory footprint on jobtracker. 
-        Following are the parameters, by which you can control the laziness
-        of the job initialization. The following parameters can be 
-        configured in capacity-scheduler.xml
+        Following are the parameters, by which you can control the
+        initialization of jobs per-queue.
         </p>
         
         <table>
           <tr><th>Name</th><th>Description</th></tr>
           <tr>
             <td>
-              mapred.capacity-scheduler.queue.&lt;queue-name&gt;.maximum-initialized-jobs-per-user
+              mapred.capacity-scheduler.maximum-system-jobs
             </td>
             <td>
-              Maximum number of jobs which are allowed to be pre-initialized for
-              a particular user in the queue. Once a job is scheduled, i.e. 
-              it starts running, then that job is not considered
-              while scheduler computes the maximum job a user is allowed to
-              initialize. 
+              Maximum number of jobs in the system which can be initialized,
+              concurrently, by the CapacityScheduler.
+              
+              Individual queue limits on initialized jobs are directly 
+              proportional to their queue capacities.
             </td>
           </tr>
           <tr>
             <td>
-              mapred.capacity-scheduler.init-poll-interval
+              mapred.capacity-scheduler.queue.&lt;queue-name&gt;.maximum-initialized-active-tasks
             </td>
             <td>
-              Amount of time in miliseconds which is used to poll the scheduler
-              job queue to look for jobs to be initialized.
+              The maximum number of tasks, across all jobs in the queue, 
+              which can be initialized concurrently. Once the queue's jobs 
+              exceed this limit they will be queued on disk.             
             </td>
           </tr>
           <tr>
             <td>
-              mapred.capacity-scheduler.init-worker-threads
+              mapred.capacity-scheduler.queue.&lt;queue-name&gt;.maximum-initialized-active-tasks-per-user
             </td>
             <td>
-              Number of worker threads which would be used by Initialization
-              poller to initialize jobs in a set of queue. If number mentioned 
-              in property is equal to number of job queues then a thread is 
-              assigned jobs from one queue. If the number configured is lesser than
-              number of queues, then a thread can get jobs from more than one queue
-              which it initializes in a round robin fashion. If the number configured
-              is greater than number of queues, then number of threads spawned
-              would be equal to number of job queues.
+              The maximum number of tasks per-user, across all the of the
+              user's jobs in the queue, which can be initialized concurrently. 
+              Once the user's jobs exceed this limit they will be queued on disk.
             </td>
           </tr>
+          <tr>
+            <td> 
+              mapred.capacity-scheduler.queue.&lt;queue-name&gt;.init-accept-jobs-factor
+            </td>
+            <td>
+              The multipe of (maximum-system-jobs * queue-capacity) used to
+              determine the number of jobs which are accepted by the scheduler. 
+              The default value is 10. If number of jobs submitted to the queue
+              exceeds this limit, job submission are rejected. 
+            </td>
+          </tr> 
         </table>
       </section>   
+      </section>
+      
       <section>
-        <title>Reviewing the configuration of the Capacity Scheduler</title>
+        <title>Resource based scheduling</title>
+      
+        <p>The CapacityScheduler supports scheduling of tasks on a
+        <code>TaskTracker</code>(TT) based on a job's memory requirements
+        in terms of RAM and Virtual Memory (VMEM) on the TT node.
+        A TT is conceptually composed of a fixed number of map and reduce
+        slots with fixed slot size across the cluster. A job can ask for one
+        or more slots for each of its component map and/or reduce slots. If a
+        task consumes more memory than configured the TT forcibly kills the task.
+        </p>
+
+        <p>Currently the memory based scheduling is only supported
+        in Linux platform.</p>
+        
+        <p>Additional scheduler-based config 
+        parameters are as follows:</p>
+
+        <table>
+          <tr><th>Name</th><th>Description</th></tr>
+          <tr>
+            <td>mapred.cluster.map.memory.mb</td>
+          	 <td>The size, in terms of virtual memory, of a single map slot
+             in the Map-Reduce framework, used by the scheduler.
+             A job can ask for multiple slots for a single map task via
+             <code>mapred.job.map.memory.mb</code>, upto the limit specified by
+             <code>mapred.cluster.max.map.memory.mb</code>, if the scheduler 
+             supports the feature.
+             The value of -1 indicates that this feature is turned off.
+          	 </td>
+          </tr>
+          <tr>
+            <td>mapred.cluster.reduce.memory.mb</td>
+             <td>The size, in terms of virtual memory, of a single reduce slot
+             in the Map-Reduce framework, used by the scheduler.
+             A job can ask for multiple slots for a single reduce task via
+             <code>mapred.job.reduce.memory.mb</code>, upto the limit specified by
+             <code>mapred.cluster.max.reduce.memory.mb</code>, if the scheduler supports the 
+             feature.The value of -1 indicates that this feature is turned off.
+             </td>
+          </tr>
+          <tr>
+            <td>mapred.cluster.max.map.memory.mb</td>
+            <td>The maximum size, in terms of virtual memory, of a single map
+            task launched by the Map-Reduce framework, used by the scheduler.
+            A job can ask for multiple slots for a single map task via
+            <code>mapred.job.map.memory.mb</code>, upto the limit specified by
+            <code>mapred.cluster.max.map.memory.mb</code>, if the scheduler supports the 
+            feature. The value of -1 indicates that this feature is turned off.
+            </td>
+          </tr>
+          <tr>
+            <td>mapred.cluster.max.reduce.memory.mb</td>
+            <td>The maximum size, in terms of virtual memory, of a single reduce
+            task launched by the Map-Reduce framework, used by the scheduler.
+            A job can ask for multiple slots for a single reduce task via
+            <code>mapred.job.reduce.memory.mb</code>, upto the limit specified by
+            <code>mapred.cluster.max.reduce.memory.mb</code>, if the scheduler supports the 
+            feature. The value of -1 indicates that this feature is turned off.
+            </td>
+          </tr>
+          <tr>
+            <td>mapred.job.map.memory.mb</td>
+            <td>The size, in terms of virtual memory, of a single map task
+            for the job. A job can ask for multiple slots for a single map task, 
+            rounded up to the next multiple of <code>mapred.cluster.map.memory.mb</code> and 
+            upto the limit specified by <code>mapred.cluster.max.map.memory.mb</code>, 
+            if the scheduler supports the feature. The value of -1 indicates 
+            that this feature is turned off iff <code>mapred.cluster.map.memory.mb</code> is 
+            also turned off (-1).</td>
+          </tr>
+          <tr>
+            <td>mapred.job.reduce.memory.mb</td>
+            <td>The size, in terms of virtual memory, of a single reduce task
+            for the job. A job can ask for multiple slots for a single reduce task, 
+            rounded up to the next multiple of <code>mapred.cluster.reduce.memory.mb</code> and 
+            upto the limit specified by <code>mapred.cluster.max.reduce.memory.mb</code>, 
+            if the scheduler supports the feature. The value of -1 indicates 
+            that this feature is turned off iff <code>mapred.cluster.reduce.memory.mb</code> is 
+            also turned off (-1).</td>
+          </tr>
+        </table>
+      </section>
+      
+      <section>
+        <title>Reviewing the configuration of the CapacityScheduler</title>
         <p>
           Once the installation and configuration is completed, you can review
           it after starting the MapReduce cluster from the admin UI.
@@ -370,10 +420,218 @@
               Information</em> section of the page.</li>
           <li>The properties for the queues should be visible in the <em>Scheduling
               Information</em> column against each queue.</li>
+          <li>The /scheduler web-page should show the resource usages of 
+              individual queues.</li>
         </ul>
       </section>
       
    </section>
+
+  <section>
+    <title>Example</title>
+    <p>Here is a practical example for using CapacityScheduler:</p>
+    <table>
+    <tr>
+    <td>
+<code>&lt;?xml version="1.0"?&gt;</code><br/>
+<br/>
+<code>&lt;configuration&gt;</code><br/>
+<br/>
+<code>  &lt;!-- system limit, across all queues --&gt;</code><br/>
+<br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.maximum-system-jobs&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;3000&lt;/value&gt;</code><br/>
+<code>    &lt;description&gt;Maximum number of jobs in the system which can be initialized,</code><br/>
+<code>     concurrently, by the CapacityScheduler.</code><br/>
+<code>    &lt;/description&gt;    </code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code> </code><br/>
+<code>&lt;!-- queue: queueA --&gt;</code><br/>
+<code> &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueA.capacity&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;8&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueA.supports-priority&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;false&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueA.minimum-user-limit-percent&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;20&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueA.user-limit-factor&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;10&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueA.maximum-initialized-active-tasks&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;200000&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueA.maximum-initialized-active-tasks-per-user&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;100000&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueA.init-accept-jobs-factor&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;100&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<br/>
+<code>&lt;!-- queue: queueB --&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueB.capacity&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;2&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueB.supports-priority&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;false&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueB.minimum-user-limit-percent&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;20&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueB.user-limit-factor&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;1&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueB.maximum-initialized-active-tasks&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;200000&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueB.maximum-initialized-active-tasks-per-user&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;100000&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueB.init-accept-jobs-factor&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;10&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<br/>
+<code>&lt;!-- queue: queueC --&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueC.capacity&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;30&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueC.supports-priority&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;false&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueC.minimum-user-limit-percent&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;20&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueC.user-limit-factor&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;1&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueC.maximum-initialized-active-tasks&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;200000&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueC.maximum-initialized-active-tasks-per-user&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;100000&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueC.init-accept-jobs-factor&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;10&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<br/>
+<code>&lt;!-- queue: queueD --&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueD.capacity&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;1&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueD.supports-priority&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;false&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueD.minimum-user-limit-percent&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;20&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueD.user-limit-factor&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;20&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueD.maximum-initialized-active-tasks&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;200000&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueD.maximum-initialized-active-tasks-per-user&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;100000&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueD.init-accept-jobs-factor&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;10&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<br/>
+<code>&lt;!-- queue: queueE --&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueE.capacity&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;31&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueE.supports-priority&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;false&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueE.minimum-user-limit-percent&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;20&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueE.user-limit-factor&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;1&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueE.maximum-initialized-active-tasks&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;200000&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueE.maximum-initialized-active-tasks-per-user&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;100000&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueE.init-accept-jobs-factor&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;10&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<br/>
+<code>&lt;!-- queue: queueF --&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueF.capacity&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;28&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueF.supports-priority&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;false&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueF.minimum-user-limit-percent&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;20&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueF.user-limit-factor&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;1&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueF.maximum-initialized-active-tasks&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;200000&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueF.maximum-initialized-active-tasks-per-user&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;100000&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<code>  &lt;property&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;name&gt;mapred.capacity-scheduler.queue.queueF.init-accept-jobs-factor&lt;/name&gt;</code><br/>
+<code>    &nbsp;&nbsp;&lt;value&gt;10&lt;/value&gt;</code><br/>
+<code>  &lt;/property&gt;</code><br/>
+<br/>
+<code>&lt;/configuration&gt;</code><br/>
+    </td>
+    </tr>
+    </table>
+  </section>
   </body>
   
 </document>

+ 2 - 2
src/docs/src/documentation/content/xdocs/fair_scheduler.xml

@@ -93,7 +93,7 @@
       <p>
         To run the fair scheduler in your Hadoop installation, you need to put
         it on the CLASSPATH. The easiest way is to copy the 
-        <em>hadoop-*-fairscheduler.jar</em> from
+        <em>hadoop-fairscheduler-*.jar</em> from
         <em>HADOOP_HOME/contrib/fairscheduler</em> to <em>HADOOP_HOME/lib</em>.
         Alternatively you can modify <em>HADOOP_CLASSPATH</em> to include this jar, in
         <em>HADOOP_CONF_DIR/hadoop-env.sh</em>
@@ -101,7 +101,7 @@
       <p>
         In order to compile fair scheduler, from sources execute <em> ant 
         package</em> in source folder and copy the 
-        <em>build/contrib/fair-scheduler/hadoop-*-fairscheduler.jar</em> 
+        <em>build/contrib/fair-scheduler/hadoop-fairscheduler-*.jar</em> 
         to <em>HADOOP_HOME/lib</em>
       </p>
       <p>

+ 2 - 2
src/docs/src/documentation/content/xdocs/single_node_setup.xml

@@ -146,7 +146,7 @@
         <code>$ mkdir input</code><br/>
         <code>$ cp conf/*.xml input</code><br/>
         <code>
-          $ bin/hadoop jar hadoop-*-examples.jar grep input output 'dfs[a-z.]+'
+          $ bin/hadoop jar hadoop-examples-*.jar grep input output 'dfs[a-z.]+'
         </code><br/>
         <code>$ cat output/*</code>
       </p>
@@ -252,7 +252,7 @@
         <p>
           Run some of the examples provided:<br/>
           <code>
-            $ bin/hadoop jar hadoop-*-examples.jar grep input output 'dfs[a-z.]+'
+            $ bin/hadoop jar hadoop-examples-*.jar grep input output 'dfs[a-z.]+'
           </code>
         </p>
         

+ 1 - 1
src/examples/org/apache/hadoop/examples/AggregateWordCount.java

@@ -34,7 +34,7 @@ import org.apache.hadoop.mapred.lib.aggregate.ValueAggregatorJob;
  * text input files, breaks each line into words and counts them. The output is
  * a locally sorted list of words and the count of how often they occurred.
  * 
- * To run: bin/hadoop jar hadoop-*-examples.jar aggregatewordcount <i>in-dir</i>
+ * To run: bin/hadoop jar hadoop-examples-*.jar aggregatewordcount <i>in-dir</i>
  * <i>out-dir</i> <i>numOfReducers</i> textinputformat
  * 
  */

+ 1 - 1
src/examples/org/apache/hadoop/examples/AggregateWordHistogram.java

@@ -32,7 +32,7 @@ import org.apache.hadoop.mapred.lib.aggregate.ValueAggregatorJob;
  * This is an example Aggregated Hadoop Map/Reduce application. Computes the
  * histogram of the words in the input texts.
  * 
- * To run: bin/hadoop jar hadoop-*-examples.jar aggregatewordhist <i>in-dir</i>
+ * To run: bin/hadoop jar hadoop-examples-*.jar aggregatewordhist <i>in-dir</i>
  * <i>out-dir</i> <i>numOfReducers</i> textinputformat
  * 
  */

+ 2 - 2
src/examples/org/apache/hadoop/examples/dancing/package.html

@@ -59,8 +59,8 @@ Both applications have been added to the examples jar, so they can be
 run as:
 
 <pre>
-bin/hadoop jar hadoop-*-examples.jar pentomino pent-outdir
-bin/hadoop jar hadoop-*-examples.jar sudoku puzzle.txt
+bin/hadoop jar hadoop-examples-*.jar pentomino pent-outdir
+bin/hadoop jar hadoop-examples-*.jar sudoku puzzle.txt
 </pre>
 
 <p>

+ 1 - 1
src/examples/org/apache/hadoop/examples/terasort/TeraGen.java

@@ -55,7 +55,7 @@ import org.apache.hadoop.util.ToolRunner;
  *
  * <p>
  * To run the program: 
- * <b>bin/hadoop jar hadoop-*-examples.jar teragen 10000000000 in-dir</b>
+ * <b>bin/hadoop jar hadoop-examples-*.jar teragen 10000000000 in-dir</b>
  */
 public class TeraGen extends Configured implements Tool {
 

+ 1 - 1
src/examples/org/apache/hadoop/examples/terasort/TeraSort.java

@@ -45,7 +45,7 @@ import org.apache.hadoop.util.ToolRunner;
  * finish. 
  * <p>
  * To run the program: 
- * <b>bin/hadoop jar hadoop-*-examples.jar terasort in-dir out-dir</b>
+ * <b>bin/hadoop jar hadoop-examples-*.jar terasort in-dir out-dir</b>
  */
 public class TeraSort extends Configured implements Tool {
   private static final Log LOG = LogFactory.getLog(TeraSort.class);

+ 1 - 1
src/examples/org/apache/hadoop/examples/terasort/TeraValidate.java

@@ -44,7 +44,7 @@ import org.apache.hadoop.util.ToolRunner;
  * Any output from the reduce is problem report.
  * <p>
  * To run the program: 
- * <b>bin/hadoop jar hadoop-*-examples.jar teravalidate out-dir report-dir</b>
+ * <b>bin/hadoop jar hadoop-examples-*.jar teravalidate out-dir report-dir</b>
  * <p>
  * If there is any output, something is wrong and the output of the reduce
  * will have the problem report.

+ 32 - 9
src/mapred/org/apache/hadoop/mapred/lib/CombineFileInputFormat.java

@@ -20,12 +20,12 @@ package org.apache.hadoop.mapred.lib;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.HashSet;
 import java.util.List;
 import java.util.HashMap;
 import java.util.Set;
 import java.util.Iterator;
 import java.util.Map;
-import java.util.Map.Entry;
 
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
@@ -73,6 +73,9 @@ public abstract class CombineFileInputFormat<K, V>
   // across multiple pools.
   private ArrayList<MultiPathFilter> pools = new  ArrayList<MultiPathFilter>();
 
+  // mapping from a rack name to the set of Nodes in the rack 
+  private static HashMap<String, Set<String>> rackToNodes = 
+                            new HashMap<String, Set<String>>();
   /**
    * Specify the maximum size (in bytes) of each split. Each split is
    * approximately equal to the specified size.
@@ -214,6 +217,8 @@ public abstract class CombineFileInputFormat<K, V>
     getMoreSplits(job, myPaths.toArray(new Path[myPaths.size()]), 
                   maxSize, minSizeNode, minSizeRack, splits);
 
+    // free up rackToNodes map
+    rackToNodes.clear();
     return splits.toArray(new CombineFileSplit[splits.size()]);    
   }
 
@@ -341,7 +346,7 @@ public abstract class CombineFileInputFormat<K, V>
             // create this split.
             if (maxSize != 0 && curSplitSize >= maxSize) {
               // create an input split and add it to the splits array
-              addCreatedSplit(job, splits, racks, validBlocks);
+              addCreatedSplit(job, splits, getHosts(racks), validBlocks);
               createdSplit = true;
               break;
             }
@@ -360,7 +365,7 @@ public abstract class CombineFileInputFormat<K, V>
           if (minSizeRack != 0 && curSplitSize >= minSizeRack) {
             // if there is a mimimum size specified, then create a single split
             // otherwise, store these blocks into overflow data structure
-            addCreatedSplit(job, splits, racks, validBlocks);
+            addCreatedSplit(job, splits, getHosts(racks), validBlocks);
           } else {
             // There were a few blocks in this rack that remained to be processed.
             // Keep them in 'overflow' block list. These will be combined later.
@@ -393,7 +398,7 @@ public abstract class CombineFileInputFormat<K, V>
       // create this split.
       if (maxSize != 0 && curSplitSize >= maxSize) {
         // create an input split and add it to the splits array
-        addCreatedSplit(job, splits, racks, validBlocks);
+        addCreatedSplit(job, splits, getHosts(racks), validBlocks);
         curSplitSize = 0;
         validBlocks.clear();
         racks.clear();
@@ -402,7 +407,7 @@ public abstract class CombineFileInputFormat<K, V>
 
     // Process any remaining blocks, if any.
     if (!validBlocks.isEmpty()) {
-      addCreatedSplit(job, splits, racks, validBlocks);
+      addCreatedSplit(job, splits, getHosts(racks), validBlocks);
     }
   }
 
@@ -412,13 +417,12 @@ public abstract class CombineFileInputFormat<K, V>
    */
   private void addCreatedSplit(JobConf job,
                                List<CombineFileSplit> splitList, 
-                               List<String> racks, 
+                               List<String> locations, 
                                ArrayList<OneBlockInfo> validBlocks) {
     // create an input split
     Path[] fl = new Path[validBlocks.size()];
     long[] offset = new long[validBlocks.size()];
     long[] length = new long[validBlocks.size()];
-    String[] rackLocations = racks.toArray(new String[racks.size()]);
     for (int i = 0; i < validBlocks.size(); i++) {
       fl[i] = validBlocks.get(i).onepath; 
       offset[i] = validBlocks.get(i).offset;
@@ -427,7 +431,7 @@ public abstract class CombineFileInputFormat<K, V>
 
      // add this split to the list that is returned
     CombineFileSplit thissplit = new CombineFileSplit(job, fl, offset, 
-                                                      length, rackLocations);
+                                   length, locations.toArray(new String[0]));
     splitList.add(thissplit); 
   }
 
@@ -484,7 +488,9 @@ public abstract class CombineFileInputFormat<K, V>
               rackToBlocks.put(rack, blklist);
             }
             blklist.add(oneblock);
-          }
+            // Add this host to rackToNodes map
+            addHostToRack(oneblock.racks[j], oneblock.hosts[j]);
+         }
 
           // add this block to the node --> block map
           for (int j = 0; j < oneblock.hosts.length; j++) {
@@ -547,6 +553,23 @@ public abstract class CombineFileInputFormat<K, V>
     }
   }
 
+  private static void addHostToRack(String rack, String host) {
+    Set<String> hosts = rackToNodes.get(rack);
+    if (hosts == null) {
+      hosts = new HashSet<String>();
+      rackToNodes.put(rack, hosts);
+    }
+    hosts.add(host);
+  }
+  
+  private static List<String> getHosts(List<String> racks) {
+    List<String> hosts = new ArrayList<String>();
+    for (String rack : racks) {
+      hosts.addAll(rackToNodes.get(rack));
+    }
+    return hosts;
+  }
+  
   /**
    * Accept a path only if any one of filters given in the
    * constructor do. 

+ 240 - 0
src/test/org/apache/hadoop/fs/TestHarFileSystem.java

@@ -0,0 +1,240 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs;
+
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FsShell;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.MiniMRCluster;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TextInputFormat;
+import org.apache.hadoop.mapred.TextOutputFormat;
+import org.apache.hadoop.tools.HadoopArchives;
+import org.apache.hadoop.util.ToolRunner;
+
+import junit.framework.TestCase;
+
+/**
+ * test the har file system
+ * create a har filesystem
+ * run fs commands
+ * and then run a map reduce job
+ */
+public class TestHarFileSystem extends TestCase {
+  private Path inputPath;
+  private MiniDFSCluster dfscluster;
+  private MiniMRCluster mapred;
+  private FileSystem fs;
+  private Path filea, fileb, filec, filed;
+  private Path archivePath;
+  
+  protected void setUp() throws Exception {
+    super.setUp();
+    dfscluster = new MiniDFSCluster(new JobConf(), 2, true, null);
+    fs = dfscluster.getFileSystem();
+    mapred = new MiniMRCluster(2, fs.getUri().toString(), 1);
+    inputPath = new Path(fs.getHomeDirectory(), "test"); 
+    filea = new Path(inputPath,"a");
+    fileb = new Path(inputPath,"b");
+    filec = new Path(inputPath,"c");
+    // check for har containing escape worthy characters
+    // in there name
+    filed = new Path(inputPath, "d%d");
+    archivePath = new Path(fs.getHomeDirectory(), "tmp");
+  }
+  
+  protected void tearDown() throws Exception {
+    try {
+      if (mapred != null) {
+        mapred.shutdown();
+      }
+      if (dfscluster != null) {
+        dfscluster.shutdown();
+      }
+    } catch(Exception e) {
+      System.err.println(e);
+    }
+    super.tearDown();
+  }
+  
+  static class TextMapperReducer implements Mapper<LongWritable, Text, Text, Text>, 
+            Reducer<Text, Text, Text, Text> {
+    
+    public void configure(JobConf conf) {
+      //do nothing 
+    }
+
+    public void map(LongWritable key, Text value, OutputCollector<Text, Text> output, Reporter reporter) throws IOException {
+      output.collect(value, new Text(""));
+    }
+
+    public void close() throws IOException {
+      // do nothing
+    }
+
+    public void reduce(Text key, Iterator<Text> values, OutputCollector<Text, Text> output, Reporter reporter) throws IOException {
+      while(values.hasNext()) { 
+        values.next();
+        output.collect(key, null);
+      }
+    }
+  }
+  
+  public void testArchives() throws Exception {
+    fs.mkdirs(inputPath);
+    
+    FSDataOutputStream out = fs.create(filea); 
+    out.write("a".getBytes());
+    out.close();
+    out = fs.create(fileb);
+    out.write("b".getBytes());
+    out.close();
+    out = fs.create(filec);
+    out.write("c".getBytes());
+    out.close();
+    out = fs.create(filed);
+    out.write("d".getBytes());
+    out.close();
+    Configuration conf = mapred.createJobConf();
+    
+    // check to see if fs.har.impl.disable.cache is true
+    boolean archivecaching = conf.getBoolean("fs.har.impl.disable.cache", false);
+    assertTrue(archivecaching);
+    HadoopArchives har = new HadoopArchives(conf);
+    String[] args = new String[3];
+    //check for destination not specfied
+    args[0] = "-archiveName";
+    args[1] = "foo.har";
+    args[2] = inputPath.toString();
+    int ret = ToolRunner.run(har, args);
+    assertTrue(ret != 0);
+    args = new String[4];
+    //check for wrong archiveName
+    args[0] = "-archiveName";
+    args[1] = "/d/foo.har";
+    args[2] = inputPath.toString();
+    args[3] = archivePath.toString();
+    ret = ToolRunner.run(har, args);
+    assertTrue(ret != 0);
+//  se if dest is a file 
+    args[1] = "foo.har";
+    args[3] = filec.toString();
+    ret = ToolRunner.run(har, args);
+    assertTrue(ret != 0);
+    //this is a valid run
+    args[0] = "-archiveName";
+    args[1] = "foo.har";
+    args[2] = inputPath.toString();
+    args[3] = archivePath.toString();
+    ret = ToolRunner.run(har, args);
+    //checl for the existenece of the archive
+    assertTrue(ret == 0);
+    ///try running it again. it should not 
+    // override the directory
+    ret = ToolRunner.run(har, args);
+    assertTrue(ret != 0);
+    Path finalPath = new Path(archivePath, "foo.har");
+    Path fsPath = new Path(inputPath.toUri().getPath());
+    String relative = fsPath.toString().substring(1);
+    Path filePath = new Path(finalPath, relative);
+    //make it a har path 
+    Path harPath = new Path("har://" + filePath.toUri().getPath());
+    assertTrue(fs.exists(new Path(finalPath, "_index")));
+    assertTrue(fs.exists(new Path(finalPath, "_masterindex")));
+    assertTrue(!fs.exists(new Path(finalPath, "_logs")));
+    //creation tested
+    //check if the archive is same
+    // do ls and cat on all the files
+    FsShell shell = new FsShell(conf);
+    args = new String[2];
+    args[0] = "-ls";
+    args[1] = harPath.toString();
+    ret = ToolRunner.run(shell, args);
+    // ls should work.
+    assertTrue((ret == 0));
+    //now check for contents of filea
+    // fileb and filec
+    Path harFilea = new Path(harPath, "a");
+    Path harFileb = new Path(harPath, "b");
+    Path harFilec = new Path(harPath, "c");
+    Path harFiled = new Path(harPath, "d%d");
+    FileSystem harFs = harFilea.getFileSystem(conf);
+    FSDataInputStream fin = harFs.open(harFilea);
+    byte[] b = new byte[4];
+    int readBytes = fin.read(b);
+    fin.close();
+    assertTrue("strings are equal ", (b[0] == "a".getBytes()[0]));
+    fin = harFs.open(harFileb);
+    fin.read(b);
+    fin.close();
+    assertTrue("strings are equal ", (b[0] == "b".getBytes()[0]));
+    fin = harFs.open(harFilec);
+    fin.read(b);
+    fin.close();
+    assertTrue("strings are equal ", (b[0] == "c".getBytes()[0]));
+    fin = harFs.open(harFiled);
+    fin.read(b);
+    fin.close();
+    assertTrue("strings are equal ", (b[0] == "d".getBytes()[0]));
+    
+    // ok all files match 
+    // run a map reduce job
+    Path outdir = new Path(fs.getHomeDirectory(), "mapout"); 
+    JobConf jobconf = mapred.createJobConf();
+    FileInputFormat.addInputPath(jobconf, harPath);
+    jobconf.setInputFormat(TextInputFormat.class);
+    jobconf.setOutputFormat(TextOutputFormat.class);
+    FileOutputFormat.setOutputPath(jobconf, outdir);
+    jobconf.setMapperClass(TextMapperReducer.class);
+    jobconf.setMapOutputKeyClass(Text.class);
+    jobconf.setMapOutputValueClass(Text.class);
+    jobconf.setReducerClass(TextMapperReducer.class);
+    jobconf.setNumReduceTasks(1);
+    JobClient.runJob(jobconf);
+    args[1] = outdir.toString();
+    ret = ToolRunner.run(shell, args);
+    
+    FileStatus[] status = fs.globStatus(new Path(outdir, "part*"));
+    Path reduceFile = status[0].getPath();
+    FSDataInputStream reduceIn = fs.open(reduceFile);
+    b = new byte[8];
+    reduceIn.read(b);
+    //assuming all the 8 bytes were read.
+    Text readTxt = new Text(b);
+    assertTrue("a\nb\nc\nd\n".equals(readTxt.toString()));
+    assertTrue("number of bytes left should be -1", reduceIn.read(b) == -1);
+    reduceIn.close();
+  }
+}

+ 164 - 0
src/test/org/apache/hadoop/io/compress/TestCodec.java

@@ -19,13 +19,22 @@ package org.apache.hadoop.io.compress;
 
 import java.io.BufferedInputStream;
 import java.io.BufferedOutputStream;
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
 import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
 import java.util.Arrays;
 import java.util.Random;
+import java.util.zip.GZIPInputStream;
 import java.util.zip.GZIPOutputStream;
 
 import junit.framework.TestCase;
@@ -45,8 +54,11 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.io.SequenceFile.CompressionType;
+import org.apache.hadoop.io.compress.CompressorStream;
 import org.apache.hadoop.io.compress.CompressionOutputStream;
 import org.apache.hadoop.io.compress.zlib.BuiltInGzipDecompressor;
+import org.apache.hadoop.io.compress.zlib.BuiltInZlibDeflater;
+import org.apache.hadoop.io.compress.zlib.BuiltInZlibInflater;
 import org.apache.hadoop.io.compress.zlib.ZlibCompressor.CompressionLevel;
 import org.apache.hadoop.io.compress.zlib.ZlibCompressor.CompressionStrategy;
 import org.apache.hadoop.io.compress.zlib.ZlibFactory;
@@ -447,4 +459,156 @@ public class TestCodec extends TestCase {
     super(name);
   }
 
+  public void testCodecPoolAndGzipDecompressor() {
+    // BuiltInZlibInflater should not be used as the GzipCodec decompressor.
+    // Assert that this is the case.
+
+    // Don't use native libs for this test.
+    Configuration conf = new Configuration();
+    conf.setBoolean("hadoop.native.lib", false);
+    assertFalse("ZlibFactory is using native libs against request",
+        ZlibFactory.isNativeZlibLoaded(conf));
+
+    // This should give us a BuiltInZlibInflater.
+    Decompressor zlibDecompressor = ZlibFactory.getZlibDecompressor(conf);
+    assertNotNull("zlibDecompressor is null!", zlibDecompressor);
+    assertTrue("ZlibFactory returned unexpected inflator",
+        zlibDecompressor instanceof BuiltInZlibInflater);
+
+    // its createOutputStream() just wraps the existing stream in a
+    // java.util.zip.GZIPOutputStream.
+    CompressionCodecFactory ccf = new CompressionCodecFactory(conf);
+    CompressionCodec codec = ccf.getCodec(new Path("foo.gz"));
+    assertTrue("Codec for .gz file is not GzipCodec", 
+               codec instanceof GzipCodec);
+
+    // make sure we don't get a null decompressor
+    Decompressor codecDecompressor = codec.createDecompressor();
+    if (null == codecDecompressor) {
+      fail("Got null codecDecompressor");
+    }
+
+    // Asking the CodecPool for a decompressor for GzipCodec
+    // should not return null
+    Decompressor poolDecompressor = CodecPool.getDecompressor(codec);
+    if (null == poolDecompressor) {
+      fail("Got null poolDecompressor");
+    }
+    // return a couple decompressors
+    CodecPool.returnDecompressor(zlibDecompressor);
+    CodecPool.returnDecompressor(poolDecompressor);
+    Decompressor poolDecompressor2 = CodecPool.getDecompressor(codec);
+    if (poolDecompressor.getClass() == BuiltInGzipDecompressor.class) {
+      if (poolDecompressor == poolDecompressor2) {
+        fail("Reused java gzip decompressor in pool");
+      }
+    } else {
+      if (poolDecompressor != poolDecompressor2) {
+        fail("Did not reuse native gzip decompressor in pool");
+      }
+    }
+  }
+
+  public void testGzipCodecRead() throws IOException {
+    // Create a gzipped file and try to read it back, using a decompressor
+    // from the CodecPool.
+
+    // Don't use native libs for this test.
+    Configuration conf = new Configuration();
+    conf.setBoolean("hadoop.native.lib", false);
+    assertFalse("ZlibFactory is using native libs against request",
+        ZlibFactory.isNativeZlibLoaded(conf));
+
+    // Ensure that the CodecPool has a BuiltInZlibInflater in it.
+    Decompressor zlibDecompressor = ZlibFactory.getZlibDecompressor(conf);
+    assertNotNull("zlibDecompressor is null!", zlibDecompressor);
+    assertTrue("ZlibFactory returned unexpected inflator",
+        zlibDecompressor instanceof BuiltInZlibInflater);
+    CodecPool.returnDecompressor(zlibDecompressor);
+
+    // Now create a GZip text file.
+    String tmpDir = System.getProperty("test.build.data", "/tmp/");
+    Path f = new Path(new Path(tmpDir), "testGzipCodecRead.txt.gz");
+    BufferedWriter bw = new BufferedWriter(new OutputStreamWriter(
+      new GZIPOutputStream(new FileOutputStream(f.toString()))));
+    final String msg = "This is the message in the file!";
+    bw.write(msg);
+    bw.close();
+
+    // Now read it back, using the CodecPool to establish the
+    // decompressor to use.
+    CompressionCodecFactory ccf = new CompressionCodecFactory(conf);
+    CompressionCodec codec = ccf.getCodec(f);
+    Decompressor decompressor = CodecPool.getDecompressor(codec);
+    FileSystem fs = FileSystem.getLocal(conf);
+    InputStream is = fs.open(f);
+    is = codec.createInputStream(is, decompressor);
+    BufferedReader br = new BufferedReader(new InputStreamReader(is));
+    String line = br.readLine();
+    assertEquals("Didn't get the same message back!", msg, line);
+    br.close();
+  }
+
+  private void verifyGzipFile(String filename, String msg) throws IOException {
+    BufferedReader r = new BufferedReader(new InputStreamReader(
+        new GZIPInputStream(new FileInputStream(filename))));
+    try {
+      String line = r.readLine();
+      assertEquals("Got invalid line back from " + filename, msg, line);
+    } finally {
+      r.close();
+      new File(filename).delete();
+    }
+  }
+
+  public void testGzipCodecWrite() throws IOException {
+    // Create a gzipped file using a compressor from the CodecPool,
+    // and try to read it back via the regular GZIPInputStream.
+
+    // Don't use native libs for this test.
+    Configuration conf = new Configuration();
+    conf.setBoolean("hadoop.native.lib", false);
+    assertFalse("ZlibFactory is using native libs against request",
+        ZlibFactory.isNativeZlibLoaded(conf));
+
+    // Ensure that the CodecPool has a BuiltInZlibDeflater in it.
+    Compressor zlibCompressor = ZlibFactory.getZlibCompressor(conf);
+    assertNotNull("zlibCompressor is null!", zlibCompressor);
+    assertTrue("ZlibFactory returned unexpected deflator",
+        zlibCompressor instanceof BuiltInZlibDeflater);
+    CodecPool.returnCompressor(zlibCompressor);
+
+    // Create a GZIP text file via the Compressor interface.
+    CompressionCodecFactory ccf = new CompressionCodecFactory(conf);
+    CompressionCodec codec = ccf.getCodec(new Path("foo.gz"));
+    assertTrue("Codec for .gz file is not GzipCodec", codec instanceof GzipCodec);
+
+    final String msg = "This is the message we are going to compress.";
+    final String tmpDir = System.getProperty("test.build.data", "/tmp/");
+    final String fileName = new Path(new Path(tmpDir),
+        "testGzipCodecWrite.txt.gz").toString();
+
+    BufferedWriter w = null;
+    Compressor gzipCompressor = CodecPool.getCompressor(codec);
+    if (null != gzipCompressor) {
+      // If it gives us back a Compressor, we should be able to use this
+      // to write files we can then read back with Java's gzip tools.
+      OutputStream os = new CompressorStream(new FileOutputStream(fileName),
+          gzipCompressor);
+      w = new BufferedWriter(new OutputStreamWriter(os));
+      w.write(msg);
+      w.close();
+      CodecPool.returnCompressor(gzipCompressor);
+
+      verifyGzipFile(fileName, msg);
+    }
+
+    // Create a gzip text file via codec.getOutputStream().
+    w = new BufferedWriter(new OutputStreamWriter(
+        codec.createOutputStream(new FileOutputStream(fileName))));
+    w.write(msg);
+    w.close();
+
+    verifyGzipFile(fileName, msg);
+  }
 }

+ 12 - 22
src/test/org/apache/hadoop/mapred/lib/TestCombineFileInputFormat.java

@@ -18,11 +18,6 @@
 package org.apache.hadoop.mapred.lib;
 
 import java.io.IOException;
-import java.io.DataOutputStream;
-import java.util.BitSet;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Random;
 
 import junit.framework.TestCase;
 
@@ -30,17 +25,12 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.fs.BlockLocation;
-import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.SequenceFile.CompressionType;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.JobConf;
@@ -151,14 +141,14 @@ public class TestCombineFileInputFormat extends TestCase{
       assertEquals(fileSplit.getPath(1).getName(), file2.getName());
       assertEquals(fileSplit.getOffset(1), BLOCKSIZE);
       assertEquals(fileSplit.getLength(1), BLOCKSIZE);
-      assertEquals(fileSplit.getLocations()[0], "/r2");
+      assertEquals(fileSplit.getLocations()[0], hosts2[0]); // should be on r2
       fileSplit = (CombineFileSplit) splits[1];
       assertEquals(fileSplit.getNumPaths(), 1);
       assertEquals(fileSplit.getLocations().length, 1);
       assertEquals(fileSplit.getPath(0).getName(), file1.getName());
       assertEquals(fileSplit.getOffset(0), 0);
       assertEquals(fileSplit.getLength(0), BLOCKSIZE);
-      assertEquals(fileSplit.getLocations()[0], "/r1");
+      assertEquals(fileSplit.getLocations()[0], hosts1[0]); // should be on r1
 
       // create another file on 3 datanodes and 3 racks.
       dfs.startDataNodes(conf, 1, true, null, rack3, hosts3, null);
@@ -186,7 +176,7 @@ public class TestCombineFileInputFormat extends TestCase{
       assertEquals(fileSplit.getPath(2).getName(), file3.getName());
       assertEquals(fileSplit.getOffset(2), 2 * BLOCKSIZE);
       assertEquals(fileSplit.getLength(2), BLOCKSIZE);
-      assertEquals(fileSplit.getLocations()[0], "/r3");
+      assertEquals(fileSplit.getLocations()[0], hosts3[0]); // should be on r3
       fileSplit = (CombineFileSplit) splits[1];
       assertEquals(fileSplit.getNumPaths(), 2);
       assertEquals(fileSplit.getLocations().length, 1);
@@ -196,14 +186,14 @@ public class TestCombineFileInputFormat extends TestCase{
       assertEquals(fileSplit.getPath(1).getName(), file2.getName());
       assertEquals(fileSplit.getOffset(1), BLOCKSIZE);
       assertEquals(fileSplit.getLength(1), BLOCKSIZE);
-      assertEquals(fileSplit.getLocations()[0], "/r2");
+      assertEquals(fileSplit.getLocations()[0], hosts2[0]); // should be on r2
       fileSplit = (CombineFileSplit) splits[2];
       assertEquals(fileSplit.getNumPaths(), 1);
       assertEquals(fileSplit.getLocations().length, 1);
       assertEquals(fileSplit.getPath(0).getName(), file1.getName());
       assertEquals(fileSplit.getOffset(0), 0);
       assertEquals(fileSplit.getLength(0), BLOCKSIZE);
-      assertEquals(fileSplit.getLocations()[0], "/r1");
+      assertEquals(fileSplit.getLocations()[0], hosts1[0]); // should be on r1
 
       // create file4 on all three racks
       Path file4 = new Path(dir4 + "/file4");
@@ -229,7 +219,7 @@ public class TestCombineFileInputFormat extends TestCase{
       assertEquals(fileSplit.getPath(2).getName(), file3.getName());
       assertEquals(fileSplit.getOffset(2), 2 * BLOCKSIZE);
       assertEquals(fileSplit.getLength(2), BLOCKSIZE);
-      assertEquals(fileSplit.getLocations()[0], "/r3");
+      assertEquals(fileSplit.getLocations()[0], hosts3[0]); // should be on r3
       fileSplit = (CombineFileSplit) splits[1];
       assertEquals(fileSplit.getNumPaths(), 2);
       assertEquals(fileSplit.getLocations().length, 1);
@@ -239,14 +229,14 @@ public class TestCombineFileInputFormat extends TestCase{
       assertEquals(fileSplit.getPath(1).getName(), file2.getName());
       assertEquals(fileSplit.getOffset(1), BLOCKSIZE);
       assertEquals(fileSplit.getLength(1), BLOCKSIZE);
-      assertEquals(fileSplit.getLocations()[0], "/r2");
+      assertEquals(fileSplit.getLocations()[0], hosts2[0]); // should be on r2
       fileSplit = (CombineFileSplit) splits[2];
       assertEquals(fileSplit.getNumPaths(), 1);
       assertEquals(fileSplit.getLocations().length, 1);
       assertEquals(fileSplit.getPath(0).getName(), file1.getName());
       assertEquals(fileSplit.getOffset(0), 0);
       assertEquals(fileSplit.getLength(0), BLOCKSIZE);
-      assertEquals(fileSplit.getLocations()[0], "/r1");
+      assertEquals(fileSplit.getLocations()[0], hosts1[0]); // should be on r1
 
       // maximum split size is 2 blocks 
       inFormat = new DummyInputFormat();
@@ -385,7 +375,7 @@ public class TestCombineFileInputFormat extends TestCase{
       assertEquals(fileSplit.getPath(0).getName(), file1.getName());
       assertEquals(fileSplit.getOffset(0), 0);
       assertEquals(fileSplit.getLength(0), BLOCKSIZE);
-      assertEquals(fileSplit.getLocations()[0], "/r1");
+      assertEquals(fileSplit.getLocations()[0], hosts1[0]); // should be on r1
 
       // maximum split size is 7 blocks and min is 3 blocks
       inFormat = new DummyInputFormat();
@@ -431,15 +421,15 @@ public class TestCombineFileInputFormat extends TestCase{
       fileSplit = (CombineFileSplit) splits[0];
       assertEquals(fileSplit.getNumPaths(), 2);
       assertEquals(fileSplit.getLocations().length, 1);
-      assertEquals(fileSplit.getLocations()[0], "/r2");
+      assertEquals(fileSplit.getLocations()[0], hosts2[0]); // should be on r2
       fileSplit = (CombineFileSplit) splits[1];
       assertEquals(fileSplit.getNumPaths(), 1);
       assertEquals(fileSplit.getLocations().length, 1);
-      assertEquals(fileSplit.getLocations()[0], "/r1");
+      assertEquals(fileSplit.getLocations()[0], hosts1[0]); // should be on r1
       fileSplit = (CombineFileSplit) splits[2];
       assertEquals(fileSplit.getNumPaths(), 6);
       assertEquals(fileSplit.getLocations().length, 1);
-      assertEquals(fileSplit.getLocations()[0], "/r3");
+      assertEquals(fileSplit.getLocations()[0], hosts3[0]); // should be on r3
     } finally {
       if (dfs != null) {
         dfs.shutdown();

+ 0 - 15
src/test/org/apache/hadoop/metrics2/impl/TestMetricsConfig.java

@@ -109,21 +109,6 @@ public class TestMetricsConfig {
                  t2i1.getString("bar"));
   }
 
-  /**
-   * Should throw if missing config files
-   */
-  @Test public void testMissingFiles() {
-    try {
-      MetricsConfig.create("JobTracker");
-    }
-    catch (MetricsConfigException e) {
-      assertTrue("expected the 'cannot locate configuration' exception",
-                 e.getMessage().startsWith("Cannot locate configuration"));
-      return;
-    }
-    fail("should've thrown");
-  }
-
   /**
    * Test the config file load order
    * @throws Exception

+ 692 - 0
src/tools/org/apache/hadoop/tools/HadoopArchives.java

@@ -0,0 +1,692 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.tools;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.HarFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mapred.SequenceFileRecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.lib.NullOutputFormat;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+
+/**
+ * a archive creation utility.
+ * This class provides methods that can be used 
+ * to create hadoop archives. For understanding of 
+ * Hadoop archives look at {@link HarFileSystem}.
+ */
+public class HadoopArchives implements Tool {
+  private static final Log LOG = LogFactory.getLog(HadoopArchives.class);
+  
+  private static final String NAME = "har"; 
+  static final String SRC_LIST_LABEL = NAME + ".src.list";
+  static final String DST_DIR_LABEL = NAME + ".dest.path";
+  static final String TMP_DIR_LABEL = NAME + ".tmp.dir";
+  static final String JOB_DIR_LABEL = NAME + ".job.dir";
+  static final String SRC_COUNT_LABEL = NAME + ".src.count";
+  static final String TOTAL_SIZE_LABEL = NAME + ".total.size";
+  static final String DST_HAR_LABEL = NAME + ".archive.name";
+  // size of each part file
+  // its fixed for now.
+  static final long partSize = 2 * 1024 * 1024 * 1024l;
+
+  private static final String usage = "archive"
+  + " -archiveName NAME <src>* <dest>" +
+  "\n";
+  
+ 
+  private JobConf conf;
+
+  public void setConf(Configuration conf) {
+    if (conf instanceof JobConf) {
+      this.conf = (JobConf) conf;
+    } else {
+      this.conf = new JobConf(conf, HadoopArchives.class);
+    }
+  }
+
+  public Configuration getConf() {
+    return this.conf;
+  }
+
+  public HadoopArchives(Configuration conf) {
+    setConf(conf);
+  }
+
+  // check the src paths
+  private static void checkPaths(Configuration conf, List<Path> paths) throws
+  IOException {
+    for (Path p : paths) {
+      FileSystem fs = p.getFileSystem(conf);
+      if (!fs.exists(p)) {
+        throw new FileNotFoundException("Source " + p + " does not exist.");
+      }
+    }
+  }
+
+  /**
+   * this assumes that there are two types of files file/dir
+   * @param fs the input filesystem
+   * @param p the top level path 
+   * @param out the list of paths output of recursive ls
+   * @throws IOException
+   */
+  private void recursivels(FileSystem fs, Path p, List<FileStatus> out) 
+  throws IOException {
+    FileStatus fstatus = fs.getFileStatus(p);
+    if (!fstatus.isDir()) {
+      out.add(fstatus);
+      return;
+    }
+    else {
+      out.add(fstatus);
+      FileStatus[] listStatus = fs.listStatus(p);
+      for (FileStatus stat: listStatus) {
+        recursivels(fs, stat.getPath(), out);
+      }
+    }
+  }
+
+  /**
+   * Input format of a hadoop archive job responsible for 
+   * generating splits of the file list
+   */
+
+  static class HArchiveInputFormat implements InputFormat<LongWritable, Text> {
+
+    //generate input splits from the src file lists
+    public InputSplit[] getSplits(JobConf jconf, int numSplits)
+    throws IOException {
+      String srcfilelist = jconf.get(SRC_LIST_LABEL, "");
+      if ("".equals(srcfilelist)) {
+          throw new IOException("Unable to get the " +
+              "src file for archive generation.");
+      }
+      long totalSize = jconf.getLong(TOTAL_SIZE_LABEL, -1);
+      if (totalSize == -1) {
+        throw new IOException("Invalid size of files to archive");
+      }
+      //we should be safe since this is set by our own code
+      Path src = new Path(srcfilelist);
+      FileSystem fs = src.getFileSystem(jconf);
+      FileStatus fstatus = fs.getFileStatus(src);
+      ArrayList<FileSplit> splits = new ArrayList<FileSplit>(numSplits);
+      LongWritable key = new LongWritable();
+      Text value = new Text();
+      SequenceFile.Reader reader = null;
+      // the remaining bytes in the file split
+      long remaining = fstatus.getLen();
+      // the count of sizes calculated till now
+      long currentCount = 0L;
+      // the endposition of the split
+      long lastPos = 0L;
+      // the start position of the split
+      long startPos = 0L;
+      long targetSize = totalSize/numSplits;
+      // create splits of size target size so that all the maps 
+      // have equals sized data to read and write to.
+      try {
+        reader = new SequenceFile.Reader(fs, src, jconf);
+        while(reader.next(key, value)) {
+          if (currentCount + key.get() > targetSize && currentCount != 0){
+            long size = lastPos - startPos;
+            splits.add(new FileSplit(src, startPos, size, (String[]) null));
+            remaining = remaining - size;
+            startPos = lastPos;
+            currentCount = 0L;
+          }
+          currentCount += key.get();
+          lastPos = reader.getPosition();
+        }
+        // the remaining not equal to the target size.
+        if (remaining != 0) {
+          splits.add(new FileSplit(src, startPos, remaining, (String[])null));
+        }
+      }
+      finally { 
+        reader.close();
+      }
+      return splits.toArray(new FileSplit[splits.size()]);
+    }
+
+    public RecordReader<LongWritable, Text> getRecordReader(InputSplit split,
+        JobConf job, Reporter reporter) throws IOException {
+      return new SequenceFileRecordReader<LongWritable, Text>(job,
+                 (FileSplit)split);
+    }
+  }
+
+  private boolean checkValidName(String name) {
+    Path tmp = new Path(name);
+    if (tmp.depth() != 1) {
+      return false;
+    }
+    if (name.endsWith(".har")) 
+      return true;
+    return false;
+  }
+  
+
+  private Path largestDepth(List<Path> paths) {
+    Path deepest = paths.get(0);
+    for (Path p: paths) {
+      if (p.depth() > deepest.depth()) {
+        deepest = p;
+      }
+    }
+    return deepest;
+  }
+  
+  // this method is tricky. This method writes 
+  // the top level directories in such a way so that 
+  // the output only contains valid directoreis in archives.
+  // so for an input path specified by the user 
+  // as /user/hadoop
+  // we need to index 
+  // / as the root 
+  // /user as a directory
+  // /user/hadoop as a directory
+  // so for multiple input paths it makes sure that it
+  // does the right thing.
+  // so if the user specifies the input directories as 
+  // /user/harry and /user/hadoop
+  // we need to write / and user as its child
+  // and /user and harry and hadoop as its children
+  private void writeTopLevelDirs(SequenceFile.Writer srcWriter, 
+      List<Path> paths) throws IOException {
+    //these are qualified paths 
+    List<Path> justDirs = new ArrayList<Path>();
+    for (Path p: paths) {
+      if (!p.getFileSystem(getConf()).isFile(p)) {
+        justDirs.add(new Path(p.toUri().getPath()));
+      }
+      else {
+        justDirs.add(new Path(p.getParent().toUri().getPath()));
+      }
+    }
+    
+    //get the largest depth path
+    // this is tricky
+    TreeMap<String, HashSet<String>> allpaths = new TreeMap<String, HashSet<String>>();
+    Path deepest = largestDepth(paths);
+    Path root = new Path(Path.SEPARATOR);
+    for (int i = 0; i < deepest.depth(); i++) {
+      List<Path> parents = new ArrayList<Path>();
+      for (Path p: justDirs) {
+        if (p.compareTo(root) == 0){
+          //don nothing
+        }
+        else {
+          Path parent = p.getParent();
+          if (null != parent) {
+            if (allpaths.containsKey(parent.toString())) {
+              HashSet<String> children = allpaths.get(parent.toString());
+              children.add(p.getName());
+            } 
+            else {
+              HashSet<String> children = new HashSet<String>();
+              children.add(p.getName());
+              allpaths.put(parent.toString(), children);
+            }
+            parents.add(parent);
+          }
+        }
+      }
+      justDirs = parents;
+    }
+    Set<Map.Entry<String, HashSet<String>>> keyVals = allpaths.entrySet();
+    for (Map.Entry<String, HashSet<String>> entry : keyVals) {
+      HashSet<String> children = entry.getValue();
+      String toWrite = entry.getKey() + " dir ";
+      StringBuffer sbuff = new StringBuffer();
+      sbuff.append(toWrite);
+      for (String child: children) {
+        sbuff.append(child + " ");
+      }
+      toWrite = sbuff.toString();
+      srcWriter.append(new LongWritable(0L), new Text(toWrite));
+    }
+  }
+  
+  /**archive the given source paths into
+   * the dest
+   * @param srcPaths the src paths to be archived
+   * @param dest the dest dir that will contain the archive
+   */
+  public void archive(List<Path> srcPaths, String archiveName, Path dest) 
+  throws IOException {
+    checkPaths(conf, srcPaths);
+    int numFiles = 0;
+    long totalSize = 0;
+    conf.set(DST_HAR_LABEL, archiveName);
+    Path outputPath = new Path(dest, archiveName);
+    FileOutputFormat.setOutputPath(conf, outputPath);
+    FileSystem outFs = outputPath.getFileSystem(conf);
+    if (outFs.exists(outputPath) || outFs.isFile(dest)) {
+      throw new IOException("Invalid Output.");
+    }
+    conf.set(DST_DIR_LABEL, outputPath.toString());
+    final String randomId = DistCp.getRandomId();
+    Path jobDirectory = new Path(new JobClient(conf).getSystemDir(),
+                          NAME + "_" + randomId);
+    conf.set(JOB_DIR_LABEL, jobDirectory.toString());
+    //get a tmp directory for input splits
+    FileSystem jobfs = jobDirectory.getFileSystem(conf);
+    jobfs.mkdirs(jobDirectory);
+    Path srcFiles = new Path(jobDirectory, "_har_src_files");
+    conf.set(SRC_LIST_LABEL, srcFiles.toString());
+    SequenceFile.Writer srcWriter = SequenceFile.createWriter(jobfs, conf,
+        srcFiles, LongWritable.class, Text.class, 
+        SequenceFile.CompressionType.NONE);
+    // get the list of files 
+    // create single list of files and dirs
+    try {
+      // write the top level dirs in first 
+      writeTopLevelDirs(srcWriter, srcPaths);
+      srcWriter.sync();
+      // these are the input paths passed 
+      // from the command line
+      // we do a recursive ls on these paths 
+      // and then write them to the input file 
+      // one at a time
+      for (Path src: srcPaths) {
+        FileSystem fs = src.getFileSystem(conf);
+        ArrayList<FileStatus> allFiles = new ArrayList<FileStatus>();
+        recursivels(fs, src, allFiles);
+        for (FileStatus stat: allFiles) {
+          String toWrite = "";
+          long len = stat.isDir()? 0:stat.getLen();
+          if (stat.isDir()) {
+            toWrite = "" + fs.makeQualified(stat.getPath()) + " dir ";
+            //get the children 
+            FileStatus[] list = fs.listStatus(stat.getPath());
+            StringBuffer sbuff = new StringBuffer();
+            sbuff.append(toWrite);
+            for (FileStatus stats: list) {
+              sbuff.append(stats.getPath().getName() + " ");
+            }
+            toWrite = sbuff.toString();
+          }
+          else {
+            toWrite +=  fs.makeQualified(stat.getPath()) + " file ";
+          }
+          srcWriter.append(new LongWritable(len), new 
+              Text(toWrite));
+          srcWriter.sync();
+          numFiles++;
+          totalSize += len;
+        }
+      }
+    } finally {
+      srcWriter.close();
+    }
+    //increase the replication of src files
+    jobfs.setReplication(srcFiles, (short) 10);
+    conf.setInt(SRC_COUNT_LABEL, numFiles);
+    conf.setLong(TOTAL_SIZE_LABEL, totalSize);
+    int numMaps = (int)(totalSize/partSize);
+    //run atleast one map.
+    conf.setNumMapTasks(numMaps == 0? 1:numMaps);
+    conf.setNumReduceTasks(1);
+    conf.setInputFormat(HArchiveInputFormat.class);
+    conf.setOutputFormat(NullOutputFormat.class);
+    conf.setMapperClass(HArchivesMapper.class);
+    conf.setReducerClass(HArchivesReducer.class);
+    conf.setMapOutputKeyClass(IntWritable.class);
+    conf.setMapOutputValueClass(Text.class);
+    conf.set("hadoop.job.history.user.location", "none");
+    FileInputFormat.addInputPath(conf, jobDirectory);
+    //make sure no speculative execution is done
+    conf.setSpeculativeExecution(false);
+    JobClient.runJob(conf);
+    //delete the tmp job directory
+    try {
+      jobfs.delete(jobDirectory, true);
+    } catch(IOException ie) {
+      LOG.info("Unable to clean tmp directory " + jobDirectory);
+    }
+  }
+
+  static class HArchivesMapper 
+  implements Mapper<LongWritable, Text, IntWritable, Text> {
+    private JobConf conf = null;
+    int partId = -1 ; 
+    Path tmpOutputDir = null;
+    Path tmpOutput = null;
+    String partname = null;
+    FSDataOutputStream partStream = null;
+    FileSystem destFs = null;
+    byte[] buffer;
+    int buf_size = 128 * 1024;
+    
+    // configure the mapper and create 
+    // the part file.
+    // use map reduce framework to write into
+    // tmp files. 
+    public void configure(JobConf conf) {
+      this.conf = conf;
+      // this is tightly tied to map reduce
+      // since it does not expose an api 
+      // to get the partition
+      partId = conf.getInt("mapred.task.partition", -1);
+      // create a file name using the partition
+      // we need to write to this directory
+      tmpOutputDir = FileOutputFormat.getWorkOutputPath(conf);
+      // get the output path and write to the tmp 
+      // directory 
+      partname = "part-" + partId;
+      tmpOutput = new Path(tmpOutputDir, partname);
+      try {
+        destFs = tmpOutput.getFileSystem(conf);
+        //this was a stale copy
+        if (destFs.exists(tmpOutput)) {
+          destFs.delete(tmpOutput, false);
+        }
+        partStream = destFs.create(tmpOutput);
+      } catch(IOException ie) {
+        throw new RuntimeException("Unable to open output file " + tmpOutput);
+      }
+      buffer = new byte[buf_size];
+    }
+
+    // copy raw data.
+    public void copyData(Path input, FSDataInputStream fsin, 
+        FSDataOutputStream fout, Reporter reporter) throws IOException {
+      try {
+        for (int cbread=0; (cbread = fsin.read(buffer))>= 0;) {
+          fout.write(buffer, 0,cbread);
+          reporter.progress();
+        }
+      } finally {
+        fsin.close();
+      }
+    }
+    
+    // the relative path of p. basically 
+    // getting rid of schema. Parsing and doing 
+    // string manipulation is not good - so
+    // just use the path api to do it.
+    private Path makeRelative(Path p) {
+      Path retPath = new Path(p.toUri().getPath());
+      return retPath;
+    }
+    
+    static class MapStat {
+      private String pathname;
+      private boolean isDir;
+      private List<String> children;
+      public MapStat(String line) {
+        String[] splits = line.split(" ");
+        pathname = splits[0];
+        if ("dir".equals(splits[1])) {
+          isDir = true;
+        }
+        else {
+          isDir = false;
+        }
+        if (isDir) {
+          children = new ArrayList<String>();
+          for (int i = 2; i < splits.length; i++) {
+            children.add(splits[i]);
+          }
+        }
+      }
+    }
+    // read files from the split input 
+    // and write it onto the part files.
+    // also output hash(name) and string 
+    // for reducer to create index 
+    // and masterindex files.
+    public void map(LongWritable key, Text value,
+        OutputCollector<IntWritable, Text> out,
+        Reporter reporter) throws IOException {
+      String line  = value.toString();
+      MapStat mstat = new MapStat(line);
+      Path srcPath = new Path(mstat.pathname);
+      String towrite = null;
+      Path relPath = makeRelative(srcPath);
+      int hash = HarFileSystem.getHarHash(relPath);
+      long startPos = partStream.getPos();
+      if (mstat.isDir) { 
+        towrite = relPath.toString() + " " + "dir none " + 0 + " " + 0 + " ";
+        StringBuffer sbuff = new StringBuffer();
+        sbuff.append(towrite);
+        for (String child: mstat.children) {
+          sbuff.append(child + " ");
+        }
+        towrite = sbuff.toString();
+        //reading directories is also progress
+        reporter.progress();
+      }
+      else {
+        FileSystem srcFs = srcPath.getFileSystem(conf);
+        FileStatus srcStatus = srcFs.getFileStatus(srcPath);
+        FSDataInputStream input = srcFs.open(srcStatus.getPath());
+        reporter.setStatus("Copying file " + srcStatus.getPath() + 
+            " to archive.");
+        copyData(srcStatus.getPath(), input, partStream, reporter);
+        towrite = relPath.toString() + " file " + partname + " " + startPos
+        + " " + srcStatus.getLen() + " ";
+      }
+      out.collect(new IntWritable(hash), new Text(towrite));
+    }
+    
+    public void close() throws IOException {
+      // close the part files.
+      partStream.close();
+    }
+  }
+  
+  /** the reduce for creating the index and the master index 
+   * 
+   */
+  static class HArchivesReducer implements Reducer<IntWritable, 
+  Text, Text, Text> {
+    private JobConf conf = null;
+    private long startIndex = 0;
+    private long endIndex = 0;
+    private long startPos = 0;
+    private Path masterIndex = null;
+    private Path index = null;
+    private FileSystem fs = null;
+    private FSDataOutputStream outStream = null;
+    private FSDataOutputStream indexStream = null;
+    private int numIndexes = 1000;
+    private Path tmpOutputDir = null;
+    private int written = 0;
+    private int keyVal = 0;
+    
+    // configure 
+    public void configure(JobConf conf) {
+      this.conf = conf;
+      tmpOutputDir = FileOutputFormat.getWorkOutputPath(this.conf);
+      masterIndex = new Path(tmpOutputDir, "_masterindex");
+      index = new Path(tmpOutputDir, "_index");
+      try {
+        fs = masterIndex.getFileSystem(conf);
+        if (fs.exists(masterIndex)) {
+          fs.delete(masterIndex, false);
+        }
+        if (fs.exists(index)) {
+          fs.delete(index, false);
+        }
+        indexStream = fs.create(index);
+        outStream = fs.create(masterIndex);
+        String version = HarFileSystem.VERSION + " \n";
+        outStream.write(version.getBytes());
+        
+      } catch(IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+    
+    // create the index and master index. The input to 
+    // the reduce is already sorted by the hash of the 
+    // files. SO we just need to write it to the index. 
+    // We update the masterindex as soon as we update 
+    // numIndex entries.
+    public void reduce(IntWritable key, Iterator<Text> values,
+        OutputCollector<Text, Text> out,
+        Reporter reporter) throws IOException {
+      keyVal = key.get();
+      while(values.hasNext()) {
+        Text value = values.next();
+        String towrite = value.toString() + "\n";
+        indexStream.write(towrite.getBytes());
+        written++;
+        if (written > numIndexes -1) {
+          // every 1000 indexes we report status
+          reporter.setStatus("Creating index for archives");
+          reporter.progress();
+          endIndex = keyVal;
+          String masterWrite = startIndex + " " + endIndex + " " + startPos 
+                              +  " " + indexStream.getPos() + " \n" ;
+          outStream.write(masterWrite.getBytes());
+          startPos = indexStream.getPos();
+          startIndex = endIndex;
+          written = 0;
+        }
+      }
+    }
+    
+    public void close() throws IOException {
+      //write the last part of the master index.
+      if (written > 0) {
+        String masterWrite = startIndex + " " + keyVal + " " + startPos  +
+                             " " + indexStream.getPos() + " \n";
+        outStream.write(masterWrite.getBytes());
+      }
+      // close the streams
+      outStream.close();
+      indexStream.close();
+      // try increasing the replication 
+      fs.setReplication(index, (short) 10);
+      fs.setReplication(masterIndex, (short) 10);
+    }
+    
+  }
+  
+  /** the main driver for creating the archives
+   *  it takes at least two command line parameters. The src and the 
+   *  dest. It does an lsr on the source paths.
+   *  The mapper created archuves and the reducer creates 
+   *  the archive index.
+   */
+
+  public int run(String[] args) throws Exception {
+    try {
+      List<Path> srcPaths = new ArrayList<Path>();
+      Path destPath = null;
+      // check we were supposed to archive or 
+      // unarchive
+      String archiveName = null;
+      if (args.length < 4) {
+        System.out.println(usage);
+        throw new IOException("Invalid usage.");
+      }
+      if (!"-archiveName".equals(args[0])) {
+        System.out.println(usage);
+        throw new IOException("Archive Name not specified.");
+      }
+      archiveName = args[1];
+      if (!checkValidName(archiveName)) {
+        System.out.println(usage);
+        throw new IOException("Invalid name for archives. " + archiveName);
+      }
+      for (int i = 2; i < args.length; i++) {
+        if (i == (args.length - 1)) {
+          destPath = new Path(args[i]);
+        }
+        else {
+          srcPaths.add(new Path(args[i]));
+        }
+      }
+      if (srcPaths.size() == 0) {
+        System.out.println(usage);
+        throw new IOException("Invalid Usage: No input sources specified.");
+      }
+      // do a glob on the srcPaths and then pass it on
+      List<Path> globPaths = new ArrayList<Path>();
+      for (Path p: srcPaths) {
+        FileSystem fs = p.getFileSystem(getConf());
+        FileStatus[] statuses = fs.globStatus(p);
+        for (FileStatus status: statuses) {
+          globPaths.add(fs.makeQualified(status.getPath()));
+        }
+      }
+      archive(globPaths, archiveName, destPath);
+    } catch(IOException ie) {
+      System.err.println(ie.getLocalizedMessage());
+      return -1;
+    }
+    return 0;
+  }
+
+  /** the main functions **/
+  public static void main(String[] args) {
+    JobConf job = new JobConf(HadoopArchives.class);
+    HadoopArchives harchives = new HadoopArchives(job);
+    int ret = 0;
+
+    try{
+      ret = ToolRunner.run(harchives, args);
+    } catch(Exception e) {
+      LOG.debug("Exception in archives  ", e);
+      System.err.println("Exception in archives");
+      System.err.println(e.getLocalizedMessage());
+      System.exit(1);
+    }
+    System.exit(ret);
+  }
+}