Browse Source

Merge r1569890 through r1570692 from trunk.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-5535@1570694 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 11 years ago
parent
commit
78325b08c5
28 changed files with 558 additions and 129 deletions
  1. 7 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 5 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
  3. 7 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java
  4. 45 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/AclCommands.java
  5. 25 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Progress.java
  6. 42 40
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/loadGenerator/LoadGenerator.java
  7. 44 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestProgress.java
  8. 11 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  9. 14 15
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  10. 1 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
  11. 43 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/LsrPBImage.java
  12. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewerPB.java
  13. 5 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageXmlWriter.java
  14. 17 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
  15. 34 25
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
  16. 37 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
  17. 37 18
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java
  18. 61 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testAclCLI.xml
  19. 6 0
      hadoop-mapreduce-project/CHANGES.txt
  20. 4 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestStagingCleanup.java
  21. 6 0
      hadoop-yarn-project/CHANGES.txt
  22. 5 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClusterMetrics.java
  23. 8 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/NodesListManager.java
  24. 15 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
  25. 8 6
      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
  26. 5 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java
  27. 63 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
  28. 1 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java

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

@@ -326,6 +326,11 @@ Trunk (Unreleased)
     HADOOP-10344. Fix TestAclCommands after merging HADOOP-10338 patch.
     (cnauroth)
 
+    HADOOP-10352. Recursive setfacl erroneously attempts to apply default ACL to
+    files. (cnauroth)
+
+    HADOOP-10354. TestWebHDFS fails after merge of HDFS-4685 to trunk. (cnauroth)
+
   OPTIMIZATIONS
 
     HADOOP-7761. Improve the performance of raw comparisons. (todd)
@@ -396,6 +401,8 @@ Release 2.4.0 - UNRELEASED
     HADOOP-10328. loadGenerator exit code is not reliable.
     (Haohui Mai via cnauroth)
 
+    HADOOP-10355. Fix TestLoadGenerator#testLoadGenerator. (Haohui Mai via jing9)
+
 Release 2.3.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 5 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java

