Browse Source

merge trunk into HDFS-4949 branch

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-4949@1535563 13f79535-47bb-0310-9956-ffa450edef68
Andrew Wang 11 years ago
parent
commit
e87b2a3684
87 changed files with 4580 additions and 580 deletions
  1. 15 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 275 28
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
  3. 42 16
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java
  4. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/filter/AbstractPatternFilter.java
  5. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/package-info.java
  6. 1 1
      hadoop-common-project/hadoop-common/src/site/apt/FileSystemShell.apt.vm
  7. 34 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FCStatisticsBaseTest.java
  8. 53 10
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/filter/TestPatternFilter.java
  9. 138 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/sink/TestFileSink.java
  10. 41 1
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  11. 7 1
      hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
  12. 10 0
      hadoop-hdfs-project/hadoop-hdfs/pom.xml
  13. 10 1
      hadoop-hdfs-project/hadoop-hdfs/src/CMakeLists.txt
  14. 12 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs.cmd
  15. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  16. 12 18
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
  17. 7 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java
  18. 6 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
  19. 42 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatus.java
  20. 44 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatusDefault.java
  21. 11 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
  22. 9 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/resources/DatanodeWebHdfsMethods.java
  23. 1 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
  24. 6 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
  25. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java
  26. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/JMXGet.java
  27. 16 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/snapshot/SnapshotDiff.java
  28. 7 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
  29. 75 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java
  30. 8 33
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
  31. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/util/posix_util.c
  32. 116 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfs-dust.js
  33. 265 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.dust.html
  34. 43 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html
  35. 126 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js
  36. 13 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer-block-info.dust.html
  37. 26 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.dust.html
  38. 86 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
  39. 182 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
  40. 0 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dust-full-2.0.0.min.js
  41. 8 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dust-helpers-1.1.1.min.js
  42. 258 0
      hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HdfsNfsGateway.apt.vm
  43. 74 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
  44. 24 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java
  45. 54 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestURLConnectionFactory.java
  46. 6 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTimeouts.java
  47. 358 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestDelegationTokenRemoteFetcher.java
  48. 23 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestJMXGet.java
  49. 160 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestTools.java
  50. 8 0
      hadoop-mapreduce-project/CHANGES.txt
  51. 0 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java
  52. 49 22
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTextOutputFormat.java
  53. 1 0
      hadoop-project/src/site/site.xml
  54. 507 68
      hadoop-tools/hadoop-archives/src/test/java/org/apache/hadoop/tools/TestHadoopArchives.java
  55. 7 0
      hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/io/IdentifierResolver.java
  56. 35 0
      hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/io/KeyOnlyTextInputWriter.java
  57. 90 0
      hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/io/KeyOnlyTextOutputReader.java
  58. 2 2
      hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/io/TextInputWriter.java
  59. 51 0
      hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestStreamingOutputOnlyKeys.java
  60. 68 0
      hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/io/TestKeyOnlyTextOutputReader.java
  61. 24 2
      hadoop-yarn-project/CHANGES.txt
  62. 2 3
      hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd
  63. 125 23
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/HAUtil.java
  64. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RMAdminCLI.java
  65. 243 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestRMAdminCLI.java
  66. 98 16
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/conf/TestHAUtil.java
  67. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMHAProtocolService.java
  68. 30 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java
  69. 0 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/Queue.java
  70. 0 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
  71. 0 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
  72. 1 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
  73. 1 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java
  74. 5 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerConfiguration.java
  75. 25 34
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java
  76. 0 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
  77. 27 65
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/QueueACLsTestBase.java
  78. 3 100
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreTestBase.java
  79. 120 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java
  80. 80 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStore.java
  81. 9 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStoreZKClientConnections.java
  82. 73 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerQueueACLs.java
  83. 55 30
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
  84. 62 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerQueueACLs.java
  85. 34 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
  86. 27 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/FairScheduler.apt.vm
  87. 2 2
      pom.xml

+ 15 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -363,6 +363,15 @@ Release 2.3.0 - UNRELEASED
     HADOOP-9078. enhance unit-test coverage of class
     HADOOP-9078. enhance unit-test coverage of class
     org.apache.hadoop.fs.FileContext (Ivan A. Veselovsky via jeagles)
     org.apache.hadoop.fs.FileContext (Ivan A. Veselovsky via jeagles)
 
 
+    HDFS-5276. FileSystem.Statistics should use thread-local counters to avoid
+    multi-threaded performance issues on read/write.  (Colin Patrick McCabe)
+
+    HADOOP-9291. enhance unit-test coverage of package o.a.h.metrics2 (Ivan A.
+    Veselovsky via jeagles)
+
+    HADOOP-10064. Upgrade to maven antrun plugin version 1.7 (Arpit Agarwal via
+    jeagles)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     HADOOP-9748. Reduce blocking on UGI.ensureInitialized (daryn)
     HADOOP-9748. Reduce blocking on UGI.ensureInitialized (daryn)
@@ -405,6 +414,9 @@ Release 2.3.0 - UNRELEASED
     HADOOP-9981. globStatus should minimize its listStatus and getFileStatus
     HADOOP-9981. globStatus should minimize its listStatus and getFileStatus
     calls.  (Contributed by Colin Patrick McCabe)
     calls.  (Contributed by Colin Patrick McCabe)
 
 
+    HADOOP-9016. HarFsInputStream.skip(long) must never return negative value.
+    (Ivan A. Veselovsky via jeagles)
+
 Release 2.2.1 - UNRELEASED
 Release 2.2.1 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -434,6 +446,9 @@ Release 2.2.1 - UNRELEASED
     HADOOP-10040. hadoop.cmd in UNIX format and would not run by default on
     HADOOP-10040. hadoop.cmd in UNIX format and would not run by default on
     Windows. (cnauroth)
     Windows. (cnauroth)
 
 
+    HADOOP-10055. FileSystemShell.apt.vm doc has typo "numRepicas".
+    (Akira Ajisaka via cnauroth)
+
 Release 2.2.0 - 2013-10-13
 Release 2.2.0 - 2013-10-13
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 275 - 28
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.fs;
 import java.io.Closeable;
 import java.io.Closeable;
 import java.io.FileNotFoundException;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
+import java.lang.ref.WeakReference;
 import java.net.URI;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URISyntaxException;
 import java.security.PrivilegedExceptionAction;
 import java.security.PrivilegedExceptionAction;
@@ -31,6 +32,7 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.HashSet;
 import java.util.IdentityHashMap;
 import java.util.IdentityHashMap;
 import java.util.Iterator;
 import java.util.Iterator;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
 import java.util.NoSuchElementException;
 import java.util.NoSuchElementException;
@@ -2501,28 +2503,149 @@ public abstract class FileSystem extends Configured implements Closeable {
     }
     }
   }
   }
   
   