@@ -569,6 +569,11 @@ public class RawLocalFileSystem extends FileSystem {
         //expected format
         //-rw-------    1 username groupname ...
         String permission = t.nextToken();
+        if (permission.length() > FsPermission.MAX_PERMISSION_LENGTH) {
+          //files with ACLs might have a '+'
+          permission = permission.substring(0,
+            FsPermission.MAX_PERMISSION_LENGTH);
+        }
         setPermission(FsPermission.valueOf(permission));
         t.nextToken();
 

+ 7 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java

@@ -48,6 +48,9 @@ public class FsPermission implements Writable {
     WritableFactories.setFactory(ImmutableFsPermission.class, FACTORY);
   }
 
+  /** Maximum acceptable length of a permission string to parse */
+  public static final int MAX_PERMISSION_LENGTH = 10;
+
   /** Create an immutable {@link FsPermission} object. */
   public static FsPermission createImmutable(short permission) {
     return new ImmutableFsPermission(permission);
@@ -319,9 +322,10 @@ public class FsPermission implements Writable {
     if (unixSymbolicPermission == null) {
       return null;
     }
-    else if (unixSymbolicPermission.length() != 10) {
-      throw new IllegalArgumentException("length != 10(unixSymbolicPermission="
-          + unixSymbolicPermission + ")");
+    else if (unixSymbolicPermission.length() != MAX_PERMISSION_LENGTH) {
+      throw new IllegalArgumentException(String.format(
+        "length != %d(unixSymbolicPermission=%s)", MAX_PERMISSION_LENGTH,
+        unixSymbolicPermission));
     }
 
     int n = 0;

+ 45 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/AclCommands.java

@@ -22,6 +22,8 @@ import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 
+import com.google.common.collect.Lists;
+
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -231,6 +233,7 @@ class AclCommands extends FsCommand {
     CommandFormat cf = new CommandFormat(0, Integer.MAX_VALUE, "b", "k", "R",
         "m", "x", "-set");
     List<AclEntry> aclEntries = null;
+    List<AclEntry> accessAclEntries = null;
 
     @Override
     protected void processOptions(LinkedList<String> args) throws IOException {
@@ -263,6 +266,19 @@ class AclCommands extends FsCommand {
       if (args.size() > 1) {
         throw new HadoopIllegalArgumentException("Too many arguments");
       }
+
+      // In recursive mode, save a separate list of just the access ACL entries.
+      // Only directories may have a default ACL.  When a recursive operation
+      // encounters a file under the specified path, it must pass only the
+      // access ACL entries.
+      if (isRecursive() && (oneModifyOption || setOption)) {
+        accessAclEntries = Lists.newArrayList();
+        for (AclEntry entry: aclEntries) {
+          if (entry.getScope() == AclEntryScope.ACCESS) {
+            accessAclEntries.add(entry);
+          }
+        }
+      }
     }
 
     @Override
@@ -272,11 +288,37 @@ class AclCommands extends FsCommand {
       } else if (cf.getOpt("k")) {
         item.fs.removeDefaultAcl(item.path);
       } else if (cf.getOpt("m")) {
-        item.fs.modifyAclEntries(item.path, aclEntries);
+        List<AclEntry> entries = getAclEntries(item);
+        if (!entries.isEmpty()) {
+          item.fs.modifyAclEntries(item.path, entries);
+        }
       } else if (cf.getOpt("x")) {
-        item.fs.removeAclEntries(item.path, aclEntries);
+        List<AclEntry> entries = getAclEntries(item);
+        if (!entries.isEmpty()) {
+          item.fs.removeAclEntries(item.path, entries);
+        }
       } else if (cf.getOpt("-set")) {
-        item.fs.setAcl(item.path, aclEntries);
+        List<AclEntry> entries = getAclEntries(item);
+        if (!entries.isEmpty()) {
+          item.fs.setAcl(item.path, entries);
+        }
+      }
+    }
+
+    /**
+     * Returns the ACL entries to use in the API call for the given path.  For a
+     * recursive operation, returns all specified ACL entries if the item is a
+     * directory or just the access ACL entries if the item is a file.  For a
+     * non-recursive operation, returns all specified ACL entries.
+     *
+     * @param item PathData path to check
+     * @return List<AclEntry> ACL entries to use in the API call
+     */
+    private List<AclEntry> getAclEntries(PathData item) {
+      if (isRecursive()) {
+        return item.stat.isDirectory() ? aclEntries : accessAclEntries;
+      } else {
+        return aclEntries;
       }
     }
   }

+ 25 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Progress.java

@@ -160,6 +160,31 @@ public class Progress {
 
   /** Called during execution on a leaf node to set its progress. */
   public synchronized void set(float progress) {
+    if (Float.isNaN(progress)) {
+      progress = 0;
+      LOG.warn("Illegal progress value found, progress is Float.NaN. " +
+        "Progress will be changed to 0");
+    }
+    else if (progress == Float.NEGATIVE_INFINITY) {
+      progress = 0;
+      LOG.warn("Illegal progress value found, progress is " +
+        "Float.NEGATIVE_INFINITY. Progress will be changed to 0");
+    }
+    else if (progress < 0) {
+      progress = 0;
+      LOG.warn("Illegal progress value found, progress is less than 0." +
+        " Progress will be changed to 0");
+    }
+    else if (progress > 1) {
+      progress = 1;
+      LOG.warn("Illegal progress value found, progress is larger than 1." +
+        " Progress will be changed to 1");
+    }
+    else if (progress == Float.POSITIVE_INFINITY) {
+      progress = 1;
+      LOG.warn("Illegal progress value found, progress is " +
+        "Float.POSITIVE_INFINITY. Progress will be changed to 1");
+    }
     this.progress = progress;
   }
 

+ 42 - 40
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/loadGenerator/LoadGenerator.java

@@ -45,6 +45,8 @@ import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 
+import com.google.common.base.Preconditions;
+
 /** The load generator is a tool for testing NameNode behavior under
  * different client loads.
  * It allows the user to generate different mixes of read, write,
@@ -488,7 +490,35 @@ public class LoadGenerator extends Configured implements Tool {
     
     return initFileDirTables();
   }
-  
+
+  private static void parseScriptLine(String line, ArrayList<Long> duration,
+      ArrayList<Double> readProb, ArrayList<Double> writeProb) {
+    String[] a = line.split("\\s");
+
+    if (a.length != 3) {
+      throw new IllegalArgumentException("Incorrect number of parameters: "
+          + line);
+    }
+
+    try {
+      long d = Long.parseLong(a[0]);
+      double r = Double.parseDouble(a[1]);
+      double w = Double.parseDouble(a[2]);
+
+      Preconditions.checkArgument(d >= 0, "Invalid duration: " + d);
+      Preconditions.checkArgument(0 <= r && r <= 1.0,
+          "The read probability must be [0, 1]: " + r);
+      Preconditions.checkArgument(0 <= w && w <= 1.0,
+          "The read probability must be [0, 1]: " + w);
+
+      readProb.add(r);
+      duration.add(d);
+      writeProb.add(w);
+    } catch (NumberFormatException nfe) {
+      throw new IllegalArgumentException("Cannot parse: " + line);
+    }
+  }
+
   /**
    * Read a script file of the form: lines of text with duration in seconds,
    * read probability and write probability, separated by white space.
@@ -508,47 +538,19 @@ public class LoadGenerator extends Configured implements Tool {
     String line;
     // Read script, parse values, build array of duration, read and write probs
 
-    while ((line = br.readLine()) != null) {
-      lineNum++;
-      if (line.startsWith("#") || line.isEmpty()) // skip comments and blanks
-        continue;
-
-      String[] a = line.split("\\s");
-      if (a.length != 3) {
-        System.err.println("Line " + lineNum
-            + ": Incorrect number of parameters: " + line);
-      }
-
-      try {
-        long d = Long.parseLong(a[0]);
-        if (d < 0) {
-          System.err.println("Line " + lineNum + ": Invalid duration: " + d);
-          return -1;
-        }
-
-        double r = Double.parseDouble(a[1]);
-        if (r < 0.0 || r > 1.0) {
-          System.err.println("Line " + lineNum
-              + ": The read probability must be [0, 1]: " + r);
-          return -1;
-        }
-
-        double w = Double.parseDouble(a[2]);
-        if (w < 0.0 || w > 1.0) {
-          System.err.println("Line " + lineNum
-              + ": The read probability must be [0, 1]: " + r);
-          return -1;
-        }
+    try {
+      while ((line = br.readLine()) != null) {
+        lineNum++;
+        if (line.startsWith("#") || line.isEmpty()) // skip comments and blanks
+          continue;
 
-        readProb.add(r);
-        duration.add(d);
-        writeProb.add(w);
-      } catch (NumberFormatException nfe) {
-        System.err.println(lineNum + ": Can't parse: " + line);
-        return -1;
-      } finally {
-        IOUtils.cleanup(LOG, br);
+        parseScriptLine(line, duration, readProb, writeProb);
       }
+    } catch (IllegalArgumentException e) {
+      System.err.println("Line: " + lineNum + ", " + e.getMessage());
+      return -1;
+    } finally {
+      IOUtils.cleanup(LOG, br);
     }
     
     // Copy vectors to arrays of values, to avoid autoboxing overhead later

+ 44 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestProgress.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.util;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestProgress {
+
+  @Test
+  public void testSet(){
+    Progress progress = new Progress();
+    progress.set(Float.NaN);
+    Assert.assertEquals(0, progress.getProgress(), 0.0);
+
+    progress.set(Float.NEGATIVE_INFINITY);
+    Assert.assertEquals(0,progress.getProgress(),0.0);
+
+    progress.set(-1);
+    Assert.assertEquals(0,progress.getProgress(),0.0);
+
+    progress.set((float) 1.1);
+    Assert.assertEquals(1,progress.getProgress(),0.0);
+
+    progress.set(Float.POSITIVE_INFINITY);
+    Assert.assertEquals(1,progress.getProgress(),0.0);
+  }
+}

+ 11 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -424,6 +424,9 @@ Release 2.4.0 - UNRELEASED
     HDFS-5775. Consolidate the code for serialization in CacheManager
     (Haohui Mai via brandonli)
 
+    HDFS-5935. New Namenode UI FS browser should throw smarter error messages.
+    (Travis Thompson via jing9)
+
   OPTIMIZATIONS
 
     HDFS-5790. LeaseManager.findPath is very slow when many leases need recovery
@@ -543,6 +546,11 @@ Release 2.4.0 - UNRELEASED
     HDFS-5944. LeaseManager:findLeaseWithPrefixPath can't handle path like /a/b/
     and cause SecondaryNameNode failed do checkpoint (Yunjiong Zhao via brandonli)
 
+    HDFS-5982. Need to update snapshot manager when applying editlog for deleting
+    a snapshottable directory. (jing9)
+
+    HDFS-5988. Bad fsimage always generated after upgrade. (wang)
+
   BREAKDOWN OF HDFS-5698 SUBTASKS AND RELATED JIRAS
 
     HDFS-5717. Save FSImage header in protobuf. (Haohui Mai via jing9)
@@ -600,6 +608,9 @@ Release 2.4.0 - UNRELEASED
     HDFS-5959. Fix typo at section name in FSImageFormatProtobuf.java.
     (Akira Ajisaka via suresh)
 
+    HDFS-5981. PBImageXmlWriter generates malformed XML.
+    (Haohui Mai via cnauroth)
+
 Release 2.3.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 14 - 15
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -1317,20 +1317,12 @@ public class FSDirectory implements Closeable {
       if (!deleteAllowed(inodesInPath, src) ) {
         filesRemoved = -1;
       } else {
-        // Before removing the node, first check if the targetNode is for a
-        // snapshottable dir with snapshots, or its descendants have
-        // snapshottable dir with snapshots
-        final INode targetNode = inodesInPath.getLastINode();
         List<INodeDirectorySnapshottable> snapshottableDirs = 
             new ArrayList<INodeDirectorySnapshottable>();
-        checkSnapshot(targetNode, snapshottableDirs);
+        checkSnapshot(inodesInPath.getLastINode(), snapshottableDirs);
         filesRemoved = unprotectedDelete(inodesInPath, collectedBlocks,
             removedINodes, now);
-        if (snapshottableDirs.size() > 0) {
-          // There are some snapshottable directories without snapshots to be
-          // deleted. Need to update the SnapshotManager.
-          namesystem.removeSnapshottableDirs(snapshottableDirs);
-        }
+        namesystem.removeSnapshottableDirs(snapshottableDirs);
       }
     } finally {
       writeUnlock();
@@ -1392,18 +1384,25 @@ public class FSDirectory implements Closeable {
    * @param src a string representation of a path to an inode
    * @param mtime the time the inode is removed
    * @throws SnapshotAccessControlException if path is in RO snapshot
-   */ 
+   */
   void unprotectedDelete(String src, long mtime) throws UnresolvedLinkException,
-      QuotaExceededException, SnapshotAccessControlException {
+      QuotaExceededException, SnapshotAccessControlException, IOException {
     assert hasWriteLock();
     BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
     List<INode> removedINodes = new ChunkedArrayList<INode>();
 
     final INodesInPath inodesInPath = rootDir.getINodesInPath4Write(
         normalizePath(src), false);
-    final long filesRemoved = deleteAllowed(inodesInPath, src) ? 
-        unprotectedDelete(inodesInPath, collectedBlocks, 
-            removedINodes, mtime) : -1;
+    long filesRemoved = -1;
+    if (deleteAllowed(inodesInPath, src)) {
+      List<INodeDirectorySnapshottable> snapshottableDirs = 
+          new ArrayList<INodeDirectorySnapshottable>();
+      checkSnapshot(inodesInPath.getLastINode(), snapshottableDirs);
+      filesRemoved = unprotectedDelete(inodesInPath, collectedBlocks,
+          removedINodes, mtime);
+      namesystem.removeSnapshottableDirs(snapshottableDirs); 
+    }
+
     if (filesRemoved >= 0) {
       getFSNamesystem().removePathAndBlocks(src, collectedBlocks, 
           removedINodes);

+ 1 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java

@@ -700,9 +700,7 @@ public class FSImageFormat {
       localName =
           renameReservedComponentOnUpgrade(localName, getLayoutVersion());
       INode inode = loadINode(localName, isSnapshotINode, in, counter);
-      if (updateINodeMap
-          && NameNodeLayoutVersion.supports(
-              LayoutVersion.Feature.ADD_INODE_ID, getLayoutVersion())) {
+      if (updateINodeMap) {
         namesystem.dir.addToInodeMap(inode);
       }
       return inode;

+ 43 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/LsrPBImage.java

@@ -28,6 +28,8 @@ import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
@@ -63,6 +65,9 @@ import com.google.common.io.LimitInputStream;
  * output of the lsr command.
  */
 final class LsrPBImage {
+
+  private static final Log LOG = LogFactory.getLog(LsrPBImage.class);
+
   private final Configuration conf;
   private final PrintWriter out;
   private String[] stringTable;
@@ -133,6 +138,10 @@ final class LsrPBImage {
 
   private void list(String parent, long dirId) {
     INode inode = inodes.get(dirId);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Listing directory id " + dirId + " parent '" + parent
+          + "' (INode is " + inode + ")");
+    }
     listINode(parent.isEmpty() ? "/" : parent, inode);
     long[] children = dirmap.get(dirId);
     if (children == null) {
@@ -189,6 +198,9 @@ final class LsrPBImage {
   }
 
   private void loadINodeDirectorySection(InputStream in) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Loading directory section");
+    }
     while (true) {
       INodeDirectorySection.DirEntry e = INodeDirectorySection.DirEntry
           .parseDelimitedFrom(in);
@@ -205,10 +217,21 @@ final class LsrPBImage {
         l[i] = refList.get(refId).getReferredId();
       }
       dirmap.put(e.getParent(), l);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Loaded directory (parent " + e.getParent()
+            + ") with " + e.getChildrenCount() + " children and "
+            + e.getRefChildrenCount() + " reference children");
+      }
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Loaded " + dirmap.size() + " directories");
     }
   }
 
   private void loadINodeReferenceSection(InputStream in) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Loading inode reference section");
+    }
     while (true) {
       INodeReferenceSection.INodeReference e = INodeReferenceSection
           .INodeReference.parseDelimitedFrom(in);
@@ -216,24 +239,44 @@ final class LsrPBImage {
         break;
       }
       refList.add(e);
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Loaded inode reference named '" + e.getName()
+            + "' referring to id " + e.getReferredId() + "");
+      }
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Loaded " + refList.size() + " inode references");
     }
   }
 
   private void loadINodeSection(InputStream in) throws IOException {
     INodeSection s = INodeSection.parseDelimitedFrom(in);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Found " + s.getNumInodes() + " inodes in inode section");
+    }
     for (int i = 0; i < s.getNumInodes(); ++i) {
       INodeSection.INode p = INodeSection.INode.parseDelimitedFrom(in);
       inodes.put(p.getId(), p);
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Loaded inode id " + p.getId() + " type " + p.getType()
+            + " name '" + p.getName().toStringUtf8() + "'");
+      }
     }
   }
 
   private void loadStringTable(InputStream in) throws IOException {
     StringTableSection s = StringTableSection.parseDelimitedFrom(in);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Found " + s.getNumEntry() + " strings in string section");
+    }
     stringTable = new String[s.getNumEntry() + 1];
     for (int i = 0; i < s.getNumEntry(); ++i) {
       StringTableSection.Entry e = StringTableSection.Entry
           .parseDelimitedFrom(in);
       stringTable[e.getId()] = e.getStr();
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Loaded string " + e.getStr());
+      }
     }
   }
 }

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

@@ -33,6 +33,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.IOUtils;
 
 /**
  * OfflineImageViewer to dump the contents of an Hadoop image file to XML or the
@@ -164,7 +165,7 @@ public class OfflineImageViewerPB {
     } catch (IOException e) {
       System.err.println("Encountered exception.  Exiting: " + e.getMessage());
     } finally {
-      out.close();
+      IOUtils.cleanup(null, out);
     }
 
   }

+ 5 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageXmlWriter.java

@@ -78,7 +78,7 @@ public final class PBImageXmlWriter {
     FileInputStream fin = null;
     try {
       fin = new FileInputStream(file.getFD());
-      out.print("<?xml version=\"1.0\"?>\n");
+      out.print("<?xml version=\"1.0\"?>\n<fsimage>");
 
       ArrayList<FileSummary.Section> sections = Lists.newArrayList(summary
           .getSectionsList());
@@ -138,6 +138,7 @@ public final class PBImageXmlWriter {
           break;
         }
       }
+      out.print("</fsimage>\n");
     } finally {
       IOUtils.cleanup(null, fin);
     }
@@ -229,6 +230,7 @@ public final class PBImageXmlWriter {
       }
       dumpINodeReference(e);
     }
+    out.print("</INodeReferenceSection>");
   }
 
   private void dumpINodeReference(INodeReferenceSection.INodeReference r) {
@@ -301,7 +303,7 @@ public final class PBImageXmlWriter {
         .o("genstampV1Limit", s.getGenstampV1Limit())
         .o("lastAllocatedBlockId", s.getLastAllocatedBlockId())
         .o("txid", s.getTransactionId());
-    out.print("<NameSection>\n");
+    out.print("</NameSection>\n");
   }
 
   private String dumpPermission(long permission) {
@@ -375,7 +377,7 @@ public final class PBImageXmlWriter {
       }
       out.print("</diff>");
     }
-    out.print("<SnapshotDiffSection>\n");
+    out.print("</SnapshotDiffSection>\n");
   }
 
   private void dumpSnapshotSection(InputStream in) throws IOException {

+ 17 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js

@@ -66,10 +66,23 @@
 
   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>';
-      }
+      switch(jqxhr.status) {
+        case 401:
+          var msg = '<p>Authentication failed when trying to open ' + url + ': Unauthrozied.</p>';
+          break;
+        case 403:
+          if(jqxhr.responseJSON !== undefined && jqxhr.responseJSON.RemoteException !== undefined) {
+            var msg = '<p>' + jqxhr.responseJSON.RemoteException.message + "</p>";
+            break;
+          }
+          var msg = '<p>Permission denied when trying to open ' + url + ': ' + err + '</p>';
+          break;
+        case 404:
+          var msg = '<p>Path does not exist on HDFS or WebHDFS is disabled.  Please check your path or enable WebHDFS</p>';
+          break;
+        default:
+          var msg = '<p>Failed to retreive data from ' + url + ': ' + err + '</p>';
+        }
       show_err_msg(msg);
     };
   }

+ 34 - 25
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java

@@ -330,13 +330,14 @@ public class TestDFSUpgradeFromImage {
    * paths to test renaming on upgrade
    */
   @Test