+  /**
+   * Tracks statistics about how many reads, writes, and so forth have been
+   * done in a FileSystem.
+   * 
+   * Since there is only one of these objects per FileSystem, there will 
+   * typically be many threads writing to this object.  Almost every operation
+   * on an open file will involve a write to this object.  In contrast, reading
+   * statistics is done infrequently by most programs, and not at all by others.
+   * Hence, this is optimized for writes.
+   * 
+   * Each thread writes to its own thread-local area of memory.  This removes 
+   * contention and allows us to scale up to many, many threads.  To read
+   * statistics, the reader thread totals up the contents of all of the 
+   * thread-local data areas.
+   */
   public static final class Statistics {
   public static final class Statistics {
+    /**
+     * Statistics data.
+     * 
+     * There is only a single writer to thread-local StatisticsData objects.
+     * Hence, volatile is adequate here-- we do not need AtomicLong or similar
+     * to prevent lost updates.
+     * The Java specification guarantees that updates to volatile longs will
+     * be perceived as atomic with respect to other threads, which is all we
+     * need.
+     */
+    private static class StatisticsData {
+      volatile long bytesRead;
+      volatile long bytesWritten;
+      volatile int readOps;
+      volatile int largeReadOps;
+      volatile int writeOps;
+      /**
+       * Stores a weak reference to the thread owning this StatisticsData.
+       * This allows us to remove StatisticsData objects that pertain to
+       * threads that no longer exist.
+       */
+      final WeakReference<Thread> owner;
+
+      StatisticsData(WeakReference<Thread> owner) {
+        this.owner = owner;
+      }
+
+      /**
+       * Add another StatisticsData object to this one.
+       */
+      void add(StatisticsData other) {
+        this.bytesRead += other.bytesRead;
+        this.bytesWritten += other.bytesWritten;
+        this.readOps += other.readOps;
+        this.largeReadOps += other.largeReadOps;
+        this.writeOps += other.writeOps;
+      }
+
+      /**
+       * Negate the values of all statistics.
+       */
+      void negate() {
+        this.bytesRead = -this.bytesRead;
+        this.bytesWritten = -this.bytesWritten;
+        this.readOps = -this.readOps;
+        this.largeReadOps = -this.largeReadOps;
+        this.writeOps = -this.writeOps;
+      }
+
+      @Override
+      public String toString() {
+        return bytesRead + " bytes read, " + bytesWritten + " bytes written, "
+            + readOps + " read ops, " + largeReadOps + " large read ops, "
+            + writeOps + " write ops";
+      }
+    }
+
+    private interface StatisticsAggregator<T> {
+      void accept(StatisticsData data);
+      T aggregate();
+    }
+
     private final String scheme;
     private final String scheme;
-    private AtomicLong bytesRead = new AtomicLong();
-    private AtomicLong bytesWritten = new AtomicLong();
-    private AtomicInteger readOps = new AtomicInteger();
-    private AtomicInteger largeReadOps = new AtomicInteger();
-    private AtomicInteger writeOps = new AtomicInteger();
+
+    /**
+     * rootData is data that doesn't belong to any thread, but will be added
+     * to the totals.  This is useful for making copies of Statistics objects,
+     * and for storing data that pertains to threads that have been garbage
+     * collected.  Protected by the Statistics lock.
+     */
+    private final StatisticsData rootData;
+
+    /**
+     * Thread-local data.
+     */
+    private final ThreadLocal<StatisticsData> threadData;
     
     
+    /**
+     * List of all thread-local data areas.  Protected by the Statistics lock.
+     */
+    private LinkedList<StatisticsData> allData;
+
     public Statistics(String scheme) {
     public Statistics(String scheme) {
       this.scheme = scheme;
       this.scheme = scheme;
+      this.rootData = new StatisticsData(null);
+      this.threadData = new ThreadLocal<StatisticsData>();
+      this.allData = null;
     }
     }
 
 
     /**
     /**
      * Copy constructor.
      * Copy constructor.
      * 
      * 
-     * @param st
-     *          The input Statistics object which is cloned.
+     * @param other    The input Statistics object which is cloned.
+     */
+    public Statistics(Statistics other) {
+      this.scheme = other.scheme;
+      this.rootData = new StatisticsData(null);
+      other.visitAll(new StatisticsAggregator<Void>() {
+        @Override
+        public void accept(StatisticsData data) {
+          rootData.add(data);
+        }
+
+        public Void aggregate() {
+          return null;
+        }
+      });
+      this.threadData = new ThreadLocal<StatisticsData>();
+    }
+
+    /**
+     * Get or create the thread-local data associated with the current thread.
      */
      */
-    public Statistics(Statistics st) {
-      this.scheme = st.scheme;
-      this.bytesRead = new AtomicLong(st.bytesRead.longValue());
-      this.bytesWritten = new AtomicLong(st.bytesWritten.longValue());
+    private StatisticsData getThreadData() {
+      StatisticsData data = threadData.get();
+      if (data == null) {
+        data = new StatisticsData(
+            new WeakReference<Thread>(Thread.currentThread()));
+        threadData.set(data);
+        synchronized(this) {
+          if (allData == null) {
+            allData = new LinkedList<StatisticsData>();
+          }
+          allData.add(data);
+        }
+      }
+      return data;
     }
     }
 
 
     /**
     /**
@@ -2530,7 +2653,7 @@ public abstract class FileSystem extends Configured implements Closeable {
      * @param newBytes the additional bytes read
      * @param newBytes the additional bytes read
      */
      */
     public void incrementBytesRead(long newBytes) {
     public void incrementBytesRead(long newBytes) {
-      bytesRead.getAndAdd(newBytes);
+      getThreadData().bytesRead += newBytes;
     }
     }
     
     
     /**
     /**
@@ -2538,7 +2661,7 @@ public abstract class FileSystem extends Configured implements Closeable {
      * @param newBytes the additional bytes written
      * @param newBytes the additional bytes written
      */
      */
     public void incrementBytesWritten(long newBytes) {
     public void incrementBytesWritten(long newBytes) {
-      bytesWritten.getAndAdd(newBytes);
+      getThreadData().bytesWritten += newBytes;
     }
     }
     
     
     /**
     /**
@@ -2546,7 +2669,7 @@ public abstract class FileSystem extends Configured implements Closeable {
      * @param count number of read operations
      * @param count number of read operations
      */
      */
     public void incrementReadOps(int count) {
     public void incrementReadOps(int count) {
-      readOps.getAndAdd(count);
+      getThreadData().readOps += count;
     }
     }
 
 
     /**
     /**
@@ -2554,7 +2677,7 @@ public abstract class FileSystem extends Configured implements Closeable {
      * @param count number of large read operations
      * @param count number of large read operations
      */
      */
     public void incrementLargeReadOps(int count) {
     public void incrementLargeReadOps(int count) {
-      largeReadOps.getAndAdd(count);
+      getThreadData().largeReadOps += count;
     }
     }
 
 
     /**
     /**
@@ -2562,7 +2685,38 @@ public abstract class FileSystem extends Configured implements Closeable {
      * @param count number of write operations
      * @param count number of write operations
      */
      */
     public void incrementWriteOps(int count) {
     public void incrementWriteOps(int count) {
-      writeOps.getAndAdd(count);
+      getThreadData().writeOps += count;
+    }
+
+    /**
+     * Apply the given aggregator to all StatisticsData objects associated with
+     * this Statistics object.
+     *
+     * For each StatisticsData object, we will call accept on the visitor.
+     * Finally, at the end, we will call aggregate to get the final total. 
+     *
+     * @param         The visitor to use.
+     * @return        The total.
+     */
+    private synchronized <T> T visitAll(StatisticsAggregator<T> visitor) {
+      visitor.accept(rootData);
+      if (allData != null) {
+        for (Iterator<StatisticsData> iter = allData.iterator();
+            iter.hasNext(); ) {
+          StatisticsData data = iter.next();
+          visitor.accept(data);
+          if (data.owner.get() == null) {
+            /*
+             * If the thread that created this thread-local data no
+             * longer exists, remove the StatisticsData from our list
+             * and fold the values into rootData.
+             */
+            rootData.add(data);
+            iter.remove();
+          }
+        }
+      }
+      return visitor.aggregate();
     }
     }
 
 
     /**
     /**
@@ -2570,7 +2724,18 @@ public abstract class FileSystem extends Configured implements Closeable {
      * @return the number of bytes
      * @return the number of bytes
      */
      */
     public long getBytesRead() {
     public long getBytesRead() {
-      return bytesRead.get();
+      return visitAll(new StatisticsAggregator<Long>() {
+        private long bytesRead = 0;
+
+        @Override
+        public void accept(StatisticsData data) {
+          bytesRead += data.bytesRead;
+        }
+
+        public Long aggregate() {
+          return bytesRead;
+        }
+      });
     }
     }
     
     
     /**
     /**
@@ -2578,7 +2743,18 @@ public abstract class FileSystem extends Configured implements Closeable {
      * @return the number of bytes
      * @return the number of bytes
      */
      */
     public long getBytesWritten() {
     public long getBytesWritten() {
-      return bytesWritten.get();
+      return visitAll(new StatisticsAggregator<Long>() {
+        private long bytesWritten = 0;
+
+        @Override
+        public void accept(StatisticsData data) {
+          bytesWritten += data.bytesWritten;
+        }
+
+        public Long aggregate() {
+          return bytesWritten;
+        }
+      });
     }
     }
     
     
     /**
     /**
@@ -2586,7 +2762,19 @@ public abstract class FileSystem extends Configured implements Closeable {
      * @return number of read operations
      * @return number of read operations
      */
      */
     public int getReadOps() {
     public int getReadOps() {
-      return readOps.get() + largeReadOps.get();
+      return visitAll(new StatisticsAggregator<Integer>() {
+        private int readOps = 0;
+
+        @Override
+        public void accept(StatisticsData data) {
+          readOps += data.readOps;
+          readOps += data.largeReadOps;
+        }
+
+        public Integer aggregate() {
+          return readOps;
+        }
+      });
     }
     }
 
 
     /**
     /**
@@ -2595,7 +2783,18 @@ public abstract class FileSystem extends Configured implements Closeable {
      * @return number of large read operations
      * @return number of large read operations
      */
      */
     public int getLargeReadOps() {
     public int getLargeReadOps() {
-      return largeReadOps.get();
+      return visitAll(new StatisticsAggregator<Integer>() {
+        private int largeReadOps = 0;
+
+        @Override
+        public void accept(StatisticsData data) {
+          largeReadOps += data.largeReadOps;
+        }
+
+        public Integer aggregate() {
+          return largeReadOps;
+        }
+      });
     }
     }
 
 
     /**
     /**
@@ -2604,22 +2803,70 @@ public abstract class FileSystem extends Configured implements Closeable {
      * @return number of write operations
      * @return number of write operations
      */
      */
     public int getWriteOps() {
     public int getWriteOps() {
-      return writeOps.get();
+      return visitAll(new StatisticsAggregator<Integer>() {
+        private int writeOps = 0;
+
+        @Override
+        public void accept(StatisticsData data) {
+          writeOps += data.writeOps;
+        }
+
+        public Integer aggregate() {
+          return writeOps;
+        }
+      });
     }
     }
 
 
+
     @Override
     @Override
     public String toString() {
     public String toString() {
-      return bytesRead + " bytes read, " + bytesWritten + " bytes written, "
-          + readOps + " read ops, " + largeReadOps + " large read ops, "
-          + writeOps + " write ops";
+      return visitAll(new StatisticsAggregator<String>() {
+        private StatisticsData total = new StatisticsData(null);
+
+        @Override
+        public void accept(StatisticsData data) {
+          total.add(data);
+        }
+
+        public String aggregate() {
+          return total.toString();
+        }
+      });
     }
     }
-    
+
     /**
     /**
-     * Reset the counts of bytes to 0.
+     * Resets all statistics to 0.
+     *
+     * In order to reset, we add up all the thread-local statistics data, and
+     * set rootData to the negative of that.
+     *
+     * This may seem like a counterintuitive way to reset the statsitics.  Why
+     * can't we just zero out all the thread-local data?  Well, thread-local
+     * data can only be modified by the thread that owns it.  If we tried to
+     * modify the thread-local data from this thread, our modification might get
+     * interleaved with a read-modify-write operation done by the thread that
+     * owns the data.  That would result in our update getting lost.
+     *
+     * The approach used here avoids this problem because it only ever reads
+     * (not writes) the thread-local data.  Both reads and writes to rootData
+     * are done under the lock, so we're free to modify rootData from any thread
+     * that holds the lock.
      */
      */
     public void reset() {
     public void reset() {
-      bytesWritten.set(0);
-      bytesRead.set(0);
+      visitAll(new StatisticsAggregator<Void>() {
+        private StatisticsData total = new StatisticsData(null);
+
+        @Override
+        public void accept(StatisticsData data) {
+          total.add(data);
+        }
+
+        public Void aggregate() {
+          total.negate();
+          rootData.add(total);
+          return null;
+        }
+      });
     }
     }
     
     
     /**
     /**

+ 42 - 16
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java

@@ -898,11 +898,15 @@ public class HarFileSystem extends FileSystem {
       private long position, start, end;
       private long position, start, end;
       //The underlying data input stream that the
       //The underlying data input stream that the
       // underlying filesystem will return.
       // underlying filesystem will return.
-      private FSDataInputStream underLyingStream;
+      private final FSDataInputStream underLyingStream;
       //one byte buffer
       //one byte buffer
-      private byte[] oneBytebuff = new byte[1];
+      private final byte[] oneBytebuff = new byte[1];
+      
       HarFsInputStream(FileSystem fs, Path path, long start,
       HarFsInputStream(FileSystem fs, Path path, long start,
           long length, int bufferSize) throws IOException {
           long length, int bufferSize) throws IOException {
+        if (length < 0) {
+          throw new IllegalArgumentException("Negative length ["+length+"]");
+        }
         underLyingStream = fs.open(path, bufferSize);
         underLyingStream = fs.open(path, bufferSize);
         underLyingStream.seek(start);
         underLyingStream.seek(start);
         // the start of this file in the part file
         // the start of this file in the part file
@@ -916,7 +920,7 @@ public class HarFileSystem extends FileSystem {
       @Override
       @Override
       public synchronized int available() throws IOException {
       public synchronized int available() throws IOException {
         long remaining = end - underLyingStream.getPos();
         long remaining = end - underLyingStream.getPos();
-        if (remaining > (long)Integer.MAX_VALUE) {
+        if (remaining > Integer.MAX_VALUE) {
           return Integer.MAX_VALUE;
           return Integer.MAX_VALUE;
         }
         }
         return (int) remaining;
         return (int) remaining;
@@ -948,10 +952,14 @@ public class HarFileSystem extends FileSystem {
         return (ret <= 0) ? -1: (oneBytebuff[0] & 0xff);
         return (ret <= 0) ? -1: (oneBytebuff[0] & 0xff);
       }
       }
       
       
+      // NB: currently this method actually never executed becusae
+      // java.io.DataInputStream.read(byte[]) directly delegates to 
+      // method java.io.InputStream.read(byte[], int, int).
+      // However, potentially it can be invoked, so leave it intact for now.
       @Override
       @Override
       public synchronized int read(byte[] b) throws IOException {
       public synchronized int read(byte[] b) throws IOException {
-        int ret = read(b, 0, b.length);
-        if (ret != -1) {
+        final int ret = read(b, 0, b.length);
+        if (ret > 0) {
           position += ret;
           position += ret;
         }
         }
         return ret;
         return ret;
@@ -980,15 +988,19 @@ public class HarFileSystem extends FileSystem {
       public synchronized long skip(long n) throws IOException {
       public synchronized long skip(long n) throws IOException {
         long tmpN = n;
         long tmpN = n;
         if (tmpN > 0) {
         if (tmpN > 0) {
-          if (position + tmpN > end) {
-            tmpN = end - position;
-          }
+          final long actualRemaining = end - position; 
+          if (tmpN > actualRemaining) {
+            tmpN = actualRemaining;
+          }   
           underLyingStream.seek(tmpN + position);
           underLyingStream.seek(tmpN + position);
           position += tmpN;
           position += tmpN;
           return tmpN;
           return tmpN;
-        }
-        return (tmpN < 0)? -1 : 0;
-      }
+        }   
+        // NB: the contract is described in java.io.InputStream.skip(long):
+        // this method returns the number of bytes actually skipped, so,
+        // the return value should never be negative. 
+        return 0;
+      }   
       
       
       @Override
       @Override
       public synchronized long getPos() throws IOException {
       public synchronized long getPos() throws IOException {
@@ -996,14 +1008,23 @@ public class HarFileSystem extends FileSystem {
       }
       }
       
       
       @Override
       @Override
-      public synchronized void seek(long pos) throws IOException {
-        if (pos < 0 || (start + pos > end)) {
-          throw new IOException("Failed to seek: EOF");
-        }
+      public synchronized void seek(final long pos) throws IOException {
+        validatePosition(pos);
         position = start + pos;
         position = start + pos;
         underLyingStream.seek(position);
         underLyingStream.seek(position);
       }
       }
 
 
+      private void validatePosition(final long pos) throws IOException {
+        if (pos < 0) {
+          throw new IOException("Negative position: "+pos);
+         }
+         final long length = end - start;
+         if (pos > length) {
+           throw new IOException("Position behind the end " +
+               "of the stream (length = "+length+"): " + pos);
+         }
+      }
+
       @Override
       @Override
       public boolean seekToNewSource(long targetPos) throws IOException {
       public boolean seekToNewSource(long targetPos) throws IOException {
         // do not need to implement this
         // do not need to implement this
@@ -1020,7 +1041,12 @@ public class HarFileSystem extends FileSystem {
       throws IOException {
       throws IOException {
         int nlength = length;
         int nlength = length;
         if (start + nlength + pos > end) {
         if (start + nlength + pos > end) {
-          nlength = (int) (end - (start + pos));
+          // length corrected to the real remaining length:
+          nlength = (int) (end - start - pos);
+        }
+        if (nlength <= 0) {
+          // EOS:
+          return -1;
         }
         }
         return underLyingStream.read(pos + start , b, offset, nlength);
         return underLyingStream.read(pos + start , b, offset, nlength);
       }
       }

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/filter/AbstractPatternFilter.java

@@ -112,7 +112,7 @@ public abstract class AbstractPatternFilter extends MetricsFilter {
       return false;
       return false;
     }
     }
     // Reject if no match in whitelist only mode
     // Reject if no match in whitelist only mode
-    if (ipat != null && epat == null) {
+    if (!includeTagPatterns.isEmpty() && excludeTagPatterns.isEmpty()) {
       return false;
       return false;
     }
     }
     return true;
     return true;

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/package-info.java

@@ -234,7 +234,7 @@
     patterns.
     patterns.
   </p>
   </p>
   <p>Similarly, you can specify the <code>record.filter</code> and
   <p>Similarly, you can specify the <code>record.filter</code> and
-    <code>metrics.filter</code> options, which operate at record and metric
+    <code>metric.filter</code> options, which operate at record and metric
     level, respectively. Filters can be combined to optimize
     level, respectively. Filters can be combined to optimize
     the filtering efficiency.</p>
     the filtering efficiency.</p>
 
 

+ 1 - 1
hadoop-common-project/hadoop-common/src/site/apt/FileSystemShell.apt.vm

@@ -381,7 +381,7 @@ rmr
 
 
 setrep
 setrep
 
 
-   Usage: <<<hdfs dfs -setrep [-R] [-w] <numRepicas> <path> >>>
+   Usage: <<<hdfs dfs -setrep [-R] [-w] <numReplicas> <path> >>>
 
 
    Changes the replication factor of a file. If <path> is a directory then
    Changes the replication factor of a file. If <path> is a directory then
    the command recursively changes the replication factor of all files under
    the command recursively changes the replication factor of all files under

+ 34 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FCStatisticsBaseTest.java

@@ -27,6 +27,8 @@ import org.apache.hadoop.fs.FileSystem.Statistics;
 import org.junit.Assert;
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.Test;
 
 
+import com.google.common.util.concurrent.Uninterruptibles;
+
 import static org.apache.hadoop.fs.FileContextTestHelper.*;
 import static org.apache.hadoop.fs.FileContextTestHelper.*;
 
 
 /**
 /**
@@ -44,6 +46,38 @@ public abstract class FCStatisticsBaseTest {
   //fc should be set appropriately by the deriving test.
   //fc should be set appropriately by the deriving test.
   protected static FileContext fc = null;
   protected static FileContext fc = null;
   
   
+  @Test(timeout=60000)
+  public void testStatisticsOperations() throws Exception {
+    final Statistics stats = new Statistics("file");
+    Assert.assertEquals(0L, stats.getBytesRead());
+    Assert.assertEquals(0L, stats.getBytesWritten());
+    Assert.assertEquals(0, stats.getWriteOps());
+    stats.incrementBytesWritten(1000);
+    Assert.assertEquals(1000L, stats.getBytesWritten());
+    Assert.assertEquals(0, stats.getWriteOps());
+    stats.incrementWriteOps(123);
+    Assert.assertEquals(123, stats.getWriteOps());
+    
+    Thread thread = new Thread() {
+      @Override
+      public void run() {
+        stats.incrementWriteOps(1);
+      }
+    };
+    thread.start();
+    Uninterruptibles.joinUninterruptibly(thread);
+    Assert.assertEquals(124, stats.getWriteOps());
+    // Test copy constructor and reset function
+    Statistics stats2 = new Statistics(stats);
+    stats.reset();
+    Assert.assertEquals(0, stats.getWriteOps());
+    Assert.assertEquals(0L, stats.getBytesWritten());
+    Assert.assertEquals(0L, stats.getBytesRead());
+    Assert.assertEquals(124, stats2.getWriteOps());
+    Assert.assertEquals(1000L, stats2.getBytesWritten());
+    Assert.assertEquals(0L, stats2.getBytesRead());
+  }
+
   @Test
   @Test
   public void testStatistics() throws IOException, URISyntaxException {
   public void testStatistics() throws IOException, URISyntaxException {
     URI fsUri = getFsUri();
     URI fsUri = getFsUri();

+ 53 - 10
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/filter/TestPatternFilter.java

@@ -23,9 +23,11 @@ import java.util.List;
 
 
 import org.apache.commons.configuration.SubsetConfiguration;
 import org.apache.commons.configuration.SubsetConfiguration;
 import org.junit.Test;
 import org.junit.Test;
+
 import static org.junit.Assert.*;
 import static org.junit.Assert.*;
 import static org.mockito.Mockito.*;
 import static org.mockito.Mockito.*;
 
 
+import org.apache.hadoop.metrics2.MetricsFilter;
 import org.apache.hadoop.metrics2.MetricsRecord;
 import org.apache.hadoop.metrics2.MetricsRecord;
 import org.apache.hadoop.metrics2.MetricsTag;
 import org.apache.hadoop.metrics2.MetricsTag;
 import org.apache.hadoop.metrics2.impl.ConfigBuilder;
 import org.apache.hadoop.metrics2.impl.ConfigBuilder;
@@ -53,7 +55,7 @@ public class TestPatternFilter {
         .add("p.include.tags", "foo:f").subset("p");
         .add("p.include.tags", "foo:f").subset("p");
     shouldAccept(wl, "foo");
     shouldAccept(wl, "foo");
     shouldAccept(wl, Arrays.asList(tag("bar", "", ""),
     shouldAccept(wl, Arrays.asList(tag("bar", "", ""),
-                                   tag("foo", "", "f")));
+                                   tag("foo", "", "f")), new boolean[] {false, true});
     shouldAccept(wl, mockMetricsRecord("foo", Arrays.asList(
     shouldAccept(wl, mockMetricsRecord("foo", Arrays.asList(
       tag("bar", "", ""), tag("foo", "", "f"))));
       tag("bar", "", ""), tag("foo", "", "f"))));
     shouldReject(wl, "bar");
     shouldReject(wl, "bar");
@@ -78,7 +80,7 @@ public class TestPatternFilter {
       tag("bar", "", ""))));
       tag("bar", "", ""))));
     shouldReject(bl, "foo");
     shouldReject(bl, "foo");
     shouldReject(bl, Arrays.asList(tag("bar", "", ""),
     shouldReject(bl, Arrays.asList(tag("bar", "", ""),
-                                   tag("foo", "", "f")));
+                                   tag("foo", "", "f")), new boolean[] {true, false});
     shouldReject(bl, mockMetricsRecord("foo", Arrays.asList(
     shouldReject(bl, mockMetricsRecord("foo", Arrays.asList(
       tag("bar", "", ""))));
       tag("bar", "", ""))));
     shouldReject(bl, mockMetricsRecord("bar", Arrays.asList(
     shouldReject(bl, mockMetricsRecord("bar", Arrays.asList(
@@ -125,15 +127,61 @@ public class TestPatternFilter {
     shouldAccept(c, mockMetricsRecord("foo", Arrays.asList(
     shouldAccept(c, mockMetricsRecord("foo", Arrays.asList(
       tag("foo", "", "f"))));
       tag("foo", "", "f"))));
   }
   }
-
+  
   static void shouldAccept(SubsetConfiguration conf, String s) {
   static void shouldAccept(SubsetConfiguration conf, String s) {
     assertTrue("accepts "+ s, newGlobFilter(conf).accepts(s));
     assertTrue("accepts "+ s, newGlobFilter(conf).accepts(s));
     assertTrue("accepts "+ s, newRegexFilter(conf).accepts(s));
     assertTrue("accepts "+ s, newRegexFilter(conf).accepts(s));
   }
   }
 
 
+  // Version for one tag:
   static void shouldAccept(SubsetConfiguration conf, List<MetricsTag> tags) {
   static void shouldAccept(SubsetConfiguration conf, List<MetricsTag> tags) {
-    assertTrue("accepts "+ tags, newGlobFilter(conf).accepts(tags));
-    assertTrue("accepts "+ tags, newRegexFilter(conf).accepts(tags));
+    shouldAcceptImpl(true, conf, tags, new boolean[] {true});
+  }
+  // Version for multiple tags: 
+  static void shouldAccept(SubsetConfiguration conf, List<MetricsTag> tags, 
+      boolean[] expectedAcceptedSpec) {
+    shouldAcceptImpl(true, conf, tags, expectedAcceptedSpec);
+  }
+
+  // Version for one tag:
+  static void shouldReject(SubsetConfiguration conf, List<MetricsTag> tags) {
+    shouldAcceptImpl(false, conf, tags, new boolean[] {false});
+  }
+  // Version for multiple tags: 
+  static void shouldReject(SubsetConfiguration conf, List<MetricsTag> tags, 
+      boolean[] expectedAcceptedSpec) {
+    shouldAcceptImpl(false, conf, tags, expectedAcceptedSpec);
+  }
+  
+  private static void shouldAcceptImpl(final boolean expectAcceptList,  
+      SubsetConfiguration conf, List<MetricsTag> tags, boolean[] expectedAcceptedSpec) {
+    final MetricsFilter globFilter = newGlobFilter(conf);
+    final MetricsFilter regexFilter = newRegexFilter(conf);
+    
+    // Test acceptance of the tag list:  
+    assertEquals("accepts "+ tags, expectAcceptList, globFilter.accepts(tags));
+    assertEquals("accepts "+ tags, expectAcceptList, regexFilter.accepts(tags));
+    
+    // Test results on each of the individual tags:
+    int acceptedCount = 0;
+    for (int i=0; i<tags.size(); i++) {
+      MetricsTag tag = tags.get(i);
+      boolean actGlob = globFilter.accepts(tag);
+      boolean actRegex = regexFilter.accepts(tag);
+      assertEquals("accepts "+tag, expectedAcceptedSpec[i], actGlob);
+      // Both the filters should give the same result:
+      assertEquals(actGlob, actRegex);
+      if (actGlob) {
+        acceptedCount++;
+      }
+    }
+    if (expectAcceptList) {
+      // At least one individual tag should be accepted:
+      assertTrue("No tag of the following accepted: " + tags, acceptedCount > 0);
+    } else {
+      // At least one individual tag should be rejected: 
+      assertTrue("No tag of the following rejected: " + tags, acceptedCount < tags.size());
+    }
   }
   }
 
 
   /**
   /**
@@ -152,11 +200,6 @@ public class TestPatternFilter {
     assertTrue("rejects "+ s, !newRegexFilter(conf).accepts(s));
     assertTrue("rejects "+ s, !newRegexFilter(conf).accepts(s));
   }
   }
 
 
-  static void shouldReject(SubsetConfiguration conf, List<MetricsTag> tags) {
-    assertTrue("rejects "+ tags, !newGlobFilter(conf).accepts(tags));
-    assertTrue("rejects "+ tags, !newRegexFilter(conf).accepts(tags));
-  }
-
   /**
   /**
    * Asserts that filters with the given configuration reject the given record.
    * Asserts that filters with the given configuration reject the given record.
    * 
    * 

+ 138 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/sink/TestFileSink.java

@@ -0,0 +1,138 @@
+/*
+ * 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.metrics2.sink;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.annotation.Metric;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.annotation.Metric.Type;
+import org.apache.hadoop.metrics2.impl.ConfigBuilder;
+import org.apache.hadoop.metrics2.impl.MetricsSystemImpl;
+import org.apache.hadoop.metrics2.impl.TestMetricsConfig;
+import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
+import org.junit.After;
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+public class TestFileSink {
+  
+  private File outFile;
+
+  // The 2 sample metric classes:
+  @Metrics(name="testRecord1", context="test1")
+  static class MyMetrics1 {
+    @Metric(value={"testTag1", ""}, type=Type.TAG) 
+    String testTag1() { return "testTagValue1"; }
+    
+    @Metric(value={"testTag2", ""}, type=Type.TAG) 
+    String gettestTag2() { return "testTagValue2"; }
+    
+    @Metric(value={"testMetric1", "An integer gauge"},always=true) 
+    MutableGaugeInt testMetric1;
+    
+    @Metric(value={"testMetric2", "An integer gauge"},always=true) 
+    MutableGaugeInt testMetric2;
+
+    public MyMetrics1 registerWith(MetricsSystem ms) {
+      return ms.register("m1", null, this);
+    }
+  }
+  
+  @Metrics(name="testRecord2", context="test1")
+  static class MyMetrics2 {
+    @Metric(value={"testTag22", ""}, type=Type.TAG) 
+    String testTag1() { return "testTagValue22"; }
+
+    public MyMetrics2 registerWith(MetricsSystem ms) {
+      return ms.register("m2", null, this);
+    }
+  }
+  
+  private File getTestTempFile(String prefix, String suffix) throws IOException {
+    String tmpPath = System.getProperty("java.io.tmpdir", "/tmp");
+    String user = System.getProperty("user.name", "unknown-user");
+    File dir = new File(tmpPath + "/" + user);
+    dir.mkdirs();
+    return File.createTempFile(prefix, suffix, dir);
+  }
+  
+  @Test(timeout=6000) 
+  public void testFileSink() throws IOException {
+    outFile = getTestTempFile("test-file-sink-", ".out");
+    final String outPath = outFile.getAbsolutePath();  
+    
+    // NB: specify large period to avoid multiple metrics snapshotting: 
+    new ConfigBuilder().add("*.period", 10000)
+        .add("test.sink.mysink0.class", FileSink.class.getName())
+        .add("test.sink.mysink0.filename", outPath)
+        // NB: we filter by context to exclude "metricssystem" context metrics:
+        .add("test.sink.mysink0.context", "test1")
+        .save(TestMetricsConfig.getTestFilename("hadoop-metrics2-test"));
+    MetricsSystemImpl ms = new MetricsSystemImpl("test");
+    ms.start();
+
+    final MyMetrics1 mm1 
+      = new MyMetrics1().registerWith(ms);
+    new MyMetrics2().registerWith(ms);
+
+    mm1.testMetric1.incr();
+    mm1.testMetric2.incr(2);
+
+    ms.publishMetricsNow(); // publish the metrics
+    ms.stop();
+    ms.shutdown();
+    
+    InputStream is = new FileInputStream(outFile);
+    ByteArrayOutputStream baos = new ByteArrayOutputStream((int)outFile.length());
+    IOUtils.copyBytes(is, baos, 1024, true);
+    String outFileContent = new String(baos.toByteArray(), "UTF-8");
+
+    // Check the out file content. Should be something like the following:
+    //1360244820087 test1.testRecord1: Context=test1, testTag1=testTagValue1, testTag2=testTagValue2, Hostname=myhost, testMetric1=1, testMetric2=2
+    //1360244820089 test1.testRecord2: Context=test1, testTag22=testTagValue22, Hostname=myhost
+    
+    // Note that in the below expression we allow tags and metrics to go in arbitrary order.  
+    Pattern expectedContentPattern = Pattern.compile(
+        // line #1:
+        "^\\d+\\s+test1.testRecord1:\\s+Context=test1,\\s+" +
+        "(testTag1=testTagValue1,\\s+testTag2=testTagValue2|testTag2=testTagValue2,\\s+testTag1=testTagValue1)," +
+        "\\s+Hostname=.*,\\s+(testMetric1=1,\\s+testMetric2=2|testMetric2=2,\\s+testMetric1=1)" +
+        // line #2:
+        "$[\\n\\r]*^\\d+\\s+test1.testRecord2:\\s+Context=test1," +
+        "\\s+testTag22=testTagValue22,\\s+Hostname=.*$[\\n\\r]*", 
+         Pattern.MULTILINE);
+     assertTrue(expectedContentPattern.matcher(outFileContent).matches());
+  }
+  
+  @After
+  public void after() {
+    if (outFile != null) {
+      outFile.delete();
+      assertTrue(!outFile.exists());
+    }
+  }
+}

+ 41 - 1
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -255,7 +255,15 @@ Release 2.3.0 - UNRELEASED
 
 
     HDFS-5342. Provide more information in the FSNamesystem JMX interfaces.
     HDFS-5342. Provide more information in the FSNamesystem JMX interfaces.
     (Haohui Mai via jing9)
     (Haohui Mai via jing9)
- 
+
+    HDFS-5334. Implement dfshealth.jsp in HTML pages. (Haohui Mai via jing9)
+
+    HDFS-5379. Update links to datanode information in dfshealth.html. (Haohui
+    Mai via jing9)
+
+    HDFS-5382. Implement the UI of browsing filesystems in HTML 5 page. (Haohui
+    Mai via jing9)
+
   IMPROVEMENTS
   IMPROVEMENTS
 
 
     HDFS-5267. Remove volatile from LightWeightHashSet. (Junping Du via llu)
     HDFS-5267. Remove volatile from LightWeightHashSet. (Junping Du via llu)
@@ -319,10 +327,24 @@ Release 2.3.0 - UNRELEASED
     HDFS-5130. Add test for snapshot related FsShell and DFSAdmin commands.
     HDFS-5130. Add test for snapshot related FsShell and DFSAdmin commands.
     (Binglin Chang via jing9)
     (Binglin Chang via jing9)
 
 
+    HDFS-5374. Remove deadcode in DFSOutputStream. (suresh)
+
+    HDFS-4511. Cover package org.apache.hadoop.hdfs.tools with unit test
+    (Andrey Klochkov via jeagles)
+
+    HDFS-4885. Improve the verifyBlockPlacement() API in BlockPlacementPolicy.
+    (Junping Du via szetszwo)
+
+    HDFS-5363. Refactor WebHdfsFileSystem: move SPENGO-authenticated connection
+    creation to URLConnectionFactory. (Haohui Mai via jing9)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     HDFS-5239.  Allow FSNamesystem lock fairness to be configurable (daryn)
     HDFS-5239.  Allow FSNamesystem lock fairness to be configurable (daryn)
 
 
+    HDFS-5341. Reduce fsdataset lock duration during directory scanning.
+    (Qus-Jiawei via kihwal)
+
   BUG FIXES
   BUG FIXES
     HDFS-5034.  Remove debug prints from GetFileLinkInfo (Andrew Wang via Colin
     HDFS-5034.  Remove debug prints from GetFileLinkInfo (Andrew Wang via Colin
     Patrick McCabe)
     Patrick McCabe)
@@ -354,6 +376,14 @@ Release 2.3.0 - UNRELEASED
 
 
     HDFS-4376. Fix race conditions in Balancer.  (Junping Du via szetszwo)
     HDFS-4376. Fix race conditions in Balancer.  (Junping Du via szetszwo)
 
 
+    HDFS-5375. hdfs.cmd does not expose several snapshot commands. (cnauroth)
+
+    HDFS-5336. DataNode should not output 'StartupProgress' metrics.
+    (Akira Ajisaka via cnauroth)
+
+    HDFS-5400.  DFS_CLIENT_MMAP_CACHE_THREAD_RUNS_PER_TIMEOUT constant is set
+    to the wrong value.  (Colin Patrick McCabe)
+
 Release 2.2.1 - UNRELEASED
 Release 2.2.1 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -365,6 +395,9 @@ Release 2.2.1 - UNRELEASED
     HDFS-5360. Improvement of usage message of renameSnapshot and
     HDFS-5360. Improvement of usage message of renameSnapshot and
     deleteSnapshot. (Shinichi Yamashita via wang)
     deleteSnapshot. (Shinichi Yamashita via wang)
 
 
+    HDFS-5331. make SnapshotDiff.java to a o.a.h.util.Tool interface implementation. 
+    (Vinayakumar B via umamahesh)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES
@@ -400,6 +433,13 @@ Release 2.2.1 - UNRELEASED
     HDFS-5370. Typo in Error Message: different between range in condition
     HDFS-5370. Typo in Error Message: different between range in condition
     and range in error message. (Kousuke Saruta via suresh)
     and range in error message. (Kousuke Saruta via suresh)
 
 
+    HDFS-5365. Fix libhdfs compile error on FreeBSD9. (Radim Kolar via cnauroth)
+    
+    HDFS-5347. Add HDFS NFS user guide. (brandonli)
+
+    HDFS-5403. WebHdfs client cannot communicate with older WebHdfs servers
+    post HDFS-5306. (atm)
+
 Release 2.2.0 - 2013-10-13
 Release 2.2.0 - 2013-10-13
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 7 - 1
hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml

@@ -83,7 +83,7 @@
        <Class name="org.apache.hadoop.mapred.Task$TaskReporter" />
        <Class name="org.apache.hadoop.mapred.Task$TaskReporter" />
        <Method name="run" />
        <Method name="run" />
        <Bug pattern="DM_EXIT" />
        <Bug pattern="DM_EXIT" />
-     </Match>
+     </Match>     
      <!--
      <!--
        We need to cast objects between old and new api objects
        We need to cast objects between old and new api objects
      -->
      -->
@@ -325,6 +325,12 @@
        <Field name="modification" />
        <Field name="modification" />
        <Bug pattern="VO_VOLATILE_INCREMENT" />
        <Bug pattern="VO_VOLATILE_INCREMENT" />
      </Match>
      </Match>
+      <!-- Replace System.exit() call with ExitUtil.terminate() -->
+     <Match>
+       <Class name="org.apache.hadoop.hdfs.tools.JMXGet"/>
+       <Method name="main" />
+       <Bug pattern="NP_NULL_ON_SOME_PATH" />
+     </Match>    
      <Match>
      <Match>
        <Class name="org.apache.hadoop.hdfs.server.datanode.ReplicaInfo" />
        <Class name="org.apache.hadoop.hdfs.server.datanode.ReplicaInfo" />
        <Method name="setDirInternal" />
        <Method name="setDirInternal" />

+ 10 - 0
hadoop-hdfs-project/hadoop-hdfs/pom.xml

@@ -176,6 +176,11 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
       <artifactId>xmlenc</artifactId>
       <artifactId>xmlenc</artifactId>
       <scope>compile</scope>
       <scope>compile</scope>
     </dependency>
     </dependency>
+    <dependency>
+      <groupId>io.netty</groupId>
+      <artifactId>netty</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
   </dependencies>
 
 
   <build>
   <build>
@@ -542,6 +547,11 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
             <exclude>src/main/docs/releasenotes.html</exclude>
             <exclude>src/main/docs/releasenotes.html</exclude>
             <exclude>src/contrib/**</exclude>
             <exclude>src/contrib/**</exclude>
             <exclude>src/site/resources/images/*</exclude>
             <exclude>src/site/resources/images/*</exclude>
+            <exclude>src/main/webapps/static/dust-full-2.0.0.min.js</exclude>
+            <exclude>src/main/webapps/static/dust-helpers-1.1.1.min.js</exclude>
+            <exclude>src/main/webapps/hdfs/dfshealth.dust.html</exclude>
+            <exclude>src/main/webapps/hdfs/explorer-block-info.dust.html</exclude>
+            <exclude>src/main/webapps/hdfs/explorer.dust.html</exclude>
           </excludes>
           </excludes>
         </configuration>
         </configuration>
       </plugin>
       </plugin>

+ 10 - 1
hadoop-hdfs-project/hadoop-hdfs/src/CMakeLists.txt

@@ -62,6 +62,11 @@ endfunction()
 INCLUDE(CheckCSourceCompiles)
 INCLUDE(CheckCSourceCompiles)
 CHECK_C_SOURCE_COMPILES("int main(void) { static __thread int i = 0; return 0; }" HAVE_BETTER_TLS)
 CHECK_C_SOURCE_COMPILES("int main(void) { static __thread int i = 0; return 0; }" HAVE_BETTER_TLS)
 
 
+# Check if we need to link dl library to get dlopen.
+# dlopen on Linux is in separate library but on FreeBSD its in libc
+INCLUDE(CheckLibraryExists)
+CHECK_LIBRARY_EXISTS(dl dlopen "" NEED_LINK_DL)
+
 find_package(JNI REQUIRED)
 find_package(JNI REQUIRED)
 if (NOT GENERATED_JAVAH)
 if (NOT GENERATED_JAVAH)
     # Must identify where the generated headers have been placed
     # Must identify where the generated headers have been placed
@@ -89,9 +94,13 @@ add_dual_library(hdfs
     main/native/libhdfs/jni_helper.c
     main/native/libhdfs/jni_helper.c
     main/native/libhdfs/hdfs.c
     main/native/libhdfs/hdfs.c
 )
 )
+if (NEED_LINK_DL)
+   set(LIB_DL dl)
+endif(NEED_LINK_DL)
+
 target_link_dual_libraries(hdfs
 target_link_dual_libraries(hdfs
     ${JAVA_JVM_LIBRARY}
     ${JAVA_JVM_LIBRARY}
-    dl
+    ${LIB_DL}
     pthread
     pthread
 )
 )
 dual_output_directory(hdfs target/usr/local/lib)
 dual_output_directory(hdfs target/usr/local/lib)

+ 12 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs.cmd

@@ -123,6 +123,14 @@ goto :eof
   set CLASS=org.apache.hadoop.hdfs.tools.GetGroups
   set CLASS=org.apache.hadoop.hdfs.tools.GetGroups
   goto :eof
   goto :eof
 
 
+:snapshotDiff
+  set CLASS=org.apache.hadoop.hdfs.tools.snapshot.SnapshotDiff
+  goto :eof
+
+:lsSnapshottableDir
+  set CLASS=org.apache.hadoop.hdfs.tools.snapshot.LsSnapshottableDir
+  goto :eof
+
 @rem This changes %1, %2 etc. Hence those cannot be used after calling this.
 @rem This changes %1, %2 etc. Hence those cannot be used after calling this.
 :make_command_arguments
 :make_command_arguments
   if "%1" == "--config" (
   if "%1" == "--config" (
@@ -164,7 +172,10 @@ goto :eof
   @echo   fetchdt              fetch a delegation token from the NameNode
   @echo   fetchdt              fetch a delegation token from the NameNode
   @echo   getconf              get config values from configuration
   @echo   getconf              get config values from configuration
   @echo   groups               get the groups which users belong to
   @echo   groups               get the groups which users belong to
-  @echo                        Use -help to see options
+  @echo   snapshotDiff         diff two snapshots of a directory or diff the
+  @echo                        current directory contents with a snapshot
+  @echo   lsSnapshottableDir   list all snapshottable dirs owned by the current user
+  @echo 						Use -help to see options
   @echo.
   @echo.
   @echo Most commands print help when invoked w/o parameters.
   @echo Most commands print help when invoked w/o parameters.
 
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

@@ -401,7 +401,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int DFS_CLIENT_MMAP_CACHE_SIZE_DEFAULT = 1024;
   public static final int DFS_CLIENT_MMAP_CACHE_SIZE_DEFAULT = 1024;
   public static final String DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS = "dfs.client.mmap.cache.timeout.ms";
   public static final String DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS = "dfs.client.mmap.cache.timeout.ms";
   public static final long DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS_DEFAULT  = 15 * 60 * 1000;
   public static final long DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS_DEFAULT  = 15 * 60 * 1000;
-  public static final String DFS_CLIENT_MMAP_CACHE_THREAD_RUNS_PER_TIMEOUT = "dfs.client.mmap.cache.timeout.ms";
+  public static final String DFS_CLIENT_MMAP_CACHE_THREAD_RUNS_PER_TIMEOUT = "dfs.client.mmap.cache.thread.runs.per.timeout";
   public static final int DFS_CLIENT_MMAP_CACHE_THREAD_RUNS_PER_TIMEOUT_DEFAULT  = 4;
   public static final int DFS_CLIENT_MMAP_CACHE_THREAD_RUNS_PER_TIMEOUT_DEFAULT  = 4;
 
 
   // property for fsimage compression
   // property for fsimage compression

+ 12 - 18
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java

@@ -47,7 +47,6 @@ import org.apache.hadoop.fs.FSOutputSummer;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.Syncable;
 import org.apache.hadoop.fs.Syncable;
-import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
@@ -138,7 +137,7 @@ public class DFSOutputStream extends FSOutputSummer
   private long currentSeqno = 0;
   private long currentSeqno = 0;
   private long lastQueuedSeqno = -1;
   private long lastQueuedSeqno = -1;
   private long lastAckedSeqno = -1;
   private long lastAckedSeqno = -1;
-  private long bytesCurBlock = 0; // bytes writen in current block
+  private long bytesCurBlock = 0; // bytes written in current block
   private int packetSize = 0; // write packet size, not including the header.
   private int packetSize = 0; // write packet size, not including the header.
   private int chunksPerPacket = 0;
   private int chunksPerPacket = 0;
   private final AtomicReference<IOException> lastException = new AtomicReference<IOException>();
   private final AtomicReference<IOException> lastException = new AtomicReference<IOException>();
@@ -458,8 +457,7 @@ public class DFSOutputStream extends FSOutputSummer
           }
           }
         }
         }
 
 
-        Packet one = null;
-
+        Packet one;
         try {
         try {
           // process datanode IO errors if any
           // process datanode IO errors if any
           boolean doSleep = false;
           boolean doSleep = false;
@@ -504,7 +502,7 @@ public class DFSOutputStream extends FSOutputSummer
             if(DFSClient.LOG.isDebugEnabled()) {
             if(DFSClient.LOG.isDebugEnabled()) {
               DFSClient.LOG.debug("Allocating new block");
               DFSClient.LOG.debug("Allocating new block");
             }
             }
-            nodes = nextBlockOutputStream(src);
+            nodes = nextBlockOutputStream();
             initDataStreaming();
             initDataStreaming();
           } else if (stage == BlockConstructionStage.PIPELINE_SETUP_APPEND) {
           } else if (stage == BlockConstructionStage.PIPELINE_SETUP_APPEND) {
             if(DFSClient.LOG.isDebugEnabled()) {
             if(DFSClient.LOG.isDebugEnabled()) {
@@ -569,9 +567,6 @@ public class DFSOutputStream extends FSOutputSummer
           }
           }
           lastPacket = Time.now();
           lastPacket = Time.now();
           
           
-          if (one.isHeartbeatPacket()) {  //heartbeat packet
-          }
-          
           // update bytesSent
           // update bytesSent
           long tmpBytesSent = one.getLastByteOffsetBlock();
           long tmpBytesSent = one.getLastByteOffsetBlock();
           if (bytesSent < tmpBytesSent) {
           if (bytesSent < tmpBytesSent) {
@@ -690,7 +685,7 @@ public class DFSOutputStream extends FSOutputSummer
     }
     }
 
 
     //
     //
-    // Processes reponses from the datanodes.  A packet is removed 
+    // Processes responses from the datanodes.  A packet is removed
     // from the ackQueue when its response arrives.
     // from the ackQueue when its response arrives.
     //
     //
     private class ResponseProcessor extends Daemon {
     private class ResponseProcessor extends Daemon {
@@ -732,18 +727,18 @@ public class DFSOutputStream extends FSOutputSummer
             }
             }
             
             
             assert seqno != PipelineAck.UNKOWN_SEQNO : 
             assert seqno != PipelineAck.UNKOWN_SEQNO : 
-              "Ack for unkown seqno should be a failed ack: " + ack;
+              "Ack for unknown seqno should be a failed ack: " + ack;
             if (seqno == Packet.HEART_BEAT_SEQNO) {  // a heartbeat ack
             if (seqno == Packet.HEART_BEAT_SEQNO) {  // a heartbeat ack
               continue;
               continue;
             }
             }
 
 
             // a success ack for a data packet
             // a success ack for a data packet
-            Packet one = null;
+            Packet one;
             synchronized (dataQueue) {
             synchronized (dataQueue) {
               one = ackQueue.getFirst();
               one = ackQueue.getFirst();
             }
             }
             if (one.seqno != seqno) {
             if (one.seqno != seqno) {
-              throw new IOException("Responseprocessor: Expecting seqno " +
+              throw new IOException("ResponseProcessor: Expecting seqno " +
                                     " for block " + block +
                                     " for block " + block +
                                     one.seqno + " but received " + seqno);
                                     one.seqno + " but received " + seqno);
             }
             }
@@ -1052,7 +1047,7 @@ public class DFSOutputStream extends FSOutputSummer
      * Must get block ID and the IDs of the destinations from the namenode.
      * Must get block ID and the IDs of the destinations from the namenode.
      * Returns the list of target datanodes.
      * Returns the list of target datanodes.
      */
      */
-    private DatanodeInfo[] nextBlockOutputStream(String client) throws IOException {
+    private DatanodeInfo[] nextBlockOutputStream() throws IOException {
       LocatedBlock lb = null;
       LocatedBlock lb = null;
       DatanodeInfo[] nodes = null;
       DatanodeInfo[] nodes = null;
       int count = dfsClient.getConf().nBlockWriteRetry;
       int count = dfsClient.getConf().nBlockWriteRetry;
@@ -1210,8 +1205,7 @@ public class DFSOutputStream extends FSOutputSummer
     }
     }
 
 
     private LocatedBlock locateFollowingBlock(long start,
     private LocatedBlock locateFollowingBlock(long start,
-        DatanodeInfo[] excludedNodes) 
-        throws IOException, UnresolvedLinkException {
+        DatanodeInfo[] excludedNodes)  throws IOException {
       int retries = dfsClient.getConf().nBlockWriteLocateFollowingRetry;
       int retries = dfsClient.getConf().nBlockWriteLocateFollowingRetry;
       long sleeptime = 400;
       long sleeptime = 400;
       while (true) {
       while (true) {
@@ -1283,7 +1277,7 @@ public class DFSOutputStream extends FSOutputSummer
    * Create a socket for a write pipeline
    * Create a socket for a write pipeline
    * @param first the first datanode 
    * @param first the first datanode 
    * @param length the pipeline length
    * @param length the pipeline length
-   * @param client
+   * @param client client
    * @return the socket connected to the first datanode
    * @return the socket connected to the first datanode
    */
    */
   static Socket createSocketForPipeline(final DatanodeInfo first,
   static Socket createSocketForPipeline(final DatanodeInfo first,
@@ -1475,7 +1469,7 @@ public class DFSOutputStream extends FSOutputSummer
           //
           //
           // Rather than wait around for space in the queue, we should instead try to
           // Rather than wait around for space in the queue, we should instead try to
           // return to the caller as soon as possible, even though we slightly overrun
           // return to the caller as soon as possible, even though we slightly overrun
-          // the MAX_PACKETS iength.
+          // the MAX_PACKETS length.
           Thread.currentThread().interrupt();
           Thread.currentThread().interrupt();
           break;
           break;
         }
         }
@@ -1696,7 +1690,7 @@ public class DFSOutputStream extends FSOutputSummer
         }
         }
       }
       }
       // If 1) any new blocks were allocated since the last flush, or 2) to
       // If 1) any new blocks were allocated since the last flush, or 2) to
-      // update length in NN is requried, then persist block locations on
+      // update length in NN is required, then persist block locations on
       // namenode.
       // namenode.
       if (persistBlocks.getAndSet(false) || updateLength) {
       if (persistBlocks.getAndSet(false) || updateLength) {
         try {
         try {

+ 7 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java

@@ -95,18 +95,17 @@ public abstract class BlockPlacementPolicy {
   }
   }
 
 
   /**
   /**
-   * Verify that the block is replicated on at least minRacks different racks
-   * if there is more than minRacks rack in the system.
+   * Verify if the block's placement meets requirement of placement policy,
+   * i.e. replicas are placed on no less than minRacks racks in the system.
    * 
    * 
    * @param srcPath the full pathname of the file to be verified
    * @param srcPath the full pathname of the file to be verified
    * @param lBlk block with locations
    * @param lBlk block with locations
-   * @param minRacks number of racks the block should be replicated to
-   * @return the difference between the required and the actual number of racks
-   * the block is replicated to.
+   * @param numOfReplicas replica number of file to be verified
+   * @return the result of verification
    */
    */
-  abstract public int verifyBlockPlacement(String srcPath,
-                                           LocatedBlock lBlk,
-                                           int minRacks);
+  abstract public BlockPlacementStatus verifyBlockPlacement(String srcPath,
+      LocatedBlock lBlk,
+      int numOfReplicas);
   /**
   /**
    * Decide whether deleting the specified replica of the block still makes 
    * Decide whether deleting the specified replica of the block still makes 
    * the block conform to the configured block placement policy.
    * the block conform to the configured block placement policy.

+ 6 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java

@@ -655,22 +655,22 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
   }
   }
 
 
   @Override
   @Override
-  public int verifyBlockPlacement(String srcPath,
-                                  LocatedBlock lBlk,
-                                  int minRacks) {
+  public BlockPlacementStatus verifyBlockPlacement(String srcPath,
+      LocatedBlock lBlk, int numberOfReplicas) {
     DatanodeInfo[] locs = lBlk.getLocations();
     DatanodeInfo[] locs = lBlk.getLocations();
     if (locs == null)
     if (locs == null)
       locs = DatanodeDescriptor.EMPTY_ARRAY;
       locs = DatanodeDescriptor.EMPTY_ARRAY;
     int numRacks = clusterMap.getNumOfRacks();
     int numRacks = clusterMap.getNumOfRacks();
     if(numRacks <= 1) // only one rack
     if(numRacks <= 1) // only one rack
-      return 0;
-    minRacks = Math.min(minRacks, numRacks);
+      return new BlockPlacementStatusDefault(
+          Math.min(numRacks, numberOfReplicas), numRacks);
+    int minRacks = Math.min(2, numberOfReplicas);
     // 1. Check that all locations are different.
     // 1. Check that all locations are different.
     // 2. Count locations on different racks.
     // 2. Count locations on different racks.
     Set<String> racks = new TreeSet<String>();
     Set<String> racks = new TreeSet<String>();
     for (DatanodeInfo dn : locs)
     for (DatanodeInfo dn : locs)
       racks.add(dn.getNetworkLocation());
       racks.add(dn.getNetworkLocation());
-    return minRacks - racks.size();
+    return new BlockPlacementStatusDefault(racks.size(), minRacks);
   }
   }
 
 
   @Override
   @Override

+ 42 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatus.java

@@ -0,0 +1,42 @@
+/**
+ * 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.hdfs.server.blockmanagement;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface BlockPlacementStatus {
+
+  /**
+   * Boolean value to identify if replicas of this block satisfy requirement of 
+   * placement policy
+   * @return if replicas satisfy placement policy's requirement 
+   */
+  public boolean isPlacementPolicySatisfied();
+  
+  /**
+   * Get description info for log or printed in case replicas are failed to meet
+   * requirement of placement policy
+   * @return description in case replicas are failed to meet requirement of
+   * placement policy
+   */
+  public String getErrorDescription();
+
+}

+ 44 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementStatusDefault.java

@@ -0,0 +1,44 @@
+/**
+ * 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.hdfs.server.blockmanagement;
+
+public class BlockPlacementStatusDefault implements BlockPlacementStatus {
+
+  private int requiredRacks = 0;
+  private int currentRacks = 0;
+  
+  public BlockPlacementStatusDefault(int currentRacks, int requiredRacks){
+    this.requiredRacks = requiredRacks;
+    this.currentRacks = currentRacks;
+  }
+  
+  @Override
+  public boolean isPlacementPolicySatisfied() {
+    return requiredRacks <= currentRacks;
+  }
+
+  @Override
+  public String getErrorDescription() {
+    if (isPlacementPolicySatisfied()) {
+      return null;
+    }
+    return "Block should be additionally replicated on " + 
+        (requiredRacks - currentRacks) + " more rack(s).";
+  }
+
+}

+ 11 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java

@@ -191,6 +191,11 @@ public class DirectoryScanner implements Runnable {
 
 
     private final FsVolumeSpi volume;
     private final FsVolumeSpi volume;
 
 
+    /**
+     * Get the file's length in async block scan
+     */
+    private final long blockFileLength;
+
     private final static Pattern CONDENSED_PATH_REGEX =
     private final static Pattern CONDENSED_PATH_REGEX =
         Pattern.compile("(?<!^)(\\\\|/){2,}");
         Pattern.compile("(?<!^)(\\\\|/){2,}");
     
     
@@ -235,6 +240,7 @@ public class DirectoryScanner implements Runnable {
         getCondensedPath(vol.getBasePath());
         getCondensedPath(vol.getBasePath());
       this.blockSuffix = blockFile == null ? null :
       this.blockSuffix = blockFile == null ? null :
         getSuffix(blockFile, condensedVolPath);
         getSuffix(blockFile, condensedVolPath);
+      this.blockFileLength = (blockFile != null) ? blockFile.length() : 0; 
       if (metaFile == null) {
       if (metaFile == null) {
         this.metaSuffix = null;
         this.metaSuffix = null;
       } else if (blockFile == null) {
       } else if (blockFile == null) {
@@ -251,6 +257,10 @@ public class DirectoryScanner implements Runnable {
         new File(volume.getBasePath(), blockSuffix);
         new File(volume.getBasePath(), blockSuffix);
     }
     }
 
 
+    long getBlockFileLength() {
+      return blockFileLength;
+    }
+
     File getMetaFile() {
     File getMetaFile() {
       if (metaSuffix == null) {
       if (metaSuffix == null) {
         return null;
         return null;
@@ -458,7 +468,7 @@ public class DirectoryScanner implements Runnable {
             // Block metadata file exits and block file is missing
             // Block metadata file exits and block file is missing
             addDifference(diffRecord, statsRecord, info);
             addDifference(diffRecord, statsRecord, info);
           } else if (info.getGenStamp() != memBlock.getGenerationStamp()
           } else if (info.getGenStamp() != memBlock.getGenerationStamp()
-              || info.getBlockFile().length() != memBlock.getNumBytes()) {
+              || info.getBlockFileLength() != memBlock.getNumBytes()) {
             // Block metadata file is missing or has wrong generation stamp,
             // Block metadata file is missing or has wrong generation stamp,
             // or block file length is different than expected
             // or block file length is different than expected
             statsRecord.mismatchBlocks++;
             statsRecord.mismatchBlocks++;

+ 9 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/resources/DatanodeWebHdfsMethods.java

@@ -413,8 +413,15 @@ public class DatanodeWebHdfsMethods {
       final long n = length.getValue() != null ?
       final long n = length.getValue() != null ?
         Math.min(length.getValue(), in.getVisibleLength() - offset.getValue()) :
         Math.min(length.getValue(), in.getVisibleLength() - offset.getValue()) :
         in.getVisibleLength() - offset.getValue();
         in.getVisibleLength() - offset.getValue();
-      return Response.ok(new OpenEntity(in, n, dfsclient)).type(
-          MediaType.APPLICATION_OCTET_STREAM).build();
+
+      /**
+       * Allow the Web UI to perform an AJAX request to get the data.
+       */
+      return Response.ok(new OpenEntity(in, n, dfsclient))
+          .type(MediaType.APPLICATION_OCTET_STREAM)
+          .header("Access-Control-Allow-Methods", "GET")
+          .header("Access-Control-Allow-Origin", "*")
+          .build();
     }
     }
     case GETFILECHECKSUM:
     case GETFILECHECKSUM:
     {
     {

+ 1 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -270,10 +270,6 @@ public class NameNode implements NameNodeStatusMXBean {
 
 
   static NameNodeMetrics metrics;
   static NameNodeMetrics metrics;
   private static final StartupProgress startupProgress = new StartupProgress();
   private static final StartupProgress startupProgress = new StartupProgress();
-  static {
-    StartupProgressMetrics.register(startupProgress);
-  }
-
   /** Return the {@link FSNamesystem} object.
   /** Return the {@link FSNamesystem} object.
    * @return {@link FSNamesystem} object.
    * @return {@link FSNamesystem} object.
    */
    */
@@ -485,6 +481,7 @@ public class NameNode implements NameNodeStatusMXBean {
     loginAsNameNodeUser(conf);
     loginAsNameNodeUser(conf);
 
 
     NameNode.initMetrics(conf, this.getRole());
     NameNode.initMetrics(conf, this.getRole());
+    StartupProgressMetrics.register(startupProgress);
 
 
     if (NamenodeRole.NAMENODE == role) {
     if (NamenodeRole.NAMENODE == role) {
       startHttpServer(conf);
       startHttpServer(conf);

+ 6 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java

@@ -50,6 +50,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementStatus;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
@@ -374,9 +375,10 @@ public class NamenodeFsck {
                     locs.length + " replica(s).");
                     locs.length + " replica(s).");
       }
       }
       // verify block placement policy
       // verify block placement policy
-      int missingRacks = BlockPlacementPolicy.getInstance(conf, null, networktopology).
-                           verifyBlockPlacement(path, lBlk, Math.min(2,targetFileReplication));
-      if (missingRacks > 0) {
+      BlockPlacementStatus blockPlacementStatus = 
+          BlockPlacementPolicy.getInstance(conf, null, networktopology).
+              verifyBlockPlacement(path, lBlk, targetFileReplication);
+      if (!blockPlacementStatus.isPlacementPolicySatisfied()) {
         res.numMisReplicatedBlocks++;
         res.numMisReplicatedBlocks++;
         misReplicatedPerFile++;
         misReplicatedPerFile++;
         if (!showFiles) {
         if (!showFiles) {
@@ -385,9 +387,7 @@ public class NamenodeFsck {
           out.print(path + ": ");
           out.print(path + ": ");
         }
         }
         out.println(" Replica placement policy is violated for " + 
         out.println(" Replica placement policy is violated for " + 
-                    block +
-                    ". Block should be additionally replicated on " + 
-                    missingRacks + " more rack(s).");
+                    block + ". " + blockPlacementStatus.getErrorDescription());
       }
       }
       report.append(i + ". " + blkName + " len=" + block.getNumBytes());
       report.append(i + ". " + blkName + " len=" + block.getNumBytes());
       if (locs.length == 0) {
       if (locs.length == 0) {

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java

@@ -53,6 +53,7 @@ import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.GenericOptionsParser;
 import org.apache.hadoop.util.GenericOptionsParser;
 
 
 import com.google.common.base.Charsets;
 import com.google.common.base.Charsets;
@@ -86,7 +87,7 @@ public class DelegationTokenFetcher {
     err.println("  --print             Print the delegation token");
     err.println("  --print             Print the delegation token");
     err.println();
     err.println();
     GenericOptionsParser.printGenericCommandUsage(err);
     GenericOptionsParser.printGenericCommandUsage(err);
-    System.exit(1);
+    ExitUtil.terminate(1);    
   }
   }
 
 
   private static Collection<Token<?>> readTokens(Path file, Configuration conf)
   private static Collection<Token<?>> readTokens(Path file, Configuration conf)

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/JMXGet.java

@@ -43,6 +43,7 @@ import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
 import org.apache.commons.cli.ParseException;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.util.ExitUtil;
 
 
 /**
 /**
  * tool to get data from NameNode or DataNode using MBeans currently the
  * tool to get data from NameNode or DataNode using MBeans currently the
@@ -295,7 +296,7 @@ public class JMXGet {
       // invalid arguments
       // invalid arguments
       err("Invalid args");
       err("Invalid args");
       printUsage(opts);
       printUsage(opts);
-      System.exit(-1);
+      ExitUtil.terminate(-1);      
     }
     }
 
 
     JMXGet jm = new JMXGet();
     JMXGet jm = new JMXGet();
@@ -317,7 +318,7 @@ public class JMXGet {
 
 
     if (commandLine.hasOption("help")) {
     if (commandLine.hasOption("help")) {
       printUsage(opts);
       printUsage(opts);
-      System.exit(0);
+      ExitUtil.terminate(0);
     }
     }
 
 
     // rest of args
     // rest of args
@@ -342,6 +343,6 @@ public class JMXGet {
       res = -1;
       res = -1;
     }
     }
 
 
-    System.exit(res);
+    ExitUtil.terminate(res);
   }
   }
 }
 }

+ 16 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/snapshot/SnapshotDiff.java

@@ -20,12 +20,14 @@ package org.apache.hadoop.hdfs.tools.snapshot;
 import java.io.IOException;
 import java.io.IOException;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
 
 
 /**
 /**
  * A tool used to get the difference report between two snapshots, or between
  * A tool used to get the difference report between two snapshots, or between
@@ -38,7 +40,7 @@ import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
  * </pre>
  * </pre>
  */
  */
 @InterfaceAudience.Private
 @InterfaceAudience.Private
-public class SnapshotDiff {
+public class SnapshotDiff extends Configured implements Tool {
   private static String getSnapshotName(String name) {
   private static String getSnapshotName(String name) {
     if (Path.CUR_DIR.equals(name)) { // current directory
     if (Path.CUR_DIR.equals(name)) { // current directory
       return "";
       return "";
@@ -57,7 +59,8 @@ public class SnapshotDiff {
     return name.substring(i + HdfsConstants.DOT_SNAPSHOT_DIR.length() + 1);
     return name.substring(i + HdfsConstants.DOT_SNAPSHOT_DIR.length() + 1);
   }
   }
   
   
-  public static void main(String[] argv) throws IOException {
+  @Override
+  public int run(String[] argv) throws Exception {
     String description = "SnapshotDiff <snapshotDir> <from> <to>:\n" +
     String description = "SnapshotDiff <snapshotDir> <from> <to>:\n" +
     "\tGet the difference between two snapshots, \n" + 
     "\tGet the difference between two snapshots, \n" + 
     "\tor between a snapshot and the current tree of a directory.\n" +
     "\tor between a snapshot and the current tree of a directory.\n" +
@@ -67,15 +70,14 @@ public class SnapshotDiff {
     
     
     if(argv.length != 3) {
     if(argv.length != 3) {
       System.err.println("Usage: \n" + description);
       System.err.println("Usage: \n" + description);
-      System.exit(1);
+      return 1;
     }
     }
     
     
-    Configuration conf = new Configuration();
-    FileSystem fs = FileSystem.get(conf);
+    FileSystem fs = FileSystem.get(getConf());
     if (! (fs instanceof DistributedFileSystem)) {
     if (! (fs instanceof DistributedFileSystem)) {
       System.err.println(
       System.err.println(
           "SnapshotDiff can only be used in DistributedFileSystem");
           "SnapshotDiff can only be used in DistributedFileSystem");
-      System.exit(1);
+      return 1;
     }
     }
     DistributedFileSystem dfs = (DistributedFileSystem) fs;
     DistributedFileSystem dfs = (DistributedFileSystem) fs;
     
     
@@ -89,7 +91,14 @@ public class SnapshotDiff {
     } catch (IOException e) {
     } catch (IOException e) {
       String[] content = e.getLocalizedMessage().split("\n");
       String[] content = e.getLocalizedMessage().split("\n");
       System.err.println("snapshotDiff: " + content[0]);
       System.err.println("snapshotDiff: " + content[0]);
+      return 1;
     }
     }
+    return 0;
+  }
+
+  public static void main(String[] argv) throws Exception {
+    int rc = ToolRunner.run(new SnapshotDiff(), argv);
+    System.exit(rc);
   }
   }
 
 
 }
 }

+ 7 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java

@@ -299,10 +299,15 @@ public class JsonUtil {
   }
   }
 
 
   /** Convert a Json map to an DatanodeInfo object. */
   /** Convert a Json map to an DatanodeInfo object. */
-  private static DatanodeInfo toDatanodeInfo(final Map<?, ?> m) {
+  static DatanodeInfo toDatanodeInfo(final Map<?, ?> m) {
     if (m == null) {
     if (m == null) {
       return null;
       return null;
     }
     }
+    
+    Object infoSecurePort = m.get("infoSecurePort");
+    if (infoSecurePort == null) {
+      infoSecurePort = 0l; // same as the default value in hdfs.proto
+    }
 
 
     return new DatanodeInfo(
     return new DatanodeInfo(
         (String)m.get("ipAddr"),
         (String)m.get("ipAddr"),
@@ -310,7 +315,7 @@ public class JsonUtil {
         (String)m.get("storageID"),
         (String)m.get("storageID"),
         (int)(long)(Long)m.get("xferPort"),
         (int)(long)(Long)m.get("xferPort"),
         (int)(long)(Long)m.get("infoPort"),
         (int)(long)(Long)m.get("infoPort"),
-        (int)(long)(Long)m.get("infoSecurePort"),
+        (int)(long)(Long)infoSecurePort,
         (int)(long)(Long)m.get("ipcPort"),
         (int)(long)(Long)m.get("ipcPort"),
 
 
         (Long)m.get("capacity"),
         (Long)m.get("capacity"),

+ 75 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java

@@ -19,49 +19,114 @@
 package org.apache.hadoop.hdfs.web;
 package org.apache.hadoop.hdfs.web;
 
 
 import java.io.IOException;
 import java.io.IOException;
+import java.net.HttpURLConnection;
 import java.net.URL;
 import java.net.URL;
 import java.net.URLConnection;
 import java.net.URLConnection;
 
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.web.resources.HttpOpParam;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
 
 
 /**
 /**
  * Utilities for handling URLs
  * Utilities for handling URLs
  */
  */
-@InterfaceAudience.LimitedPrivate({"HDFS"})
+@InterfaceAudience.LimitedPrivate({ "HDFS" })
 @InterfaceStability.Unstable
 @InterfaceStability.Unstable
 public class URLConnectionFactory {
 public class URLConnectionFactory {
+  private static final Log LOG = LogFactory.getLog(URLConnectionFactory.class);
+
+  /** SPNEGO authenticator */
+  private static final KerberosUgiAuthenticator AUTH = new KerberosUgiAuthenticator();
+
   /**
   /**
    * Timeout for socket connects and reads
    * Timeout for socket connects and reads
    */
    */
-  public final static int DEFAULT_SOCKET_TIMEOUT = 1*60*1000; // 1 minute
+  public final static int DEFAULT_SOCKET_TIMEOUT = 1 * 60 * 1000; // 1 minute
+
+  public static final URLConnectionFactory DEFAULT_CONNECTION_FACTORY = new URLConnectionFactory(
+      DEFAULT_SOCKET_TIMEOUT);
 
 
-  public static final URLConnectionFactory DEFAULT_CONNECTION_FACTORY = new URLConnectionFactory(DEFAULT_SOCKET_TIMEOUT);
-  
   private int socketTimeout;
   private int socketTimeout;
 
 
+  /** Configure connections for AuthenticatedURL */
+  private ConnectionConfigurator connConfigurator = new ConnectionConfigurator() {
+    @Override
+    public HttpURLConnection configure(HttpURLConnection conn)
+        throws IOException {
+      URLConnectionFactory.setTimeouts(conn, socketTimeout);
+      return conn;
+    }
+  };
+
   public URLConnectionFactory(int socketTimeout) {
   public URLConnectionFactory(int socketTimeout) {
     this.socketTimeout = socketTimeout;
     this.socketTimeout = socketTimeout;
   }
   }
-  
+
   /**
   /**
    * Opens a url with read and connect timeouts
    * Opens a url with read and connect timeouts
-   * @param url to open
+   *
+   * @param url
+   *          to open
    * @return URLConnection
    * @return URLConnection
    * @throws IOException
    * @throws IOException
    */
    */
   public URLConnection openConnection(URL url) throws IOException {
   public URLConnection openConnection(URL url) throws IOException {
     URLConnection connection = url.openConnection();
     URLConnection connection = url.openConnection();
-    setTimeouts(connection);
-    return connection;    
+    if (connection instanceof HttpURLConnection) {
+      connConfigurator.configure((HttpURLConnection) connection);
+    }
+    return connection;
+  }
+
+  /**
+   * Opens a url with read and connect timeouts
+   *
+   * @param url URL to open
+   * @return URLConnection
+   * @throws IOException
+   * @throws AuthenticationException
+   */
+  public URLConnection openConnection(HttpOpParam.Op op, URL url)
+      throws IOException, AuthenticationException {
+    if (op.getRequireAuth()) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("open AuthenticatedURL connection" + url);
+      }
+      UserGroupInformation.getCurrentUser().checkTGTAndReloginFromKeytab();
+      final AuthenticatedURL.Token authToken = new AuthenticatedURL.Token();
+      return new AuthenticatedURL(AUTH, connConfigurator).openConnection(url,
+          authToken);
+    } else {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("open URL connection");
+      }
+      return openConnection(url);
+    }
+  }
+
+  public ConnectionConfigurator getConnConfigurator() {
+    return connConfigurator;
+  }
+
+  public void setConnConfigurator(ConnectionConfigurator connConfigurator) {
+    this.connConfigurator = connConfigurator;
   }
   }
 
 
   /**
   /**
    * Sets timeout parameters on the given URLConnection.
    * Sets timeout parameters on the given URLConnection.
    * 
    * 
-   * @param connection URLConnection to set
+   * @param connection
+   *          URLConnection to set
+   * @param socketTimeout
+   *          the connection and read timeout of the connection.
    */
    */
-  public void setTimeouts(URLConnection connection) {
+  static void setTimeouts(URLConnection connection, int socketTimeout) {
     connection.setConnectTimeout(socketTimeout);
     connection.setConnectTimeout(socketTimeout);
     connection.setReadTimeout(socketTimeout);
     connection.setReadTimeout(socketTimeout);
   }
   }

+ 8 - 33
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java

@@ -94,9 +94,7 @@ import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
-import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.TokenRenewer;
 import org.apache.hadoop.security.token.TokenRenewer;
@@ -119,20 +117,9 @@ public class WebHdfsFileSystem extends FileSystem
   /** Http URI: http://namenode:port/{PATH_PREFIX}/path/to/file */
   /** Http URI: http://namenode:port/{PATH_PREFIX}/path/to/file */
   public static final String PATH_PREFIX = "/" + SCHEME + "/v" + VERSION;
   public static final String PATH_PREFIX = "/" + SCHEME + "/v" + VERSION;
 
 
-  /** SPNEGO authenticator */
-  private static final KerberosUgiAuthenticator AUTH = new KerberosUgiAuthenticator();
   /** Default connection factory may be overridden in tests to use smaller timeout values */
   /** Default connection factory may be overridden in tests to use smaller timeout values */
   URLConnectionFactory connectionFactory = URLConnectionFactory.DEFAULT_CONNECTION_FACTORY;
   URLConnectionFactory connectionFactory = URLConnectionFactory.DEFAULT_CONNECTION_FACTORY;
-  /** Configures connections for AuthenticatedURL */
-  private final ConnectionConfigurator CONN_CONFIGURATOR =
-    new ConnectionConfigurator() {
-      @Override
-      public HttpURLConnection configure(HttpURLConnection conn)
-          throws IOException {
-        connectionFactory.setTimeouts(conn);
-        return conn;
-      }
-    };
+
   /** Delegation token kind */
   /** Delegation token kind */
   public static final Text TOKEN_KIND = new Text("WEBHDFS delegation");
   public static final Text TOKEN_KIND = new Text("WEBHDFS delegation");
   /** Token selector */
   /** Token selector */
@@ -504,16 +491,7 @@ public class WebHdfsFileSystem extends FileSystem
         throws IOException {
         throws IOException {
       final HttpURLConnection conn;
       final HttpURLConnection conn;
       try {
       try {
-        if (op.getRequireAuth()) {
-          LOG.debug("open AuthenticatedURL connection");
-          UserGroupInformation.getCurrentUser().checkTGTAndReloginFromKeytab();
-          final AuthenticatedURL.Token authToken = new AuthenticatedURL.Token();
-          conn = new AuthenticatedURL(AUTH, CONN_CONFIGURATOR).openConnection(
-            url, authToken);
-        } else {
-          LOG.debug("open URL connection");
-          conn = (HttpURLConnection)connectionFactory.openConnection(url);
-        }
+        conn = (HttpURLConnection) connectionFactory.openConnection(op, url);
       } catch (AuthenticationException e) {
       } catch (AuthenticationException e) {
         throw new IOException(e);
         throw new IOException(e);
       }
       }
@@ -635,8 +613,10 @@ public class WebHdfsFileSystem extends FileSystem
       checkRetry = false;
       checkRetry = false;
       
       
       //Step 2) Submit another Http request with the URL from the Location header with data.
       //Step 2) Submit another Http request with the URL from the Location header with data.
-      conn = (HttpURLConnection)connectionFactory.openConnection(new URL(redirect));
-      conn.setRequestProperty("Content-Type", MediaType.APPLICATION_OCTET_STREAM);
+      conn = (HttpURLConnection) connectionFactory.openConnection(new URL(
+          redirect));
+      conn.setRequestProperty("Content-Type",
+          MediaType.APPLICATION_OCTET_STREAM);
       conn.setChunkedStreamingMode(32 << 10); //32kB-chunk
       conn.setChunkedStreamingMode(32 << 10); //32kB-chunk
       connect();
       connect();
       return conn;
       return conn;
@@ -658,7 +638,8 @@ public class WebHdfsFileSystem extends FileSystem
           disconnect();
           disconnect();
   
   
           checkRetry = false;
           checkRetry = false;
-          conn = (HttpURLConnection)connectionFactory.openConnection(new URL(redirect));
+          conn = (HttpURLConnection) connectionFactory.openConnection(new URL(
+              redirect));
           connect();
           connect();
         }
         }
 
 
@@ -892,12 +873,6 @@ public class WebHdfsFileSystem extends FileSystem
       .write(bufferSize);
       .write(bufferSize);
   }
   }
 
 
-  @SuppressWarnings("deprecation")
-  @Override
-  public boolean delete(final Path f) throws IOException {
-    return delete(f, true);
-  }
-
   @Override
   @Override
   public boolean delete(Path f, boolean recursive) throws IOException {
   public boolean delete(Path f, boolean recursive) throws IOException {
     final HttpOpParam.Op op = DeleteOpParam.Op.DELETE;
     final HttpOpParam.Op op = DeleteOpParam.Op.DELETE;

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/native/util/posix_util.c

@@ -27,6 +27,7 @@
 #include <sys/stat.h>
 #include <sys/stat.h>
 #include <sys/types.h>
 #include <sys/types.h>
 #include <unistd.h>
 #include <unistd.h>
+#include <limits.h>
 
 
 static pthread_mutex_t gTempdirLock = PTHREAD_MUTEX_INITIALIZER;
 static pthread_mutex_t gTempdirLock = PTHREAD_MUTEX_INITIALIZER;
 
 

+ 116 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfs-dust.js

@@ -0,0 +1,116 @@
+/**
+ * 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.
+ */
+(function ($, dust, exports) {
+  "use strict";
+
+  var filters = {
+    'fmt_bytes': function (v) {
+      var UNITS = ['B', 'KB', 'MB', 'GB', 'TB', 'PB', 'ZB'];
+      var prev = 0, i = 0;
+      while (Math.floor(v) > 0 && i < UNITS.length) {
+        prev = v;
+        v /= 1024;
+        i += 1;
+      }
+
+      if (i > 0 && i < UNITS.length) {
+        v = prev;
+        i -= 1;
+      }
+      return Math.round(v * 100) / 100 + ' ' + UNITS[i];
+    },
+
+    'fmt_percentage': function (v) {
+      return Math.round(v * 100) / 100 + '%';
+    },
+
+    'fmt_time': function (v) {
+      var s = Math.floor(v / 1000), h = Math.floor(s / 3600);
+      s -= h * 3600;
+      var m = Math.floor(s / 60);
+      s -= m * 60;
+
+      var res = s + " sec";
+      if (m !== 0) {
+        res = m + " mins, " + res;
+      }
+
+      if (h !== 0) {
+        res = h + " hrs, " + res;
+      }
+
+      return res;
+    }
+  };
+  $.extend(dust.filters, filters);
+
+  /**
+   * Load templates from external sources in sequential orders, and
+   * compile them. The loading order is important to resolve dependency.
+   *
+   * The code compile the templates on the client sides, which should be
+   * precompiled once we introduce the infrastructure in the building
+   * system.
+   *
+   * templates is an array of tuples in the format of {url, name}.
+   */
+  function load_templates(dust, templates, success_cb, error_cb) {
+    if (templates.length === 0) {
+      success_cb();
+      return;
+    }
+
+    var t = templates.shift();
+    $.get(t.url, function (tmpl) {
+      var c = dust.compile(tmpl, t.name);
+      dust.loadSource(c);
+      load_templates(dust, templates, success_cb, error_cb);
+    }).error(function (jqxhr, text, err) {
+      error_cb(t.url, jqxhr, text, err);
+    });
+  }
+
+  /**
+   * Load a sequence of JSON.
+   *
+   * beans is an array of tuples in the format of {url, name}.
+   */
+  function load_json(beans, success_cb, error_cb) {
+    var data = {}, error = false, to_be_completed = beans.length;
+
+    $.each(beans, function(idx, b) {
+      if (error) {
+        return false;
+      }
+      $.get(b.url, function (resp) {
+        data[b.name] = resp;
+        to_be_completed -= 1;
+        if (to_be_completed === 0) {
+          success_cb(data);
+        }
+      }).error(function (jqxhr, text, err) {
+        error = true;
+        error_cb(b.url, jqxhr, text, err);
+      });
+    });
+  }
+
+  exports.load_templates = load_templates;
+  exports.load_json = load_json;
+
+}($, dust, window));

+ 265 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.dust.html

@@ -0,0 +1,265 @@
+<div class="page-header">
+  {#nnstat}
+  <h1>NameNode '{HostAndPort}' ({State})</h1>
+  {/nnstat}
+</div>
+
+{#nn}
+{@if cond="{DistinctVersionCount} > 1"}
+<div class="alert alert-dismissable alert-success">
+  <button type="button" class="close" data-dismiss="alert" aria-hidden="true">&times;</button>
+
+  There are {DistinctVersionCount} versions of datanodes currently live: 
+  {#DistinctVersions}
+  {key} ({value}) {@sep},{/sep}
+  {/DistinctVersions}
+</div>
+{/if}
+
+{@if cond="{NumberOfMissingBlocks} > 0"}
+<div class="alert alert-dismissable alert-warning">
+  <button type="button" class="close" data-dismiss="alert" aria-hidden="true">&times;</button>
+
+  <p>There are {NumberOfMissingBlocks} missing blocks. The following files may be corrupted:</p>
+  <br/>
+  <div class="well">
+    {#CorruptFiles}
+    {.}<br/>
+    {/CorruptFiles}
+  </div>
+  <p>Please check the logs or run fsck in order to identify the missing blocks. See the Hadoop FAQ for common causes and potential solutions.</p>
+</div>
+{/if}
+{/nn}
+
+<div class="panel panel-primary">
+  <div class="panel-heading">Overview</div>
+  <div class="panel-body">
+    {#nn}
+    <table class="table table-bordered">
+      <tr><th>Started:</th><td>{NNStarted}</td></tr>
+      <tr><th>Version:</th><td>{Version}</td></tr>
+      <tr><th>Compiled:</th><td>{CompileInfo}</td></tr>
+      <tr><th>Cluster ID:</th><td>{ClusterId}</td></tr>
+      <tr><th>Block Pool ID:</th><td>{BlockPoolId}</td></tr>
+    </table>
+    {/nn}
+  </div>
+</div>
+
+<a href="explorer.html">Browse the filesystem</a>  <a href="/logs/">NameNode Logs</a>
+
+<hr/>
+
+<div class="panel panel-primary">
+  <div class="panel-heading">Cluster Summary</div>
+  <div class="panel-body">
+
+    <p>
+      Security is {#nnstat}{#SecurityEnabled}on{:else}off{/SecurityEnabled}{/nnstat}.</p>
+    <p>{#nn}{#Safemode}{.}{:else}Safemode is off.{/Safemode}{/nn}</p>
+
+    <p>
+      {#fs}
+      {TotalLoad} files and directories, {BlocksTotal} blocks = {FilesTotal} total filesystem object(s).
+      {#helper_fs_max_objects/}
+      {/fs}
+    </p>
+    {#mem.HeapMemoryUsage}
+    <p>Heap Memory used {used|fmt_bytes} of {committed|fmt_bytes} Heap Memory. Max Heap Memory is {max|fmt_bytes}. </p>
+    {/mem.HeapMemoryUsage}
+
+    {#mem.NonHeapMemoryUsage}
+    <p>Non Heap Memory used {used|fmt_bytes} of {committed|fmt_bytes} Commited Non Heap Memory. Max Non Heap Memory is {max|fmt_bytes}. </p>
+    {/mem.NonHeapMemoryUsage}
+
+    {#nn}
+    <table class="table table-bordered table-striped">
+      <tr><th> Configured Capacity:</th><td>{Total|fmt_bytes}</td></tr>
+      <tr><th> DFS Used:</th><td>{Used|fmt_bytes}</td></tr>
+      <tr><th> Non DFS Used:</th><td>{NonDfsUsedSpace|fmt_bytes}</td></tr>
+      <tr><th> DFS Remaining:</th><td>{Free|fmt_bytes}</td></tr>
+      <tr><th> DFS Used%:</th><td>{PercentUsed|fmt_percentage}</td></tr>
+      <tr><th> DFS Remaining%:</th><td>{PercentRemaining|fmt_percentage}</td></tr>
+      <tr><th> Block Pool Used:</th><td>{BlockPoolUsedSpace|fmt_bytes}</td></tr>
+      <tr><th> Block Pool Used%:</th><td>{PercentBlockPoolUsed|fmt_percentage}</td></tr>
+      <tr><th> DataNodes usages% (Min/Median/Max/stdDev): </th>
+	<td>{#NodeUsage.nodeUsage}{min} / {median} / {max} / {stdDev}{/NodeUsage.nodeUsage}</td></tr>
+      {/nn}
+
+      {#fs}
+      <tr><th><a href="#nodelist-operation">Live Nodes</a></th><td>{NumLiveDataNodes} (Decommissioned: {NumDecomLiveDataNodes})</td></tr>
+      <tr><th><a href="#nodelist-operation">Dead Nodes</a></th><td>{NumDeadDataNodes} (Decommissioned: {NumDecomDeadDataNodes})</td></tr>
+      <tr><th><a href="#nodelist-decom">Decommissioning Nodes</a></th><td>{NumDecommissioningDataNodes}</td></tr>
+      <tr><th title="Excludes missing blocks.">Number of Under-Replicated Blocks</th><td>{UnderReplicatedBlocks}</td></tr>
+      {/fs}
+    </table>
+  </div>
+</div>
+
+<hr/>
+<div class="panel panel-primary">
+  <div class="panel-heading">NameNode Journal Status</div>
+  <div class="panel-body">
+    <p><b>Current transaction ID:</b> {nn.JournalTransactionInfo.LastAppliedOrWrittenTxId}</p>
+    <table class="table" title="NameNode Journals">
+      <thead>
+	<tr><th>Journal Manager</th><th>State</th></tr>
+      </thead>
+      <tbody>
+	{#nn.NameJournalStatus}
+	<tr><td>{manager}</td><td>{stream}</td></tr>
+	{/nn.NameJournalStatus}
+      </tbody>
+    </table>
+  </div>
+</div>
+
+<hr/>
+<div class="panel panel-primary">
+  <div class="panel-heading">NameNode Storage</div>
+  <div class="panel-body">
+    <table class="table" title="NameNode Storage">
+      <thead><tr><td><b>Storage Directory</b></td><td><b>Type</b></td><td><b>State</b></td></tr></thead>
+      {#nn.NameDirStatuses}
+      {#active}{#helper_dir_status type="Active"/}{/active}
+      {#failed}{#helper_dir_status type="Failed"/}{/failed}
+      {/nn.NameDirStatuses}
+    </table>
+  </div>
+</div>
+<hr/>
+
+<div class="panel panel-primary">
+  <div class="panel-heading">Snapshot Summary</div>
+  <div class="panel-body">
+    {#fs.SnapshotStats}
+    <table class="table" title="Snapshot Summary">
+      <thead><tr><td><b>Snapshottable directories</b></td>
+	  <td><b>Snapshotted directories</b></td></tr>
+      </thead>
+      <tbody>
+	<tr>
+	  <td>{SnapshottableDirectories}</td>
+	  <td>{Snapshots}</td>
+	</tr>
+      </tbody>
+    </table>
+    {/fs.SnapshotStats}
+  </div>
+</div>
+<hr/>
+
+{#startup}
+<div class="panel panel-primary">
+  <div class="panel-heading">Startup Progress</div>
+  <div class="panel-body">
+    <p>Elapsed Time: {elapsedTime|fmt_time}, Percent Complete: {percentComplete|fmt_percentage}</p>
+    <table class="table">
+      <thead>
+	<tr>
+	  <th>Phase</th>
+	  <th>Completion</th>
+	  <th>Elapsed Time</th>
+	</tr>
+      </thead>
+      <tbody>
+	{#phases}
+	<tr class="phase">
+	  <td class="startupdesc">{desc} {file} {size|fmt_bytes}</td>
+	  <td>{percentComplete|fmt_percentage}</td>
+	  <td>{elapsedTime|fmt_time}</td>
+	</tr>
+	{#steps root_file=file}
+	<tr class="step">
+	  <td class="startupdesc">{stepDesc} {stepFile} {stepSize|fmt_bytes} ({count}/{total})</td>
+	  <td>{percentComplete|fmt_percentage}</td>
+	  <td></td>
+	</tr>
+	{/steps}
+	{/phases}
+    </table>
+  </div>
+</div>
+{/startup}
+
+<hr/>
+<div class="panel panel-primary">
+  <div class="panel-heading">Datanode Information</div>
+  <div class="panel-body">
+    <div class="panel panel-default" id="nodelist-operation">
+      <div class="panel-heading">Nodes in operation</div>
+      <div class="panel-body">
+        <table class="table">
+          <thead>
+            <tr>
+              <th>Node</th>
+              <th>Last contact</th>
+              <th>Admin State</th>
+              <th>Capacity</th>
+              <th>Used</th>
+              <th>Non DFS Used</th>
+              <th>Remaining</th>
+              <th>Blocks</th>
+              <th>Block pool used</th>
+              <th>Failed Volumes</th>
+            </tr>
+          </thead>
+          {#nn.LiveNodes}
+          <tr>
+            <td>{name} ({xferaddr})</td>
+            <td>{lastContact}</td>
+            <td>{adminState}</td>
+            <td>{capacity|fmt_bytes}</td>
+            <td>{used|fmt_bytes}</td>
+            <td>{nonDfsUsedSpace|fmt_bytes}</td>
+            <td>{remaining|fmt_bytes}</td>
+            <td>{numBlocks}</td>
+            <td>{blockPoolUsed|fmt_bytes} ({blockPoolUsedPercent|fmt_percentage})</td>
+            <td>{volfails}</td>
+          </tr>
+          {/nn.LiveNodes}
+          {#nn.DeadNodes}
+          <tr class="danger">
+            <td>{name} ({xferaddr})</td>
+            <td>{lastContact}</td>
+            <td>Dead{?decomissioned}, Decomissioned{/decomissioned}</td>
+            <td>-</td>
+            <td>-</td>
+            <td>-</td>
+            <td>-</td>
+            <td>-</td>
+            <td>-</td>
+            <td>-</td>
+          </tr>
+          {/nn.DeadNodes}
+        </table>
+      </div>
+    </div>
+    <div class="panel panel-default" id="nodelist-decom">
+      <div class="panel-heading">Nodes being decomissioned</div>
+      <div class="panel-body">
+        <table class="table">
+          <thead>
+            <tr>
+              <th>Node</th>
+              <th>Last contact</th>
+              <th>Under replicated blocks</th>
+              <th>Blocks with no live replicas</th>
+              <th>Under Replicated Blocks <br/>In files under construction</th>
+            </tr>
+          </thead>
+          {#nn.DecomNodes}
+          <tr>
+            <td>{name} ({xferaddr})</td>
+            <td>{lastContact}</td>
+            <td>{underReplicatedBlocks}</td>
+            <td>{decommissionOnlyReplicas}</td>
+	    <td>{underReplicateInOpenFiles}</td>
+	  </tr>
+	  {/nn.DecomNodes}
+	</table>
+      </div>
+    </div>
+  </div>
+</div>

+ 43 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html

@@ -0,0 +1,43 @@
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Strict//EN"
+    "http://www.w3.org/TR/xhtml1/DTD/xhtml1-strict.dtd">
+<html xmlns="http://www.w3.org/1999/xhtml">
+<head>
+<link rel="stylesheet" type="text/css" href="//netdna.bootstrapcdn.com/bootstrap/3.0.0/css/bootstrap.min.css" />
+<link rel="stylesheet" type="text/css" href="/static/hadoop.css" />
+<title>Namenode information</title>
+</head>
+<body>
+<div class="container">
+<div class="alert alert-danger" id="alert-panel" style="display:none">
+<button type="button" class="close" onclick="$('#alert-panel').hide();">&times;</button>
+<div class="alert-body" id="alert-panel-body"></div>
+</div>
+<div id="panel"></div>
+</div>
+<script type="text/javascript" src="//ajax.googleapis.com/ajax/libs/jquery/2.0.3/jquery.min.js">
+</script><script type="text/javascript" src="//netdna.bootstrapcdn.com/bootstrap/3.0.0/js/bootstrap.min.js">
+</script><script type="text/javascript" src="/static/dust-full-2.0.0.min.js">
+</script><script type="text/javascript" src="/static/dust-helpers-1.1.1.min.js">
+</script><script type="text/javascript" src="dfs-dust.js">
+</script><script type="text/javascript" src="dfshealth.js">
+</script>
+<hr />
+<p><a href="http://hadoop.apache.org/core">Hadoop</a>, 2013.</p>
+</body>
+</html>

+ 126 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js

@@ -0,0 +1,126 @@
+/**
+ * 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.
+ */
+(function () {
+  "use strict";
+
+  var data = {};
+
+  function render() {
+    var helpers = {
+      'helper_fs_max_objects': function (chunk, ctx, bodies, params) {
+        var o = ctx.current();
+        if (o.MaxObjects > 0) {
+          chunk.write('(' + Math.round((o.FilesTotal + o.BlockTotal) / o.MaxObjects * 100) * 100 + ')%');
+        }
+      },
+
+      'helper_dir_status': function (chunk, ctx, bodies, params) {
+        var j = ctx.current();
+        for (var i in j) {
+          chunk.write('<tr><td>' + i + '</td><td>' + j[i] + '</td><td>' + params.type + '</td></tr>');
+        }
+      }
+    };
+
+    var base = dust.makeBase(helpers);
+
+    var TEMPLATES = [ { 'name': 'dfshealth', 'url': 'dfshealth.dust.html' } ];
+
+    load_templates(dust, TEMPLATES, function() {
+      dust.render('dfshealth', base.push(data), function(err, out) {
+        $('#panel').html(out);
+      });
+    }, function () {
+      show_err_msg('Failed to load the page.');
+    });
+  }
+
+  var BEANS = [
+    {"name": "nn",      "url": "/jmx?qry=Hadoop:service=NameNode,name=NameNodeInfo"},
+    {"name": "nnstat",  "url": "/jmx?qry=Hadoop:service=NameNode,name=NameNodeStatus"},
+    {"name": "fs",      "url": "/jmx?qry=Hadoop:service=NameNode,name=FSNamesystemState"},
+    {"name": "mem",     "url": "/jmx?qry=java.lang:type=Memory"},
+    {"name": "startup", "url": "/startupProgress"}
+  ];
+
+  // Workarounds for the fact that JMXJsonServlet returns non-standard JSON strings
+  function data_workaround(d) {
+    function node_map_to_array(nodes) {
+      var res = [];
+      for (var n in nodes) {
+        var p = nodes[n];
+        p.name = n;
+        res.push(p);
+      }
+      return res;
+    }
+
+    function startup_progress_workaround(r) {
+      function rename_property(o, s, d) {
+        if (o[s] !== undefined) {
+          o[d] = o[s];
+          delete o[s];
+        }
+      }
+      r.percentComplete *= 100;
+      $.each(r.phases, function (idx, p) {
+        p.percentComplete *= 100;
+        $.each(p.steps, function (idx2, s) {
+          s.percentComplete *= 100;
+          // dust.js is confused by these optional keys in nested
+          // structure, rename them
+          rename_property(s, "desc", "stepDesc");
+          rename_property(s, "file", "stepFile");
+          rename_property(s, "size", "stepSize");
+        });
+      });
+      return r;
+    }
+
+    d.nn.JournalTransactionInfo = JSON.parse(d.nn.JournalTransactionInfo);
+    d.nn.NameJournalStatus = JSON.parse(d.nn.NameJournalStatus);
+    d.nn.NameDirStatuses = JSON.parse(d.nn.NameDirStatuses);
+    d.nn.NodeUsage = JSON.parse(d.nn.NodeUsage);
+    d.nn.LiveNodes = node_map_to_array(JSON.parse(d.nn.LiveNodes));
+    d.nn.DeadNodes = node_map_to_array(JSON.parse(d.nn.DeadNodes));
+    d.nn.DecomNodes = node_map_to_array(JSON.parse(d.nn.DecomNodes));
+    d.nn.CorruptFiles = JSON.parse(d.nn.CorruptFiles);
+
+    d.fs.SnapshotStats = JSON.parse(d.fs.SnapshotStats);
+    d.startup = startup_progress_workaround(d.startup);
+    return d;
+  }
+
+  function show_err_msg(msg) {
+    $('#alert-panel-body').html(msg);
+    $('#alert-panel').show();
+  }
+
+  load_json(
+    BEANS,
+    function(d) {
+      for (var k in d) {
+        data[k] = k === "startup" ? d[k] : d[k].beans[0];
+      }
+      data = data_workaround(data);
+      render();
+    },
+    function (url, jqxhr, text, err) {
+      show_err_msg('<p>Failed to retrieve data from ' + url + ', cause: ' + err + '</p>');
+    });
+})();

+ 13 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer-block-info.dust.html

@@ -0,0 +1,13 @@
+{#block}
+<p>Block ID: {blockId}</p>
+<p>Block Pool ID: {blockPoolId}</p>
+<p>Generation Stamp: {generationStamp}</p>
+<p>Size: {numBytes}</p>
+{/block}
+<p>Availability:
+<ul>
+{#locations}
+<li>{hostName}</li>
+{/locations}
+</ul>
+</p>

+ 26 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.dust.html

@@ -0,0 +1,26 @@
+<table class="table">
+<thead>
+<tr>
+<th>Permission</th>
+<th>Owner</th>
+<th>Group</th>
+<th>Size</th>
+<th>Replication</th>
+<th>Block Size</th>
+<th>Name</th>
+</tr>
+</thead>
+<tbody>
+{#FileStatus}
+<tr>
+<td>{#helper_to_permission/}</td>
+<td>{owner}</td>
+<td>{group}</td>
+<td>{length|fmt_bytes}</td>
+<td>{replication}</td>
+<td>{blockSize|fmt_bytes}</td>
+<td><a style="cursor:pointer" inode-type="{type}" class="explorer-browse-links" inode-path="{pathSuffix}">{pathSuffix}</a></td>
+</tr>
+{/FileStatus}
+</tbody>
+</table>

+ 86 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html

@@ -0,0 +1,86 @@
+<!--
+    Licensed to the Apache Software Foundation (ASF) under one or more
+    contributor license agreements.  See the NOTICE file distributed with
+    this work for additional information regarding copyright ownership.
+    The ASF licenses this file to You under the Apache License, Version 2.0
+    (the "License"); you may not use this file except in compliance with
+    the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+  -->
+<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Strict//EN"
+	  "http://www.w3.org/TR/xhtml1/DTD/xhtml1-strict.dtd">
+<html xmlns="http://www.w3.org/1999/xhtml">
+  <head>
+    <link rel="stylesheet" type="text/css" href="//netdna.bootstrapcdn.com/bootstrap/3.0.0/css/bootstrap.min.css" />
+    <link rel="stylesheet" type="text/css" href="/static/hadoop.css" />
+    <title>Browsing HDFS</title>
+  </head>
+  <body>
+    <div class="modal" id="file-info" tabindex="-1" role="dialog" aria-hidden="true">
+      <div class="modal-dialog">
+	<div class="modal-content">
+	  <div class="modal-header"><button type="button" class="close" data-dismiss="modal" aria-hidden="true">&times;</button>
+	    <h4 class="modal-title" id="file-info-title">File information</h4>
+	  </div>
+	  <div class="modal-body" id="file-info-body">
+	    <a id="file-info-download">Download</a>
+            <a id="file-info-preview" style="cursor:pointer">Tail the file (last 32K)</a>
+	    <hr />
+	    <div class="panel panel-info" id="file-info-blockinfo-panel">
+	      <div class="panel-heading">
+		Block information -- 
+		<select class="btn btn-default" id="file-info-blockinfo-list">
+		</select>
+	      </div>
+	      <div class="panel-body" id="file-info-blockinfo-body"></div>
+	    </div>
+	    <div class="panel panel-info" id="file-info-tail" style="display:none">
+	      <div class="panel-heading">File contents</div>
+	      <div class="panel-body">
+		<div class="input-group-sm">
+		<textarea class="form-control" style="height: 150px" id="file-info-preview-body"></textarea>
+		</div>
+	      </div>
+	    </div>
+	  </div>
+	  <div class="modal-footer"><button type="button" class="btn btn-primary"
+					    data-dismiss="modal">Close</button></div>
+	</div>
+      </div>
+    </div>
+    <div class="container">
+      <div class="page-header">
+	<h1>Browse Directory</h1>
+      </div>
+      <div class="alert alert-danger" id="alert-panel" style="display:none">
+	<button type="button" class="close" onclick="$('#alert-panel').hide();">&times;</button>
+	<div class="alert-body" id="alert-panel-body"></div>
+      </div>
+      <div class="row">
+	<form onsubmit="return false;">
+	  <div class="input-group"><input type="text" class="form-control" id=
+					  "directory" /> <span class="input-group-btn"><button class="btn btn-default"
+											       type="submit" id="btn-nav-directory"><span class="input-group-btn">Go!</span></button></span></div>
+	</form>
+      </div>
+      <br />
+      <div id="panel"></div>
+    </div>
+    <script type="text/javascript" src="//ajax.googleapis.com/ajax/libs/jquery/2.0.3/jquery.min.js">
+    </script><script type="text/javascript" src="//netdna.bootstrapcdn.com/bootstrap/3.0.0/js/bootstrap.min.js">
+    </script><script type="text/javascript" src="/static/dust-full-2.0.0.min.js">
+    </script><script type="text/javascript" src="/static/dust-helpers-1.1.1.min.js">
+    </script><script type="text/javascript" src="dfs-dust.js">
+    </script><script type="text/javascript" src="explorer.js">
+    </script>
+    <hr />
+    <p><a href="http://hadoop.apache.org/core">Hadoop</a>, 2013.</p>
+  </body>
+</html>

+ 182 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js

@@ -0,0 +1,182 @@
+/**
+ * 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.
+ */
+(function() {
+  "use strict";
+
+  // The chunk size of tailing the files, i.e., how many bytes will be shown
+  // in the preview.
+  var TAIL_CHUNK_SIZE = 32768;
+  var helpers = {
+    'helper_to_permission': function(chunk, ctx, bodies, params) {
+      var p = ctx.current().permission;
+      var dir = ctx.current().type == 'DIRECTORY' ? 'd' : '-';
+      var symbols = [ '---', '--x', '-w-', '-wx', 'r--', 'r-x', 'rw-', 'rwx' ];
+      var sticky = p > 1000;
+
+      var res = "";
+      for (var i = 0; i < 3; ++i) {
+	res = symbols[(p % 10)] + res;
+	p = Math.floor(p / 10);
+      }
+
+      if (sticky) {
+	var exec = ((parms.perm % 10) & 1) == 1;
+	res[res.length - 1] = exec ? 't' : 'T';
+      }
+
+      chunk.write(dir + res);
+      return chunk;
+    }
+  };
+
+  var base = dust.makeBase(helpers);
+  var current_directory = "";
+
+  function show_err_msg(msg) {
+    $('#alert-panel-body').html(msg);
+    $('#alert-panel').show();
+  }
+
+  function network_error_handler(url) {
+    return function (jqxhr, text, err) {
+      var msg = '<p>Failed to retreive data from ' + url + ', cause: ' + err + '</p>';
+      if (url.indexOf('/webhdfs/v1') === 0)  {
+        msg += '<p>WebHDFS might be disabled. WebHDFS is required to browse the filesystem.</p>';
+      }
+      show_err_msg(msg);
+    };
+  }
+
+  function append_path(prefix, s) {
+    var l = prefix.length;
+    var p = l > 0 && prefix[l - 1] == '/' ? prefix.substring(0, l - 1) : prefix;
+    return p + '/' + s;
+  }
+
+  function get_response(data, type) {
+    return data[type] !== undefined ? data[type] : null;
+  }
+
+  function get_response_err_msg(data) {
+    var msg = data.RemoteException !== undefined ? data.RemoteException.message : "";
+    return msg;
+  }
+
+  function view_file_details(path, abs_path) {
+    function show_block_info(blocks) {
+      var menus = $('#file-info-blockinfo-list');
+      menus.empty();
+
+      menus.data("blocks", blocks);
+      menus.change(function() {
+        var d = $(this).data('blocks')[$(this).val()];
+        if (d === undefined) {
+          return;
+        }
+
+        dust.render('block-info', d, function(err, out) {
+          $('#file-info-blockinfo-body').html(out);
+        });
+
+      });
+      for (var i = 0; i < blocks.length; ++i) {
+        var item = $('<option value="' + i + '">Block ' + i + '</option>');
+        menus.append(item);
+      }
+      menus.change();
+    }
+
+    var url = '/webhdfs/v1' + abs_path + '?op=GET_BLOCK_LOCATIONS';
+    $.ajax({"url": url, "crossDomain": true}).done(function(data) {
+      var d = get_response(data, "LocatedBlocks");
+      if (d === null) {
+        show_err_msg(get_response_err_msg(data));
+        return;
+      }
+
+      $('#file-info-tail').hide();
+      $('#file-info-title').text("File information - " + path);
+
+      var download_url = '/webhdfs/v1' + abs_path + '/?op=OPEN';
+
+      $('#file-info-download').attr('href', download_url);
+      $('#file-info-preview').click(function() {
+        var offset = d.fileLength - TAIL_CHUNK_SIZE;
+        var url = offset > 0 ? download_url + '&offset=' + offset : download_url;
+        $.get(url, function(t) {
+          $('#file-info-preview-body').val(t);
+          $('#file-info-tail').show();
+        }, "text").error(network_error_handler(url));
+      });
+
+      if (d.fileLength > 0) {
+        show_block_info(d.locatedBlocks);
+        $('#file-info-blockinfo-panel').show();
+      } else {
+        $('#file-info-blockinfo-panel').hide();
+      }
+      $('#file-info').modal();
+    }).error(network_error_handler(url));
+  }
+
+  function browse_directory(dir) {
+    var url = '/webhdfs/v1' + dir + '?op=LISTSTATUS';
+    $.get(url, function(data) {
+      var d = get_response(data, "FileStatuses");
+      if (d === null) {
+        show_err_msg(get_response_err_msg(data));
+        return;
+      }
+
+      current_directory = dir;
+      $('#directory').val(dir);
+      dust.render('explorer', base.push(d), function(err, out) {
+        $('#panel').html(out);
+
+        $('.explorer-browse-links').click(function() {
+          var type = $(this).attr('inode-type');
+          var path = $(this).attr('inode-path');
+          var abs_path = append_path(current_directory, path);
+          if (type == 'DIRECTORY') {
+            browse_directory(abs_path);
+          } else {
+            view_file_details(path, abs_path);
+          }
+        });
+      });
+    }).error(network_error_handler(url));
+  }
+
+
+  function init() {
+    var templates = [
+      { 'name': 'explorer', 'url': 'explorer.dust.html'},
+      { 'name': 'block-info', 'url': 'explorer-block-info.dust.html'}
+    ];
+
+    load_templates(dust, templates, function () {
+      var b = function() { browse_directory($('#directory').val()); };
+      $('#btn-nav-directory').click(b);
+      browse_directory('/');
+    }, function (url, jqxhr, text, err) {
+      network_error_handler(url)(jqxhr, text, err);
+    });
+  }
+
+  init();
+})();

File diff suppressed because it is too large
+ 0 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dust-full-2.0.0.min.js


+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dust-helpers-1.1.1.min.js

@@ -0,0 +1,8 @@
+(function(k){function n(b){b=b.f();return"object"===typeof b&&!0===b.h}function p(b,c){return"function"===typeof c?c.toString():c}function l(b,c,d,a,e){a=a||{};var m=d.a,g,f,l=a.d||"";if("undefined"!==typeof a.key)g=k.b.c(a.key,b,c);else if(n(c))g=c.f().i,c.f().g&&(e=function(){return!1});else return h.log("No key specified for filter in:"+l+" helper "),b;f=k.b.c(a.value,b,c);if(e(q(f,a.type,c),q(g,a.type,c))){n(c)&&(c.f().g=!0);if(m)return b.e(m,c);h.log("Missing body block in the "+l+" helper ")}else if(d["else"])return b.e(d["else"],
+c);return b}function q(b,c,d){if(b)switch(c||typeof b){case "number":return+b;case "string":return String(b);case "boolean":return Boolean("false"===b?!1:b);case "date":return new Date(b);case "context":return d.get(b)}return b}var h="undefined"!==typeof console?console:{log:function(){}};k.b={tap:function(b,c,d){var a=b;"function"===typeof b&&(!0===b.l?a=b():(a="",c.c(function(b){a+=b;return""}).e(b,d).p(),""===a&&(a=!1)));return a},sep:function(b,c,d){return c.stack.index===c.stack.m-1?b:d.a?d.a(b,
+c):b},idx:function(b,c,d){return d.a?d.a(b,c.push(c.stack.index)):b},contextDump:function(b,c,d,a){a=a||{};d=a.o||"output";a=a.key||"current";d=k.b.c(d,b,c);a=k.b.c(a,b,c);c="full"===a?JSON.stringify(c.stack,p,2):JSON.stringify(c.stack.head,p,2);return"console"===d?(h.log(c),b):b.write(c)},"if":function(b,c,d,a){var e=d.a,m=d["else"];if(a&&a.j){a=a.j;a=k.b.c(a,b,c);if(eval(a)){if(e)return b.e(d.a,c);h.log("Missing body block in the if helper!");return b}if(m)return b.e(d["else"],c)}else h.log("No condition given in the if helper!");
+return b},math:function(b,c,d,a){if(a&&"undefined"!==typeof a.key&&a.method){var e=a.key,m=a.method,g=a.n;a=a.round;var f=null,e=k.b.c(e,b,c),g=k.b.c(g,b,c);switch(m){case "mod":0!==g&&-0!==g||h.log("operand for divide operation is 0/-0: expect Nan!");f=parseFloat(e)%parseFloat(g);break;case "add":f=parseFloat(e)+parseFloat(g);break;case "subtract":f=parseFloat(e)-parseFloat(g);break;case "multiply":f=parseFloat(e)*parseFloat(g);break;case "divide":0!==g&&-0!==g||h.log("operand for divide operation is 0/-0: expect Nan/Infinity!");
+f=parseFloat(e)/parseFloat(g);break;case "ceil":f=Math.ceil(parseFloat(e));break;case "floor":f=Math.floor(parseFloat(e));break;case "round":f=Math.round(parseFloat(e));break;case "abs":f=Math.abs(parseFloat(e));break;default:h.log("method passed is not supported")}if(null!==f)return a&&(f=Math.round(f)),d&&d.a?b.e(d.a,c.push({h:!0,g:!1,i:f})):b.write(f)}else h.log("Key is a required parameter for math helper along with method/operand!");return b},select:function(b,c,d,a){var e=d.a;if(a&&"undefined"!==
+typeof a.key){a=k.b.c(a.key,b,c);if(e)return b.e(d.a,c.push({h:!0,g:!1,i:a}));h.log("Missing body block in the select helper ")}else h.log("No key given in the select helper!");return b},eq:function(b,c,d,a){a&&(a.d="eq");return l(b,c,d,a,function(a,b){return b===a})},ne:function(b,c,d,a){return a?(a.d="ne",l(b,c,d,a,function(a,b){return b!==a})):b},lt:function(b,c,d,a){if(a)return a.d="lt",l(b,c,d,a,function(a,b){return b<a})},lte:function(b,c,d,a){return a?(a.d="lte",l(b,c,d,a,function(a,b){return b<=
+a})):b},gt:function(b,c,d,a){return a?(a.d="gt",l(b,c,d,a,function(a,b){return b>a})):b},gte:function(b,c,d,a){return a?(a.d="gte",l(b,c,d,a,function(a,b){return b>=a})):b},"default":function(b,c,d,a){a&&(a.d="default");return l(b,c,d,a,function(){return!0})},size:function(b,c,d,a){c=0;var e;a=a||{};if((a=a.key)&&!0!==a)if(k.isArray(a))c=a.length;else if(!isNaN(parseFloat(a))&&isFinite(a))c=a;else if("object"===typeof a)for(e in c=0,a)Object.hasOwnProperty.call(a,e)&&c++;else c=(a+"").length;else c=
+0;return b.write(c)}}})("undefined"!==typeof exports?module.k=require("dustjs-linkedin"):dust);

+ 258 - 0
hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HdfsNfsGateway.apt.vm

@@ -0,0 +1,258 @@
+
+~~ Licensed under the Apache License, Version 2.0 (the "License");
+~~ you may not use this file except in compliance with the License.
+~~ You may obtain a copy of the License at
+~~
+~~   http://www.apache.org/licenses/LICENSE-2.0
+~~
+~~ Unless required by applicable law or agreed to in writing, software
+~~ distributed under the License is distributed on an "AS IS" BASIS,
+~~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+~~ See the License for the specific language governing permissions and
+~~ limitations under the License. See accompanying LICENSE file.
+
+  ---
+  Hadoop Distributed File System-${project.version} - HDFS NFS Gateway
+  ---
+  ---
+  ${maven.build.timestamp}
+
+HDFS NFS Gateway
+
+  \[ {{{./index.html}Go Back}} \]
+
+%{toc|section=1|fromDepth=0}
+
+* {Overview}
+
+  The NFS Gateway supports NFSv3 and allows HDFS to be mounted as part of the client's local file system.
+  Currently NFS Gateway supports and enables the following usage patterns:
+
+   * Users can browse the HDFS file system through their local file system
+     on NFSv3 client compatible operating systems.
+
+   * Users can download files from the the HDFS file system on to their
+     local file system.
+
+   * Users can upload files from their local file system directly to the
+     HDFS file system.
+
+   * Users can stream data directly to HDFS through the mount point. File
+     append is supported but random write is not supported. 
+
+  The NFS gateway machine needs the same thing to run an HDFS client like Hadoop JAR files, HADOOP_CONF directory.
+  The NFS gateway can be on the same host as DataNode, NameNode, or any HDFS client. 
+
+
+* {Configuration}
+
+   NFS gateway can work with its default settings in most cases. However, it's
+   strongly recommended for the users to update a few configuration properties based on their use
+   cases. All the related configuration properties can be added or updated in hdfs-site.xml.
+  
+   * If the client mounts the export with access time update allowed, make sure the following 
+    property is not disabled in the configuration file. Only NameNode needs to restart after 
+    this property is changed. On some Unix systems, the user can disable access time update
+    by mounting the export with "noatime".
+
+----
+<property>
+  <name>dfs.access.time.precision</name>
+  <value>3600000</value>
+  <description>The access time for HDFS file is precise upto this value. 
+    The default value is 1 hour. Setting a value of 0 disables
+    access times for HDFS.
+  </description>
+</property>
+----
+
+   * Users are expected to update the file dump directory. NFS client often 
+      reorders writes. Sequential writes can arrive at the NFS gateway at random
+      order. This directory is used to temporarily save out-of-order writes
+      before writing to HDFS. For each file, the out-of-order writes are dumped after 
+      they are accumulated to exceed certain threshold (e.g., 1MB) in memory.
+      One needs to make sure the directory has enough
+      space. For example, if the application uploads 10 files with each having 
+      100MB, it is recommended for this directory to have roughly 1GB space in case if a
+      worst-case write reorder happens to every file. Only NFS gateway needs to restart after 
+      this property is updated.
+
+----
+  <property>    
+    <name>dfs.nfs3.dump.dir</name>    
+    <value>/tmp/.hdfs-nfs</value>
+  </property>
+---- 
+
+  * By default, the export can be mounted by any client. To better control the access,
+    users can update the following property. The value string contains machine name and
+    access privilege, separated by whitespace
+    characters. Machine name format can be single host, wildcards, and IPv4 networks.The
+    access privilege uses rw or ro to specify readwrite or readonly access of the machines to exports. If the access
+    privilege is not provided, the default is read-only. Entries are separated by ";".
+    For example: "192.168.0.0/22 rw ; host*.example.com ; host1.test.org ro;". Only NFS gateway needs to restart after 
+    this property is updated.
+
+----
+<property>
+  <name>dfs.nfs.exports.allowed.hosts</name>
+  <value>* rw</value>
+</property>
+----
+
+  * Customize log settings. To get NFS debug trace, users can edit the log4j.property file 
+   to add the following. Note, debug trace, especially for ONCRPC, can be very verbose.
+
+    To change logging level:
+
+----------------------------------------------- 
+    log4j.logger.org.apache.hadoop.hdfs.nfs=DEBUG
+----------------------------------------------- 
+
+    To get more details of ONCRPC requests:
+
+----------------------------------------------- 
+    log4j.logger.org.apache.hadoop.oncrpc=DEBUG
+----------------------------------------------- 
+
+
+* {Start and stop NFS gateway service}
+
+  Three daemons are required to provide NFS service: rpcbind (or portmap), mountd and nfsd.
+  The NFS gateway process has both nfsd and mountd. It shares the HDFS root "/" as the
+  only export. It is recommended to use the portmap included in NFS gateway package. Even
+  though NFS gateway works with portmap/rpcbind provide by most Linux distributions, the
+  package included portmap is needed on some Linux systems such as REHL6.2 due to an 
+  {{{https://bugzilla.redhat.com/show_bug.cgi?id=731542}rpcbind bug}}. More detailed discussions can
+  be found in {{{https://issues.apache.org/jira/browse/HDFS-4763}HDFS-4763}}.
+
+   [[1]] Stop nfs/rpcbind/portmap services provided by the platform (commands can be different on various Unix platforms):
+      
+-------------------------
+     service nfs stop
+      
+     service rpcbind stop
+-------------------------
+
+
+   [[2]] Start package included portmap (needs root privileges):
+
+-------------------------
+     hadoop portmap
+  
+     OR
+
+     hadoop-daemon.sh start portmap
+-------------------------
+
+   [[3]] Start mountd and nfsd.
+   
+     No root privileges are required for this command. However, ensure that the user starting
+     the Hadoop cluster and the user starting the NFS gateway are same.
+
+-------------------------
+     hadoop nfs3
+
+     OR
+
+     hadoop-daemon.sh start nfs3
+-------------------------
+
+     Note, if the hadoop-daemon.sh script starts the NFS gateway, its log can be found in the hadoop log folder.
+
+
+   [[4]] Stop NFS gateway services.
+
+-------------------------
+      hadoop-daemon.sh stop nfs3
+
+      hadoop-daemon.sh stop portmap
+-------------------------
+
+
+* {Verify validity of NFS related services}
+
+    [[1]] Execute the following command to verify if all the services are up and running:
+
+-------------------------
+       rpcinfo -p $nfs_server_ip
+-------------------------
+
+     You should see output similar to the following:
+
+-------------------------
+       program vers proto   port
+
+       100005    1   tcp   4242  mountd
+
+       100005    2   udp   4242  mountd
+
+       100005    2   tcp   4242  mountd
+
+       100000    2   tcp    111  portmapper
+
+       100000    2   udp    111  portmapper
+
+       100005    3   udp   4242  mountd
+
+       100005    1   udp   4242  mountd
+
+       100003    3   tcp   2049  nfs
+
+       100005    3   tcp   4242  mountd
+-------------------------
+
+    [[2]]  Verify if the HDFS namespace is exported and can be mounted.
+
+-------------------------
+        showmount -e $nfs_server_ip                         
+-------------------------
+
+      You should see output similar to the following:
+     
+-------------------------
+        Exports list on $nfs_server_ip :
+
+        / (everyone)
+-------------------------
+
+
+* {Mount the export “/”}
+
+  Currently NFS v3 only uses TCP as the transportation protocol. 
+  NLM is not supported so mount option "nolock" is needed. It's recommended to use
+  hard mount. This is because, even after the client sends all data to 
+  NFS gateway, it may take NFS gateway some extra time to transfer data to HDFS 
+  when writes were reorderd by NFS client Kernel.
+ 
+  If soft mount has to be used, the user should give it a relatively 
+  long timeout (at least no less than the default timeout on the host) .
+
+  The users can mount the HDFS namespace as shown below:
+
+-------------------------------------------------------------------  
+       mount -t nfs -o vers=3,proto=tcp,nolock $server:/  $mount_point
+-------------------------------------------------------------------
+
+  Then the users can access HDFS as part of the local file system except that, 
+  hard link and random write are not supported yet.
+
+* {User authentication and mapping}
+
+  NFS gateway in this release uses AUTH_UNIX style authentication. When the user on NFS client
+  accesses the mount point, NFS client passes the UID to NFS gateway. 
+  NFS gateway does a lookup to find user name from the UID, and then passes the
+  username to the HDFS along with the HDFS requests.
+  For example, if the NFS client has current user as "admin", when the user accesses
+  the mounted directory, NFS gateway will access HDFS as user "admin". To access HDFS
+  as the user "hdfs", one needs to switch the current user to "hdfs" on the client system
+  when accessing the mounted directory.
+
+  The system administrator must ensure that the user on NFS client host has the same
+  name and UID as that on the NFS gateway host. This is usually not a problem if
+  the same user management system (e.g., LDAP/NIS) is used to create and deploy users on
+  HDFS nodes and NFS client node. In case the user account is created manually in different hosts, one might need to 
+  modify UID (e.g., do "usermod -u 123 myusername") on either NFS client or NFS gateway host
+  in order to make it the same on both sides. More technical details of RPC AUTH_UNIX can be found
+  in {{{http://tools.ietf.org/html/rfc1057}RPC specification}}.
+

+ 74 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java

@@ -83,7 +83,6 @@ import org.apache.log4j.RollingFileAppender;
 import org.junit.Test;
 import org.junit.Test;
 
 
 import com.google.common.collect.Sets;
 import com.google.common.collect.Sets;
-import org.mockito.Mockito;
 import static org.mockito.Mockito.*;
 import static org.mockito.Mockito.*;
 
 
 /**
 /**
@@ -892,6 +891,80 @@ public class TestFsck {
       }
       }
     }
     }
   }
   }
+  
+  /**
+   * Tests that the # of misreplaced replicas is correct
+   * @throws IOException
+   */
+  @Test
+  public void testFsckMisPlacedReplicas() throws IOException {
+    // Desired replication factor
+    final short REPL_FACTOR = 2;
+    // Number of replicas to actually start
+    short NUM_DN = 2;
+    // Number of blocks to write
+    final short NUM_BLOCKS = 3;
+    // Set a small-ish blocksize
+    final long blockSize = 512;
+    
+    String [] racks = {"/rack1", "/rack1"};
+    String [] hosts = {"host1", "host2"};
+    
+    Configuration conf = new Configuration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
+    
+    MiniDFSCluster cluster = null;
+    DistributedFileSystem dfs = null;
+    
+    try {
+      // Startup a minicluster
+      cluster = 
+          new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DN).hosts(hosts)
+          .racks(racks).build();
+      assertNotNull("Failed Cluster Creation", cluster);
+      cluster.waitClusterUp();
+      dfs = (DistributedFileSystem) cluster.getFileSystem();
+      assertNotNull("Failed to get FileSystem", dfs);
+      
+      // Create a file that will be intentionally under-replicated
+      final String pathString = new String("/testfile");
+      final Path path = new Path(pathString);
+      long fileLen = blockSize * NUM_BLOCKS;
+      DFSTestUtil.createFile(dfs, path, fileLen, REPL_FACTOR, 1);
+      
+      // Create an under-replicated file
+      NameNode namenode = cluster.getNameNode();
+      NetworkTopology nettop = cluster.getNamesystem().getBlockManager()
+          .getDatanodeManager().getNetworkTopology();
+      // Add a new node on different rack, so previous blocks' replicas 
+      // are considered to be misplaced
+      nettop.add(DFSTestUtil.getDatanodeDescriptor("/rack2", "/host3"));
+      NUM_DN++;
+      
+      Map<String,String[]> pmap = new HashMap<String, String[]>();
+      Writer result = new StringWriter();
+      PrintWriter out = new PrintWriter(result, true);
+      InetAddress remoteAddress = InetAddress.getLocalHost();
+      NamenodeFsck fsck = new NamenodeFsck(conf, namenode, nettop, pmap, out, 
+          NUM_DN, (short)REPL_FACTOR, remoteAddress);
+      
+      // Run the fsck and check the Result
+      final HdfsFileStatus file = 
+          namenode.getRpcServer().getFileInfo(pathString);
+      assertNotNull(file);
+      Result res = new Result(conf);
+      fsck.check(pathString, file, res);
+      // check misReplicatedBlock number.
+      assertEquals(res.numMisReplicatedBlocks, NUM_BLOCKS);
+    } finally {
+      if(dfs != null) {
+        dfs.close();
+      }
+      if(cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
 
 
   /** Test fsck with FileNotFound */
   /** Test fsck with FileNotFound */
   @Test
   @Test

+ 24 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java

@@ -17,6 +17,7 @@
  */
  */
 package org.apache.hadoop.hdfs.web;
 package org.apache.hadoop.hdfs.web;
 
 
+import java.util.HashMap;
 import java.util.Map;
 import java.util.Map;
 
 
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
@@ -58,4 +59,27 @@ public class TestJsonUtil {
     System.out.println("fs2     = " + fs2);
     System.out.println("fs2     = " + fs2);
     Assert.assertEquals(fstatus, fs2);
     Assert.assertEquals(fstatus, fs2);
   }
   }
+  
+  @Test
+  public void testToDatanodeInfoWithoutSecurePort() {
+    Map<String, Object> response = new HashMap<String, Object>();
+    
+    response.put("ipAddr", "127.0.0.1");
+    response.put("hostName", "localhost");
+    response.put("storageID", "fake-id");
+    response.put("xferPort", 1337l);
+    response.put("infoPort", 1338l);
+    // deliberately don't include an entry for "infoSecurePort"
+    response.put("ipcPort", 1339l);
+    response.put("capacity", 1024l);
+    response.put("dfsUsed", 512l);
+    response.put("remaining", 512l);
+    response.put("blockPoolUsed", 512l);
+    response.put("lastUpdate", 0l);
+    response.put("xceiverCount", 4096l);
+    response.put("networkLocation", "foo.bar.baz");
+    response.put("adminState", "NORMAL");
+    
+    JsonUtil.toDatanodeInfo(response);
+  }
 }
 }

+ 54 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestURLConnectionFactory.java

@@ -0,0 +1,54 @@
+/**
+ * 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.hdfs.web;
+
+import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.util.List;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+
+public final class TestURLConnectionFactory {
+
+  @Test
+  public void testConnConfiguratior() throws IOException {
+    final URL u = new URL("http://localhost");
+    final List<HttpURLConnection> conns = Lists.newArrayList();
+    URLConnectionFactory fc = new URLConnectionFactory(
+        URLConnectionFactory.DEFAULT_SOCKET_TIMEOUT);
+
+    fc.setConnConfigurator(new ConnectionConfigurator() {
+      @Override
+      public HttpURLConnection configure(HttpURLConnection conn)
+          throws IOException {
+        Assert.assertEquals(u, conn.getURL());
+        conns.add(conn);
+        return conn;
+      }
+    });
+
+    fc.openConnection(u);
+    Assert.assertEquals(1, conns.size());
+  }
+}

+ 6 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTimeouts.java

@@ -25,9 +25,11 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.io.InputStreamReader;
 import java.io.OutputStream;
 import java.io.OutputStream;
+import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
 import java.net.ServerSocket;
 import java.net.ServerSocket;
 import java.net.Socket;
 import java.net.Socket;
+import java.net.SocketAddress;
 import java.net.SocketTimeoutException;
 import java.net.SocketTimeoutException;
 import java.nio.channels.SocketChannel;
 import java.nio.channels.SocketChannel;
 import java.util.ArrayList;
 import java.util.ArrayList;
@@ -41,6 +43,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
@@ -71,8 +74,9 @@ public class TestWebHdfsTimeouts {
   @Before
   @Before
   public void setUp() throws Exception {
   public void setUp() throws Exception {
     Configuration conf = WebHdfsTestUtil.createConf();
     Configuration conf = WebHdfsTestUtil.createConf();
-    nnHttpAddress = NameNode.getHttpAddress(conf);
-    serverSocket = new ServerSocket(nnHttpAddress.getPort(), CONNECTION_BACKLOG);
+    serverSocket = new ServerSocket(0, CONNECTION_BACKLOG);
+    nnHttpAddress = new InetSocketAddress("localhost", serverSocket.getLocalPort());
+    conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "localhost:" + serverSocket.getLocalPort());
     fs = WebHdfsTestUtil.getWebHdfsFileSystem(conf);
     fs = WebHdfsTestUtil.getWebHdfsFileSystem(conf);
     fs.connectionFactory = connectionFactory;
     fs.connectionFactory = connectionFactory;
     clients = new ArrayList<SocketChannel>();
     clients = new ArrayList<SocketChannel>();

+ 358 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestDelegationTokenRemoteFetcher.java

@@ -0,0 +1,358 @@
+/**
+ * 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 static org.jboss.netty.handler.codec.http.HttpMethod.GET;
+import static org.jboss.netty.handler.codec.http.HttpResponseStatus.OK;
+import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.charset.Charset;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.Executors;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.HftpFileSystem;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.tools.DelegationTokenFetcher;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.token.Token;
+import org.apache.log4j.Logger;
+import org.jboss.netty.bootstrap.ServerBootstrap;
+import org.jboss.netty.buffer.ChannelBuffer;
+import org.jboss.netty.buffer.ChannelBuffers;
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelFutureListener;
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.channel.ChannelPipeline;
+import org.jboss.netty.channel.ChannelPipelineFactory;
+import org.jboss.netty.channel.Channels;
+import org.jboss.netty.channel.ExceptionEvent;
+import org.jboss.netty.channel.MessageEvent;
+import org.jboss.netty.channel.SimpleChannelUpstreamHandler;
+import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory;
+import org.jboss.netty.handler.codec.http.DefaultHttpResponse;
+import org.jboss.netty.handler.codec.http.HttpChunkAggregator;
+import org.jboss.netty.handler.codec.http.HttpHeaders;
+import org.jboss.netty.handler.codec.http.HttpRequest;
+import org.jboss.netty.handler.codec.http.HttpRequestDecoder;
+import org.jboss.netty.handler.codec.http.HttpResponse;
+import org.jboss.netty.handler.codec.http.HttpResponseEncoder;
+import org.jboss.netty.handler.codec.http.HttpResponseStatus;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.UnmodifiableIterator;
+
+public class TestDelegationTokenRemoteFetcher {
+  private static final Logger LOG = Logger
+      .getLogger(TestDelegationTokenRemoteFetcher.class);
+
+  private static final String EXP_DATE = "124123512361236";
+  private static final String tokenFile = "http.file.dta";
+
+  private int httpPort;
+  private String serviceUrl;
+  private FileSystem fileSys;
+  private Configuration conf;
+  private ServerBootstrap bootstrap;
+  private Token<DelegationTokenIdentifier> testToken;
+  private volatile AssertionError assertionError;
+  
+  @Before
+  public void init() throws Exception {
+    conf = new Configuration();
+    fileSys = FileSystem.getLocal(conf);
+    httpPort = NetUtils.getFreeSocketPort();
+    serviceUrl = "http://localhost:" + httpPort;
+    testToken = createToken(serviceUrl);
+  }
+
+  @After
+  public void clean() throws IOException {
+    if (fileSys != null)
+      fileSys.delete(new Path(tokenFile), true);
+    if (bootstrap != null)
+      bootstrap.releaseExternalResources();
+  }
+
+  /**
+   * try to fetch token without http server with IOException
+   */
+  @Test
+  public void testTokenFetchFail() throws Exception {
+    try {
+      DelegationTokenFetcher.main(new String[] { "-webservice=" + serviceUrl,
+          tokenFile });
+      fail("Token fetcher shouldn't start in absense of NN");
+    } catch (IOException ex) {
+    }
+  }
+  
+  /**
+   * try to fetch token without http server with IOException
+   */
+  @Test
+  public void testTokenRenewFail() {
+    try {
+      DelegationTokenFetcher.renewDelegationToken(serviceUrl, testToken);
+      fail("Token fetcher shouldn't be able to renew tokens in absense of NN");
+    } catch (IOException ex) {
+    } 
+  }     
+  
+  /**
+   * try cancel token without http server with IOException
+   */
+  @Test
+  public void expectedTokenCancelFail() {
+    try {
+      DelegationTokenFetcher.cancelDelegationToken(serviceUrl, testToken);
+      fail("Token fetcher shouldn't be able to cancel tokens in absense of NN");
+    } catch (IOException ex) {
+    } 
+  }
+  
+  /**
+   * try fetch token and get http response with error
+   */
+  @Test  
+  public void expectedTokenRenewErrorHttpResponse() {
+    bootstrap = startHttpServer(httpPort, testToken, serviceUrl);
+    try {
+      DelegationTokenFetcher.renewDelegationToken(serviceUrl + "/exception", 
+          createToken(serviceUrl));
+      fail("Token fetcher shouldn't be able to renew tokens using an invalid"
+          + " NN URL");
+    } catch (IOException ex) {
+    } 
+    if (assertionError != null)
+      throw assertionError;
+  }
+  
+  /**
+   *   
+   *
+   */
+  @Test
+  public void testCancelTokenFromHttp() throws IOException {
+    bootstrap = startHttpServer(httpPort, testToken, serviceUrl);
+    DelegationTokenFetcher.cancelDelegationToken(serviceUrl, testToken);
+    if (assertionError != null)
+      throw assertionError;
+  }
+  
+  /**
+   * Call renew token using http server return new expiration time
+   */
+  @Test
+  public void testRenewTokenFromHttp() throws IOException {
+    bootstrap = startHttpServer(httpPort, testToken, serviceUrl);
+    assertTrue("testRenewTokenFromHttp error",
+        Long.valueOf(EXP_DATE) == DelegationTokenFetcher.renewDelegationToken(
+            serviceUrl, testToken));
+    if (assertionError != null)
+      throw assertionError;
+  }
+
+  /**
+   * Call fetch token using http server 
+   */
+  @Test
+  public void expectedTokenIsRetrievedFromHttp() throws Exception {
+    bootstrap = startHttpServer(httpPort, testToken, serviceUrl);
+    DelegationTokenFetcher.main(new String[] { "-webservice=" + serviceUrl,
+        tokenFile });
+    Path p = new Path(fileSys.getWorkingDirectory(), tokenFile);
+    Credentials creds = Credentials.readTokenStorageFile(p, conf);
+    Iterator<Token<?>> itr = creds.getAllTokens().iterator();
+    assertTrue("token not exist error", itr.hasNext());
+    Token<?> fetchedToken = itr.next();
+    Assert.assertArrayEquals("token wrong identifier error",
+        testToken.getIdentifier(), fetchedToken.getIdentifier());
+    Assert.assertArrayEquals("token wrong password error",
+        testToken.getPassword(), fetchedToken.getPassword());
+    if (assertionError != null)
+      throw assertionError;
+  }
+  
+  private static Token<DelegationTokenIdentifier> createToken(String serviceUri) {
+    byte[] pw = "hadoop".getBytes();
+    byte[] ident = new DelegationTokenIdentifier(new Text("owner"), new Text(
+        "renewer"), new Text("realuser")).getBytes();
+    Text service = new Text(serviceUri);
+    return new Token<DelegationTokenIdentifier>(ident, pw,
+        HftpFileSystem.TOKEN_KIND, service);
+  }
+
+  private interface Handler {
+    void handle(Channel channel, Token<DelegationTokenIdentifier> token,
+        String serviceUrl) throws IOException;
+  }
+
+  private class FetchHandler implements Handler {
+    
+    @Override
+    public void handle(Channel channel, Token<DelegationTokenIdentifier> token,
+        String serviceUrl) throws IOException {
+      Assert.assertEquals(testToken, token);
+
+      Credentials creds = new Credentials();
+      creds.addToken(new Text(serviceUrl), token);
+      DataOutputBuffer out = new DataOutputBuffer();
+      creds.write(out);
+      int fileLength = out.getData().length;
+      ChannelBuffer cbuffer = ChannelBuffers.buffer(fileLength);
+      cbuffer.writeBytes(out.getData());
+      HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK);
+      response.setHeader(HttpHeaders.Names.CONTENT_LENGTH,
+          String.valueOf(fileLength));
+      response.setContent(cbuffer);
+      channel.write(response).addListener(ChannelFutureListener.CLOSE);
+    }
+  }
+
+  private class RenewHandler implements Handler {
+    
+    @Override
+    public void handle(Channel channel, Token<DelegationTokenIdentifier> token,
+        String serviceUrl) throws IOException {
+      Assert.assertEquals(testToken, token);
+      byte[] bytes = EXP_DATE.getBytes();
+      ChannelBuffer cbuffer = ChannelBuffers.buffer(bytes.length);
+      cbuffer.writeBytes(bytes);
+      HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK);
+      response.setHeader(HttpHeaders.Names.CONTENT_LENGTH,
+          String.valueOf(bytes.length));
+      response.setContent(cbuffer);
+      channel.write(response).addListener(ChannelFutureListener.CLOSE);
+    }
+  }
+  
+  private class ExceptionHandler implements Handler {
+
+    @Override
+    public void handle(Channel channel, Token<DelegationTokenIdentifier> token,
+        String serviceUrl) throws IOException {
+      Assert.assertEquals(testToken, token);
+      HttpResponse response = new DefaultHttpResponse(HTTP_1_1, 
+          HttpResponseStatus.METHOD_NOT_ALLOWED);
+      channel.write(response).addListener(ChannelFutureListener.CLOSE);
+    }    
+  }
+  
+  private class CancelHandler implements Handler {
+
+    @Override
+    public void handle(Channel channel, Token<DelegationTokenIdentifier> token,
+        String serviceUrl) throws IOException {
+      Assert.assertEquals(testToken, token);
+      HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK);
+      channel.write(response).addListener(ChannelFutureListener.CLOSE);
+    }    
+  }
+  
+  private final class CredentialsLogicHandler extends
+      SimpleChannelUpstreamHandler {
+
+    private final Token<DelegationTokenIdentifier> token;
+    private final String serviceUrl;
+    private ImmutableMap<String, Handler> routes = ImmutableMap.of(
+        "/exception", new ExceptionHandler(),
+        "/cancelDelegationToken", new CancelHandler(),
+        "/getDelegationToken", new FetchHandler() , 
+        "/renewDelegationToken", new RenewHandler());
+
+    public CredentialsLogicHandler(Token<DelegationTokenIdentifier> token,
+        String serviceUrl) {
+      this.token = token;
+      this.serviceUrl = serviceUrl;
+    }
+
+    @Override
+    public void messageReceived(ChannelHandlerContext ctx, final MessageEvent e)
+        throws Exception {
+      HttpRequest request = (HttpRequest) e.getMessage();
+      if (request.getMethod() != GET) {
+        return;
+      }
+      UnmodifiableIterator<Map.Entry<String, Handler>> iter = routes.entrySet()
+          .iterator();
+      while (iter.hasNext()) {
+        Map.Entry<String, Handler> entry = iter.next();
+        if (request.getUri().contains(entry.getKey())) {
+          Handler handler = entry.getValue();
+          try {
+            handler.handle(e.getChannel(), token, serviceUrl);
+          } catch (AssertionError ee) {
+            TestDelegationTokenRemoteFetcher.this.assertionError = ee;
+            HttpResponse response = new DefaultHttpResponse(HTTP_1_1, 
+                HttpResponseStatus.BAD_REQUEST);
+            response.setContent(ChannelBuffers.copiedBuffer(ee.getMessage(), 
+                Charset.defaultCharset()));
+            e.getChannel().write(response).addListener(ChannelFutureListener.CLOSE);
+          }
+          return;
+        }
+      }
+    }
+
+    @Override
+    public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e)
+        throws Exception {
+      Channel ch = e.getChannel();
+      Throwable cause = e.getCause();
+
+      if (LOG.isDebugEnabled())
+        LOG.debug(cause.getMessage());
+      ch.close().addListener(ChannelFutureListener.CLOSE);
+    }
+  }
+
+  private ServerBootstrap startHttpServer(int port,
+      final Token<DelegationTokenIdentifier> token, final String url) {
+    ServerBootstrap bootstrap = new ServerBootstrap(
+        new NioServerSocketChannelFactory(Executors.newCachedThreadPool(),
+            Executors.newCachedThreadPool()));
+
+    bootstrap.setPipelineFactory(new ChannelPipelineFactory() {
+      @Override
+      public ChannelPipeline getPipeline() throws Exception {
+        return Channels.pipeline(new HttpRequestDecoder(),
+            new HttpChunkAggregator(65536), new HttpResponseEncoder(),
+            new CredentialsLogicHandler(token, url));
+      }
+    });
+    bootstrap.bind(new InetSocketAddress("localhost", port));
+    return bootstrap;
+  }
+  
+}

+ 23 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestJMXGet.java

@@ -21,9 +21,13 @@ package org.apache.hadoop.tools;
 import static org.apache.hadoop.test.MetricsAsserts.assertGauge;
 import static org.apache.hadoop.test.MetricsAsserts.assertGauge;
 import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
 import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 
 import java.io.File;
 import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
+import java.io.PipedInputStream;
+import java.io.PipedOutputStream;
+import java.io.PrintStream;
 import java.util.Random;
 import java.util.Random;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
@@ -92,6 +96,7 @@ public class TestJMXGet {
     //jmx.init();
     //jmx.init();
     //jmx = new JMXGet();
     //jmx = new JMXGet();
     jmx.init(); // default lists namenode mbeans only
     jmx.init(); // default lists namenode mbeans only
+    assertTrue("error printAllValues", checkPrintAllValues(jmx));
 
 
     //get some data from different source
     //get some data from different source
     assertEquals(numDatanodes, Integer.parseInt(
     assertEquals(numDatanodes, Integer.parseInt(
@@ -103,7 +108,24 @@ public class TestJMXGet {
 
 
     cluster.shutdown();
     cluster.shutdown();
   }
   }
-
+  
+  private static boolean checkPrintAllValues(JMXGet jmx) throws Exception {
+    int size = 0; 
+    byte[] bytes = null;
+    String pattern = "List of all the available keys:";
+    PipedOutputStream pipeOut = new PipedOutputStream();
+    PipedInputStream pipeIn = new PipedInputStream(pipeOut);
+    System.setErr(new PrintStream(pipeOut));
+    jmx.printAllValues();
+    if ((size = pipeIn.available()) != 0) {
+      bytes = new byte[size];
+      pipeIn.read(bytes, 0, bytes.length);            
+    }
+    pipeOut.close();
+    pipeIn.close();
+    return bytes != null ? new String(bytes).contains(pattern) : false;
+  }
+  
   /**
   /**
    * test JMX connection to DataNode..
    * test JMX connection to DataNode..
    * @throws Exception 
    * @throws Exception 

+ 160 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestTools.java

@@ -0,0 +1,160 @@
+package org.apache.hadoop.tools;
+
+/**
+ * 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.
+ */
+import static org.junit.Assert.*;
+
+import java.io.ByteArrayOutputStream;
+import java.io.PipedInputStream;
+import java.io.PipedOutputStream;
+import java.io.PrintStream;
+import org.apache.hadoop.hdfs.tools.DFSAdmin;
+import org.apache.hadoop.hdfs.tools.DelegationTokenFetcher;
+import org.apache.hadoop.hdfs.tools.JMXGet;
+import org.apache.hadoop.util.ExitUtil;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.hadoop.util.ExitUtil.ExitException;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.io.ByteStreams;
+
+public class TestTools {
+
+  private static final int PIPE_BUFFER_SIZE = 1024 * 5;
+  private final static String INVALID_OPTION = "-invalidOption";
+  private static final String[] OPTIONS = new String[2];
+
+  @BeforeClass
+  public static void before() {
+    ExitUtil.disableSystemExit();
+    OPTIONS[1] = INVALID_OPTION;
+  }
+
+  @Test  
+  public void testDelegationTokenFetcherPrintUsage() {
+    String pattern = "Options:";
+    checkOutput(new String[] { "-help" }, pattern, System.out,
+        DelegationTokenFetcher.class);
+  }
+
+  @Test  
+  public void testDelegationTokenFetcherErrorOption() {
+    String pattern = "ERROR: Only specify cancel, renew or print.";
+    checkOutput(new String[] { "-cancel", "-renew" }, pattern, System.err,
+        DelegationTokenFetcher.class);
+  }
+
+  @Test  
+  public void testJMXToolHelp() {
+    String pattern = "usage: jmxget options are:";
+    checkOutput(new String[] { "-help" }, pattern, System.out, JMXGet.class);
+  }
+
+  @Test  
+  public void testJMXToolAdditionParameter() {
+    String pattern = "key = -addition";
+    checkOutput(new String[] { "-service=NameNode", "-server=localhost",
+        "-addition" }, pattern, System.err, JMXGet.class);
+  }
+
+  @Test
+  public void testDFSAdminInvalidUsageHelp() {
+    ImmutableSet<String> args = ImmutableSet.of("-report", "-saveNamespace",
+        "-rollEdits", "-restoreFailedStorage", "-refreshNodes",
+        "-finalizeUpgrade", "-metasave", "-refreshUserToGroupsMappings",
+        "-printTopology", "-refreshNamenodes", "-deleteBlockPool",
+        "-setBalancerBandwidth", "-fetchImage");
+    try {
+      for (String arg : args)
+        assertTrue(ToolRunner.run(new DFSAdmin(), fillArgs(arg)) == -1);
+      
+      assertTrue(ToolRunner.run(new DFSAdmin(),
+          new String[] { "-help", "-some" }) == 0);
+    } catch (Exception e) {
+      fail("testDFSAdminHelp error" + e);
+    }
+
+    String pattern = "Usage: java DFSAdmin";
+    checkOutput(new String[] { "-cancel", "-renew" }, pattern, System.err,
+        DFSAdmin.class);
+  }
+
+  private static String[] fillArgs(String arg) {
+    OPTIONS[0] = arg;
+    return OPTIONS;
+  }
+
+  private void checkOutput(String[] args, String pattern, PrintStream out,
+      Class<?> clazz) {       
+    ByteArrayOutputStream outBytes = new ByteArrayOutputStream();
+    try {
+      PipedOutputStream pipeOut = new PipedOutputStream();
+      PipedInputStream pipeIn = new PipedInputStream(pipeOut, PIPE_BUFFER_SIZE);
+      if (out == System.out) {
+        System.setOut(new PrintStream(pipeOut));
+      } else if (out == System.err) {
+        System.setErr(new PrintStream(pipeOut));
+      }
+
+      if (clazz == DelegationTokenFetcher.class) {
+        expectDelegationTokenFetcherExit(args);
+      } else if (clazz == JMXGet.class) {
+        expectJMXGetExit(args);
+      } else if (clazz == DFSAdmin.class) {
+        expectDfsAdminPrint(args);
+      }
+      pipeOut.close();
+      ByteStreams.copy(pipeIn, outBytes);      
+      pipeIn.close();
+      assertTrue(new String(outBytes.toByteArray()).contains(pattern));            
+    } catch (Exception ex) {
+      fail("checkOutput error " + ex);
+    }
+  }
+
+  private void expectDfsAdminPrint(String[] args) {
+    try {
+      ToolRunner.run(new DFSAdmin(), args);
+    } catch (Exception ex) {
+      fail("expectDelegationTokenFetcherExit ex error " + ex);
+    }
+  }
+
+  private static void expectDelegationTokenFetcherExit(String[] args) {
+    try {
+      DelegationTokenFetcher.main(args);
+      fail("should call exit");
+    } catch (ExitException e) {
+      ExitUtil.resetFirstExitException();
+    } catch (Exception ex) {
+      fail("expectDelegationTokenFetcherExit ex error " + ex);
+    }
+  }
+
+  private static void expectJMXGetExit(String[] args) {
+    try {
+      JMXGet.main(args);
+      fail("should call exit");
+    } catch (ExitException e) {
+      ExitUtil.resetFirstExitException();
+    } catch (Exception ex) {
+      fail("expectJMXGetExit ex error " + ex);
+    }
+  }
+}

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

@@ -209,6 +209,9 @@ Release 2.2.1 - UNRELEASED
     MAPREDUCE-5463. Deprecate SLOTS_MILLIS counters (Tzuyoshi Ozawa via Sandy
     MAPREDUCE-5463. Deprecate SLOTS_MILLIS counters (Tzuyoshi Ozawa via Sandy
     Ryza)
     Ryza)
 
 
+    MAPREDUCE-5457. Add a KeyOnlyTextOutputReader to enable streaming to write
+    out text files without separators (Sandy Ryza)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES
@@ -222,6 +225,9 @@ Release 2.2.1 - UNRELEASED
     MAPREDUCE-5518. Fixed typo "can't read paritions file". (Albert Chu
     MAPREDUCE-5518. Fixed typo "can't read paritions file". (Albert Chu
     via devaraj)
     via devaraj)
 
 
+    MAPREDUCE-5561. org.apache.hadoop.mapreduce.v2.app.job.impl.TestJobImpl
+    testcase failing on trunk (Karthik Kambatla via jlowe)
+
 Release 2.2.0 - 2013-10-13
 Release 2.2.0 - 2013-10-13
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -1497,6 +1503,8 @@ Release 0.23.10 - UNRELEASED
     MAPREDUCE-5586. TestCopyMapper#testCopyFailOnBlockSizeDifference fails when
     MAPREDUCE-5586. TestCopyMapper#testCopyFailOnBlockSizeDifference fails when
     run from hadoop-tools/hadoop-distcp directory (jeagles)
     run from hadoop-tools/hadoop-distcp directory (jeagles)
 
 
+    MAPREDUCE-5587. TestTextOutputFormat fails on JDK7 (jeagles)
+
 Release 0.23.9 - 2013-07-08
 Release 0.23.9 - 2013-07-08
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 0 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java

@@ -415,7 +415,6 @@ public class TestJobImpl {
           TaskEventType.T_ATTEMPT_FAILED));
           TaskEventType.T_ATTEMPT_FAILED));
       }
       }
     }
     }
-    assertJobState(job, JobStateInternal.FAIL_ABORT);
 
 
     dispatcher.await();
     dispatcher.await();
     //Verify abortJob is called once and the job failed
     //Verify abortJob is called once and the job failed

+ 49 - 22
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTextOutputFormat.java

@@ -18,13 +18,24 @@
 
 
 package org.apache.hadoop.mapred;
 package org.apache.hadoop.mapred;
 
 
-import java.io.*;
-import junit.framework.TestCase;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
 
 
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.io.*;
+import org.junit.Test;
 
 
-public class TestTextOutputFormat extends TestCase {
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.compress.DefaultCodec;
+import org.apache.hadoop.io.compress.CompressionInputStream;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.util.LineReader;
+
+public class TestTextOutputFormat {
   private static JobConf defaultConf = new JobConf();
   private static JobConf defaultConf = new JobConf();
 
 
   private static FileSystem localFs = null;
   private static FileSystem localFs = null;
@@ -38,12 +49,13 @@ public class TestTextOutputFormat extends TestCase {
   // A random task attempt id for testing.
   // A random task attempt id for testing.
   private static String attempt = "attempt_200707121733_0001_m_000000_0";
   private static String attempt = "attempt_200707121733_0001_m_000000_0";
 
 
-  private static Path workDir = 
+  private static Path workDir =
     new Path(new Path(
     new Path(new Path(
-                      new Path(System.getProperty("test.build.data", "."), 
-                               "data"), 
+                      new Path(System.getProperty("test.build.data", "."),
+                               "data"),
                       FileOutputCommitter.TEMP_DIR_NAME), "_" + attempt);
                       FileOutputCommitter.TEMP_DIR_NAME), "_" + attempt);
 
 
+  @Test
   public void testFormat() throws Exception {
   public void testFormat() throws Exception {
     JobConf job = new JobConf();
     JobConf job = new JobConf();
     job.set(JobContext.TASK_ATTEMPT_ID, attempt);
     job.set(JobContext.TASK_ATTEMPT_ID, attempt);
@@ -53,7 +65,7 @@ public class TestTextOutputFormat extends TestCase {
     if (!fs.mkdirs(workDir)) {
     if (!fs.mkdirs(workDir)) {
       fail("Failed to create output directory");
       fail("Failed to create output directory");
     }
     }
-    String file = "test.txt";
+    String file = "test_format.txt";
 
 
     // A reporter that does nothing
     // A reporter that does nothing
     Reporter reporter = Reporter.NULL;
     Reporter reporter = Reporter.NULL;
@@ -90,10 +102,11 @@ public class TestTextOutputFormat extends TestCase {
     expectedOutput.append(key1).append("\n");
     expectedOutput.append(key1).append("\n");
     expectedOutput.append(key2).append('\t').append(val2).append("\n");
     expectedOutput.append(key2).append('\t').append(val2).append("\n");
     String output = UtilsForTests.slurp(expectedFile);
     String output = UtilsForTests.slurp(expectedFile);
-    assertEquals(output, expectedOutput.toString());
+    assertEquals(expectedOutput.toString(), output);
 
 
   }
   }
 
 
+  @Test
   public void testFormatWithCustomSeparator() throws Exception {
   public void testFormatWithCustomSeparator() throws Exception {
     JobConf job = new JobConf();
     JobConf job = new JobConf();
     String separator = "\u0001";
     String separator = "\u0001";
@@ -105,7 +118,7 @@ public class TestTextOutputFormat extends TestCase {
     if (!fs.mkdirs(workDir)) {
     if (!fs.mkdirs(workDir)) {
       fail("Failed to create output directory");
       fail("Failed to create output directory");
     }
     }
-    String file = "test.txt";
+    String file = "test_custom.txt";
 
 
     // A reporter that does nothing
     // A reporter that does nothing
     Reporter reporter = Reporter.NULL;
     Reporter reporter = Reporter.NULL;
@@ -142,27 +155,27 @@ public class TestTextOutputFormat extends TestCase {
     expectedOutput.append(key1).append("\n");
     expectedOutput.append(key1).append("\n");
     expectedOutput.append(key2).append(separator).append(val2).append("\n");
     expectedOutput.append(key2).append(separator).append(val2).append("\n");
     String output = UtilsForTests.slurp(expectedFile);
     String output = UtilsForTests.slurp(expectedFile);
-    assertEquals(output, expectedOutput.toString());
+    assertEquals(expectedOutput.toString(), output);
 
 
   }
   }
+
   /**
   /**
    * test compressed file
    * test compressed file
    * @throws IOException
    * @throws IOException
    */
    */
- public void testCompress() throws IOException{
+  @Test
+  public void testCompress() throws IOException {
    JobConf job = new JobConf();
    JobConf job = new JobConf();
-   String separator = "\u0001";
-   job.set("mapreduce.output.textoutputformat.separator", separator);
    job.set(JobContext.TASK_ATTEMPT_ID, attempt);
    job.set(JobContext.TASK_ATTEMPT_ID, attempt);
    job.set(org.apache.hadoop.mapreduce.lib.output.FileOutputFormat.COMPRESS,"true");
    job.set(org.apache.hadoop.mapreduce.lib.output.FileOutputFormat.COMPRESS,"true");
-   
+
    FileOutputFormat.setOutputPath(job, workDir.getParent().getParent());
    FileOutputFormat.setOutputPath(job, workDir.getParent().getParent());
    FileOutputFormat.setWorkOutputPath(job, workDir);
    FileOutputFormat.setWorkOutputPath(job, workDir);
    FileSystem fs = workDir.getFileSystem(job);
    FileSystem fs = workDir.getFileSystem(job);
    if (!fs.mkdirs(workDir)) {
    if (!fs.mkdirs(workDir)) {
      fail("Failed to create output directory");
      fail("Failed to create output directory");
    }
    }
-   String file = "test.txt";
+   String file = "test_compress.txt";
 
 
    // A reporter that does nothing
    // A reporter that does nothing
    Reporter reporter = Reporter.NULL;
    Reporter reporter = Reporter.NULL;
@@ -189,16 +202,30 @@ public class TestTextOutputFormat extends TestCase {
    } finally {
    } finally {
      theRecordWriter.close(reporter);
      theRecordWriter.close(reporter);
    }
    }
-   File expectedFile = new File(new Path(workDir, file).toString());
    StringBuffer expectedOutput = new StringBuffer();
    StringBuffer expectedOutput = new StringBuffer();
-   expectedOutput.append(key1).append(separator).append(val1).append("\n");
+   expectedOutput.append(key1).append("\t").append(val1).append("\n");
    expectedOutput.append(val1).append("\n");
    expectedOutput.append(val1).append("\n");
    expectedOutput.append(val2).append("\n");
    expectedOutput.append(val2).append("\n");
    expectedOutput.append(key2).append("\n");
    expectedOutput.append(key2).append("\n");
    expectedOutput.append(key1).append("\n");
    expectedOutput.append(key1).append("\n");
-   expectedOutput.append(key2).append(separator).append(val2).append("\n");
-   String output = UtilsForTests.slurp(expectedFile);
-   assertEquals(output, expectedOutput.toString());
+   expectedOutput.append(key2).append("\t").append(val2).append("\n");
+
+   DefaultCodec codec = new DefaultCodec();
+   codec.setConf(job);
+   Path expectedFile = new Path(workDir, file + codec.getDefaultExtension());
+   final FileInputStream istream = new FileInputStream(expectedFile.toString());
+   CompressionInputStream cistream = codec.createInputStream(istream);
+   LineReader reader = new LineReader(cistream);
+
+   String output = "";
+   Text out = new Text();
+   while (reader.readLine(out) > 0) {
+     output += out;
+     output += "\n";
+   }
+   reader.close();
+
+   assertEquals(expectedOutput.toString(), output);
  }
  }
   public static void main(String[] args) throws Exception {
   public static void main(String[] args) throws Exception {
     new TestTextOutputFormat().testFormat();
     new TestTextOutputFormat().testFormat();

+ 1 - 0
hadoop-project/src/site/site.xml

@@ -80,6 +80,7 @@
       <item name="HttpFS Gateway" href="hadoop-hdfs-httpfs/index.html"/>
       <item name="HttpFS Gateway" href="hadoop-hdfs-httpfs/index.html"/>
       <item name="Short Circuit Local Reads" 
       <item name="Short Circuit Local Reads" 
           href="hadoop-project-dist/hadoop-hdfs/ShortCircuitLocalReads.html"/>
           href="hadoop-project-dist/hadoop-hdfs/ShortCircuitLocalReads.html"/>
+      <item name="HDFS NFS Gateway" href="hadoop-project-dist/hadoop-hdfs/HdfsNfsGateway.html"/>
     </menu>
     </menu>
 
 
     <menu name="MapReduce" inherit="top">
     <menu name="MapReduce" inherit="top">

+ 507 - 68
hadoop-tools/hadoop-archives/src/test/java/org/apache/hadoop/tools/TestHadoopArchives.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.tools;
 package org.apache.hadoop.tools;
 
 
 import java.io.ByteArrayOutputStream;
 import java.io.ByteArrayOutputStream;
+import java.io.FilterInputStream;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.io.PrintStream;
 import java.net.URI;
 import java.net.URI;
@@ -30,9 +31,13 @@ import java.util.StringTokenizer;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.FsShell;
+import org.apache.hadoop.fs.HarFileSystem;
+import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
@@ -42,6 +47,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.log4j.Level;
 import org.apache.log4j.Level;
 import org.junit.After;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Assert;
+import static org.junit.Assert.*;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
 
 
@@ -62,19 +68,36 @@ public class TestHadoopArchives {
   private static final String inputDir = "input";
   private static final String inputDir = "input";
 
 
   private Path inputPath;
   private Path inputPath;
+  private Path archivePath;
+  private final List<String> fileList = new ArrayList<String>();
   private MiniDFSCluster dfscluster;
   private MiniDFSCluster dfscluster;
 
 
   private Configuration conf;
   private Configuration conf;
   private FileSystem fs;
   private FileSystem fs;
-  private Path archivePath;
 
 
-  static private Path createFile(Path dir, String filename, FileSystem fs)
-      throws IOException {
-    final Path f = new Path(dir, filename);
+  private static String createFile(Path root, FileSystem fs, String... dirsAndFile
+      ) throws IOException {
+    String fileBaseName = dirsAndFile[dirsAndFile.length - 1]; 
+    return createFile(root, fs, fileBaseName.getBytes("UTF-8"), dirsAndFile);
+  }
+  
+  private static String createFile(Path root, FileSystem fs, byte[] fileContent, String... dirsAndFile
+    ) throws IOException {
+    StringBuilder sb = new StringBuilder();
+    for (String segment: dirsAndFile) {
+      if (sb.length() > 0) {
+        sb.append(Path.SEPARATOR);  
+      }
+      sb.append(segment);
+    }
+    final Path f = new Path(root, sb.toString());
     final FSDataOutputStream out = fs.create(f);
     final FSDataOutputStream out = fs.create(f);
-    out.write(filename.getBytes());
-    out.close();
-    return f;
+    try {
+         out.write(fileContent);
+    } finally {
+      out.close();
+    }
+    return sb.toString();
   }
   }
 
 
   @Before
   @Before
@@ -86,102 +109,80 @@ public class TestHadoopArchives {
     conf.set(CapacitySchedulerConfiguration.PREFIX
     conf.set(CapacitySchedulerConfiguration.PREFIX
         + CapacitySchedulerConfiguration.ROOT + ".default."
         + CapacitySchedulerConfiguration.ROOT + ".default."
         + CapacitySchedulerConfiguration.CAPACITY, "100");
         + CapacitySchedulerConfiguration.CAPACITY, "100");
-    dfscluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).format(true)
-        .build();
+    dfscluster = new MiniDFSCluster
+      .Builder(conf)
+      .checkExitOnShutdown(true)
+      .numDataNodes(2)
+      .format(true)
+      .racks(null)
+      .build();
 
 
     fs = dfscluster.getFileSystem();
     fs = dfscluster.getFileSystem();
-    inputPath = new Path(fs.getHomeDirectory(), inputDir);
+    
+    // prepare archive path:
     archivePath = new Path(fs.getHomeDirectory(), "archive");
     archivePath = new Path(fs.getHomeDirectory(), "archive");
+    fs.delete(archivePath, true);
+    
+    // prepare input path:
+    inputPath = new Path(fs.getHomeDirectory(), inputDir);
+    fs.delete(inputPath, true);
     fs.mkdirs(inputPath);
     fs.mkdirs(inputPath);
-    createFile(inputPath, "a", fs);
-    createFile(inputPath, "b", fs);
-    createFile(inputPath, "c", fs);
+    // create basic input files:
+    fileList.add(createFile(inputPath, fs, "a"));
+    fileList.add(createFile(inputPath, fs, "b"));
+    fileList.add(createFile(inputPath, fs, "c"));
   }
   }
 
 
   @After
   @After
   public void tearDown() throws Exception {
   public void tearDown() throws Exception {
-    try {
-      if (dfscluster != null) {
-        dfscluster.shutdown();
-      }
-      if (dfscluster != null) {
-        dfscluster.shutdown();
-      }
-    } catch (Exception e) {
-      System.err.println(e);
+    if (dfscluster != null) {
+      dfscluster.shutdown();
     }
     }
   }
   }
 
 
   @Test
   @Test
   public void testRelativePath() throws Exception {
   public void testRelativePath() throws Exception {
-    fs.delete(archivePath, true);
-
     final Path sub1 = new Path(inputPath, "dir1");
     final Path sub1 = new Path(inputPath, "dir1");
     fs.mkdirs(sub1);
     fs.mkdirs(sub1);
-    createFile(sub1, "a", fs);
+    createFile(inputPath, fs, sub1.getName(), "a");
     final FsShell shell = new FsShell(conf);
     final FsShell shell = new FsShell(conf);
 
 
     final List<String> originalPaths = lsr(shell, "input");
     final List<String> originalPaths = lsr(shell, "input");
-    System.out.println("originalPath: " + originalPaths);
-    final URI uri = fs.getUri();
-    final String prefix = "har://hdfs-" + uri.getHost() + ":" + uri.getPort()
-        + archivePath.toUri().getPath() + Path.SEPARATOR;
+    System.out.println("originalPaths: " + originalPaths);
 
 
-    {
-      final String harName = "foo.har";
-      final String[] args = { "-archiveName", harName, "-p", "input", "*",
-          "archive" };
-      System.setProperty(HadoopArchives.TEST_HADOOP_ARCHIVES_JAR_PATH,
-          HADOOP_ARCHIVES_JAR);
-      final HadoopArchives har = new HadoopArchives(conf);
-      Assert.assertEquals(0, ToolRunner.run(har, args));
+    // make the archive:
+    final String fullHarPathStr = makeArchive();
 
 
-      // compare results
-      final List<String> harPaths = lsr(shell, prefix + harName);
-      Assert.assertEquals(originalPaths, harPaths);
-    }
+    // compare results:
+    final List<String> harPaths = lsr(shell, fullHarPathStr);
+    Assert.assertEquals(originalPaths, harPaths);
   }
   }
   
   
 @Test
 @Test
   public void testPathWithSpaces() throws Exception {
   public void testPathWithSpaces() throws Exception {
-    fs.delete(archivePath, true);
-
     // create files/directories with spaces
     // create files/directories with spaces
-    createFile(inputPath, "c c", fs);
+    createFile(inputPath, fs, "c c");
     final Path sub1 = new Path(inputPath, "sub 1");
     final Path sub1 = new Path(inputPath, "sub 1");
     fs.mkdirs(sub1);
     fs.mkdirs(sub1);
-    createFile(sub1, "file x y z", fs);
-    createFile(sub1, "file", fs);
-    createFile(sub1, "x", fs);
-    createFile(sub1, "y", fs);
-    createFile(sub1, "z", fs);
+    createFile(sub1, fs, "file x y z");
+    createFile(sub1, fs, "file");
+    createFile(sub1, fs, "x");
+    createFile(sub1, fs, "y");
+    createFile(sub1, fs, "z");
     final Path sub2 = new Path(inputPath, "sub 1 with suffix");
     final Path sub2 = new Path(inputPath, "sub 1 with suffix");
     fs.mkdirs(sub2);
     fs.mkdirs(sub2);
-    createFile(sub2, "z", fs);
+    createFile(sub2, fs, "z");
 
 
     final FsShell shell = new FsShell(conf);
     final FsShell shell = new FsShell(conf);
-
     final String inputPathStr = inputPath.toUri().getPath();
     final String inputPathStr = inputPath.toUri().getPath();
-
     final List<String> originalPaths = lsr(shell, inputPathStr);
     final List<String> originalPaths = lsr(shell, inputPathStr);
-    final URI uri = fs.getUri();
-    final String prefix = "har://hdfs-" + uri.getHost() + ":" + uri.getPort()
-        + archivePath.toUri().getPath() + Path.SEPARATOR;
-
-    {// Enable space replacement
-      final String harName = "foo.har";
-      final String[] args = { "-archiveName", harName, "-p", inputPathStr, "*",
-          archivePath.toString() };
-      System.setProperty(HadoopArchives.TEST_HADOOP_ARCHIVES_JAR_PATH,
-          HADOOP_ARCHIVES_JAR);
-      final HadoopArchives har = new HadoopArchives(conf);
-      Assert.assertEquals(0, ToolRunner.run(har, args));
 
 
-      // compare results
-      final List<String> harPaths = lsr(shell, prefix + harName);
-      Assert.assertEquals(originalPaths, harPaths);
-    }
+    // make the archive:
+    final String fullHarPathStr = makeArchive();
 
 
+    // compare results
+    final List<String> harPaths = lsr(shell, fullHarPathStr);
+    Assert.assertEquals(originalPaths, harPaths);
   }
   }
 
 
   private static List<String> lsr(final FsShell shell, String dir)
   private static List<String> lsr(final FsShell shell, String dir)
@@ -222,4 +223,442 @@ public class TestHadoopArchives {
         .println("lsr paths = " + paths.toString().replace(", ", ",\n  "));
         .println("lsr paths = " + paths.toString().replace(", ", ",\n  "));
     return paths;
     return paths;
   }
   }
+  
+  @Test
+  public void testReadFileContent() throws Exception {
+    fileList.add(createFile(inputPath, fs, "c c"));
+    final Path sub1 = new Path(inputPath, "sub 1");
+    fs.mkdirs(sub1);
+    fileList.add(createFile(inputPath, fs, sub1.getName(), "file x y z"));
+    fileList.add(createFile(inputPath, fs, sub1.getName(), "file"));
+    fileList.add(createFile(inputPath, fs, sub1.getName(), "x"));
+    fileList.add(createFile(inputPath, fs, sub1.getName(), "y"));
+    fileList.add(createFile(inputPath, fs, sub1.getName(), "z"));
+    final Path sub2 = new Path(inputPath, "sub 1 with suffix");
+    fs.mkdirs(sub2);
+    fileList.add(createFile(inputPath, fs, sub2.getName(), "z"));
+    // Generate a big binary file content:
+    final byte[] binContent = prepareBin();
+    fileList.add(createFile(inputPath, fs, binContent, sub2.getName(), "bin"));
+    fileList.add(createFile(inputPath, fs, new byte[0], sub2.getName(), "zero-length"));
+
+    final String fullHarPathStr = makeArchive();
+
+    // Create fresh HarFs:
+    final HarFileSystem harFileSystem = new HarFileSystem(fs);
+    try {
+      final URI harUri = new URI(fullHarPathStr);
+      harFileSystem.initialize(harUri, fs.getConf());
+      // now read the file content and compare it against the expected:
+      int readFileCount = 0;
+      for (final String pathStr0 : fileList) {
+        final Path path = new Path(fullHarPathStr + Path.SEPARATOR + pathStr0);
+        final String baseName = path.getName();
+        final FileStatus status = harFileSystem.getFileStatus(path);
+        if (status.isFile()) {
+          // read the file:
+          final byte[] actualContentSimple = readAllSimple(
+              harFileSystem.open(path), true);
+          
+          final byte[] actualContentBuffer = readAllWithBuffer(
+              harFileSystem.open(path), true);
+          assertArrayEquals(actualContentSimple, actualContentBuffer);
+          
+          final byte[] actualContentFully = readAllWithReadFully(
+              actualContentSimple.length,
+              harFileSystem.open(path), true);
+          assertArrayEquals(actualContentSimple, actualContentFully);
+          
+          final byte[] actualContentSeek = readAllWithSeek(
+              actualContentSimple.length,
+              harFileSystem.open(path), true);
+          assertArrayEquals(actualContentSimple, actualContentSeek);
+          
+          final byte[] actualContentRead4
+          = readAllWithRead4(harFileSystem.open(path), true);
+          assertArrayEquals(actualContentSimple, actualContentRead4);
+          
+          final byte[] actualContentSkip = readAllWithSkip(
+              actualContentSimple.length, 
+              harFileSystem.open(path), 
+              harFileSystem.open(path), 
+              true);
+          assertArrayEquals(actualContentSimple, actualContentSkip);
+          
+          if ("bin".equals(baseName)) {
+            assertArrayEquals(binContent, actualContentSimple);
+          } else if ("zero-length".equals(baseName)) {
+            assertEquals(0, actualContentSimple.length);
+          } else {
+            String actual = new String(actualContentSimple, "UTF-8");
+            assertEquals(baseName, actual);
+          }
+          readFileCount++;
+        }
+      }
+      assertEquals(fileList.size(), readFileCount);
+    } finally {
+      harFileSystem.close();
+    }
+  }
+  
+  private static byte[] readAllSimple(FSDataInputStream fsdis, boolean close) throws IOException {
+    final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    try {
+      int b;
+      while (true) {
+        b = fsdis.read();
+        if (b < 0) {
+          break;
+        } else {
+          baos.write(b);
+        }
+      }
+      baos.close();
+      return baos.toByteArray();
+    } finally {
+      if (close) {
+        fsdis.close();
+      }
+    }
+  }
+
+  private static byte[] readAllWithBuffer(FSDataInputStream fsdis, boolean close)
+      throws IOException {
+    try {
+      final int available = fsdis.available();
+      final byte[] buffer;
+      final ByteArrayOutputStream baos;
+      if (available < 0) {
+        buffer = new byte[1024];
+        baos = new ByteArrayOutputStream(buffer.length * 2);
+      } else {
+        buffer = new byte[available];
+        baos = new ByteArrayOutputStream(available);
+      }
+      int readIntoBuffer = 0;
+      int read; 
+      while (true) {
+        read = fsdis.read(buffer, readIntoBuffer, buffer.length - readIntoBuffer);
+        if (read < 0) {
+          // end of stream:
+          if (readIntoBuffer > 0) {
+            baos.write(buffer, 0, readIntoBuffer);
+          }
+          return baos.toByteArray();
+        } else {
+          readIntoBuffer += read;
+          if (readIntoBuffer == buffer.length) {
+            // buffer is full, need to clean the buffer.
+            // drop the buffered data to baos:
+            baos.write(buffer);
+            // reset the counter to start reading to the buffer beginning:
+            readIntoBuffer = 0;
+          } else if (readIntoBuffer > buffer.length) {
+            throw new IOException("Read more than the buffer length: "
+                + readIntoBuffer + ", buffer length = " + buffer.length);
+          }
+        }
+      }
+    } finally {
+      if (close) {
+        fsdis.close();
+      }
+    }
+  }
+
+  private static byte[] readAllWithReadFully(int totalLength, FSDataInputStream fsdis, boolean close)
+      throws IOException {
+    final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    // Simulate reading of some data structures of known length:
+    final byte[] buffer = new byte[17];
+    final int times = totalLength / buffer.length;
+    final int remainder = totalLength % buffer.length;
+    // it would be simpler to leave the position tracking to the 
+    // InputStream, but we need to check the methods #readFully(2) 
+    // and #readFully(4) that receive the position as a parameter:
+    int position = 0;
+    try {
+      // read "data structures":
+      for (int i=0; i<times; i++) {
+        fsdis.readFully(position, buffer);
+        position += buffer.length;
+        baos.write(buffer);
+      }
+      if (remainder > 0) {
+        // read the remainder:
+        fsdis.readFully(position, buffer, 0, remainder);
+        position += remainder;
+        baos.write(buffer, 0, remainder);
+      }
+      try {
+        fsdis.readFully(position, buffer, 0, 1);
+        assertTrue(false);
+      } catch (IOException ioe) {
+        // okay
+      }
+      assertEquals(totalLength, position);
+      final byte[] result = baos.toByteArray();
+      assertEquals(totalLength, result.length);
+      return result;
+    } finally {
+      if (close) {
+        fsdis.close();
+      }
+    }
+  }
+
+  private static byte[] readAllWithRead4(FSDataInputStream fsdis, boolean close)
+      throws IOException {
+    try {
+      final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      final byte[] buffer = new byte[17];
+      int totalRead = 0;
+      int read;
+      while (true) {
+        read = fsdis.read(totalRead, buffer, 0, buffer.length);
+        if (read > 0) {
+          totalRead += read;
+          baos.write(buffer, 0, read);
+        } else if (read < 0) {
+          break; // EOF
+        } else {
+          // read == 0:
+          // zero result may be returned *only* in case if the 4th 
+          // parameter is 0. Since in our case this is 'buffer.length',
+          // zero return value clearly indicates a bug: 
+          throw new AssertionError("FSDataInputStream#read(4) returned 0, while " +
+          		" the 4th method parameter is " + buffer.length + ".");
+        }
+      }
+      final byte[] result = baos.toByteArray();
+      return result;
+    } finally {
+      if (close) {
+        fsdis.close();
+      }
+    }
+  }
+  
+  private static byte[] readAllWithSeek(final int totalLength, 
+      final FSDataInputStream fsdis, final boolean close)
+      throws IOException {
+    final byte[] result = new byte[totalLength];
+    long pos;
+    try {
+      // read the data in the reverse order, from 
+      // the tail to the head by pieces of 'buffer' length:
+      final byte[] buffer = new byte[17];
+      final int times = totalLength / buffer.length;
+      int read;
+      int expectedRead;
+      for (int i=times; i>=0; i--) {
+        pos = i * buffer.length;
+        fsdis.seek(pos);
+        // check that seek is successful:
+        assertEquals(pos, fsdis.getPos());
+        read = fsdis.read(buffer);
+        // check we read right number of bytes:
+        if (i == times) {
+          expectedRead = totalLength % buffer.length; // remainder
+          if (expectedRead == 0) {
+            // zero remainder corresponds to the EOS, so
+            // by the contract of DataInpitStream#read(byte[]) -1 should be 
+            // returned:
+            expectedRead = -1;
+          }
+        } else {
+          expectedRead = buffer.length;
+        }
+        assertEquals(expectedRead, read);
+        if (read > 0) {
+          System.arraycopy(buffer, 0, result, (int)pos, read);
+        }
+      }
+      
+      // finally, check that #seek() to not existing position leads to IOE:
+      expectSeekIOE(fsdis, Long.MAX_VALUE, "Seek to Long.MAX_VALUE should lead to IOE.");
+      expectSeekIOE(fsdis, Long.MIN_VALUE, "Seek to Long.MIN_VALUE should lead to IOE.");
+      long pp = -1L;
+      expectSeekIOE(fsdis, pp, "Seek to "+pp+" should lead to IOE.");
+      
+      // NB: is is *possible* to #seek(length), but *impossible* to #seek(length + 1):
+      fsdis.seek(totalLength);
+      assertEquals(totalLength, fsdis.getPos());
+      pp = totalLength + 1;
+      expectSeekIOE(fsdis, pp, "Seek to the length position + 1 ("+pp+") should lead to IOE.");
+      
+      return result;
+    } finally {
+      if (close) {
+        fsdis.close();
+      }
+    }
+  }  
+
+  private static void expectSeekIOE(FSDataInputStream fsdis, long seekPos, String message) {
+    try {
+      fsdis.seek(seekPos);
+      assertTrue(message + " (Position = " + fsdis.getPos() + ")", false);
+    } catch (IOException ioe) {
+      // okay
+    }
+  }
+  
+  /*
+   * Reads data by chunks from 2 input streams:
+   * reads chunk from stream 1, and skips this chunk in the stream 2;
+   * Then reads next chunk from stream 2, and skips this chunk in stream 1. 
+   */
+  private static byte[] readAllWithSkip(
+      final int totalLength, 
+      final FSDataInputStream fsdis1, 
+      final FSDataInputStream fsdis2, 
+      final boolean close)
+      throws IOException {
+    // test negative skip arg: 
+    assertEquals(0, fsdis1.skip(-1));
+    // test zero skip arg: 
+    assertEquals(0, fsdis1.skip(0));
+    
+    final ByteArrayOutputStream baos = new ByteArrayOutputStream(totalLength);
+    try {
+      // read the data in the reverse order, from 
+      // the tail to the head by pieces of 'buffer' length:
+      final byte[] buffer = new byte[17];
+      final int times = totalLength / buffer.length;
+      final int remainder = totalLength % buffer.length;
+      long skipped;
+      long expectedPosition;
+      int toGo;
+      for (int i=0; i<=times; i++) {
+        toGo = (i < times) ? buffer.length : remainder;
+        if (i % 2 == 0) {
+          fsdis1.readFully(buffer, 0, toGo);
+          skipped = skipUntilZero(fsdis2, toGo);
+        } else {
+          fsdis2.readFully(buffer, 0, toGo);
+          skipped = skipUntilZero(fsdis1, toGo);
+        }
+        if (i < times) {
+          assertEquals(buffer.length, skipped);
+          expectedPosition = (i + 1) * buffer.length;
+        } else { 
+          // remainder:
+          if (remainder > 0) {
+            assertEquals(remainder, skipped);
+          } else {
+            assertEquals(0, skipped);
+          }
+          expectedPosition = totalLength;
+        }
+        // check if the 2 streams have equal and correct positions:
+        assertEquals(expectedPosition, fsdis1.getPos());
+        assertEquals(expectedPosition, fsdis2.getPos());
+        // save the read data:
+        if (toGo > 0) {
+          baos.write(buffer, 0, toGo);
+        }
+      }
+
+      // finally, check up if ended stream cannot skip:
+      assertEquals(0, fsdis1.skip(-1));
+      assertEquals(0, fsdis1.skip(0));
+      assertEquals(0, fsdis1.skip(1));
+      assertEquals(0, fsdis1.skip(Long.MAX_VALUE));
+      
+      return baos.toByteArray();
+    } finally {
+      if (close) {
+        fsdis1.close();
+        fsdis2.close();
+      }
+    }
+  }  
+  
+  private static long skipUntilZero(final FilterInputStream fis, 
+      final long toSkip) throws IOException {
+    long skipped = 0;
+    long remainsToSkip = toSkip;
+    long s;
+    while (skipped < toSkip) {
+      s = fis.skip(remainsToSkip); // actually skippped
+      if (s == 0) {
+        return skipped; // EOF or impossible to skip.
+      }
+      skipped += s; 
+      remainsToSkip -= s;
+    }
+    return skipped;
+  }
+  
+  private static byte[] prepareBin() {
+    byte[] bb = new byte[77777];
+    for (int i=0; i<bb.length; i++) {
+      // Generate unique values, as possible:
+      double d = Math.log(i + 2);
+      long bits = Double.doubleToLongBits(d);
+      bb[i] = (byte)bits;
+    }
+    return bb;
+  }
+
+  /*
+   * Run the HadoopArchives tool to create an archive on the 
+   * given file system.
+   */
+  private String makeArchive() throws Exception {
+    final String inputPathStr = inputPath.toUri().getPath();
+    System.out.println("inputPathStr = " + inputPathStr);
+
+    final URI uri = fs.getUri();
+    final String prefix = "har://hdfs-" + uri.getHost() + ":" + uri.getPort()
+        + archivePath.toUri().getPath() + Path.SEPARATOR;
+
+    final String harName = "foo.har";
+    final String fullHarPathStr = prefix + harName;
+    final String[] args = { "-archiveName", harName, "-p", inputPathStr, "*",
+        archivePath.toString() };
+    System.setProperty(HadoopArchives.TEST_HADOOP_ARCHIVES_JAR_PATH,
+        HADOOP_ARCHIVES_JAR);
+    final HadoopArchives har = new HadoopArchives(conf);
+    assertEquals(0, ToolRunner.run(har, args));
+    return fullHarPathStr;
+  }
+  
+  @Test
+  /*
+   * Tests copying from archive file system to a local file system
+   */
+  public void testCopyToLocal() throws Exception {
+    final String fullHarPathStr = makeArchive();
+
+    // make path to copy the file to:
+    final String tmpDir
+      = System.getProperty("test.build.data","build/test/data") + "/work-dir/har-fs-tmp";
+    final Path tmpPath = new Path(tmpDir);
+    final LocalFileSystem localFs = FileSystem.getLocal(new Configuration());
+    localFs.delete(tmpPath, true);
+    localFs.mkdirs(tmpPath);
+    assertTrue(localFs.exists(tmpPath));
+    
+    // Create fresh HarFs:
+    final HarFileSystem harFileSystem = new HarFileSystem(fs);
+    try {
+      final URI harUri = new URI(fullHarPathStr);
+      harFileSystem.initialize(harUri, fs.getConf());
+      
+      final Path sourcePath = new Path(fullHarPathStr + Path.SEPARATOR + "a");
+      final Path targetPath = new Path(tmpPath, "straus");
+      // copy the Har file to a local file system:
+      harFileSystem.copyToLocalFile(false, sourcePath, targetPath);
+      FileStatus straus = localFs.getFileStatus(targetPath);
+      // the file should contain just 1 character:
+      assertEquals(1, straus.getLen());
+    } finally {
+      harFileSystem.close();
+      localFs.delete(tmpPath, true);      
+    }
+  }
+  
 }
 }

+ 7 - 0
hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/io/IdentifierResolver.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.streaming.io;
 package org.apache.hadoop.streaming.io;
 
 
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.typedbytes.TypedBytesWritable;
 import org.apache.hadoop.typedbytes.TypedBytesWritable;
 
 
@@ -34,6 +35,7 @@ public class IdentifierResolver {
   public static final String TEXT_ID = "text";
   public static final String TEXT_ID = "text";
   public static final String RAW_BYTES_ID = "rawbytes";
   public static final String RAW_BYTES_ID = "rawbytes";
   public static final String TYPED_BYTES_ID = "typedbytes";
   public static final String TYPED_BYTES_ID = "typedbytes";
+  public static final String KEY_ONLY_TEXT_ID = "keyonlytext";
   
   
   private Class<? extends InputWriter> inputWriterClass = null;
   private Class<? extends InputWriter> inputWriterClass = null;
   private Class<? extends OutputReader> outputReaderClass = null;
   private Class<? extends OutputReader> outputReaderClass = null;
@@ -55,6 +57,11 @@ public class IdentifierResolver {
       setOutputReaderClass(TypedBytesOutputReader.class);
       setOutputReaderClass(TypedBytesOutputReader.class);
       setOutputKeyClass(TypedBytesWritable.class);
       setOutputKeyClass(TypedBytesWritable.class);
       setOutputValueClass(TypedBytesWritable.class);
       setOutputValueClass(TypedBytesWritable.class);
+    } else if (identifier.equalsIgnoreCase(KEY_ONLY_TEXT_ID)) {
+      setInputWriterClass(KeyOnlyTextInputWriter.class);
+      setOutputReaderClass(KeyOnlyTextOutputReader.class);
+      setOutputKeyClass(Text.class);
+      setOutputValueClass(NullWritable.class);
     } else { // assume TEXT_ID
     } else { // assume TEXT_ID
       setInputWriterClass(TextInputWriter.class);
       setInputWriterClass(TextInputWriter.class);
       setOutputReaderClass(TextOutputReader.class);
       setOutputReaderClass(TextOutputReader.class);

+ 35 - 0
hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/io/KeyOnlyTextInputWriter.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.streaming.io;
+
+import java.io.IOException;
+
+
+public class KeyOnlyTextInputWriter extends TextInputWriter {
+
+  @Override
+  public void writeKey(Object key) throws IOException {
+    writeUTF8(key);
+    clientOut.write('\n');
+  }
+
+  @Override
+  public void writeValue(Object value) throws IOException {}
+
+}

+ 90 - 0
hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/io/KeyOnlyTextOutputReader.java

@@ -0,0 +1,90 @@
+/**
+ * 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.io;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.UnsupportedEncodingException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.streaming.PipeMapRed;
+import org.apache.hadoop.util.LineReader;
+
+/**
+ * OutputReader that reads the client's output as text, interpreting each line
+ * as a key and outputting NullWritables for values.
+ */
+public class KeyOnlyTextOutputReader extends OutputReader<Text, NullWritable> {
+
+  private LineReader lineReader;
+  private byte[] bytes;
+  private DataInput clientIn;
+  private Configuration conf;
+  private Text key;
+  private Text line;
+  
+  @Override
+  public void initialize(PipeMapRed pipeMapRed) throws IOException {
+    super.initialize(pipeMapRed);
+    clientIn = pipeMapRed.getClientInput();
+    conf = pipeMapRed.getConfiguration();
+    lineReader = new LineReader((InputStream)clientIn, conf);
+    key = new Text();
+    line = new Text();
+  }
+  
+  @Override
+  public boolean readKeyValue() throws IOException {
+    if (lineReader.readLine(line) <= 0) {
+      return false;
+    }
+    bytes = line.getBytes();
+    key.set(bytes, 0, line.getLength());
+
+    line.clear();
+    return true;
+  }
+  
+  @Override
+  public Text getCurrentKey() throws IOException {
+    return key;
+  }
+  
+  @Override
+  public NullWritable getCurrentValue() throws IOException {
+    return NullWritable.get();
+  }
+
+  @Override
+  public String getLastOutput() {
+    if (bytes != null) {
+      try {
+        return new String(bytes, "UTF-8");
+      } catch (UnsupportedEncodingException e) {
+        return "<undecodable>";
+      }
+    } else {
+      return null;
+    }
+  }
+  
+}

+ 2 - 2
hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/io/TextInputWriter.java

@@ -30,7 +30,7 @@ import org.apache.hadoop.streaming.PipeMapRed;
  */
  */
 public class TextInputWriter extends InputWriter<Object, Object> {
 public class TextInputWriter extends InputWriter<Object, Object> {
   
   
-  private DataOutput clientOut;
+  protected DataOutput clientOut;
   private byte[] inputSeparator;
   private byte[] inputSeparator;
   
   
   @Override
   @Override
@@ -53,7 +53,7 @@ public class TextInputWriter extends InputWriter<Object, Object> {
   }
   }
   
   
   // Write an object to the output stream using UTF-8 encoding
   // Write an object to the output stream using UTF-8 encoding
-  private void writeUTF8(Object object) throws IOException {
+  protected void writeUTF8(Object object) throws IOException {
     byte[] bval;
     byte[] bval;
     int valSize;
     int valSize;
     if (object instanceof BytesWritable) {
     if (object instanceof BytesWritable) {

+ 51 - 0
hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/TestStreamingOutputOnlyKeys.java

@@ -0,0 +1,51 @@
+/**
+ * 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.IOException;
+
+import org.junit.Test;
+
+public class TestStreamingOutputOnlyKeys extends TestStreaming {
+
+  public TestStreamingOutputOnlyKeys() throws IOException {
+    super();
+  }
+  
+  @Test
+  public void testOutputOnlyKeys() throws Exception {
+    args.add("-jobconf"); args.add("stream.reduce.input" +
+        "=keyonlytext");
+    args.add("-jobconf"); args.add("stream.reduce.output" +
+        "=keyonlytext");
+    super.testCommandLine();
+  }
+  
+  @Override
+  public String getExpectedOutput() {
+    return outputExpect.replaceAll("\t", "");
+  }
+  
+  @Override
+  @Test
+  public void testCommandLine() {
+    // Do nothing
+  }
+
+}

+ 68 - 0
hadoop-tools/hadoop-streaming/src/test/java/org/apache/hadoop/streaming/io/TestKeyOnlyTextOutputReader.java

@@ -0,0 +1,68 @@
+/**
+ * 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.io;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.IOException;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.streaming.PipeMapRed;
+import org.apache.hadoop.streaming.PipeMapper;
+import org.junit.Test;
+
+public class TestKeyOnlyTextOutputReader {
+  @Test
+  public void testKeyOnlyTextOutputReader() throws IOException {
+    String text = "key,value\nkey2,value2\nnocomma\n";
+    PipeMapRed pipeMapRed = new MyPipeMapRed(text);
+    KeyOnlyTextOutputReader outputReader = new KeyOnlyTextOutputReader();
+    outputReader.initialize(pipeMapRed);
+    outputReader.readKeyValue();
+    Assert.assertEquals(new Text("key,value"), outputReader.getCurrentKey());
+    outputReader.readKeyValue();
+    Assert.assertEquals(new Text("key2,value2"), outputReader.getCurrentKey());
+    outputReader.readKeyValue();
+    Assert.assertEquals(new Text("nocomma"), outputReader.getCurrentKey());
+    Assert.assertEquals(false, outputReader.readKeyValue());
+  }
+  
+  private class MyPipeMapRed extends PipeMapper {
+    private DataInput clientIn;
+    private Configuration conf = new Configuration();
+    
+    public MyPipeMapRed(String text) {
+      clientIn = new DataInputStream(new ByteArrayInputStream(text.getBytes()));
+    }
+    
+    @Override
+    public DataInput getClientInput() {
+      return clientIn;
+    }
+    
+    @Override
+    public Configuration getConfiguration() {
+      return conf;
+    }
+  }
+}

+ 24 - 2
hadoop-yarn-project/CHANGES.txt

@@ -61,11 +61,12 @@ Release 2.3.0 - UNRELEASED
 
 
     YARN-976. Document the meaning of a virtual core. (Sandy Ryza)
     YARN-976. Document the meaning of a virtual core. (Sandy Ryza)
 
 
-    YARN-1258. Allow configuring the Fair Scheduler root queue (Sandy Ryza)
-
     YARN-1182. MiniYARNCluster creates and inits the RM/NM only on start()
     YARN-1182. MiniYARNCluster creates and inits the RM/NM only on start()
     (Karthik Kambatla via Sandy Ryza)
     (Karthik Kambatla via Sandy Ryza)
 
 
+    HADOOP-9598. Improve code coverage of RMAdminCLI (Aleksey Gorshkov and
+    Andrey Klochkov via jeagles)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES
@@ -84,6 +85,12 @@ Release 2.3.0 - UNRELEASED
     YARN-1300. SLS tests fail because conf puts YARN properties in
     YARN-1300. SLS tests fail because conf puts YARN properties in
     fair-scheduler.xml (Ted Yu via Sandy Ryza)
     fair-scheduler.xml (Ted Yu via Sandy Ryza)
 
 
+    YARN-1183. MiniYARNCluster shutdown takes several minutes intermittently
+    (Andrey Klochkov via jeagles)
+
+    YARN-1305. RMHAProtocolService#serviceInit should handle HAUtil's
+    IllegalArgumentException (Tsuyoshi Ozawa via bikas)
+
 Release 2.2.1 - UNRELEASED
 Release 2.2.1 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -95,6 +102,12 @@ Release 2.2.1 - UNRELEASED
     YARN-305. Fair scheduler logs too many "Node offered to app" messages.
     YARN-305. Fair scheduler logs too many "Node offered to app" messages.
     (Lohit Vijayarenu via Sandy Ryza)
     (Lohit Vijayarenu via Sandy Ryza)
 
 
+    YARN-1258. Allow configuring the Fair Scheduler root queue (Sandy Ryza)
+
+    YARN-1288. Make Fair Scheduler ACLs more user friendly (Sandy Ryza)
+
+    YARN-1315. TestQueueACLs should also test FairScheduler (Sandy Ryza)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES
@@ -120,6 +133,15 @@ Release 2.2.1 - UNRELEASED
     YARN-1295. In UnixLocalWrapperScriptBuilder, using bash -c can cause Text
     YARN-1295. In UnixLocalWrapperScriptBuilder, using bash -c can cause Text
     file busy errors (Sandy Ryza)
     file busy errors (Sandy Ryza)
 
 
+    YARN-1185. Fixed FileSystemRMStateStore to not leave partial files that
+    prevent subsequent ResourceManager recovery. (Omkar Vinit Joshi via vinodkv)
+
+    YARN-1331. yarn.cmd exits with NoClassDefFoundError trying to run rmadmin or
+    logs. (cnauroth)
+
+    YARN-1330. Fair Scheduler: defaultQueueSchedulingPolicy does not take effect
+    (Sandy Ryza)
+
 Release 2.2.0 - 2013-10-13
 Release 2.2.0 - 2013-10-13
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 2 - 3
hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd

@@ -149,7 +149,7 @@ goto :eof
   goto :eof
   goto :eof
 
 
 :rmadmin
 :rmadmin
-  set CLASS=org.apache.hadoop.yarn.server.resourcemanager.tools.RMAdmin
+  set CLASS=org.apache.hadoop.yarn.client.cli.RMAdminCLI
   set YARN_OPTS=%YARN_OPTS% %YARN_CLIENT_OPTS%
   set YARN_OPTS=%YARN_OPTS% %YARN_CLIENT_OPTS%
   goto :eof
   goto :eof
 
 
@@ -200,7 +200,7 @@ goto :eof
   goto :eof
   goto :eof
 
 
 :logs
 :logs
-  set CLASS=org.apache.hadoop.yarn.logaggregation.LogDumper
+  set CLASS=org.apache.hadoop.yarn.client.cli.LogsCLI
   set YARN_OPTS=%YARN_OPTS% %YARN_CLIENT_OPTS%
   set YARN_OPTS=%YARN_OPTS% %YARN_CLIENT_OPTS%
   goto :eof
   goto :eof
 
 
@@ -237,7 +237,6 @@ goto :eof
   @echo        where COMMAND is one of:
   @echo        where COMMAND is one of:
   @echo   resourcemanager      run the ResourceManager
   @echo   resourcemanager      run the ResourceManager
   @echo   nodemanager          run a nodemanager on each slave
   @echo   nodemanager          run a nodemanager on each slave
-  @echo   historyserver        run job history servers as a standalone daemon
   @echo   rmadmin              admin tools
   @echo   rmadmin              admin tools
   @echo   version              print the version
   @echo   version              print the version
   @echo   jar ^<jar^>          run a jar file
   @echo   jar ^<jar^>          run a jar file

+ 125 - 23
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/HAUtil.java

@@ -23,6 +23,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 
 
 import java.util.Arrays;
 import java.util.Arrays;
@@ -42,10 +43,13 @@ public class HAUtil {
           YarnConfiguration.RM_RESOURCE_TRACKER_ADDRESS,
           YarnConfiguration.RM_RESOURCE_TRACKER_ADDRESS,
           YarnConfiguration.RM_WEBAPP_ADDRESS));
           YarnConfiguration.RM_WEBAPP_ADDRESS));
 
 
+  public static final String BAD_CONFIG_MESSAGE_PREFIX =
+    "Invalid configuration! ";
+
   private HAUtil() { /* Hidden constructor */ }
   private HAUtil() { /* Hidden constructor */ }
 
 
   private static void throwBadConfigurationException(String msg) {
   private static void throwBadConfigurationException(String msg) {
-    throw new YarnRuntimeException("Invalid configuration! " + msg);
+    throw new YarnRuntimeException(BAD_CONFIG_MESSAGE_PREFIX + msg);
   }
   }
 
 
   /**
   /**
@@ -59,29 +63,137 @@ public class HAUtil {
         YarnConfiguration.DEFAULT_RM_HA_ENABLED);
         YarnConfiguration.DEFAULT_RM_HA_ENABLED);
   }
   }
 
 
+  /**
+   * Verify configuration for Resource Manager HA.
+   * @param conf Configuration
+   * @throws YarnRuntimeException
+   */
+  public static void verifyAndSetConfiguration(Configuration conf)
+    throws YarnRuntimeException {
+    verifyAndSetRMHAIds(conf);
+    verifyAndSetRMHAId(conf);
+    verifyAndSetAllRpcAddresses(conf);
+  }
+
+
+  private static void verifyAndSetRMHAIds(Configuration conf) {
+    Collection<String> ids =
+      conf.getTrimmedStringCollection(YarnConfiguration.RM_HA_IDS);
+    if (ids.size() <= 0) {
+      throwBadConfigurationException(
+        getInvalidValueMessage(YarnConfiguration.RM_HA_IDS,
+          conf.get(YarnConfiguration.RM_HA_IDS)));
+    } else if (ids.size() == 1) {
+      LOG.warn(getRMHAIdsWarningMessage(ids.toString()));
+    }
+
+    StringBuilder setValue = new StringBuilder();
+    for (String id: ids) {
+      setValue.append(id);
+      setValue.append(",");
+    }
+    conf.set(YarnConfiguration.RM_HA_IDS,
+      setValue.substring(0, setValue.length() - 1));
+  }
+
+  private static void verifyAndSetRMHAId(Configuration conf) {
+    String rmId = conf.getTrimmed(YarnConfiguration.RM_HA_ID);
+    if (rmId == null) {
+      throwBadConfigurationException(
+        getNeedToSetValueMessage(YarnConfiguration.RM_HA_ID));
+    } else {
+      Collection<String> ids = getRMHAIds(conf);
+      if (!ids.contains(rmId)) {
+        throwBadConfigurationException(
+          getRMHAIdNeedToBeIncludedMessage(ids.toString(), rmId));
+      }
+    }
+    conf.set(YarnConfiguration.RM_HA_ID, rmId);
+  }
+
+  private static void verifyAndSetConfValue(String prefix, Configuration conf) {
+    String confKey = null;
+    String confValue = null;
+    try {
+      confKey = getConfKeyForRMInstance(prefix, conf);
+      confValue = getConfValueForRMInstance(prefix, conf);
+      conf.set(prefix, confValue);
+    } catch (YarnRuntimeException yre) {
+      // Error at getRMHAId()
+      throw yre;
+    } catch (IllegalArgumentException iae) {
+      String errmsg;
+      if (confKey == null) {
+        // Error at addSuffix
+        errmsg = getInvalidValueMessage(YarnConfiguration.RM_HA_ID,
+          getRMHAId(conf));
+      } else {
+        // Error at Configuration#set.
+        errmsg = getNeedToSetValueMessage(confKey);
+      }
+      throwBadConfigurationException(errmsg);
+    }
+  }
+
+  public static void verifyAndSetAllRpcAddresses(Configuration conf) {
+    for (String confKey : RPC_ADDRESS_CONF_KEYS) {
+     verifyAndSetConfValue(confKey, conf);
+    }
+  }
+
+  /**
+   * @param conf Configuration. Please use getRMHAIds to check.
+   * @return RM Ids on success
+   */
   public static Collection<String> getRMHAIds(Configuration conf) {
   public static Collection<String> getRMHAIds(Configuration conf) {
-    return conf.getTrimmedStringCollection(YarnConfiguration.RM_HA_IDS);
+    return  conf.getStringCollection(YarnConfiguration.RM_HA_IDS);
   }
   }
 
 
   /**
   /**
-   * @param conf Configuration
+   * @param conf Configuration. Please use verifyAndSetRMHAId to check.
    * @return RM Id on success
    * @return RM Id on success
-   * @throws YarnRuntimeException for configurations without a node id
    */
    */
   @VisibleForTesting
   @VisibleForTesting
-  public static String getRMHAId(Configuration conf) {
-    String rmId = conf.get(YarnConfiguration.RM_HA_ID);
-    if (rmId == null) {
-      throwBadConfigurationException(YarnConfiguration.RM_HA_ID +
-          " needs to be set in a HA configuration");
-    }
-    return rmId;
+  static String getRMHAId(Configuration conf) {
+    return conf.get(YarnConfiguration.RM_HA_ID);
+  }
+
+  @VisibleForTesting
+  static String getNeedToSetValueMessage(String confKey) {
+    return confKey + " needs to be set in a HA configuration.";
+  }
+
+  @VisibleForTesting
+  static String getInvalidValueMessage(String confKey,
+                                              String invalidValue){
+    return "Invalid value of "  + confKey +". "
+      + "Current value is " + invalidValue;
+  }
+
+  @VisibleForTesting
+  static String getRMHAIdNeedToBeIncludedMessage(String ids,
+                                                        String rmId) {
+    return YarnConfiguration.RM_HA_IDS + "("
+      + ids +  ") need to contain " + YarnConfiguration.RM_HA_ID + "("
+      + rmId + ") in a HA configuration.";
+  }
+
+  @VisibleForTesting
+  static String getRMHAIdsWarningMessage(String ids) {
+    return  "Resource Manager HA is enabled, but " +
+      YarnConfiguration.RM_HA_IDS + " has only one id(" +
+      ids.toString() + ")";
+  }
+
+  private static String getConfKeyForRMInstance(String prefix,
+                                                Configuration conf) {
+    return addSuffix(prefix, getRMHAId(conf));
   }
   }
 
 
   private static String getConfValueForRMInstance(String prefix,
   private static String getConfValueForRMInstance(String prefix,
                                                   Configuration conf) {
                                                   Configuration conf) {
-    String confKey = addSuffix(prefix, getRMHAId(conf));
-    String retVal = conf.get(confKey);
+    String confKey = getConfKeyForRMInstance(prefix, conf);
+    String retVal = conf.getTrimmed(confKey);
     if (LOG.isTraceEnabled()) {
     if (LOG.isTraceEnabled()) {
       LOG.trace("getConfValueForRMInstance: prefix = " + prefix +
       LOG.trace("getConfValueForRMInstance: prefix = " + prefix +
           "; confKey being looked up = " + confKey +
           "; confKey being looked up = " + confKey +
@@ -96,16 +208,6 @@ public class HAUtil {
     return (value == null) ? defaultValue : value;
     return (value == null) ? defaultValue : value;
   }
   }
 
 
-  private static void setConfValue(String prefix, Configuration conf) {
-    conf.set(prefix, getConfValueForRMInstance(prefix, conf));
-  }
-
-  public static void setAllRpcAddresses(Configuration conf) {
-    for (String confKey : RPC_ADDRESS_CONF_KEYS) {
-      setConfValue(confKey, conf);
-    }
-  }
-
   /** Add non empty and non null suffix to a key */
   /** Add non empty and non null suffix to a key */
   @VisibleForTesting
   @VisibleForTesting
   public static String addSuffix(String key, String suffix) {
   public static String addSuffix(String key, String suffix) {

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RMAdminCLI.java

@@ -162,7 +162,7 @@ public class RMAdminCLI extends Configured implements Tool {
     }
     }
   }
   }
 
 
-  private ResourceManagerAdministrationProtocol createAdminProtocol() throws IOException {
+  protected ResourceManagerAdministrationProtocol createAdminProtocol() throws IOException {
     // Get the current configuration
     // Get the current configuration
     final YarnConfiguration conf = new YarnConfiguration(getConf());
     final YarnConfiguration conf = new YarnConfiguration(getConf());
     return ClientRMProxy.createRMProxy(conf, ResourceManagerAdministrationProtocol.class);
     return ClientRMProxy.createRMProxy(conf, ResourceManagerAdministrationProtocol.class);

+ 243 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestRMAdminCLI.java

@@ -0,0 +1,243 @@
+/**
+ * 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.yarn.client;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.argThat;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+
+import org.apache.hadoop.yarn.client.cli.RMAdminCLI;
+import org.apache.hadoop.yarn.server.api.ResourceManagerAdministrationProtocol;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshQueuesRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshServiceAclsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshSuperUserGroupsConfigurationRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshUserToGroupsMappingsRequest;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.ArgumentMatcher;
+
+public class TestRMAdminCLI {
+
+  private ResourceManagerAdministrationProtocol admin;
+  private RMAdminCLI rmAdminCLI;
+
+  @Before
+  public void configure() {
+    admin = mock(ResourceManagerAdministrationProtocol.class);
+    rmAdminCLI = new RMAdminCLI() {
+
+      @Override
+      protected ResourceManagerAdministrationProtocol createAdminProtocol()
+          throws IOException {
+        return admin;
+      }
+      
+    };
+  }
+  
+  @Test(timeout=500)
+  public void testRefreshQueues() throws Exception {
+    String[] args = { "-refreshQueues" };
+    assertEquals(0, rmAdminCLI.run(args));
+    verify(admin).refreshQueues(any(RefreshQueuesRequest.class));
+  }
+
+  @Test(timeout=500)
+  public void testRefreshUserToGroupsMappings() throws Exception {
+    String[] args = { "-refreshUserToGroupsMappings" };
+    assertEquals(0, rmAdminCLI.run(args));
+    verify(admin).refreshUserToGroupsMappings(
+        any(RefreshUserToGroupsMappingsRequest.class));
+  }
+
+  @Test(timeout=500)
+  public void testRefreshSuperUserGroupsConfiguration() throws Exception {
+    String[] args = { "-refreshSuperUserGroupsConfiguration" };
+    assertEquals(0, rmAdminCLI.run(args));
+    verify(admin).refreshSuperUserGroupsConfiguration(
+        any(RefreshSuperUserGroupsConfigurationRequest.class));
+  }
+
+  @Test(timeout=500)
+  public void testRefreshAdminAcls() throws Exception {
+    String[] args = { "-refreshAdminAcls" };
+    assertEquals(0, rmAdminCLI.run(args));
+    verify(admin).refreshAdminAcls(any(RefreshAdminAclsRequest.class));
+  }
+
+  @Test(timeout=500)
+  public void testRefreshServiceAcl() throws Exception {
+    String[] args = { "-refreshServiceAcl" };
+    assertEquals(0, rmAdminCLI.run(args));
+    verify(admin).refreshServiceAcls(any(RefreshServiceAclsRequest.class));
+  }
+
+  @Test(timeout=500)
+  public void testRefreshNodes() throws Exception {
+    String[] args = { "-refreshNodes" };
+    assertEquals(0, rmAdminCLI.run(args));
+    verify(admin).refreshNodes(any(RefreshNodesRequest.class));
+  }
+  
+  @Test(timeout=500)
+  public void testGetGroups() throws Exception {
+    when(admin.getGroupsForUser(eq("admin"))).thenReturn(
+        new String[] {"group1", "group2"});
+    PrintStream origOut = System.out;
+    PrintStream out = mock(PrintStream.class);
+    System.setOut(out);
+    try {
+      String[] args = { "-getGroups", "admin" };
+      assertEquals(0, rmAdminCLI.run(args));
+      verify(admin).getGroupsForUser(eq("admin"));
+      verify(out).println(argThat(new ArgumentMatcher<StringBuilder>() {
+        @Override
+        public boolean matches(Object argument) {
+          return ("" + argument).equals("admin : group1 group2");
+        }
+      }));
+    } finally {
+      System.setOut(origOut);
+    }
+  }
+
+  /**
+   * Test printing of help messages
+   */
+  @Test(timeout=500)
+  public void testHelp() throws Exception {
+    PrintStream oldOutPrintStream = System.out;
+    PrintStream oldErrPrintStream = System.err;
+    ByteArrayOutputStream dataOut = new ByteArrayOutputStream();
+    ByteArrayOutputStream dataErr = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(dataOut));
+    System.setErr(new PrintStream(dataErr));
+    try {
+      String[] args = { "-help" };
+      assertEquals(0, rmAdminCLI.run(args));
+      assertTrue(dataOut
+          .toString()
+          .contains(
+              "rmadmin is the command to execute Map-Reduce" +
+              " administrative commands."));
+      assertTrue(dataOut
+          .toString()
+          .contains(
+              "hadoop rmadmin [-refreshQueues] [-refreshNodes] [-refreshSuper" +
+              "UserGroupsConfiguration] [-refreshUserToGroupsMappings] " +
+              "[-refreshAdminAcls] [-refreshServiceAcl] [-getGroup" +
+              " [username]] [-help [cmd]]"));
+      assertTrue(dataOut
+          .toString()
+          .contains(
+              "-refreshQueues: Reload the queues' acls, states and scheduler " +
+              "specific properties."));
+      assertTrue(dataOut
+          .toString()
+          .contains(
+              "-refreshNodes: Refresh the hosts information at the " +
+              "ResourceManager."));
+      assertTrue(dataOut.toString().contains(
+          "-refreshUserToGroupsMappings: Refresh user-to-groups mappings"));
+      assertTrue(dataOut
+          .toString()
+          .contains(
+              "-refreshSuperUserGroupsConfiguration: Refresh superuser proxy" +
+              " groups mappings"));
+      assertTrue(dataOut
+          .toString()
+          .contains(
+              "-refreshAdminAcls: Refresh acls for administration of " +
+              "ResourceManager"));
+      assertTrue(dataOut
+          .toString()
+          .contains(
+              "-refreshServiceAcl: Reload the service-level authorization" +
+              " policy file"));
+      assertTrue(dataOut
+          .toString()
+          .contains(
+              "-help [cmd]: \tDisplays help for the given command or all " +
+              "commands if none"));
+
+      testError(new String[] { "-help", "-refreshQueues" },
+          "Usage: java RMAdmin [-refreshQueues]", dataErr, 0);
+      testError(new String[] { "-help", "-refreshNodes" },
+          "Usage: java RMAdmin [-refreshNodes]", dataErr, 0);
+      testError(new String[] { "-help", "-refreshUserToGroupsMappings" },
+          "Usage: java RMAdmin [-refreshUserToGroupsMappings]", dataErr, 0);
+      testError(
+          new String[] { "-help", "-refreshSuperUserGroupsConfiguration" },
+          "Usage: java RMAdmin [-refreshSuperUserGroupsConfiguration]",
+          dataErr, 0);
+      testError(new String[] { "-help", "-refreshAdminAcls" },
+          "Usage: java RMAdmin [-refreshAdminAcls]", dataErr, 0);
+      testError(new String[] { "-help", "-refreshService" },
+          "Usage: java RMAdmin [-refreshServiceAcl]", dataErr, 0);
+      testError(new String[] { "-help", "-getGroups" },
+          "Usage: java RMAdmin [-getGroups [username]]", dataErr, 0);
+
+      
+      testError(new String[] { "-help", "-badParameter" },
+          "Usage: java RMAdmin", dataErr, 0);
+      testError(new String[] { "-badParameter" },
+          "badParameter: Unknown command", dataErr, -1); 
+    } finally {
+      System.setOut(oldOutPrintStream);
+      System.setErr(oldErrPrintStream);
+    }
+  }
+
+  @Test(timeout=500)
+  public void testException() throws Exception {
+    PrintStream oldErrPrintStream = System.err;
+    ByteArrayOutputStream dataErr = new ByteArrayOutputStream();
+    System.setErr(new PrintStream(dataErr));
+    try {
+      when(admin.refreshQueues(any(RefreshQueuesRequest.class)))
+          .thenThrow(new IOException("test exception"));
+      String[] args = { "-refreshQueues" };
+
+      assertEquals(-1, rmAdminCLI.run(args));
+      verify(admin).refreshQueues(any(RefreshQueuesRequest.class));
+      assertTrue(dataErr.toString().contains("refreshQueues: test exception"));
+    } finally {
+      System.setErr(oldErrPrintStream);
+    }
+  }
+
+  private void testError(String[] args, String template,
+      ByteArrayOutputStream data, int resultCode) throws Exception {
+    assertEquals(resultCode, rmAdminCLI.run(args));
+    assertTrue(data.toString().contains(template));
+    data.reset();
+  }
+  
+}

+ 98 - 16
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/conf/TestHAUtil.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.conf;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 
 
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
@@ -27,53 +28,134 @@ import org.junit.Test;
 import java.util.Collection;
 import java.util.Collection;
 
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.fail;
 import static org.junit.Assert.fail;
 
 
 public class TestHAUtil {
 public class TestHAUtil {
   private Configuration conf;
   private Configuration conf;
 
 
-  private static final String RM1_ADDRESS = "1.2.3.4:8021";
+  private static final String RM1_ADDRESS_UNTRIMMED = "  \t\t\n 1.2.3.4:8021  \n\t ";
+  private static final String RM1_ADDRESS = RM1_ADDRESS_UNTRIMMED.trim();
   private static final String RM2_ADDRESS = "localhost:8022";
   private static final String RM2_ADDRESS = "localhost:8022";
-  private static final String RM1_NODE_ID = "rm1";
+  private static final String RM1_NODE_ID_UNTRIMMED = "rm1 ";
+  private static final String RM1_NODE_ID = RM1_NODE_ID_UNTRIMMED.trim();
   private static final String RM2_NODE_ID = "rm2";
   private static final String RM2_NODE_ID = "rm2";
+  private static final String RM3_NODE_ID = "rm3";
+  private static final String RM_INVALID_NODE_ID = ".rm";
+  private static final String RM_NODE_IDS_UNTRIMMED = RM1_NODE_ID_UNTRIMMED + "," + RM2_NODE_ID;
+  private static final String RM_NODE_IDS = RM1_NODE_ID + "," + RM2_NODE_ID;
 
 
   @Before
   @Before
   public void setUp() {
   public void setUp() {
     conf = new Configuration();
     conf = new Configuration();
-    conf.set(YarnConfiguration.RM_HA_IDS, RM1_NODE_ID + "," + RM2_NODE_ID);
-    conf.set(YarnConfiguration.RM_HA_ID, RM1_NODE_ID);
+    conf.set(YarnConfiguration.RM_HA_IDS, RM_NODE_IDS_UNTRIMMED);
+    conf.set(YarnConfiguration.RM_HA_ID, RM1_NODE_ID_UNTRIMMED);
 
 
     for (String confKey : HAUtil.RPC_ADDRESS_CONF_KEYS) {
     for (String confKey : HAUtil.RPC_ADDRESS_CONF_KEYS) {
-      conf.set(HAUtil.addSuffix(confKey, RM1_NODE_ID), RM1_ADDRESS);
+      // configuration key itself cannot contains space/tab/return chars.
+      conf.set(HAUtil.addSuffix(confKey, RM1_NODE_ID), RM1_ADDRESS_UNTRIMMED);
       conf.set(HAUtil.addSuffix(confKey, RM2_NODE_ID), RM2_ADDRESS);
       conf.set(HAUtil.addSuffix(confKey, RM2_NODE_ID), RM2_ADDRESS);
     }
     }
   }
   }
 
 
   @Test
   @Test
   public void testGetRMServiceId() throws Exception {
   public void testGetRMServiceId() throws Exception {
+    conf.set(YarnConfiguration.RM_HA_IDS, RM1_NODE_ID + "," + RM2_NODE_ID);
     Collection<String> rmhaIds = HAUtil.getRMHAIds(conf);
     Collection<String> rmhaIds = HAUtil.getRMHAIds(conf);
     assertEquals(2, rmhaIds.size());
     assertEquals(2, rmhaIds.size());
+
+    String[] ids = rmhaIds.toArray(new String[0]);
+    assertEquals(RM1_NODE_ID, ids[0]);
+    assertEquals(RM2_NODE_ID, ids[1]);
   }
   }
 
 
   @Test
   @Test
   public void testGetRMId() throws Exception {
   public void testGetRMId() throws Exception {
+    conf.set(YarnConfiguration.RM_HA_ID, RM1_NODE_ID);
     assertEquals("Does not honor " + YarnConfiguration.RM_HA_ID,
     assertEquals("Does not honor " + YarnConfiguration.RM_HA_ID,
-        RM1_NODE_ID, HAUtil.getRMHAId(conf));
-    conf = new YarnConfiguration();
-    try {
-      HAUtil.getRMHAId(conf);
-      fail("getRMHAId() fails to throw an exception when RM_HA_ID is not set");
-    } catch (YarnRuntimeException yre) {
-      // do nothing
-    }
+      RM1_NODE_ID, HAUtil.getRMHAId(conf));
+
+    conf.clear();
+    assertNull("Return null when " + YarnConfiguration.RM_HA_ID
+        + " is not set", HAUtil.getRMHAId(conf));
   }
   }
 
 
   @Test
   @Test
-  public void testSetGetRpcAddresses() throws Exception {
-    HAUtil.setAllRpcAddresses(conf);
+  public void testVerifyAndSetConfiguration() throws Exception {
+    try {
+      HAUtil.verifyAndSetConfiguration(conf);
+    } catch (YarnRuntimeException e) {
+      fail("Should not throw any exceptions.");
+    }
+
+    assertEquals("Should be saved as Trimmed collection",
+      StringUtils.getStringCollection(RM_NODE_IDS), HAUtil.getRMHAIds(conf));
+    assertEquals("Should be saved as Trimmed string",
+      RM1_NODE_ID, HAUtil.getRMHAId(conf));
     for (String confKey : HAUtil.RPC_ADDRESS_CONF_KEYS) {
     for (String confKey : HAUtil.RPC_ADDRESS_CONF_KEYS) {
       assertEquals("RPC address not set for " + confKey,
       assertEquals("RPC address not set for " + confKey,
-          RM1_ADDRESS, conf.get(confKey));
+        RM1_ADDRESS, conf.get(confKey));
+    }
+
+    conf.clear();
+    conf.set(YarnConfiguration.RM_HA_IDS, RM_INVALID_NODE_ID);
+    try {
+      HAUtil.verifyAndSetConfiguration(conf);
+    } catch (YarnRuntimeException e) {
+      assertEquals("YarnRuntimeException by getRMId()",
+        HAUtil.BAD_CONFIG_MESSAGE_PREFIX +
+          HAUtil.getNeedToSetValueMessage(YarnConfiguration.RM_HA_ID),
+        e.getMessage());
+    }
+
+    conf.clear();
+    conf.set(YarnConfiguration.RM_HA_ID, RM_INVALID_NODE_ID);
+    conf.set(YarnConfiguration.RM_HA_IDS, RM_INVALID_NODE_ID);
+    for (String confKey : HAUtil.RPC_ADDRESS_CONF_KEYS) {
+      // simulate xml with invalid node id
+      conf.set(confKey + RM_INVALID_NODE_ID, RM_INVALID_NODE_ID);
+    }
+    try {
+      HAUtil.verifyAndSetConfiguration(conf);
+    } catch (YarnRuntimeException e) {
+      assertEquals("YarnRuntimeException by addSuffix()",
+        HAUtil.BAD_CONFIG_MESSAGE_PREFIX +
+          HAUtil.getInvalidValueMessage(YarnConfiguration.RM_HA_ID,
+            RM_INVALID_NODE_ID),
+        e.getMessage());
+    }
+
+    conf.clear();
+    // simulate the case HAUtil.RPC_ADDRESS_CONF_KEYS are not set
+    conf.set(YarnConfiguration.RM_HA_ID, RM1_NODE_ID);
+    conf.set(YarnConfiguration.RM_HA_IDS, RM1_NODE_ID);
+    try {
+      HAUtil.verifyAndSetConfiguration(conf);
+      fail("Should throw YarnRuntimeException. by Configuration#set()");
+    } catch (YarnRuntimeException e) {
+      String confKey =
+        HAUtil.addSuffix(YarnConfiguration.RM_ADDRESS, RM1_NODE_ID);
+      assertEquals("YarnRuntimeException by Configuration#set()",
+        HAUtil.BAD_CONFIG_MESSAGE_PREFIX + HAUtil.getNeedToSetValueMessage(confKey),
+        e.getMessage());
+    }
+
+    // simulate the case YarnConfiguration.RM_HA_IDS doesn't contain
+    // the value of YarnConfiguration.RM_HA_ID
+    conf.clear();
+    conf.set(YarnConfiguration.RM_HA_IDS, RM2_NODE_ID + "," + RM3_NODE_ID);
+    conf.set(YarnConfiguration.RM_HA_ID, RM1_NODE_ID_UNTRIMMED);
+    for (String confKey : HAUtil.RPC_ADDRESS_CONF_KEYS) {
+      conf.set(HAUtil.addSuffix(confKey, RM1_NODE_ID), RM1_ADDRESS_UNTRIMMED);
+      conf.set(HAUtil.addSuffix(confKey, RM2_NODE_ID), RM2_ADDRESS);
+    }
+    try {
+      HAUtil.verifyAndSetConfiguration(conf);
+    } catch (YarnRuntimeException e) {
+      assertEquals("YarnRuntimeException by getRMId()'s validation",
+        HAUtil.BAD_CONFIG_MESSAGE_PREFIX +
+        HAUtil.getRMHAIdNeedToBeIncludedMessage("[rm2, rm3]", RM1_NODE_ID),
+        e.getMessage());
     }
     }
   }
   }
 }
 }

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMHAProtocolService.java

@@ -57,7 +57,7 @@ public class RMHAProtocolService extends AbstractService implements
     this.conf = conf;
     this.conf = conf;
     haEnabled = HAUtil.isHAEnabled(this.conf);
     haEnabled = HAUtil.isHAEnabled(this.conf);
     if (haEnabled) {
     if (haEnabled) {
-      HAUtil.setAllRpcAddresses(this.conf);
+      HAUtil.verifyAndSetConfiguration(conf);
       rm.setConf(this.conf);
       rm.setConf(this.conf);
     }
     }
     rm.createAndInitActiveServices();
     rm.createAndInitActiveServices();

+ 30 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java

@@ -22,6 +22,7 @@ import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.DataOutputStream;
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.List;
 
 
@@ -118,6 +119,9 @@ public class FileSystemRMStateStore extends RMStateStore {
         for (FileStatus childNodeStatus : fs.listStatus(appDir.getPath())) {
         for (FileStatus childNodeStatus : fs.listStatus(appDir.getPath())) {
           assert childNodeStatus.isFile();
           assert childNodeStatus.isFile();
           String childNodeName = childNodeStatus.getPath().getName();
           String childNodeName = childNodeStatus.getPath().getName();
+          if (checkAndRemovePartialRecord(childNodeStatus.getPath())) {
+            continue;
+          }
           byte[] childData =
           byte[] childData =
               readFile(childNodeStatus.getPath(), childNodeStatus.getLen());
               readFile(childNodeStatus.getPath(), childNodeStatus.getLen());
           if (childNodeName.startsWith(ApplicationId.appIdStrPrefix)) {
           if (childNodeName.startsWith(ApplicationId.appIdStrPrefix)) {
@@ -178,12 +182,28 @@ public class FileSystemRMStateStore extends RMStateStore {
     }
     }
   }
   }
 
 
+  private boolean checkAndRemovePartialRecord(Path record) throws IOException {
+    // If the file ends with .tmp then it shows that it failed
+    // during saving state into state store. The file will be deleted as a
+    // part of this call
+    if (record.getName().endsWith(".tmp")) {
+      LOG.error("incomplete rm state store entry found :"
+          + record);
+      fs.delete(record, false);
+      return true;
+    }
+    return false;
+  }
+
   private void loadRMDTSecretManagerState(RMState rmState) throws Exception {
   private void loadRMDTSecretManagerState(RMState rmState) throws Exception {
     FileStatus[] childNodes = fs.listStatus(rmDTSecretManagerRoot);
     FileStatus[] childNodes = fs.listStatus(rmDTSecretManagerRoot);
 
 
     for(FileStatus childNodeStatus : childNodes) {
     for(FileStatus childNodeStatus : childNodes) {
       assert childNodeStatus.isFile();
       assert childNodeStatus.isFile();
       String childNodeName = childNodeStatus.getPath().getName();
       String childNodeName = childNodeStatus.getPath().getName();
+      if (checkAndRemovePartialRecord(childNodeStatus.getPath())) {
+        continue;
+      }
       if(childNodeName.startsWith(DELEGATION_TOKEN_SEQUENCE_NUMBER_PREFIX)) {
       if(childNodeName.startsWith(DELEGATION_TOKEN_SEQUENCE_NUMBER_PREFIX)) {
         rmState.rmSecretManagerState.dtSequenceNumber =
         rmState.rmSecretManagerState.dtSequenceNumber =
             Integer.parseInt(childNodeName.split("_")[1]);
             Integer.parseInt(childNodeName.split("_")[1]);
@@ -344,10 +364,19 @@ public class FileSystemRMStateStore extends RMStateStore {
     return data;
     return data;
   }
   }
 
 
+  /*
+   * In order to make this write atomic as a part of write we will first write
+   * data to .tmp file and then rename it. Here we are assuming that rename is
+   * atomic for underlying file system.
+   */
   private void writeFile(Path outputPath, byte[] data) throws Exception {
   private void writeFile(Path outputPath, byte[] data) throws Exception {
-    FSDataOutputStream fsOut = fs.create(outputPath, false);
+    Path tempPath =
+        new Path(outputPath.getParent(), outputPath.getName() + ".tmp");
+    FSDataOutputStream fsOut = null;
+    fsOut = fs.create(tempPath, false);
     fsOut.write(data);
     fsOut.write(data);
     fsOut.close();
     fsOut.close();
+    fs.rename(tempPath, outputPath);
   }
   }
 
 
   private boolean renameFile(Path src, Path dst) throws Exception {
   private boolean renameFile(Path src, Path dst) throws Exception {

+ 0 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/Queue.java

@@ -19,12 +19,10 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
 
 import java.util.List;
 import java.util.List;
-import java.util.Map;
 
 
 import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
@@ -44,12 +42,6 @@ public interface Queue {
    */
    */
   QueueMetrics getMetrics();
   QueueMetrics getMetrics();
 
 
-  /**
-   * Get ACLs for the queue.
-   * @return ACLs for the queue
-   */
-  public Map<QueueACL, AccessControlList> getQueueAcls();
-  
   /**
   /**
    * Get queue information
    * Get queue information
    * @param includeChildQueues include child queues?
    * @param includeChildQueues include child queues?

+ 0 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java

@@ -526,11 +526,6 @@ public class LeafQueue implements CSQueue {
     return userLimitFactor;
     return userLimitFactor;
   }
   }
 
 
-  @Override
-  public synchronized Map<QueueACL, AccessControlList> getQueueAcls() {
-    return new HashMap<QueueACL, AccessControlList>(acls);
-  }
-
   @Override
   @Override
   public synchronized QueueInfo getQueueInfo(
   public synchronized QueueInfo getQueueInfo(
       boolean includeChildQueues, boolean recursive) {
       boolean includeChildQueues, boolean recursive) {

+ 0 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java

@@ -299,11 +299,6 @@ public class ParentQueue implements CSQueue {
     return state;
     return state;
   }
   }
 
 
-  @Override
-  public synchronized Map<QueueACL, AccessControlList> getQueueAcls() {
-    return new HashMap<QueueACL, AccessControlList>(acls);
-  }
-
   @Override
   @Override
   public synchronized QueueInfo getQueueInfo( 
   public synchronized QueueInfo getQueueInfo( 
       boolean includeChildQueues, boolean recursive) {
       boolean includeChildQueues, boolean recursive) {

+ 1 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java

@@ -24,14 +24,12 @@ import java.util.Collections;
 import java.util.Comparator;
 import java.util.Comparator;
 import java.util.Iterator;
 import java.util.Iterator;
 import java.util.List;
 import java.util.List;
-import java.util.Map;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -177,8 +175,7 @@ public class FSLeafQueue extends FSQueue {
       recordFactory.newRecordInstance(QueueUserACLInfo.class);
       recordFactory.newRecordInstance(QueueUserACLInfo.class);
     List<QueueACL> operations = new ArrayList<QueueACL>();
     List<QueueACL> operations = new ArrayList<QueueACL>();
     for (QueueACL operation : QueueACL.values()) {
     for (QueueACL operation : QueueACL.values()) {
-      Map<QueueACL, AccessControlList> acls = queueMgr.getQueueAcls(getName());
-      if (acls.get(operation).isUserAllowed(user)) {
+      if (hasAccess(operation, user)) {
         operations.add(operation);
         operations.add(operation);
       }
       }
     }
     }

+ 1 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java

@@ -20,13 +20,10 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
 
 
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
 
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
@@ -135,12 +132,6 @@ public abstract class FSQueue extends Schedulable implements Queue {
     return queueInfo;
     return queueInfo;
   }
   }
   
   
-  @Override
-  public Map<QueueACL, AccessControlList> getQueueAcls() {
-    Map<QueueACL, AccessControlList> acls = queueMgr.getQueueAcls(getName());
-    return new HashMap<QueueACL, AccessControlList>(acls);
-  }
-  
   @Override
   @Override
   public FSQueueMetrics getMetrics() {
   public FSQueueMetrics getMetrics() {
     return metrics;
     return metrics;
@@ -154,7 +145,7 @@ public abstract class FSQueue extends Schedulable implements Queue {
   
   
   public boolean hasAccess(QueueACL acl, UserGroupInformation user) {
   public boolean hasAccess(QueueACL acl, UserGroupInformation user) {
     // Check if the leaf-queue allows access
     // Check if the leaf-queue allows access
-    if (queueMgr.getQueueAcls(getName()).get(acl).isUserAllowed(user)) {
+    if (queueMgr.getQueueAcl(getName(), acl).isUserAllowed(user)) {
       return true;
       return true;
     }
     }
 
 

+ 5 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerConfiguration.java

@@ -50,7 +50,7 @@ public class FairSchedulerConfiguration extends Configuration {
   
   
   private static final String CONF_PREFIX =  "yarn.scheduler.fair.";
   private static final String CONF_PREFIX =  "yarn.scheduler.fair.";
 
 
-  protected static final String ALLOCATION_FILE = CONF_PREFIX + "allocation.file";
+  public static final String ALLOCATION_FILE = CONF_PREFIX + "allocation.file";
   protected static final String DEFAULT_ALLOCATION_FILE = "fair-scheduler.xml";
   protected static final String DEFAULT_ALLOCATION_FILE = "fair-scheduler.xml";
   protected static final String EVENT_LOG_DIR = "eventlog.dir";
   protected static final String EVENT_LOG_DIR = "eventlog.dir";
 
 
@@ -113,6 +113,10 @@ public class FairSchedulerConfiguration extends Configuration {
   protected static final String MAX_ASSIGN = CONF_PREFIX + "max.assign";
   protected static final String MAX_ASSIGN = CONF_PREFIX + "max.assign";
   protected static final int DEFAULT_MAX_ASSIGN = -1;
   protected static final int DEFAULT_MAX_ASSIGN = -1;
 
 
+  public FairSchedulerConfiguration() {
+    super();
+  }
+  
   public FairSchedulerConfiguration(Configuration conf) {
   public FairSchedulerConfiguration(Configuration conf) {
     super(conf);
     super(conf);
   }
   }

+ 25 - 34
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java

@@ -72,6 +72,9 @@ public class QueueManager {
    * (this is done to prevent loading a file that hasn't been fully written).
    * (this is done to prevent loading a file that hasn't been fully written).
    */
    */
   public static final long ALLOC_RELOAD_WAIT = 5 * 1000;
   public static final long ALLOC_RELOAD_WAIT = 5 * 1000;
+  
+  private static final AccessControlList EVERYBODY_ACL = new AccessControlList("*");
+  private static final AccessControlList NOBODY_ACL = new AccessControlList(" ");
 
 
   private final FairScheduler scheduler;
   private final FairScheduler scheduler;
 
 
@@ -375,31 +378,24 @@ public class QueueManager {
           queueMaxAppsDefault, defaultSchedPolicy, minSharePreemptionTimeouts,
           queueMaxAppsDefault, defaultSchedPolicy, minSharePreemptionTimeouts,
           queueAcls, fairSharePreemptionTimeout, defaultMinSharePreemptionTimeout);
           queueAcls, fairSharePreemptionTimeout, defaultMinSharePreemptionTimeout);
       
       
-      // Update metrics
+      // Make sure all queues exist
+      for (String name: queueNamesInAllocFile) {
+        getLeafQueue(name, true);
+      }
+      
       for (FSQueue queue : queues.values()) {
       for (FSQueue queue : queues.values()) {
+        // Update queue metrics
         FSQueueMetrics queueMetrics = queue.getMetrics();
         FSQueueMetrics queueMetrics = queue.getMetrics();
         queueMetrics.setMinShare(queue.getMinShare());
         queueMetrics.setMinShare(queue.getMinShare());
         queueMetrics.setMaxShare(queue.getMaxShare());
         queueMetrics.setMaxShare(queue.getMaxShare());
+        // Set scheduling policies
+        if (queuePolicies.containsKey(queue.getName())) {
+          queue.setPolicy(queuePolicies.get(queue.getName()));
+        } else {
+          queue.setPolicy(SchedulingPolicy.getDefault());
+        }
       }
       }
-      
-      // Root queue should have empty ACLs.  As a queue's ACL is the union of
-      // its ACL and all its parents' ACLs, setting the roots' to empty will
-      // neither allow nor prohibit more access to its children.
-      Map<QueueACL, AccessControlList> rootAcls =
-          new HashMap<QueueACL, AccessControlList>();
-      rootAcls.put(QueueACL.SUBMIT_APPLICATIONS, new AccessControlList(" "));
-      rootAcls.put(QueueACL.ADMINISTER_QUEUE, new AccessControlList(" "));
-      queueAcls.put(ROOT_QUEUE, rootAcls);
  
  
-      // Create all queus
-      for (String name: queueNamesInAllocFile) {
-        getLeafQueue(name, true);
-      }
-      
-      // Set custom policies as specified
-      for (Map.Entry<String, SchedulingPolicy> entry : queuePolicies.entrySet()) {
-        queues.get(entry.getKey()).setPolicy(entry.getValue());
-      }
     }
     }
   }
   }
   
   
@@ -454,10 +450,10 @@ public class QueueManager {
         policy.initialize(scheduler.getClusterCapacity());
         policy.initialize(scheduler.getClusterCapacity());
         queuePolicies.put(queueName, policy);
         queuePolicies.put(queueName, policy);
       } else if ("aclSubmitApps".equals(field.getTagName())) {
       } else if ("aclSubmitApps".equals(field.getTagName())) {
-        String text = ((Text)field.getFirstChild()).getData().trim();
+        String text = ((Text)field.getFirstChild()).getData();
         acls.put(QueueACL.SUBMIT_APPLICATIONS, new AccessControlList(text));
         acls.put(QueueACL.SUBMIT_APPLICATIONS, new AccessControlList(text));
       } else if ("aclAdministerApps".equals(field.getTagName())) {
       } else if ("aclAdministerApps".equals(field.getTagName())) {
-        String text = ((Text)field.getFirstChild()).getData().trim();
+        String text = ((Text)field.getFirstChild()).getData();
         acls.put(QueueACL.ADMINISTER_QUEUE, new AccessControlList(text));
         acls.put(QueueACL.ADMINISTER_QUEUE, new AccessControlList(text));
       } else if ("queue".endsWith(field.getTagName()) || 
       } else if ("queue".endsWith(field.getTagName()) || 
           "pool".equals(field.getTagName())) {
           "pool".equals(field.getTagName())) {
@@ -577,21 +573,16 @@ public class QueueManager {
 
 
   /**
   /**
    * Get the ACLs associated with this queue. If a given ACL is not explicitly
    * Get the ACLs associated with this queue. If a given ACL is not explicitly
-   * configured, include the default value for that ACL.
+   * configured, include the default value for that ACL.  The default for the
+   * root queue is everybody ("*") and the default for all other queues is
+   * nobody ("")
    */
    */
-  public Map<QueueACL, AccessControlList> getQueueAcls(String queue) {
-    HashMap<QueueACL, AccessControlList> out = new HashMap<QueueACL, AccessControlList>();
-    Map<QueueACL, AccessControlList> queueAcl = info.queueAcls.get(queue);
-    if (queueAcl != null) {
-      out.putAll(queueAcl);
-    }
-    if (!out.containsKey(QueueACL.ADMINISTER_QUEUE)) {
-      out.put(QueueACL.ADMINISTER_QUEUE, new AccessControlList("*"));
-    }
-    if (!out.containsKey(QueueACL.SUBMIT_APPLICATIONS)) {
-      out.put(QueueACL.SUBMIT_APPLICATIONS, new AccessControlList("*"));
+  public AccessControlList getQueueAcl(String queue, QueueACL operation) {
+    Map<QueueACL, AccessControlList> queueAcls = info.queueAcls.get(queue);
+    if (queueAcls == null || !queueAcls.containsKey(operation)) {
+      return (queue.equals(ROOT_QUEUE)) ? EVERYBODY_ACL : NOBODY_ACL;
     }
     }
-    return out;
+    return queueAcls.get(operation);
   }
   }
   
   
   static class QueueManagerInfo {
   static class QueueManagerInfo {

+ 0 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java

@@ -156,7 +156,6 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
       return queueInfo;
       return queueInfo;
     }
     }
 
 
-    @Override
     public Map<QueueACL, AccessControlList> getQueueAcls() {
     public Map<QueueACL, AccessControlList> getQueueAcls() {
       Map<QueueACL, AccessControlList> acls =
       Map<QueueACL, AccessControlList> acls =
         new HashMap<QueueACL, AccessControlList>();
         new HashMap<QueueACL, AccessControlList>();

+ 27 - 65
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestQueueACLs.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/QueueACLsTestBase.java

@@ -42,41 +42,43 @@ import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
-import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
 
 
-public class TestQueueACLs {
+public abstract class QueueACLsTestBase {
 
 
-  private static final String COMMON_USER = "common_user";
-  private static final String QUEUE_A_USER = "queueA_user";
-  private static final String QUEUE_B_USER = "queueB_user";
-  private static final String ROOT_ADMIN = "root_admin";
-  private static final String QUEUE_A_ADMIN = "queueA_admin";
-  private static final String QUEUE_B_ADMIN = "queueB_admin";
+  protected static final String COMMON_USER = "common_user";
+  protected static final String QUEUE_A_USER = "queueA_user";
+  protected static final String QUEUE_B_USER = "queueB_user";
+  protected static final String ROOT_ADMIN = "root_admin";
+  protected static final String QUEUE_A_ADMIN = "queueA_admin";
+  protected static final String QUEUE_B_ADMIN = "queueB_admin";
 
 
-  private static final String QUEUEA = "queueA";
-  private static final String QUEUEB = "queueB";
+  protected static final String QUEUEA = "queueA";
+  protected static final String QUEUEB = "queueB";
 
 
   private static final Log LOG = LogFactory.getLog(TestApplicationACLs.class);
   private static final Log LOG = LogFactory.getLog(TestApplicationACLs.class);
 
 
-  static MockRM resourceManager;
-  static Configuration conf = createConfiguration();
-  final static YarnRPC rpc = YarnRPC.create(conf);
-  final static InetSocketAddress rmAddress = conf.getSocketAddr(
-    YarnConfiguration.RM_ADDRESS, YarnConfiguration.DEFAULT_RM_ADDRESS,
-    YarnConfiguration.DEFAULT_RM_PORT);
-
-  @BeforeClass
-  public static void setup() throws InterruptedException, IOException {
+  MockRM resourceManager;
+  Configuration conf;
+  YarnRPC rpc;
+  InetSocketAddress rmAddress;
+
+  @Before
+  public void setup() throws InterruptedException, IOException {
+    conf = createConfiguration();
+    rpc = YarnRPC.create(conf);
+    rmAddress = conf.getSocketAddr(
+      YarnConfiguration.RM_ADDRESS, YarnConfiguration.DEFAULT_RM_ADDRESS,
+      YarnConfiguration.DEFAULT_RM_PORT);
+    
     AccessControlList adminACL = new AccessControlList("");
     AccessControlList adminACL = new AccessControlList("");
     conf.set(YarnConfiguration.YARN_ADMIN_ACL, adminACL.getAclString());
     conf.set(YarnConfiguration.YARN_ADMIN_ACL, adminACL.getAclString());
 
 
@@ -109,8 +111,8 @@ public class TestQueueACLs {
     }
     }
   }
   }
 
 
-  @AfterClass
-  public static void tearDown() {
+  @After
+  public void tearDown() {
     if (resourceManager != null) {
     if (resourceManager != null) {
       resourceManager.stop();
       resourceManager.stop();
     }
     }
@@ -262,45 +264,5 @@ public class TestQueueACLs {
     return userClient;
     return userClient;
   }
   }
 
 
-  private static YarnConfiguration createConfiguration() {
-    CapacitySchedulerConfiguration csConf =
-        new CapacitySchedulerConfiguration();
-    csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {
-        QUEUEA, QUEUEB });
-
-    csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + QUEUEA, 50f);
-    csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + QUEUEB, 50f);
-
-    Map<QueueACL, AccessControlList> aclsOnQueueA =
-        new HashMap<QueueACL, AccessControlList>();
-    AccessControlList submitACLonQueueA = new AccessControlList(QUEUE_A_USER);
-    submitACLonQueueA.addUser(COMMON_USER);
-    AccessControlList adminACLonQueueA = new AccessControlList(QUEUE_A_ADMIN);
-    aclsOnQueueA.put(QueueACL.SUBMIT_APPLICATIONS, submitACLonQueueA);
-    aclsOnQueueA.put(QueueACL.ADMINISTER_QUEUE, adminACLonQueueA);
-    csConf.setAcls(CapacitySchedulerConfiguration.ROOT + "." + QUEUEA,
-      aclsOnQueueA);
-
-    Map<QueueACL, AccessControlList> aclsOnQueueB =
-        new HashMap<QueueACL, AccessControlList>();
-    AccessControlList submitACLonQueueB = new AccessControlList(QUEUE_B_USER);
-    submitACLonQueueB.addUser(COMMON_USER);
-    AccessControlList adminACLonQueueB = new AccessControlList(QUEUE_B_ADMIN);
-    aclsOnQueueB.put(QueueACL.SUBMIT_APPLICATIONS, submitACLonQueueB);
-    aclsOnQueueB.put(QueueACL.ADMINISTER_QUEUE, adminACLonQueueB);
-    csConf.setAcls(CapacitySchedulerConfiguration.ROOT + "." + QUEUEB,
-      aclsOnQueueB);
-
-    Map<QueueACL, AccessControlList> aclsOnRootQueue =
-        new HashMap<QueueACL, AccessControlList>();
-    AccessControlList submitACLonRoot = new AccessControlList("");
-    AccessControlList adminACLonRoot = new AccessControlList(ROOT_ADMIN);
-    aclsOnRootQueue.put(QueueACL.SUBMIT_APPLICATIONS, submitACLonRoot);
-    aclsOnRootQueue.put(QueueACL.ADMINISTER_QUEUE, adminACLonRoot);
-    csConf.setAcls(CapacitySchedulerConfiguration.ROOT, aclsOnRootQueue);
-
-    YarnConfiguration conf = new YarnConfiguration(csConf);
-    conf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
-    return conf;
-  }
+  protected abstract Configuration createConfiguration() throws IOException;
 }
 }

+ 3 - 100
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestRMStateStore.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreTestBase.java

@@ -39,6 +39,7 @@ import junit.framework.Assert;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
@@ -75,9 +76,9 @@ import org.apache.zookeeper.ZooKeeper;
 
 
 import org.junit.Test;
 import org.junit.Test;
 
 
-public class TestRMStateStore extends ClientBaseWithFixes{
+public class RMStateStoreTestBase extends ClientBaseWithFixes{
 
 
-  public static final Log LOG = LogFactory.getLog(TestRMStateStore.class);
+  public static final Log LOG = LogFactory.getLog(RMStateStoreTestBase.class);
 
 
   static class TestDispatcher implements
   static class TestDispatcher implements
       Dispatcher, EventHandler<RMAppAttemptStoredEvent> {
       Dispatcher, EventHandler<RMAppAttemptStoredEvent> {
@@ -116,104 +117,6 @@ public class TestRMStateStore extends ClientBaseWithFixes{
     boolean isFinalStateValid() throws Exception;
     boolean isFinalStateValid() throws Exception;
   }
   }
 
 
-  @Test
-  public void testZKRMStateStoreRealZK() throws Exception {
-    TestZKRMStateStoreTester zkTester = new TestZKRMStateStoreTester();
-    testRMAppStateStore(zkTester);
-    testRMDTSecretManagerStateStore(zkTester);
-  }
-
-  @Test
-  public void testFSRMStateStore() throws Exception {
-    HdfsConfiguration conf = new HdfsConfiguration();
-    MiniDFSCluster cluster =
-        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
-    try {
-      TestFSRMStateStoreTester fsTester = new TestFSRMStateStoreTester(cluster);
-      testRMAppStateStore(fsTester);
-      testRMDTSecretManagerStateStore(fsTester);
-    } finally {
-      cluster.shutdown();
-    }
-  }
-
-  class TestZKRMStateStoreTester implements RMStateStoreHelper {
-    ZooKeeper client;
-    ZKRMStateStore store;
-
-    class TestZKRMStateStore extends ZKRMStateStore {
-      public TestZKRMStateStore(Configuration conf, String workingZnode)
-          throws Exception {
-        init(conf);
-        start();
-        assertTrue(znodeWorkingPath.equals(workingZnode));
-      }
-
-      @Override
-      public ZooKeeper getNewZooKeeper() throws IOException {
-        return client;
-      }
-    }
-
-    public RMStateStore getRMStateStore() throws Exception {
-      String workingZnode = "/Test";
-      YarnConfiguration conf = new YarnConfiguration();
-      conf.set(YarnConfiguration.ZK_RM_STATE_STORE_ADDRESS, hostPort);
-      conf.set(YarnConfiguration.ZK_RM_STATE_STORE_PARENT_PATH, workingZnode);
-      this.client = createClient();
-      this.store = new TestZKRMStateStore(conf, workingZnode);
-      return this.store;
-    }
-
-    @Override
-    public boolean isFinalStateValid() throws Exception {
-      List<String> nodes = client.getChildren(store.znodeWorkingPath, false);
-      return nodes.size() == 1;
-    }
-  }
-
-  class TestFSRMStateStoreTester implements RMStateStoreHelper {
-    Path workingDirPathURI;
-    FileSystemRMStateStore store;
-    MiniDFSCluster cluster;
-
-    class TestFileSystemRMStore extends FileSystemRMStateStore {
-      TestFileSystemRMStore(Configuration conf) throws Exception {
-        init(conf);
-        Assert.assertNull(fs);
-        assertTrue(workingDirPathURI.equals(fsWorkingPath));
-        start();
-        Assert.assertNotNull(fs);
-      }
-    }
-
-    public TestFSRMStateStoreTester(MiniDFSCluster cluster) throws Exception {
-      Path workingDirPath = new Path("/Test");
-      this.cluster = cluster;
-      FileSystem fs = cluster.getFileSystem();
-      fs.mkdirs(workingDirPath);
-      Path clusterURI = new Path(cluster.getURI());
-      workingDirPathURI = new Path(clusterURI, workingDirPath);
-      fs.close();
-    }
-
-    @Override
-    public RMStateStore getRMStateStore() throws Exception {
-      YarnConfiguration conf = new YarnConfiguration();
-      conf.set(YarnConfiguration.FS_RM_STATE_STORE_URI,
-          workingDirPathURI.toString());
-      this.store = new TestFileSystemRMStore(conf);
-      return store;
-    }
-
-    @Override
-    public boolean isFinalStateValid() throws Exception {
-      FileSystem fs = cluster.getFileSystem();
-      FileStatus[] files = fs.listStatus(workingDirPathURI);
-      return files.length == 1;
-    }
-  }
-
   void waitNotify(TestDispatcher dispatcher) {
   void waitNotify(TestDispatcher dispatcher) {
     long startTime = System.currentTimeMillis();
     long startTime = System.currentTimeMillis();
     while(!dispatcher.notified) {
     while(!dispatcher.notified) {

+ 120 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java

@@ -0,0 +1,120 @@
+/**
+ * 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.yarn.server.resourcemanager.recovery;
+
+import static org.junit.Assert.assertTrue;
+import junit.framework.Assert;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.junit.Test;
+
+public class TestFSRMStateStore extends RMStateStoreTestBase {
+
+  public static final Log LOG = LogFactory.getLog(TestFSRMStateStore.class);
+
+  class TestFSRMStateStoreTester implements RMStateStoreHelper {
+
+    Path workingDirPathURI;
+    FileSystemRMStateStore store;
+    MiniDFSCluster cluster;
+
+    class TestFileSystemRMStore extends FileSystemRMStateStore {
+
+      TestFileSystemRMStore(Configuration conf) throws Exception {
+        init(conf);
+        Assert.assertNull(fs);
+        assertTrue(workingDirPathURI.equals(fsWorkingPath));
+        start();
+        Assert.assertNotNull(fs);
+      }
+    }
+
+    public TestFSRMStateStoreTester(MiniDFSCluster cluster) throws Exception {
+      Path workingDirPath = new Path("/Test");
+      this.cluster = cluster;
+      FileSystem fs = cluster.getFileSystem();
+      fs.mkdirs(workingDirPath);
+      Path clusterURI = new Path(cluster.getURI());
+      workingDirPathURI = new Path(clusterURI, workingDirPath);
+      fs.close();
+    }
+
+    @Override
+    public RMStateStore getRMStateStore() throws Exception {
+      YarnConfiguration conf = new YarnConfiguration();
+      conf.set(YarnConfiguration.FS_RM_STATE_STORE_URI,
+          workingDirPathURI.toString());
+      this.store = new TestFileSystemRMStore(conf);
+      return store;
+    }
+
+    @Override
+    public boolean isFinalStateValid() throws Exception {
+      FileSystem fs = cluster.getFileSystem();
+      FileStatus[] files = fs.listStatus(workingDirPathURI);
+      return files.length == 1;
+    }
+  }
+
+  @Test
+  public void testFSRMStateStore() throws Exception {
+    HdfsConfiguration conf = new HdfsConfiguration();
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    try {
+      TestFSRMStateStoreTester fsTester = new TestFSRMStateStoreTester(cluster);
+      // If the state store is FileSystemRMStateStore then add corrupted entry.
+      // It should discard the entry and remove it from file system.
+      FSDataOutputStream fsOut = null;
+      FileSystemRMStateStore fileSystemRMStateStore =
+          (FileSystemRMStateStore) fsTester.getRMStateStore();
+      String appAttemptIdStr3 = "appattempt_1352994193343_0001_000003";
+      ApplicationAttemptId attemptId3 =
+          ConverterUtils.toApplicationAttemptId(appAttemptIdStr3);
+      Path rootDir =
+          new Path(fileSystemRMStateStore.fsWorkingPath, "FSRMStateRoot");
+      Path appRootDir = new Path(rootDir, "RMAppRoot");
+      Path appDir =
+          new Path(appRootDir, attemptId3.getApplicationId().toString());
+      Path tempAppAttemptFile =
+          new Path(appDir, attemptId3.toString() + ".tmp");
+      fsOut = fileSystemRMStateStore.fs.create(tempAppAttemptFile, false);
+      fsOut.write("Some random data ".getBytes());
+      fsOut.close();
+
+      testRMAppStateStore(fsTester);
+      Assert.assertFalse(fileSystemRMStateStore.fsWorkingPath
+          .getFileSystem(conf).exists(tempAppAttemptFile));
+      testRMDTSecretManagerStateStore(fsTester);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+}

+ 80 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStore.java

@@ -0,0 +1,80 @@
+/**
+ * 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.yarn.server.resourcemanager.recovery;
+
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.zookeeper.ZooKeeper;
+import org.junit.Test;
+
+public class TestZKRMStateStore extends RMStateStoreTestBase {
+
+  public static final Log LOG = LogFactory.getLog(TestZKRMStateStore.class);
+
+  class TestZKRMStateStoreTester implements RMStateStoreHelper {
+
+    ZooKeeper client;
+    ZKRMStateStore store;
+
+    class TestZKRMStateStoreInternal extends ZKRMStateStore {
+
+      public TestZKRMStateStoreInternal(Configuration conf, String workingZnode)
+          throws Exception {
+        init(conf);
+        start();
+        assertTrue(znodeWorkingPath.equals(workingZnode));
+      }
+
+      @Override
+      public ZooKeeper getNewZooKeeper() throws IOException {
+        return client;
+      }
+    }
+
+    public RMStateStore getRMStateStore() throws Exception {
+      String workingZnode = "/Test";
+      YarnConfiguration conf = new YarnConfiguration();
+      conf.set(YarnConfiguration.ZK_RM_STATE_STORE_ADDRESS, hostPort);
+      conf.set(YarnConfiguration.ZK_RM_STATE_STORE_PARENT_PATH, workingZnode);
+      this.client = createClient();
+      this.store = new TestZKRMStateStoreInternal(conf, workingZnode);
+      return this.store;
+    }
+
+    @Override
+    public boolean isFinalStateValid() throws Exception {
+      List<String> nodes = client.getChildren(store.znodeWorkingPath, false);
+      return nodes.size() == 1;
+    }
+  }
+
+  @Test
+  public void testZKRMStateStoreRealZK() throws Exception {
+    TestZKRMStateStoreTester zkTester = new TestZKRMStateStoreTester();
+    testRMAppStateStore(zkTester);
+    testRMDTSecretManagerStateStore(zkTester);
+  }
+}

+ 9 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStoreZKClientConnections.java

@@ -24,7 +24,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ha.ClientBaseWithFixes;
 import org.apache.hadoop.ha.ClientBaseWithFixes;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.TestRMStateStore.TestDispatcher;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStoreTestBase.TestDispatcher;
 import org.apache.hadoop.util.ZKUtil;
 import org.apache.hadoop.util.ZKUtil;
 
 
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.CreateMode;
@@ -43,17 +43,20 @@ import static org.junit.Assert.fail;
 
 
 public class TestZKRMStateStoreZKClientConnections extends
 public class TestZKRMStateStoreZKClientConnections extends
     ClientBaseWithFixes {
     ClientBaseWithFixes {
+
   private static final int ZK_OP_WAIT_TIME = 3000;
   private static final int ZK_OP_WAIT_TIME = 3000;
   private Log LOG =
   private Log LOG =
       LogFactory.getLog(TestZKRMStateStoreZKClientConnections.class);
       LogFactory.getLog(TestZKRMStateStoreZKClientConnections.class);
 
 
   class TestZKClient {
   class TestZKClient {
+
     ZKRMStateStore store;
     ZKRMStateStore store;
     boolean forExpire = false;
     boolean forExpire = false;
     TestForwardingWatcher watcher;
     TestForwardingWatcher watcher;
     CyclicBarrier syncBarrier = new CyclicBarrier(2);
     CyclicBarrier syncBarrier = new CyclicBarrier(2);
 
 
     protected class TestZKRMStateStore extends ZKRMStateStore {
     protected class TestZKRMStateStore extends ZKRMStateStore {
+
       public TestZKRMStateStore(Configuration conf, String workingZnode)
       public TestZKRMStateStore(Configuration conf, String workingZnode)
           throws Exception {
           throws Exception {
         init(conf);
         init(conf);
@@ -87,6 +90,7 @@ public class TestZKRMStateStoreZKClientConnections extends
 
 
     private class TestForwardingWatcher extends
     private class TestForwardingWatcher extends
         ClientBaseWithFixes.CountdownWatcher {
         ClientBaseWithFixes.CountdownWatcher {
+
       public void process(WatchedEvent event) {
       public void process(WatchedEvent event) {
         super.process(event);
         super.process(event);
         try {
         try {
@@ -187,7 +191,7 @@ public class TestZKRMStateStoreZKClientConnections extends
     }
     }
   }
   }
 
 
-  @Test (timeout = 20000)
+  @Test(timeout = 20000)
   public void testSetZKAcl() {
   public void testSetZKAcl() {
     TestZKClient zkClientTester = new TestZKClient();
     TestZKClient zkClientTester = new TestZKClient();
     YarnConfiguration conf = new YarnConfiguration();
     YarnConfiguration conf = new YarnConfiguration();
@@ -196,10 +200,11 @@ public class TestZKRMStateStoreZKClientConnections extends
       zkClientTester.store.zkClient.delete(zkClientTester.store
       zkClientTester.store.zkClient.delete(zkClientTester.store
           .znodeWorkingPath, -1);
           .znodeWorkingPath, -1);
       fail("Shouldn't be able to delete path");
       fail("Shouldn't be able to delete path");
-    } catch (Exception e) {/* expected behavior */}
+    } catch (Exception e) {/* expected behavior */
+    }
   }
   }
 
 
-  @Test (timeout = 20000)
+  @Test(timeout = 20000)
   public void testInvalidZKAclConfiguration() {
   public void testInvalidZKAclConfiguration() {
     TestZKClient zkClientTester = new TestZKClient();
     TestZKClient zkClientTester = new TestZKClient();
     YarnConfiguration conf = new YarnConfiguration();
     YarnConfiguration conf = new YarnConfiguration();

+ 73 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerQueueACLs.java

@@ -0,0 +1,73 @@
+/**
+* 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.yarn.server.resourcemanager.scheduler.capacity;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.authorize.AccessControlList;
+import org.apache.hadoop.yarn.api.records.QueueACL;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.QueueACLsTestBase;
+
+public class TestCapacitySchedulerQueueACLs extends QueueACLsTestBase {
+  @Override
+  protected Configuration createConfiguration() {
+    CapacitySchedulerConfiguration csConf =
+        new CapacitySchedulerConfiguration();
+    csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {
+        QUEUEA, QUEUEB });
+
+    csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + QUEUEA, 50f);
+    csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + QUEUEB, 50f);
+
+    Map<QueueACL, AccessControlList> aclsOnQueueA =
+        new HashMap<QueueACL, AccessControlList>();
+    AccessControlList submitACLonQueueA = new AccessControlList(QUEUE_A_USER);
+    submitACLonQueueA.addUser(COMMON_USER);
+    AccessControlList adminACLonQueueA = new AccessControlList(QUEUE_A_ADMIN);
+    aclsOnQueueA.put(QueueACL.SUBMIT_APPLICATIONS, submitACLonQueueA);
+    aclsOnQueueA.put(QueueACL.ADMINISTER_QUEUE, adminACLonQueueA);
+    csConf.setAcls(CapacitySchedulerConfiguration.ROOT + "." + QUEUEA,
+      aclsOnQueueA);
+
+    Map<QueueACL, AccessControlList> aclsOnQueueB =
+        new HashMap<QueueACL, AccessControlList>();
+    AccessControlList submitACLonQueueB = new AccessControlList(QUEUE_B_USER);
+    submitACLonQueueB.addUser(COMMON_USER);
+    AccessControlList adminACLonQueueB = new AccessControlList(QUEUE_B_ADMIN);
+    aclsOnQueueB.put(QueueACL.SUBMIT_APPLICATIONS, submitACLonQueueB);
+    aclsOnQueueB.put(QueueACL.ADMINISTER_QUEUE, adminACLonQueueB);
+    csConf.setAcls(CapacitySchedulerConfiguration.ROOT + "." + QUEUEB,
+      aclsOnQueueB);
+
+    Map<QueueACL, AccessControlList> aclsOnRootQueue =
+        new HashMap<QueueACL, AccessControlList>();
+    AccessControlList submitACLonRoot = new AccessControlList("");
+    AccessControlList adminACLonRoot = new AccessControlList(ROOT_ADMIN);
+    aclsOnRootQueue.put(QueueACL.SUBMIT_APPLICATIONS, submitACLonRoot);
+    aclsOnRootQueue.put(QueueACL.ADMINISTER_QUEUE, adminACLonRoot);
+    csConf.setAcls(CapacitySchedulerConfiguration.ROOT, aclsOnRootQueue);
+
+    csConf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
+    csConf.set("yarn.resourcemanager.scheduler.class", CapacityScheduler.class.getName());
+
+    return csConf;
+  }
+}

+ 55 - 30
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java

@@ -88,6 +88,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSc
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.DominantResourceFairnessPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.DominantResourceFairnessPolicy;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FairSharePolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FifoPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FifoPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.security.QueueACLsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.QueueACLsManager;
@@ -807,6 +808,7 @@ public class TestFairScheduler {
     out.println("<queue name=\"queueB\">");
     out.println("<queue name=\"queueB\">");
     out.println("<minResources>2048mb,0vcores</minResources>");
     out.println("<minResources>2048mb,0vcores</minResources>");
     out.println("<aclAdministerApps>alice,bob admins</aclAdministerApps>");
     out.println("<aclAdministerApps>alice,bob admins</aclAdministerApps>");
+    out.println("<schedulingPolicy>fair</schedulingPolicy>");
     out.println("</queue>");
     out.println("</queue>");
     // Give queue C no minimum
     // Give queue C no minimum
     out.println("<queue name=\"queueC\">");
     out.println("<queue name=\"queueC\">");
@@ -833,6 +835,8 @@ public class TestFairScheduler {
         + "</defaultMinSharePreemptionTimeout>");
         + "</defaultMinSharePreemptionTimeout>");
     // Set fair share preemption timeout to 5 minutes
     // Set fair share preemption timeout to 5 minutes
     out.println("<fairSharePreemptionTimeout>300</fairSharePreemptionTimeout>");
     out.println("<fairSharePreemptionTimeout>300</fairSharePreemptionTimeout>");
+    // Set default scheduling policy to DRF
+    out.println("<defaultQueueSchedulingPolicy>drf</defaultQueueSchedulingPolicy>");
     out.println("</allocations>");
     out.println("</allocations>");
     out.close();
     out.close();
 
 
@@ -865,22 +869,25 @@ public class TestFairScheduler {
     assertEquals(10, queueManager.getUserMaxApps("user1"));
     assertEquals(10, queueManager.getUserMaxApps("user1"));
     assertEquals(5, queueManager.getUserMaxApps("user2"));
     assertEquals(5, queueManager.getUserMaxApps("user2"));
 
 
+    // Root should get * ACL
+    assertEquals("*",queueManager.getQueueAcl("root",
+        QueueACL.ADMINISTER_QUEUE).getAclString());
+    assertEquals("*", queueManager.getQueueAcl("root",
+        QueueACL.SUBMIT_APPLICATIONS).getAclString());
+
     // Unspecified queues should get default ACL
     // Unspecified queues should get default ACL
-    Map<QueueACL, AccessControlList> aclsA = queueManager.getQueueAcls("root.queueA");
-    assertTrue(aclsA.containsKey(QueueACL.ADMINISTER_QUEUE));
-    assertEquals("*", aclsA.get(QueueACL.ADMINISTER_QUEUE).getAclString());
-    assertTrue(aclsA.containsKey(QueueACL.SUBMIT_APPLICATIONS));
-    assertEquals("*", aclsA.get(QueueACL.SUBMIT_APPLICATIONS).getAclString());
+    assertEquals(" ",queueManager.getQueueAcl("root.queueA",
+        QueueACL.ADMINISTER_QUEUE).getAclString());
+    assertEquals(" ", queueManager.getQueueAcl("root.queueA",
+        QueueACL.SUBMIT_APPLICATIONS).getAclString());
 
 
     // Queue B ACL
     // Queue B ACL
-    Map<QueueACL, AccessControlList> aclsB = queueManager.getQueueAcls("root.queueB");
-    assertTrue(aclsB.containsKey(QueueACL.ADMINISTER_QUEUE));
-    assertEquals("alice,bob admins", aclsB.get(QueueACL.ADMINISTER_QUEUE).getAclString());
+    assertEquals("alice,bob admins",queueManager.getQueueAcl("root.queueB",
+        QueueACL.ADMINISTER_QUEUE).getAclString());
 
 
-    // Queue c ACL
-    Map<QueueACL, AccessControlList> aclsC = queueManager.getQueueAcls("root.queueC");
-    assertTrue(aclsC.containsKey(QueueACL.SUBMIT_APPLICATIONS));
-    assertEquals("alice,bob admins", aclsC.get(QueueACL.SUBMIT_APPLICATIONS).getAclString());
+    // Queue C ACL
+    assertEquals("alice,bob admins",queueManager.getQueueAcl("root.queueC",
+        QueueACL.SUBMIT_APPLICATIONS).getAclString());
 
 
     assertEquals(120000, queueManager.getMinSharePreemptionTimeout("root." + 
     assertEquals(120000, queueManager.getMinSharePreemptionTimeout("root." + 
         YarnConfiguration.DEFAULT_QUEUE_NAME));
         YarnConfiguration.DEFAULT_QUEUE_NAME));
@@ -891,6 +898,18 @@ public class TestFairScheduler {
     assertEquals(120000, queueManager.getMinSharePreemptionTimeout("root.queueA"));
     assertEquals(120000, queueManager.getMinSharePreemptionTimeout("root.queueA"));
     assertEquals(60000, queueManager.getMinSharePreemptionTimeout("root.queueE"));
     assertEquals(60000, queueManager.getMinSharePreemptionTimeout("root.queueE"));
     assertEquals(300000, queueManager.getFairSharePreemptionTimeout());
     assertEquals(300000, queueManager.getFairSharePreemptionTimeout());
+    
+    // Verify existing queues have default scheduling policy
+    assertEquals(DominantResourceFairnessPolicy.NAME,
+        queueManager.getQueue("root").getPolicy().getName());
+    assertEquals(DominantResourceFairnessPolicy.NAME,
+        queueManager.getQueue("root.queueA").getPolicy().getName());
+    // Verify default is overriden if specified explicitly
+    assertEquals(FairSharePolicy.NAME,
+        queueManager.getQueue("root.queueB").getPolicy().getName());
+    // Verify new queue gets default scheduling policy
+    assertEquals(DominantResourceFairnessPolicy.NAME,
+        queueManager.getLeafQueue("root.newqueue", true).getPolicy().getName());
   }
   }
 
 
   @Test
   @Test
@@ -1063,21 +1082,19 @@ public class TestFairScheduler {
     assertEquals(5, queueManager.getUserMaxApps("user2"));
     assertEquals(5, queueManager.getUserMaxApps("user2"));
 
 
     // Unspecified queues should get default ACL
     // Unspecified queues should get default ACL
-    Map<QueueACL, AccessControlList> aclsA = queueManager.getQueueAcls("queueA");
-    assertTrue(aclsA.containsKey(QueueACL.ADMINISTER_QUEUE));
-    assertEquals("*", aclsA.get(QueueACL.ADMINISTER_QUEUE).getAclString());
-    assertTrue(aclsA.containsKey(QueueACL.SUBMIT_APPLICATIONS));
-    assertEquals("*", aclsA.get(QueueACL.SUBMIT_APPLICATIONS).getAclString());
+    assertEquals(" ", queueManager.getQueueAcl("root.queueA",
+        QueueACL.ADMINISTER_QUEUE).getAclString());
+    assertEquals(" ", queueManager.getQueueAcl("root.queueA",
+        QueueACL.SUBMIT_APPLICATIONS).getAclString());
 
 
     // Queue B ACL
     // Queue B ACL
-    Map<QueueACL, AccessControlList> aclsB = queueManager.getQueueAcls("root.queueB");
-    assertTrue(aclsB.containsKey(QueueACL.ADMINISTER_QUEUE));
-    assertEquals("alice,bob admins", aclsB.get(QueueACL.ADMINISTER_QUEUE).getAclString());
+    assertEquals("alice,bob admins", queueManager.getQueueAcl("root.queueB",
+        QueueACL.ADMINISTER_QUEUE).getAclString());
+
+    // Queue C ACL
+    assertEquals("alice,bob admins", queueManager.getQueueAcl("root.queueC",
+        QueueACL.SUBMIT_APPLICATIONS).getAclString());
 
 
-    // Queue c ACL
-    Map<QueueACL, AccessControlList> aclsC = queueManager.getQueueAcls("root.queueC");
-    assertTrue(aclsC.containsKey(QueueACL.SUBMIT_APPLICATIONS));
-    assertEquals("alice,bob admins", aclsC.get(QueueACL.SUBMIT_APPLICATIONS).getAclString());
 
 
     assertEquals(120000, queueManager.getMinSharePreemptionTimeout("root." +
     assertEquals(120000, queueManager.getMinSharePreemptionTimeout("root." +
         YarnConfiguration.DEFAULT_QUEUE_NAME));
         YarnConfiguration.DEFAULT_QUEUE_NAME));
@@ -1664,9 +1681,13 @@ public class TestFairScheduler {
     PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
     PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
     out.println("<?xml version=\"1.0\"?>");
     out.println("<?xml version=\"1.0\"?>");
     out.println("<allocations>");
     out.println("<allocations>");
-    out.println("<queue name=\"queue1\">");
-    out.println("<aclSubmitApps>norealuserhasthisname</aclSubmitApps>");
-    out.println("<aclAdministerApps>norealuserhasthisname</aclAdministerApps>");
+    out.println("<queue name=\"root\">");
+    out.println("  <aclSubmitApps> </aclSubmitApps>");
+    out.println("  <aclAdministerApps> </aclAdministerApps>");
+    out.println("  <queue name=\"queue1\">");
+    out.println("    <aclSubmitApps>norealuserhasthisname</aclSubmitApps>");
+    out.println("    <aclAdministerApps>norealuserhasthisname</aclAdministerApps>");
+    out.println("  </queue>");
     out.println("</queue>");
     out.println("</queue>");
     out.println("</allocations>");
     out.println("</allocations>");
     out.close();
     out.close();
@@ -1893,9 +1914,13 @@ public class TestFairScheduler {
     PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
     PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
     out.println("<?xml version=\"1.0\"?>");
     out.println("<?xml version=\"1.0\"?>");
     out.println("<allocations>");
     out.println("<allocations>");
-    out.println("<queue name=\"queue1\">");
-    out.println("<aclSubmitApps>userallow</aclSubmitApps>");
-    out.println("<aclAdministerApps>userallow</aclAdministerApps>");
+    out.println("<queue name=\"root\">");
+    out.println("  <aclSubmitApps> </aclSubmitApps>");
+    out.println("  <aclAdministerApps> </aclAdministerApps>");
+    out.println("  <queue name=\"queue1\">");
+    out.println("    <aclSubmitApps>userallow</aclSubmitApps>");
+    out.println("    <aclAdministerApps>userallow</aclAdministerApps>");
+    out.println("  </queue>");
     out.println("</queue>");
     out.println("</queue>");
     out.println("</allocations>");
     out.println("</allocations>");
     out.close();
     out.close();

+ 62 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerQueueACLs.java

@@ -0,0 +1,62 @@
+/**
+ * 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.yarn.server.resourcemanager.scheduler.fair;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.PrintWriter;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.QueueACLsTestBase;
+
+public class TestFairSchedulerQueueACLs extends QueueACLsTestBase {
+  @Override
+  protected Configuration createConfiguration() throws IOException {
+    FairSchedulerConfiguration fsConf = new FairSchedulerConfiguration();
+    
+    final String TEST_DIR = new File(System.getProperty("test.build.data",
+        "/tmp")).getAbsolutePath();
+    final String ALLOC_FILE = new File(TEST_DIR, "test-queues.xml")
+        .getAbsolutePath();
+    PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
+    out.println("<?xml version=\"1.0\"?>");
+    out.println("<allocations>");
+    out.println("<queue name=\"root\">");
+    out.println("  <aclSubmitApps> </aclSubmitApps>");
+    out.println("  <aclAdministerApps>root_admin </aclAdministerApps>");
+    out.println("  <queue name=\"queueA\">");
+    out.println("    <aclSubmitApps>queueA_user,common_user </aclSubmitApps>");
+    out.println("    <aclAdministerApps>queueA_admin </aclAdministerApps>");
+    out.println("  </queue>");
+    out.println("  <queue name=\"queueB\">");
+    out.println("    <aclSubmitApps>queueB_user,common_user </aclSubmitApps>");
+    out.println("    <aclAdministerApps>queueB_admin </aclAdministerApps>");
+    out.println("  </queue>");
+    out.println("</queue>");
+    out.println("</allocations>");
+    out.close();
+    fsConf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
+
+    fsConf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
+    fsConf.set("yarn.resourcemanager.scheduler.class", FairScheduler.class.getName());
+
+    return fsConf;
+  }
+}

+ 34 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java

@@ -22,6 +22,8 @@ import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
 import java.net.InetAddress;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.net.UnknownHostException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
@@ -34,8 +36,10 @@ import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.service.CompositeService;
 import org.apache.hadoop.service.CompositeService;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.Shell.ShellCommandExecutor;
 import org.apache.hadoop.util.Shell.ShellCommandExecutor;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.Dispatcher;
+import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factories.RecordFactory;
@@ -52,6 +56,10 @@ import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdater;
 import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdaterImpl;
 import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdaterImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceTrackerService;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceTrackerService;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptRegistrationEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUnregistrationEvent;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 
 
 /**
 /**
@@ -83,6 +91,9 @@ public class MiniYARNCluster extends CompositeService {
 
 
   private ResourceManagerWrapper resourceManagerWrapper;
   private ResourceManagerWrapper resourceManagerWrapper;
   
   
+  private ConcurrentMap<ApplicationAttemptId, Long> appMasters =
+      new ConcurrentHashMap<ApplicationAttemptId, Long>(16, 0.75f, 2);
+  
   private File testWorkDir;
   private File testWorkDir;
 
 
   // Number of nm-local-dirs per nodemanager
   // Number of nm-local-dirs per nodemanager
@@ -210,6 +221,16 @@ public class MiniYARNCluster extends CompositeService {
         };
         };
       };
       };
       resourceManager.init(conf);
       resourceManager.init(conf);
+      resourceManager.getRMContext().getDispatcher().register(RMAppAttemptEventType.class, 
+          new EventHandler<RMAppAttemptEvent>() {
+            public void handle(RMAppAttemptEvent event) {
+              if (event instanceof RMAppAttemptRegistrationEvent) {
+                appMasters.put(event.getApplicationAttemptId(), event.getTimestamp());
+              } else if (event instanceof RMAppAttemptUnregistrationEvent) {
+                appMasters.remove(event.getApplicationAttemptId());
+              }
+            }
+          });
       super.serviceInit(conf);
       super.serviceInit(conf);
     }
     }
 
 
@@ -243,9 +264,22 @@ public class MiniYARNCluster extends CompositeService {
                WebAppUtils.getRMWebAppURLWithoutScheme(getConfig()));
                WebAppUtils.getRMWebAppURLWithoutScheme(getConfig()));
     }
     }
 
 
+    private void waitForAppMastersToFinish(long timeoutMillis) throws InterruptedException {
+      long started = System.currentTimeMillis();
+      synchronized (appMasters) {
+        while (!appMasters.isEmpty() && System.currentTimeMillis() - started < timeoutMillis) {
+          appMasters.wait(1000);
+        }
+      }
+      if (!appMasters.isEmpty()) {
+        LOG.warn("Stopping RM while some app masters are still alive");
+      }
+    }
+    
     @Override
     @Override
     protected synchronized void serviceStop() throws Exception {
     protected synchronized void serviceStop() throws Exception {
       if (resourceManager != null) {
       if (resourceManager != null) {
+        waitForAppMastersToFinish(5000);
         resourceManager.stop();
         resourceManager.stop();
       }
       }
       super.serviceStop();
       super.serviceStop();

+ 27 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/FairScheduler.apt.vm

@@ -221,10 +221,14 @@ Allocation file format
      for containers, but apps submitted later may run concurrently if there is
      for containers, but apps submitted later may run concurrently if there is
      leftover space on the cluster after satisfying the earlier app's requests.
      leftover space on the cluster after satisfying the earlier app's requests.
 
 
-   * aclSubmitApps: a list of users that can submit apps to the queue. A (default)
-     value of "*" means that any users can submit apps. A queue inherits the ACL of
-     its parent, so if a queue2 descends from queue1, and user1 is in queue1's ACL,
-     and user2 is in queue2's ACL, then both users may submit to queue2.
+   * aclSubmitApps: a list of users and/or groups that can submit apps to the
+     queue. Refer to the ACLs section below for more info on the format of this
+     list and how queue ACLs work.
+
+   * aclAdministerApps: a list of users and/or groups that can administer a
+     queue.  Currently the only administrative action is killing an application.
+     Refer to the ACLs section below for more info on the format of this list
+     and how queue ACLs work.
 
 
    * minSharePreemptionTimeout: number of seconds the queue is under its minimum share
    * minSharePreemptionTimeout: number of seconds the queue is under its minimum share
      before it will try to preempt containers to take resources from other queues.
      before it will try to preempt containers to take resources from other queues.
@@ -246,6 +250,24 @@ Allocation file format
 
 
   An example allocation file is given here:
   An example allocation file is given here:
 
 
+Queue Access Control Lists (ACLs)
+
+  Queue Access Control Lists (ACLs) allow administrators to control who may
+  take actions on particular queues. They are configured with the aclSubmitApps
+  and aclAdministerApps properties, which can be set per queue. Currently the
+  only supported administrative action is killing an application. Anybody who
+  may administer a queue may also submit applications to it. These properties
+  take values in a format like "user1,user2 group1,group2" or " group1,group2".
+  An action on a queue will be permitted if its user or group is in the ACL of
+  that queue or in the ACL of any of that queue's ancestors. So if queue2
+  is inside queue1, and user1 is in queue1's ACL, and user2 is in queue2's
+  ACL, then both users may submit to queue2.
+  
+  The root queue's ACLs are "*" by default which, because ACLs are passed down,
+  means that everybody may submit to and kill applications from every queue.
+  To start restricting access, change the root queue's ACLs to something other
+  than "*". 
+
 ---
 ---
 <?xml version="1.0"?>
 <?xml version="1.0"?>
 <allocations>
 <allocations>
@@ -256,6 +278,7 @@ Allocation file format
     <weight>2.0</weight>
     <weight>2.0</weight>
     <schedulingPolicy>fair</schedulingPolicy>
     <schedulingPolicy>fair</schedulingPolicy>
     <queue name="sample_sub_queue">
     <queue name="sample_sub_queue">
+      <aclSubmitApps>charlie</aclSubmitApps>
       <minResources>5000 mb,0vcores</minResources>
       <minResources>5000 mb,0vcores</minResources>
     </queue>
     </queue>
   </queue>
   </queue>

+ 2 - 2
pom.xml

@@ -143,7 +143,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
         <plugin>
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-antrun-plugin</artifactId>
           <artifactId>maven-antrun-plugin</artifactId>
-          <version>1.6</version>
+          <version>1.7</version>
         </plugin>
         </plugin>
         <plugin>
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <groupId>org.apache.maven.plugins</groupId>
@@ -175,7 +175,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
                   <pluginExecutionFilter>
                   <pluginExecutionFilter>
                     <groupId>org.apache.maven.plugins</groupId>
                     <groupId>org.apache.maven.plugins</groupId>
                     <artifactId>maven-antrun-plugin</artifactId>
                     <artifactId>maven-antrun-plugin</artifactId>
-                    <versionRange>[1.6,)</versionRange>
+                    <versionRange>[1.7,)</versionRange>
                     <goals>
                     <goals>
                       <goal>run</goal>
                       <goal>run</goal>
                     </goals>
                     </goals>

Some files were not shown because too many files changed in this diff