-  public void testUpgradeFromRel2ReservedImage() throws IOException {
+  public void testUpgradeFromRel2ReservedImage() throws Exception {
     unpackStorage(HADOOP2_RESERVED_IMAGE);
     MiniDFSCluster cluster = null;
     // Try it once without setting the upgrade flag to ensure it fails
+    final Configuration conf = new Configuration();
     try {
       cluster =
-          new MiniDFSCluster.Builder(new Configuration())
+          new MiniDFSCluster.Builder(conf)
               .format(false)
               .startupOption(StartupOption.UPGRADE)
               .numDataNodes(0).build();
@@ -355,28 +356,15 @@ public class TestDFSUpgradeFromImage {
           ".snapshot=.user-snapshot," +
           ".reserved=.my-reserved");
       cluster =
-          new MiniDFSCluster.Builder(new Configuration())
+          new MiniDFSCluster.Builder(conf)
               .format(false)
               .startupOption(StartupOption.UPGRADE)
               .numDataNodes(0).build();
-      // Make sure the paths were renamed as expected
       DistributedFileSystem dfs = cluster.getFileSystem();
-      ArrayList<Path> toList = new ArrayList<Path>();
-      ArrayList<String> found = new ArrayList<String>();
-      toList.add(new Path("/"));
-      while (!toList.isEmpty()) {
-        Path p = toList.remove(0);
-        FileStatus[] statuses = dfs.listStatus(p);
-        for (FileStatus status: statuses) {
-          final String path = status.getPath().toUri().getPath();
-          System.out.println("Found path " + path);
-          found.add(path);
-          if (status.isDirectory()) {
-            toList.add(status.getPath());
-          }
-        }
-      }
-      String[] expected = new String[] {
+      // Make sure the paths were renamed as expected
+      // Also check that paths are present after a restart, checks that the
+      // upgraded fsimage has the same state.
+      final String[] expected = new String[] {
           "/edits",
           "/edits/.reserved",
           "/edits/.user-snapshot",
@@ -393,12 +381,33 @@ public class TestDFSUpgradeFromImage {
           "/.my-reserved/edits-touch",
           "/.my-reserved/image-touch"
       };
-
-      for (String s: expected) {
-        assertTrue("Did not find expected path " + s, found.contains(s));
+      for (int i=0; i<2; i++) {
+        // Restart the second time through this loop
+        if (i==1) {
+          cluster.finalizeCluster(conf);
+          cluster.restartNameNode(true);
+        }
+        ArrayList<Path> toList = new ArrayList<Path>();
+        toList.add(new Path("/"));
+        ArrayList<String> found = new ArrayList<String>();
+        while (!toList.isEmpty()) {
+          Path p = toList.remove(0);
+          FileStatus[] statuses = dfs.listStatus(p);
+          for (FileStatus status: statuses) {
+            final String path = status.getPath().toUri().getPath();
+            System.out.println("Found path " + path);
+            found.add(path);
+            if (status.isDirectory()) {
+              toList.add(status.getPath());
+            }
+          }
+        }
+        for (String s: expected) {
+          assertTrue("Did not find expected path " + s, found.contains(s));
+        }
+        assertEquals("Found an unexpected path while listing filesystem",
+            found.size(), expected.length);
       }
-      assertEquals("Found an unexpected path while listing filesystem",
-          found.size(), expected.length);
     } finally {
       if (cluster != null) {
         cluster.shutdown();

+ 37 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java

@@ -40,6 +40,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
@@ -128,7 +129,42 @@ public class TestSnapshotDeletion {
     exception.expectMessage(error);
     hdfs.delete(sub, true);
   }
-  
+
+  /**
+   * Test applying editlog of operation which deletes a snapshottable directory
+   * without snapshots. The snapshottable dir list in snapshot manager should be
+   * updated.
+   */
+  @Test (timeout=300000)
+  public void testApplyEditLogForDeletion() throws Exception {
+    final Path foo = new Path("/foo");
+    final Path bar1 = new Path(foo, "bar1");
+    final Path bar2 = new Path(foo, "bar2");
+    hdfs.mkdirs(bar1);
+    hdfs.mkdirs(bar2);
+
+    // allow snapshots on bar1 and bar2
+    hdfs.allowSnapshot(bar1);
+    hdfs.allowSnapshot(bar2);
+    assertEquals(2, cluster.getNamesystem().getSnapshotManager()
+        .getNumSnapshottableDirs());
+    assertEquals(2, cluster.getNamesystem().getSnapshotManager()
+        .getSnapshottableDirs().length);
+
+    // delete /foo
+    hdfs.delete(foo, true);
+    cluster.restartNameNode(0);
+    // the snapshottable dir list in snapshot manager should be empty
+    assertEquals(0, cluster.getNamesystem().getSnapshotManager()
+        .getNumSnapshottableDirs());
+    assertEquals(0, cluster.getNamesystem().getSnapshotManager()
+        .getSnapshottableDirs().length);
+    hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    hdfs.saveNamespace();
+    hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    cluster.restartNameNode(0);
+  }
+
   /**
    * Deleting directory with snapshottable descendant with snapshots must fail.
    */

+ 37 - 18
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java

@@ -18,23 +18,24 @@
 package org.apache.hadoop.hdfs.tools.offlineImageViewer;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
-import java.io.FileReader;
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.io.RandomAccessFile;
+import java.io.StringReader;
 import java.io.StringWriter;
 import java.util.HashMap;
-import java.util.Set;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.parsers.SAXParser;
+import javax.xml.parsers.SAXParserFactory;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -43,21 +44,23 @@ 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.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DFSTestUtil;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.test.PathUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
+import org.xml.sax.InputSource;
+import org.xml.sax.SAXException;
+import org.xml.sax.helpers.DefaultHandler;
+
+import com.google.common.collect.Maps;
 
 /**
  * Test function of OfflineImageViewer by: * confirming it can correctly process
@@ -85,7 +88,7 @@ public class TestOfflineImageViewer {
   }
 
   // namespace as written to dfs, to be compared with viewer's output
-  final static HashMap<String, FileStatus> writtenFiles = new HashMap<String, FileStatus>();
+  final static HashMap<String, FileStatus> writtenFiles = Maps.newHashMap();
 
   @Rule
   public TemporaryFolder folder = new TemporaryFolder();
@@ -98,7 +101,7 @@ public class TestOfflineImageViewer {
   public static void createOriginalFSImage() throws IOException {
     MiniDFSCluster cluster = null;
     try {
-      Configuration conf = new HdfsConfiguration();
+      Configuration conf = new Configuration();
       conf.setLong(
           DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_MAX_LIFETIME_KEY, 10000);
       conf.setLong(
@@ -107,11 +110,9 @@ public class TestOfflineImageViewer {
           DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
       conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTH_TO_LOCAL,
           "RULE:[2:$1@$0](JobTracker@.*FOO.COM)s/@.*//" + "DEFAULT");
-      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(4).build();
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
       cluster.waitActive();
-      FileSystem hdfs = cluster.getFileSystem();
-
-      int filesize = 256;
+      DistributedFileSystem hdfs = cluster.getFileSystem();
 
       // Create a reasonable namespace
       for (int i = 0; i < NUM_DIRS; i++) {
@@ -121,7 +122,7 @@ public class TestOfflineImageViewer {
         for (int j = 0; j < FILES_PER_DIR; j++) {
           Path file = new Path(dir, "file" + j);
           FSDataOutputStream o = hdfs.create(file);
-          o.write(new byte[filesize++]);
+          o.write(23);
           o.close();
 
           writtenFiles.put(file.toString(),
@@ -136,10 +137,15 @@ public class TestOfflineImageViewer {
         LOG.debug("got token " + t);
       }
 
+      final Path snapshot = new Path("/snapshot");
+      hdfs.mkdirs(snapshot);
+      hdfs.allowSnapshot(snapshot);
+      hdfs.mkdirs(new Path("/snapshot/1"));
+      hdfs.delete(snapshot, true);
+
       // Write results to the fsimage file
-      cluster.getNameNodeRpc()
-          .setSafeMode(SafeModeAction.SAFEMODE_ENTER, false);
-      cluster.getNameNodeRpc().saveNamespace();
+      hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER, false);
+      hdfs.saveNamespace();
 
       // Determine location of fsimage file
       originalFsimage = FSImageTestUtil.findLatestImageFile(FSImageTestUtil
@@ -249,4 +255,17 @@ public class TestOfflineImageViewer {
     int totalFiles = Integer.parseInt(matcher.group(1));
     assertEquals(totalFiles, NUM_DIRS * FILES_PER_DIR);
   }
+
+  @Test
+  public void testPBImageXmlWriter() throws IOException, SAXException,
+      ParserConfigurationException {
+    StringWriter output = new StringWriter();
+    PrintWriter o = new PrintWriter(output);
+    PBImageXmlWriter v = new PBImageXmlWriter(new Configuration(), o);
+    v.visit(new RandomAccessFile(originalFsimage, "r"));
+    SAXParserFactory spf = SAXParserFactory.newInstance();
+    SAXParser parser = spf.newSAXParser();
+    final String xml = output.getBuffer().toString();
+    parser.parse(new InputSource(new StringReader(xml)), new DefaultHandler());
+  }
 }

+ 61 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testAclCLI.xml

@@ -911,5 +911,66 @@
         </comparator>
       </comparators>
     </test>
+    <test>
+      <description>setfacl: recursive modify entries with mix of files and directories</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir -p /dir1</command>
+        <command>-fs NAMENODE -touchz /dir1/file1</command>
+        <command>-fs NAMENODE -mkdir -p /dir1/dir2</command>
+        <command>-fs NAMENODE -touchz /dir1/dir2/file2</command>
+        <command>-fs NAMENODE -setfacl -R -m user:charlie:rwx,default:user:charlie:r-x /dir1</command>
+        <command>-fs NAMENODE -getfacl -R /dir1</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -R /dir1</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>ExactComparator</type>
+          <expected-output># file: /dir1#LF## owner: USERNAME#LF## group: supergroup#LF#user::rwx#LF#user:charlie:rwx#LF#group::r-x#LF#mask::rwx#LF#other::r-x#LF#default:user::rwx#LF#default:user:charlie:r-x#LF#default:group::r-x#LF#default:mask::r-x#LF#default:other::r-x#LF##LF## file: /dir1/dir2#LF## owner: USERNAME#LF## group: supergroup#LF#user::rwx#LF#user:charlie:rwx#LF#group::r-x#LF#mask::rwx#LF#other::r-x#LF#default:user::rwx#LF#default:user:charlie:r-x#LF#default:group::r-x#LF#default:mask::r-x#LF#default:other::r-x#LF##LF## file: /dir1/dir2/file2#LF## owner: USERNAME#LF## group: supergroup#LF#user::rw-#LF#user:charlie:rwx#LF#group::r--#LF#mask::rwx#LF#other::r--#LF##LF## file: /dir1/file1#LF## owner: USERNAME#LF## group: supergroup#LF#user::rw-#LF#user:charlie:rwx#LF#group::r--#LF#mask::rwx#LF#other::r--#LF##LF#</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+    <test>
+      <description>setfacl: recursive remove entries with mix of files and directories</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir -p /dir1</command>
+        <command>-fs NAMENODE -touchz /dir1/file1</command>
+        <command>-fs NAMENODE -mkdir -p /dir1/dir2</command>
+        <command>-fs NAMENODE -touchz /dir1/dir2/file2</command>
+        <command>-fs NAMENODE -setfacl -R -m user:bob:rwx,user:charlie:rwx,default:user:bob:rwx,default:user:charlie:r-x /dir1</command>
+        <command>-fs NAMENODE -setfacl -R -x user:bob,default:user:bob /dir1</command>
+        <command>-fs NAMENODE -getfacl -R /dir1</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -R /dir1</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>ExactComparator</type>
+          <expected-output># file: /dir1#LF## owner: USERNAME#LF## group: supergroup#LF#user::rwx#LF#user:charlie:rwx#LF#group::r-x#LF#mask::rwx#LF#other::r-x#LF#default:user::rwx#LF#default:user:charlie:r-x#LF#default:group::r-x#LF#default:mask::r-x#LF#default:other::r-x#LF##LF## file: /dir1/dir2#LF## owner: USERNAME#LF## group: supergroup#LF#user::rwx#LF#user:charlie:rwx#LF#group::r-x#LF#mask::rwx#LF#other::r-x#LF#default:user::rwx#LF#default:user:charlie:r-x#LF#default:group::r-x#LF#default:mask::r-x#LF#default:other::r-x#LF##LF## file: /dir1/dir2/file2#LF## owner: USERNAME#LF## group: supergroup#LF#user::rw-#LF#user:charlie:rwx#LF#group::r--#LF#mask::rwx#LF#other::r--#LF##LF## file: /dir1/file1#LF## owner: USERNAME#LF## group: supergroup#LF#user::rw-#LF#user:charlie:rwx#LF#group::r--#LF#mask::rwx#LF#other::r--#LF##LF#</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+    <test>
+      <description>setfacl: recursive set with mix of files and directories</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir -p /dir1</command>
+        <command>-fs NAMENODE -touchz /dir1/file1</command>
+        <command>-fs NAMENODE -mkdir -p /dir1/dir2</command>
+        <command>-fs NAMENODE -touchz /dir1/dir2/file2</command>
+        <command>-fs NAMENODE -setfacl -R --set user::rwx,user:charlie:rwx,group::r-x,other::r-x,default:user:charlie:r-x /dir1</command>
+        <command>-fs NAMENODE -getfacl -R /dir1</command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm -R /dir1</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>ExactComparator</type>
+          <expected-output># file: /dir1#LF## owner: USERNAME#LF## group: supergroup#LF#user::rwx#LF#user:charlie:rwx#LF#group::r-x#LF#mask::rwx#LF#other::r-x#LF#default:user::rwx#LF#default:user:charlie:r-x#LF#default:group::r-x#LF#default:mask::r-x#LF#default:other::r-x#LF##LF## file: /dir1/dir2#LF## owner: USERNAME#LF## group: supergroup#LF#user::rwx#LF#user:charlie:rwx#LF#group::r-x#LF#mask::rwx#LF#other::r-x#LF#default:user::rwx#LF#default:user:charlie:r-x#LF#default:group::r-x#LF#default:mask::r-x#LF#default:other::r-x#LF##LF## file: /dir1/dir2/file2#LF## owner: USERNAME#LF## group: supergroup#LF#user::rwx#LF#user:charlie:rwx#LF#group::r-x#LF#mask::rwx#LF#other::r-x#LF##LF## file: /dir1/file1#LF## owner: USERNAME#LF## group: supergroup#LF#user::rwx#LF#user:charlie:rwx#LF#group::r-x#LF#mask::rwx#LF#other::r-x#LF##LF#</expected-output>
+        </comparator>
+      </comparators>
+    </test>
   </tests>
 </configuration>

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

@@ -147,6 +147,12 @@ Release 2.5.0 - UNRELEASED
 
   IMPROVEMENTS
 
+    MAPREDUCE-5671. NaN can be created by client and assign to Progress (Chen
+    He via jeagles)
+
+    MAPREDUCE-5688. TestStagingCleanup fails intermittently with JDK7 (Mit
+    Desai via jeagles)
+
   OPTIMIZATIONS
 
   BUG FIXES 

+ 4 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestStagingCleanup.java

@@ -34,6 +34,7 @@ import junit.framework.TestCase;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.Counters;
 import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.TypeConverter;
@@ -124,7 +125,7 @@ import org.junit.Test;
      when(fs.exists(stagingDir)).thenReturn(true);
      ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(),
         0);
-     ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(appId, 0);
+     ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(appId, 1);
      JobId jobid = recordFactory.newRecordInstance(JobId.class);
      jobid.setAppId(appId);
      ContainerAllocator mockAlloc = mock(ContainerAllocator.class);
@@ -149,7 +150,7 @@ import org.junit.Test;
      when(fs.exists(stagingDir)).thenReturn(true);
      ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(),
          0);
-     ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(appId, 0);
+     ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(appId, 1);
      ContainerAllocator mockAlloc = mock(ContainerAllocator.class);
      Assert.assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1);
      MRAppMaster appMaster = new TestMRApp(attemptId, mockAlloc,
@@ -282,6 +283,7 @@ import org.junit.Test;
          String diagnostic) {
        JobImpl jobImpl = mock(JobImpl.class);
        when(jobImpl.getInternalState()).thenReturn(this.jobStateInternal);
+       when(jobImpl.getAllCounters()).thenReturn(new Counters());
        JobID jobID = JobID.forName("job_1234567890000_0001");
        JobId jobId = TypeConverter.toYarn(jobID);
        when(jobImpl.getID()).thenReturn(jobId);

+ 6 - 0
hadoop-yarn-project/CHANGES.txt

@@ -321,6 +321,12 @@ Release 2.4.0 - UNRELEASED
     YARN-713. Fixed ResourceManager to not crash while building tokens when DNS
     issues happen transmittently. (Jian He via vinodkv)
 
+    YARN-1398. Fixed a deadlock in ResourceManager between users requesting
+    queue-acls and completing containers. (vinodkv)
+
+    YARN-1071. Enabled ResourceManager to recover cluster metrics
+    numDecommissionedNMs after restarting. (Jian He via zjshen)
+
 Release 2.3.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

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

@@ -91,7 +91,11 @@ public class ClusterMetrics {
   public void incrDecommisionedNMs() {
     numDecommissionedNMs.incr();
   }
-  
+
+  public void setDecommisionedNMs(int num) {
+    numDecommissionedNMs.set(num);
+  }
+
   public void decrDecommisionedNMs() {
     numDecommissionedNMs.decr();
   }

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

@@ -75,6 +75,7 @@ public class NodesListManager extends AbstractService implements
           YarnConfiguration.DEFAULT_RM_NODES_EXCLUDE_FILE_PATH);
       this.hostsReader =
           createHostsFileReader(this.includesFile, this.excludesFile);
+      setDecomissionedNMsMetrics();
       printConfiguredHosts();
     } catch (YarnException ex) {
       disableHostsFileReader(ex);
@@ -120,10 +121,16 @@ public class NodesListManager extends AbstractService implements
                   this.conf, includesFile), excludesFile.isEmpty() ? null
               : this.rmContext.getConfigurationProvider()
                   .getConfigurationInputStream(this.conf, excludesFile));
+      setDecomissionedNMsMetrics();
       printConfiguredHosts();
     }
   }
 
+  private void setDecomissionedNMsMetrics() {
+    Set<String> excludeList = hostsReader.getExcludedHosts();
+    ClusterMetrics.getMetrics().setDecommisionedNMs(excludeList.size());
+  }
+
   public boolean isValidNode(String hostName) {
     synchronized (hostsReader) {
       Set<String> hostsList = hostsReader.getHosts();
@@ -190,6 +197,7 @@ public class NodesListManager extends AbstractService implements
           conf.get(YarnConfiguration.DEFAULT_RM_NODES_EXCLUDE_FILE_PATH);
       this.hostsReader =
           createHostsFileReader(this.includesFile, this.excludesFile);
+      setDecomissionedNMsMetrics();
     } catch (IOException ioe2) {
       // Should *never* happen
       this.hostsReader = null;

+ 15 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java

@@ -34,6 +34,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -424,9 +425,22 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
         break;
     }
 
+    // Decomissioned NMs equals to the nodes missing in include list (if
+    // include list not empty) or the nodes listed in excluded list.
+    // DecomissionedNMs as per exclude list is set upfront when the
+    // exclude list is read so that RM restart can also reflect the
+    // decomissionedNMs. Note that RM is still not able to know decomissionedNMs
+    // as per include list after it restarts as they are known when those nodes
+    // come for registration.
+    // DecomissionedNMs as per include list is incremented in this transition.
     switch (finalState) {
     case DECOMMISSIONED:
-      metrics.incrDecommisionedNMs();
+      Set<String> ecludedHosts =
+          context.getNodesListManager().getHostsReader().getExcludedHosts();
+      if (!ecludedHosts.contains(hostName)
+          && !ecludedHosts.contains(NetUtils.normalizeHostName(hostName))) {
+        metrics.incrDecommisionedNMs();
+      }
       break;
     case LOST:
       metrics.incrNumLostNMs();

+ 8 - 6
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

@@ -50,7 +50,6 @@ import org.apache.hadoop.yarn.api.records.QueueState;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
-import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
@@ -1410,12 +1409,14 @@ public class LeafQueue implements CSQueue {
       FiCaSchedulerApp application, FiCaSchedulerNode node, RMContainer rmContainer, 
       ContainerStatus containerStatus, RMContainerEventType event, CSQueue childQueue) {
     if (application != null) {
+
+      boolean removed = false;
+
       // Careful! Locking order is important!
       synchronized (this) {
 
         Container container = rmContainer.getContainer();
 
-        boolean removed = false;
         // Inform the application & the node
         // Note: It's safe to assume that all state changes to RMContainer
         // happen under scheduler's lock... 
@@ -1441,13 +1442,14 @@ public class LeafQueue implements CSQueue {
               " absoluteUsedCapacity=" + getAbsoluteUsedCapacity() +
               " used=" + usedResources +
               " cluster=" + clusterResource);
-          // Inform the parent queue
-          getParent().completedContainer(clusterResource, application,
-              node, rmContainer, null, event, this);
         }
       }
 
-
+      if (removed) {
+        // Inform the parent queue _outside_ of the leaf-queue lock
+        getParent().completedContainer(clusterResource, application, node,
+          rmContainer, null, event, this);
+      }
     }
   }
 

+ 5 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java

@@ -30,6 +30,7 @@ import java.util.List;
 
 import junit.framework.Assert;
 
+import org.apache.hadoop.util.HostsFileReader;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
@@ -102,6 +103,10 @@ public class TestRMNodeTransitions {
     rmContext =
         new RMContextImpl(rmDispatcher, null, null, null,
             mock(DelegationTokenRenewer.class), null, null, null, null, null);
+    NodesListManager nodesListManager = mock(NodesListManager.class);
+    HostsFileReader reader = mock(HostsFileReader.class);
+    when(nodesListManager.getHostsReader()).thenReturn(reader);
+    ((RMContextImpl) rmContext).setNodesListManager(nodesListManager);
     scheduler = mock(YarnScheduler.class);
     doAnswer(
         new Answer<Void>() {

+ 63 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java

@@ -23,6 +23,8 @@ import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 
+import java.io.File;
+import java.io.FileOutputStream;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
@@ -38,7 +40,9 @@ import java.util.Set;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.SaslRpcServer.AuthMethod;
 import org.apache.hadoop.security.SecurityUtil;
@@ -90,12 +94,16 @@ import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.log4j.Level;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
+import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
 public class TestRMRestart {
 
+  private final static File TEMP_DIR = new File(System.getProperty(
+    "test.build.data", "/tmp"), "decommision");
+  private File hostFile = new File(TEMP_DIR + File.separator + "hostFile.txt");
   private YarnConfiguration conf;
 
   // Fake rmAddr for token-renewal
@@ -113,6 +121,11 @@ public class TestRMRestart {
     Assert.assertTrue(YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS > 1);
   }
 
+  @After
+  public void tearDown() {
+    TEMP_DIR.delete();
+  }
+
   @SuppressWarnings("rawtypes")
   @Test (timeout=180000)
   public void testRMRestart() throws Exception {
@@ -1666,6 +1679,56 @@ public class TestRMRestart {
         appsCompleted + appsCompletedCarryOn);
   }
 
+  @Test
+  public void testDecomissionedNMsMetricsOnRMRestart() throws Exception {
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.set(YarnConfiguration.RM_NODES_EXCLUDE_FILE_PATH,
+      hostFile.getAbsolutePath());
+    writeToHostsFile("");
+    MockRM rm1 = new MockRM(conf);
+    rm1.start();
+    rm1.registerNode("localhost:1234", 8000);
+    rm1.registerNode("host2:1234", 8000);
+    Assert
+      .assertEquals(0, ClusterMetrics.getMetrics().getNumDecommisionedNMs());
+    String ip = NetUtils.normalizeHostName("localhost");
+    // Add 2 hosts to exclude list.
+    writeToHostsFile("host2", ip);
+
+    // refresh nodes
+    rm1.getNodesListManager().refreshNodes(conf);
+    Assert
+      .assertEquals(2, ClusterMetrics.getMetrics().getNumDecommisionedNMs());
+
+    // restart RM.
+    MockRM rm2 = new MockRM(conf);
+    rm2.start();
+    Assert
+      .assertEquals(2, ClusterMetrics.getMetrics().getNumDecommisionedNMs());
+    rm1.stop();
+    rm2.stop();
+  }
+
+  private void writeToHostsFile(String... hosts) throws IOException {
+    if (!hostFile.exists()) {
+      TEMP_DIR.mkdirs();
+      hostFile.createNewFile();
+    }
+    FileOutputStream fStream = null;
+    try {
+      fStream = new FileOutputStream(hostFile);
+      for (int i = 0; i < hosts.length; i++) {
+        fStream.write(hosts[i].getBytes());
+        fStream.write(System.getProperty("line.separator").getBytes());
+      }
+    } finally {
+      if (fStream != null) {
+        IOUtils.closeStream(fStream);
+        fStream = null;
+      }
+    }
+  }
+
   public class TestMemoryRMStateStore extends MemoryRMStateStore {
     int count = 0;
     public int updateApp = 0;

+ 1 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java

@@ -150,7 +150,6 @@ public class TestResourceTrackerService {
     MockNM nm3 = rm.registerNode("localhost:4433", 1024);
 
     int metricCount = ClusterMetrics.getMetrics().getNumDecommisionedNMs();
-
     NodeHeartbeatResponse nodeHeartbeat = nm1.nodeHeartbeat(true);
     Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction()));
     nodeHeartbeat = nm2.nodeHeartbeat(true);
@@ -161,18 +160,17 @@ public class TestResourceTrackerService {
     writeToHostsFile("host2", ip);
 
     rm.getNodesListManager().refreshNodes(conf);
+    checkDecommissionedNMCount(rm, metricCount + 2);
 
     nodeHeartbeat = nm1.nodeHeartbeat(true);
     Assert.assertTrue(NodeAction.NORMAL.equals(nodeHeartbeat.getNodeAction()));
     nodeHeartbeat = nm2.nodeHeartbeat(true);
     Assert.assertTrue("The decommisioned metrics are not updated",
         NodeAction.SHUTDOWN.equals(nodeHeartbeat.getNodeAction()));
-    checkDecommissionedNMCount(rm, ++metricCount);
 
     nodeHeartbeat = nm3.nodeHeartbeat(true);
     Assert.assertTrue("The decommisioned metrics are not updated",
         NodeAction.SHUTDOWN.equals(nodeHeartbeat.getNodeAction()));
-    checkDecommissionedNMCount(rm, ++metricCount);
   }
 
   /**