Parcourir la source

Merge trunk r1595301 to branch

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/fs-encryption@1595303 13f79535-47bb-0310-9956-ffa450edef68
Andrew Wang il y a 11 ans
Parent
commit
1251d6ab72
100 fichiers modifiés avec 5967 ajouts et 506 suppressions
  1. 5 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 12 6
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyShell.java
  3. 15 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java
  4. 134 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclUtil.java
  5. 21 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java
  6. 5 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/ScopedAclEntries.java
  7. 27 90
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/AclCommands.java
  8. 1 46
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Ls.java
  9. 65 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java
  10. 14 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ShellBasedUnixGroupsMapping.java
  11. 8 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
  12. 24 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyShell.java
  13. 19 3
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/IdUserGroup.java
  14. 45 0
      hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/nfs/nfs3/TestIdUserGroup.java
  15. 31 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  16. 12 0
      hadoop-hdfs-project/hadoop-hdfs/src/CMakeLists.txt
  17. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/config.h.cmake
  18. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
  19. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  20. 68 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveIterator.java
  21. 77 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/FsAclPermission.java
  22. 3 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
  23. 75 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSecretManager.java
  24. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java
  25. 15 17
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  26. 23 27
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/InvalidateBlocks.java
  27. 8 43
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclStorage.java
  28. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclTransformation.java
  29. 79 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java
  30. 8 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CheckpointConf.java
  31. 20 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  32. 19 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
  33. 381 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
  34. 213 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
  35. 62 19
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  36. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java
  37. 6 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
  38. 56 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorageRetentionManager.java
  39. 5 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
  40. 16 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
  41. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java
  42. 13 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiff.java
  43. 52 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java
  44. 20 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiff.java
  45. 9 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/Snapshot.java
  46. 72 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotFSImageFormat.java
  47. 17 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
  48. 8 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CacheAdmin.java
  49. 11 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSHAAdmin.java
  50. 172 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/DelimitedImageVisitor.java
  51. 36 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/DepthCounter.java
  52. 193 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FileDistributionVisitor.java
  53. 83 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoader.java
  54. 821 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
  55. 212 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageVisitor.java
  56. 111 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/IndentedImageVisitor.java
  57. 178 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/LsImageVisitor.java
  58. 118 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/NameDistributionVisitor.java
  59. 274 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewer.java
  60. 109 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TextWriterImageVisitor.java
  61. 88 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/XmlImageVisitor.java
  62. 10 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
  63. 804 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/test/vecsum.c
  64. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/test_libhdfs_threaded.c
  65. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
  66. 7 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dfs-dust.js
  67. 30 20
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithNodeGroup.java
  68. 5 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/AclTestHelpers.java
  69. 52 34
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSAclBaseTest.java
  70. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java
  71. 43 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
  72. 6 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithAcl.java
  73. 9 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java
  74. 28 28
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestAclWithSnapshot.java
  75. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdmin.java
  76. 65 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdminMiniCluster.java
  77. 24 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testCacheAdminConf.xml
  78. 8 0
      hadoop-mapreduce-project/CHANGES.txt
  79. 4 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/LocalContainerLauncher.java
  80. 5 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Merger.java
  81. 0 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/apt/MapredAppMasterRest.apt.vm
  82. 4 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/site/apt/HistoryServerRest.apt.vm
  83. 5 2
      hadoop-project/src/site/site.xml
  84. 27 7
      hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/CopyListing.java
  85. 153 0
      hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/CopyListingFileStatus.java
  86. 6 1
      hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCp.java
  87. 1 0
      hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java
  88. 4 2
      hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptionSwitch.java
  89. 1 1
      hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java
  90. 24 35
      hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/SimpleCopyListing.java
  91. 3 3
      hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyCommitter.java
  92. 10 5
      hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java
  93. 8 7
      hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/UniformSizeInputFormat.java
  94. 3 2
      hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/lib/DynamicInputChunk.java
  95. 2 2
      hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/lib/DynamicInputFormat.java
  96. 80 3
      hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/DistCpUtils.java
  97. 11 11
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/StubContext.java
  98. 2 3
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestCopyListing.java
  99. 329 0
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpWithAcls.java
  100. 1 2
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestFileBasedCopyListing.java

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

@@ -328,6 +328,8 @@ Trunk (Unreleased)
 
     HADOOP-10583. bin/hadoop key throws NPE with no args and assorted other fixups. (clamb via tucu)
 
+    HADOOP-10586. KeyShell doesn't allow setting Options via CLI. (clamb via tucu)
+
   OPTIMIZATIONS
 
     HADOOP-7761. Improve the performance of raw comparisons. (todd)
@@ -476,6 +478,9 @@ Release 2.5.0 - UNRELEASED
     HADOOP-10585. Retry polices ignore interrupted exceptions (Daryn Sharp via
     jeagles)
 
+    HADOOP-10401. ShellBasedUnixGroupsMapping#getGroups does not always return
+    primary group first (Akira AJISAKA via Colin Patrick McCabe)
+
 Release 2.4.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 12 - 6
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyShell.java

@@ -89,6 +89,8 @@ public class KeyShell extends Configured implements Tool {
    * @throws IOException
    */
   private int init(String[] args) throws IOException {
+    final Options options = KeyProvider.options(getConf());
+
     for (int i = 0; i < args.length; i++) { // parse command line
       boolean moreTokens = (i < args.length - 1);
       if (args[i].equals("create")) {
@@ -97,7 +99,7 @@ public class KeyShell extends Configured implements Tool {
           keyName = args[++i];
         }
 
-        command = new CreateCommand(keyName);
+        command = new CreateCommand(keyName, options);
         if ("--help".equals(keyName)) {
           printKeyShellUsage();
           return -1;
@@ -127,9 +129,11 @@ public class KeyShell extends Configured implements Tool {
       } else if ("list".equals(args[i])) {
         command = new ListCommand();
       } else if ("--size".equals(args[i]) && moreTokens) {
-        getConf().set(KeyProvider.DEFAULT_BITLENGTH_NAME, args[++i]);
+        options.setBitLength(Integer.parseInt(args[++i]));
       } else if ("--cipher".equals(args[i]) && moreTokens) {
-        getConf().set(KeyProvider.DEFAULT_CIPHER_NAME, args[++i]);
+        options.setCipher(args[++i]);
+      } else if ("--description".equals(args[i]) && moreTokens) {
+        options.setDescription(args[++i]);
       } else if ("--provider".equals(args[i]) && moreTokens) {
         userSuppliedProvider = true;
         getConf().set(KeyProviderFactory.KEY_PROVIDER_PATH, args[++i]);
@@ -399,6 +403,7 @@ public class KeyShell extends Configured implements Tool {
   private class CreateCommand extends Command {
     public static final String USAGE =
       "create <keyname> [--cipher <cipher>] [--size <size>]\n" +
+      "                     [--description <description>]\n" +
       "                     [--provider <provider>] [--help]";
     public static final String DESC =
       "The create subcommand creates a new key for the name specified\n" +
@@ -408,10 +413,12 @@ public class KeyShell extends Configured implements Tool {
       "The default keysize is 256. You may specify the requested key\n" +
       "length using the --size argument.\n";
 
-    String keyName = null;
+    final String keyName;
+    final Options options;
 
-    public CreateCommand(String keyName) {
+    public CreateCommand(String keyName, Options options) {
       this.keyName = keyName;
+      this.options = options;
     }
 
     public boolean validate() {
@@ -434,7 +441,6 @@ public class KeyShell extends Configured implements Tool {
     public void execute() throws IOException, NoSuchAlgorithmException {
       warnIfTransientProvider();
       try {
-        Options options = KeyProvider.options(getConf());
         provider.createKey(keyName, options);
         out.println(keyName + " has been successfully created.");
         provider.flush();

+ 15 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java

@@ -99,6 +99,21 @@ public class FileStatus implements Writable, Comparable {
     assert (isdir && symlink == null) || !isdir;
   }
 
+  /**
+   * Copy constructor.
+   *
+   * @param other FileStatus to copy
+   */
+  public FileStatus(FileStatus other) throws IOException {
+    // It's important to call the getters here instead of directly accessing the
+    // members.  Subclasses like ViewFsFileStatus can override the getters.
+    this(other.getLen(), other.isDirectory(), other.getReplication(),
+      other.getBlockSize(), other.getModificationTime(), other.getAccessTime(),
+      other.getPermission(), other.getOwner(), other.getGroup(),
+      (other.isSymlink() ? other.getSymlink() : null),
+      other.getPath());
+  }
+
   /**
    * Get the length of this file, in bytes.
    * @return the length of this file, in bytes.

+ 134 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclUtil.java

@@ -0,0 +1,134 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.permission;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import com.google.common.collect.Lists;
+
+/**
+ * AclUtil contains utility methods for manipulating ACLs.
+ */
+@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
+@InterfaceStability.Unstable
+public final class AclUtil {
+
+  /**
+   * Given permissions and extended ACL entries, returns the full logical ACL.
+   *
+   * @param perm FsPermission containing permissions
+   * @param entries List<AclEntry> containing extended ACL entries
+   * @return List<AclEntry> containing full logical ACL
+   */
+  public static List<AclEntry> getAclFromPermAndEntries(FsPermission perm,
+      List<AclEntry> entries) {
+    List<AclEntry> acl = Lists.newArrayListWithCapacity(entries.size() + 3);
+
+    // Owner entry implied by owner permission bits.
+    acl.add(new AclEntry.Builder()
+      .setScope(AclEntryScope.ACCESS)
+      .setType(AclEntryType.USER)
+      .setPermission(perm.getUserAction())
+      .build());
+
+    // All extended access ACL entries.
+    boolean hasAccessAcl = false;
+    Iterator<AclEntry> entryIter = entries.iterator();
+    AclEntry curEntry = null;
+    while (entryIter.hasNext()) {
+      curEntry = entryIter.next();
+      if (curEntry.getScope() == AclEntryScope.DEFAULT) {
+        break;
+      }
+      hasAccessAcl = true;
+      acl.add(curEntry);
+    }
+
+    // Mask entry implied by group permission bits, or group entry if there is
+    // no access ACL (only default ACL).
+    acl.add(new AclEntry.Builder()
+      .setScope(AclEntryScope.ACCESS)
+      .setType(hasAccessAcl ? AclEntryType.MASK : AclEntryType.GROUP)
+      .setPermission(perm.getGroupAction())
+      .build());
+
+    // Other entry implied by other bits.
+    acl.add(new AclEntry.Builder()
+      .setScope(AclEntryScope.ACCESS)
+      .setType(AclEntryType.OTHER)
+      .setPermission(perm.getOtherAction())
+      .build());
+
+    // Default ACL entries.
+    if (curEntry != null && curEntry.getScope() == AclEntryScope.DEFAULT) {
+      acl.add(curEntry);
+      while (entryIter.hasNext()) {
+        acl.add(entryIter.next());
+      }
+    }
+
+    return acl;
+  }
+
+  /**
+   * Translates the given permission bits to the equivalent minimal ACL.
+   *
+   * @param perm FsPermission to translate
+   * @return List<AclEntry> containing exactly 3 entries representing the owner,
+   *   group and other permissions
+   */
+  public static List<AclEntry> getMinimalAcl(FsPermission perm) {
+    return Lists.newArrayList(
+      new AclEntry.Builder()
+        .setScope(AclEntryScope.ACCESS)
+        .setType(AclEntryType.USER)
+        .setPermission(perm.getUserAction())
+        .build(),
+      new AclEntry.Builder()
+        .setScope(AclEntryScope.ACCESS)
+        .setType(AclEntryType.GROUP)
+        .setPermission(perm.getGroupAction())
+        .build(),
+      new AclEntry.Builder()
+        .setScope(AclEntryScope.ACCESS)
+        .setType(AclEntryType.OTHER)
+        .setPermission(perm.getOtherAction())
+        .build());
+  }
+
+  /**
+   * Checks if the given entries represent a minimal ACL (contains exactly 3
+   * entries).
+   *
+   * @param entries List<AclEntry> entries to check
+   * @return boolean true if the entries represent a minimal ACL
+   */
+  public static boolean isMinimalAcl(List<AclEntry> entries) {
+    return entries.size() == 3;
+  }
+
+  /**
+   * There is no reason to instantiate this class.
+   */
+  private AclUtil() {
+  }
+}

+ 21 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java

@@ -158,6 +158,17 @@ public class FsPermission implements Writable {
     return (short)s;
   }
 
+  /**
+   * Encodes the object to a short.  Unlike {@link #toShort()}, this method may
+   * return values outside the fixed range 00000 - 01777 if extended features
+   * are encoded into this permission, such as the ACL bit.
+   *
+   * @return short extended short representation of this permission
+   */
+  public short toExtendedShort() {
+    return toShort();
+  }
+
   @Override
   public boolean equals(Object obj) {
     if (obj instanceof FsPermission) {
@@ -273,6 +284,16 @@ public class FsPermission implements Writable {
     return stickyBit;
   }
 
+  /**
+   * Returns true if there is also an ACL (access control list).
+   *
+   * @return boolean true if there is also an ACL (access control list).
+   */
+  public boolean getAclBit() {
+    // File system subclasses that support the ACL bit would override this.
+    return false;
+  }
+
   /** Set the user file creation mask (umask) */
   public static void setUMask(Configuration conf, FsPermission umask) {
     conf.set(UMASK_LABEL, String.format("%1$03o", umask.toShort()));

+ 5 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ScopedAclEntries.java → hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/ScopedAclEntries.java

@@ -15,12 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs.server.namenode;
+package org.apache.hadoop.fs.permission;
 
 import java.util.Collections;
 import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclEntryScope;
 
@@ -28,8 +29,9 @@ import org.apache.hadoop.fs.permission.AclEntryScope;
  * Groups a list of ACL entries into separate lists for access entries vs.
  * default entries.
  */
-@InterfaceAudience.Private
-final class ScopedAclEntries {
+@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
+@InterfaceStability.Unstable
+public final class ScopedAclEntries {
   private static final int PIVOT_NOT_FOUND = -1;
 
   private final List<AclEntry> accessEntries;

+ 27 - 90
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/AclCommands.java

@@ -18,7 +18,7 @@
 package org.apache.hadoop.fs.shell;
 
 import java.io.IOException;
-import java.util.Iterator;
+import java.util.Collections;
 import java.util.LinkedList;
 import java.util.List;
 
@@ -31,8 +31,10 @@ import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclEntryScope;
 import org.apache.hadoop.fs.permission.AclEntryType;
 import org.apache.hadoop.fs.permission.AclStatus;
+import org.apache.hadoop.fs.permission.AclUtil;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.permission.ScopedAclEntries;
 
 /**
  * Acl related operations
@@ -75,84 +77,43 @@ class AclCommands extends FsCommand {
 
     @Override
     protected void processPath(PathData item) throws IOException {
-      AclStatus aclStatus = item.fs.getAclStatus(item.path);
       out.println("# file: " + item);
-      out.println("# owner: " + aclStatus.getOwner());
-      out.println("# group: " + aclStatus.getGroup());
-      List<AclEntry> entries = aclStatus.getEntries();
-      if (aclStatus.isStickyBit()) {
-        String stickyFlag = "T";
-        for (AclEntry aclEntry : entries) {
-          if (aclEntry.getType() == AclEntryType.OTHER
-              && aclEntry.getScope() == AclEntryScope.ACCESS
-              && aclEntry.getPermission().implies(FsAction.EXECUTE)) {
-            stickyFlag = "t";
-            break;
-          }
-        }
-        out.println("# flags: --" + stickyFlag);
-      }
-
+      out.println("# owner: " + item.stat.getOwner());
+      out.println("# group: " + item.stat.getGroup());
       FsPermission perm = item.stat.getPermission();
-      if (entries.isEmpty()) {
-        printMinimalAcl(perm);
-      } else {
-        printExtendedAcl(perm, entries);
+      if (perm.getStickyBit()) {
+        out.println("# flags: --" +
+          (perm.getOtherAction().implies(FsAction.EXECUTE) ? "t" : "T"));
       }
 
+      List<AclEntry> entries = perm.getAclBit() ?
+        item.fs.getAclStatus(item.path).getEntries() :
+        Collections.<AclEntry>emptyList();
+      ScopedAclEntries scopedEntries = new ScopedAclEntries(
+        AclUtil.getAclFromPermAndEntries(perm, entries));
+      printAclEntriesForSingleScope(scopedEntries.getAccessEntries());
+      printAclEntriesForSingleScope(scopedEntries.getDefaultEntries());
       out.println();
     }
 
     /**
-     * Prints an extended ACL, including all extended ACL entries and also the
-     * base entries implied by the permission bits.
+     * Prints all the ACL entries in a single scope.
      *
-     * @param perm FsPermission of file
      * @param entries List<AclEntry> containing ACL entries of file
      */
-    private void printExtendedAcl(FsPermission perm, List<AclEntry> entries) {
-      // Print owner entry implied by owner permission bits.
-      out.println(new AclEntry.Builder()
-        .setScope(AclEntryScope.ACCESS)
-        .setType(AclEntryType.USER)
-        .setPermission(perm.getUserAction())
-        .build());
-
-      // Print all extended access ACL entries.
-      boolean hasAccessAcl = false;
-      Iterator<AclEntry> entryIter = entries.iterator();
-      AclEntry curEntry = null;
-      while (entryIter.hasNext()) {
-        curEntry = entryIter.next();
-        if (curEntry.getScope() == AclEntryScope.DEFAULT) {
-          break;
-        }
-        hasAccessAcl = true;
-        printExtendedAclEntry(curEntry, perm.getGroupAction());
+    private void printAclEntriesForSingleScope(List<AclEntry> entries) {
+      if (entries.isEmpty()) {
+        return;
       }
-
-      // Print mask entry implied by group permission bits, or print group entry
-      // if there is no access ACL (only default ACL).
-      out.println(new AclEntry.Builder()
-        .setScope(AclEntryScope.ACCESS)
-        .setType(hasAccessAcl ? AclEntryType.MASK : AclEntryType.GROUP)
-        .setPermission(perm.getGroupAction())
-        .build());
-
-      // Print other entry implied by other bits.
-      out.println(new AclEntry.Builder()
-        .setScope(AclEntryScope.ACCESS)
-        .setType(AclEntryType.OTHER)
-        .setPermission(perm.getOtherAction())
-        .build());
-
-      // Print default ACL entries.
-      if (curEntry != null && curEntry.getScope() == AclEntryScope.DEFAULT) {
-        out.println(curEntry);
-        // ACL sort order guarantees default mask is the second-to-last entry.
+      if (AclUtil.isMinimalAcl(entries)) {
+        for (AclEntry entry: entries) {
+          out.println(entry);
+        }
+      } else {
+        // ACL sort order guarantees mask is the second-to-last entry.
         FsAction maskPerm = entries.get(entries.size() - 2).getPermission();
-        while (entryIter.hasNext()) {
-          printExtendedAclEntry(entryIter.next(), maskPerm);
+        for (AclEntry entry: entries) {
+          printExtendedAclEntry(entry, maskPerm);
         }
       }
     }
@@ -180,30 +141,6 @@ class AclCommands extends FsCommand {
         out.println(entry);
       }
     }
-
-    /**
-     * Prints a minimal ACL, consisting of exactly 3 ACL entries implied by the
-     * permission bits.
-     *
-     * @param perm FsPermission of file
-     */
-    private void printMinimalAcl(FsPermission perm) {
-      out.println(new AclEntry.Builder()
-        .setScope(AclEntryScope.ACCESS)
-        .setType(AclEntryType.USER)
-        .setPermission(perm.getUserAction())
-        .build());
-      out.println(new AclEntry.Builder()
-        .setScope(AclEntryScope.ACCESS)
-        .setType(AclEntryType.GROUP)
-        .setPermission(perm.getGroupAction())
-        .build());
-      out.println(new AclEntry.Builder()
-        .setScope(AclEntryScope.ACCESS)
-        .setType(AclEntryType.OTHER)
-        .setPermission(perm.getOtherAction())
-        .build());
-    }
   }
 
   /**

+ 1 - 46
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Ls.java

@@ -31,8 +31,6 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.ipc.RpcNoSuchMethodException;
 
 import com.google.common.collect.Sets;
 
@@ -116,7 +114,7 @@ class Ls extends FsCommand {
     FileStatus stat = item.stat;
     String line = String.format(lineFormat,
         (stat.isDirectory() ? "d" : "-"),
-        stat.getPermission() + (hasAcl(item) ? "+" : " "),
+        stat.getPermission() + (stat.getPermission().getAclBit() ? "+" : " "),
         (stat.isFile() ? stat.getReplication() : "-"),
         stat.getOwner(),
         stat.getGroup(),
@@ -153,49 +151,6 @@ class Ls extends FsCommand {
     lineFormat = fmt.toString();
   }
 
-  /**
-   * Calls getAclStatus to determine if the given item has an ACL.  For
-   * compatibility, this method traps errors caused by the RPC method missing
-   * from the server side.  This would happen if the client was connected to an
-   * old NameNode that didn't have the ACL APIs.  This method also traps the
-   * case of the client-side FileSystem not implementing the ACL APIs.
-   * FileSystem instances that do not support ACLs are remembered.  This
-   * prevents the client from sending multiple failing RPC calls during a
-   * recursive ls.
-   *
-   * @param item PathData item to check
-   * @return boolean true if item has an ACL
-   * @throws IOException if there is a failure
-   */
-  private boolean hasAcl(PathData item) throws IOException {
-    FileSystem fs = item.fs;
-    if (aclNotSupportedFsSet.contains(fs.getUri())) {
-      // This FileSystem failed to run the ACL API in an earlier iteration.
-      return false;
-    }
-    try {
-      return !fs.getAclStatus(item.path).getEntries().isEmpty();
-    } catch (RemoteException e) {
-      // If this is a RpcNoSuchMethodException, then the client is connected to
-      // an older NameNode that doesn't support ACLs.  Keep going.
-      IOException e2 = e.unwrapRemoteException(RpcNoSuchMethodException.class);
-      if (!(e2 instanceof RpcNoSuchMethodException)) {
-        throw e;
-      }
-    } catch (IOException e) {
-      // The NameNode supports ACLs, but they are not enabled.  Keep going.
-      String message = e.getMessage();
-      if (message != null && !message.contains("ACLs has been disabled")) {
-        throw e;
-      }
-    } catch (UnsupportedOperationException e) {
-      // The underlying FileSystem doesn't implement ACLs.  Keep going.
-    }
-    // Remember that this FileSystem cannot support ACLs.
-    aclNotSupportedFsSet.add(fs.getUri());
-    return false;
-  }
-
   private int maxLength(int n, Object value) {
     return Math.max(n, (value != null) ? String.valueOf(value).length() : 0);
   }

+ 65 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.ha;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Map;
 
 import org.apache.commons.cli.Options;
@@ -33,6 +34,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.HAServiceProtocol.StateChangeRequestInfo;
 import org.apache.hadoop.ha.HAServiceProtocol.RequestSource;
 import org.apache.hadoop.util.Tool;
@@ -66,7 +68,7 @@ public abstract class HAAdmin extends Configured implements Tool {
   protected final static Map<String, UsageInfo> USAGE =
     ImmutableMap.<String, UsageInfo>builder()
     .put("-transitionToActive",
-        new UsageInfo("<serviceId>", "Transitions the service into Active state"))
+        new UsageInfo(" <serviceId> [--"+FORCEACTIVE+"]", "Transitions the service into Active state"))
     .put("-transitionToStandby",
         new UsageInfo("<serviceId>", "Transitions the service into Standby state"))
     .put("-failover",
@@ -100,6 +102,10 @@ public abstract class HAAdmin extends Configured implements Tool {
   }
 
   protected abstract HAServiceTarget resolveTarget(String string);
+  
+  protected Collection<String> getTargetIds(String targetNodeToActivate) {
+    return Arrays.asList(new String[]{targetNodeToActivate});
+  }
 
   protected String getUsageString() {
     return "Usage: HAAdmin";
@@ -133,6 +139,11 @@ public abstract class HAAdmin extends Configured implements Tool {
       printUsage(errOut, "-transitionToActive");
       return -1;
     }
+    /*  returns true if other target node is active or some exception occurred 
+        and forceActive was not set  */
+    if(isOtherTargetNodeActive(argv[0], cmd.hasOption(FORCEACTIVE))) {
+      return -1;
+    }
     HAServiceTarget target = resolveTarget(argv[0]);
     if (!checkManualStateManagementOK(target)) {
       return -1;
@@ -142,7 +153,48 @@ public abstract class HAAdmin extends Configured implements Tool {
     HAServiceProtocolHelper.transitionToActive(proto, createReqInfo());
     return 0;
   }
-
+  
+  /**
+   * Checks whether other target node is active or not
+   * @param targetNodeToActivate
+   * @return true if other target node is active or some other exception 
+   * occurred and forceActive was set otherwise false
+   * @throws IOException
+   */
+  private boolean isOtherTargetNodeActive(String targetNodeToActivate, boolean forceActive)
+      throws IOException  {
+    Collection<String> targetIds = getTargetIds(targetNodeToActivate);
+    if(targetIds == null) {
+      errOut.println("transitionToActive: No target node in the "
+          + "current configuration");
+      printUsage(errOut, "-transitionToActive");
+      return true;
+    }
+    targetIds.remove(targetNodeToActivate);
+    for(String targetId : targetIds) {
+      HAServiceTarget target = resolveTarget(targetId);
+      if (!checkManualStateManagementOK(target)) {
+        return true;
+      }
+      try {
+        HAServiceProtocol proto = target.getProxy(getConf(), 5000);
+        if(proto.getServiceStatus().getState() == HAServiceState.ACTIVE) {
+          errOut.println("transitionToActive: Node " +  targetId +" is already active");
+          printUsage(errOut, "-transitionToActive");
+          return true;
+        }
+      } catch (Exception e) {
+        //If forceActive switch is false then return true
+        if(!forceActive) {
+          errOut.println("Unexpected error occurred  " + e.getMessage());
+          printUsage(errOut, "-transitionToActive");
+          return true; 
+        }
+      }
+    }
+    return false;
+  }
+  
   private int transitionToStandby(final CommandLine cmd)
       throws IOException, ServiceFailedException {
     String[] argv = cmd.getArgs();
@@ -364,6 +416,9 @@ public abstract class HAAdmin extends Configured implements Tool {
     if ("-failover".equals(cmd)) {
       addFailoverCliOpts(opts);
     }
+    if("-transitionToActive".equals(cmd)) {
+      addTransitionToActiveCliOpts(opts);
+    }
     // Mutative commands take FORCEMANUAL option
     if ("-transitionToActive".equals(cmd) ||
         "-transitionToStandby".equals(cmd) ||
@@ -433,6 +488,14 @@ public abstract class HAAdmin extends Configured implements Tool {
     // that change state.
   }
   
+  /**
+   * Add CLI options which are specific to the transitionToActive command and
+   * no others.
+   */
+  private void addTransitionToActiveCliOpts(Options transitionToActiveCliOpts) {
+    transitionToActiveCliOpts.addOption(FORCEACTIVE, false, "force active");
+  }
+  
   private CommandLine parseOpts(String cmdName, Options opts, String[] argv) {
     try {
       // Strip off the first arg, since that's just the command name

+ 14 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ShellBasedUnixGroupsMapping.java

@@ -74,7 +74,8 @@ public class ShellBasedUnixGroupsMapping
    * Get the current user's group list from Unix by running the command 'groups'
    * NOTE. For non-existing user it will return EMPTY list
    * @param user user name
-   * @return the groups list that the <code>user</code> belongs to
+   * @return the groups list that the <code>user</code> belongs to. The primary
+   *         group is returned first.
    * @throws IOException if encounter any error when running the command
    */
   private static List<String> getUnixGroups(final String user) throws IOException {
@@ -84,6 +85,7 @@ public class ShellBasedUnixGroupsMapping
     } catch (ExitCodeException e) {
       // if we didn't get the group - just return empty list;
       LOG.warn("got exception trying to get groups for user " + user, e);
+      return new LinkedList<String>();
     }
     
     StringTokenizer tokenizer =
@@ -92,6 +94,17 @@ public class ShellBasedUnixGroupsMapping
     while (tokenizer.hasMoreTokens()) {
       groups.add(tokenizer.nextToken());
     }
+
+    // remove duplicated primary group
+    if (!Shell.WINDOWS) {
+      for (int i = 1; i < groups.size(); i++) {
+        if (groups.get(i).equals(groups.get(0))) {
+          groups.remove(i);
+          break;
+        }
+      }
+    }
+
     return groups;
   }
 }

+ 8 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java

@@ -132,11 +132,17 @@ abstract public class Shell {
                     : new String[]{"bash", "-c", "groups"};
   }
 
-  /** a Unix command to get a given user's groups list */
+  /**
+   * a Unix command to get a given user's groups list.
+   * If the OS is not WINDOWS, the command will get the user's primary group
+   * first and finally get the groups list which includes the primary group.
+   * i.e. the user's primary group will be included twice.
+   */
   public static String[] getGroupsForUserCommand(final String user) {
     //'groups username' command return is non-consistent across different unixes
     return (WINDOWS)? new String[] { WINUTILS, "groups", "-F", "\"" + user + "\""}
-                    : new String [] {"bash", "-c", "id -Gn " + user};
+                    : new String [] {"bash", "-c", "id -gn " + user
+                                     + "&& id -Gn " + user};
   }
 
   /** a Unix command to get a given netgroup's user list */

+ 24 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyShell.java

@@ -111,6 +111,30 @@ public class TestKeyShell {
     assertFalse(outContent.toString(), outContent.toString().contains("key1"));
   }
   
+  /* HADOOP-10586 KeyShell didn't allow -description. */
+  @Test
+  public void testKeySuccessfulCreationWithDescription() throws Exception {
+    outContent.reset();
+    String[] args1 = {"create", "key1", "--provider",
+                      "jceks://file" + tmpDir + "/keystore.jceks",
+                      "--description", "someDescription"};
+    int rc = 0;
+    KeyShell ks = new KeyShell();
+    ks.setConf(new Configuration());
+    rc = ks.run(args1);
+    assertEquals(0, rc);
+    assertTrue(outContent.toString().contains("key1 has been successfully " +
+        "created."));
+
+    outContent.reset();
+    String[] args2a = {"list", "--metadata", "--provider",
+                      "jceks://file" + tmpDir + "/keystore.jceks"};
+    rc = ks.run(args2a);
+    assertEquals(0, rc);
+    assertTrue(outContent.toString().contains("description"));
+    assertTrue(outContent.toString().contains("someDescription"));
+  }
+
   @Test
   public void testInvalidKeySize() throws Exception {
     String[] args1 = {"create", "key1", "--size", "56", "--provider", 

+ 19 - 3
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/IdUserGroup.java

@@ -113,7 +113,23 @@ public class IdUserGroup {
           "The new entry is to be ignored for the following reason.",
           DUPLICATE_NAME_ID_DEBUG_INFO));
   }
-      
+
+  /**
+   * uid and gid are defined as uint32 in linux. Some systems create
+   * (intended or unintended) <nfsnobody, 4294967294> kind of <name,Id>
+   * mapping, where 4294967294 is 2**32-2 as unsigned int32. As an example,
+   *   https://bugzilla.redhat.com/show_bug.cgi?id=511876.
+   * Because user or group id are treated as Integer (signed integer or int32)
+   * here, the number 4294967294 is out of range. The solution is to convert
+   * uint32 to int32, so to map the out-of-range ID to the negative side of
+   * Integer, e.g. 4294967294 maps to -2 and 4294967295 maps to -1.
+   */
+  private static Integer parseId(final String idStr) {
+    Long longVal = Long.parseLong(idStr);
+    int intVal = longVal.intValue();
+    return Integer.valueOf(intVal);
+  }
+  
   /**
    * Get the whole list of users and groups and save them in the maps.
    * @throws IOException 
@@ -134,8 +150,8 @@ public class IdUserGroup {
         }
         LOG.debug("add to " + mapName + "map:" + nameId[0] + " id:" + nameId[1]);
         // HDFS can't differentiate duplicate names with simple authentication
-        final Integer key = Integer.valueOf(nameId[1]);
-        final String value = nameId[0];        
+        final Integer key = parseId(nameId[1]);        
+        final String value = nameId[0];
         if (map.containsKey(key)) {
           final String prevValue = map.get(key);
           if (value.equals(prevValue)) {

+ 45 - 0
hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/nfs/nfs3/TestIdUserGroup.java

@@ -66,6 +66,51 @@ public class TestIdUserGroup {
     assertEquals("mapred3", gMap.get(498));
   }
 
+  @Test
+  public void testIdOutOfIntegerRange() throws IOException {
+    String GET_ALL_USERS_CMD = "echo \""
+        + "nfsnobody:x:4294967294:4294967294:Anonymous NFS User:/var/lib/nfs:/sbin/nologin\n"
+        + "nfsnobody1:x:4294967295:4294967295:Anonymous NFS User:/var/lib/nfs1:/sbin/nologin\n"
+        + "maxint:x:2147483647:2147483647:Grid Distributed File System:/home/maxint:/bin/bash\n"
+        + "minint:x:2147483648:2147483648:Grid Distributed File System:/home/minint:/bin/bash\n"
+        + "archivebackup:*:1031:4294967294:Archive Backup:/home/users/archivebackup:/bin/sh\n"
+        + "hdfs:x:11501:10787:Grid Distributed File System:/home/hdfs:/bin/bash\n"
+        + "daemon:x:2:2:daemon:/sbin:/sbin/nologin\""
+        + " | cut -d: -f1,3";
+    String GET_ALL_GROUPS_CMD = "echo \""
+        + "hdfs:*:11501:hrt_hdfs\n"
+        + "rpcuser:*:29:\n"
+        + "nfsnobody:*:4294967294:\n"
+        + "nfsnobody1:*:4294967295:\n"
+        + "maxint:*:2147483647:\n"
+        + "minint:*:2147483648:\n"
+        + "mapred3:x:498\"" 
+        + " | cut -d: -f1,3";
+    // Maps for id to name map
+    BiMap<Integer, String> uMap = HashBiMap.create();
+    BiMap<Integer, String> gMap = HashBiMap.create();
+
+    IdUserGroup.updateMapInternal(uMap, "user", GET_ALL_USERS_CMD, ":");
+    assertTrue(uMap.size() == 7);
+    assertEquals("nfsnobody", uMap.get(-2));
+    assertEquals("nfsnobody1", uMap.get(-1));
+    assertEquals("maxint", uMap.get(2147483647));
+    assertEquals("minint", uMap.get(-2147483648));
+    assertEquals("archivebackup", uMap.get(1031));
+    assertEquals("hdfs",uMap.get(11501));
+    assertEquals("daemon", uMap.get(2));
+
+    IdUserGroup.updateMapInternal(gMap, "group", GET_ALL_GROUPS_CMD, ":");
+    assertTrue(gMap.size() == 7);
+    assertEquals("hdfs",gMap.get(11501));
+    assertEquals("rpcuser", gMap.get(29));
+    assertEquals("nfsnobody", gMap.get(-2));
+    assertEquals("nfsnobody1", gMap.get(-1));
+    assertEquals("maxint", gMap.get(2147483647));
+    assertEquals("minint", gMap.get(-2147483648));
+    assertEquals("mapred3", gMap.get(498));
+  }
+
   @Test
   public void testUserUpdateSetting() throws IOException {
     IdUserGroup iug = new IdUserGroup();

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

@@ -356,6 +356,19 @@ Release 2.5.0 - UNRELEASED
 
     HDFS-6186. Pause deletion of blocks when the namenode starts up. (jing9)
 
+    HDFS-6293. Issues with OIV processing PB-based fsimages. (kihwal)
+
+    HDFS-2949. Add check to active state transition to prevent operator-induced
+    split brain. (Rushabh S Shah via kihwal)
+
+    HDFS-6287. Add vecsum test of libhdfs read access times (cmccabe)
+
+    HDFS-5683. Better audit log messages for caching operations.
+    (Abhiraj Butala via wang)
+
+    HDFS-6345. DFS.listCacheDirectives() should allow filtering based on
+    cache directive ID. (wang)
+
   OPTIMIZATIONS
 
     HDFS-6214. Webhdfs has poor throughput for files >2GB (daryn)
@@ -472,6 +485,13 @@ Release 2.5.0 - UNRELEASED
     HDFS-6370. Web UI fails to display in intranet under IE.
     (Haohui Mai via cnauroth)
 
+    HDFS-6381. Fix a typo in INodeReference.java. (Binglin Chang via jing9)
+
+    HDFS-6400. Cannot execute hdfs oiv_legacy. (Akira AJISAKA via kihwal)
+
+    HDFS-6250. Fix test failed in TestBalancerWithNodeGroup.testBalancerWithRackLocality
+    (Binglin Chang and Chen He via junping_du)
+
 Release 2.4.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -538,6 +558,17 @@ Release 2.4.1 - UNRELEASED
     HDFS-6313. WebHdfs may use the wrong NN when configured for multiple HA NNs
     (kihwal)
 
+    HDFS-6326. WebHdfs ACL compatibility is broken. (cnauroth)
+
+    HDFS-6361. TestIdUserGroup.testUserUpdateSetting failed due to out of range
+    nfsnobody Id. (Yongjun Zhang via brandonli)
+
+    HDFS-6362. InvalidateBlocks is inconsistent in usage of DatanodeUuid and
+    StorageID. (Arpit Agarwal)
+
+    HDFS-6402. Suppress findbugs warning for failure to override equals and
+    hashCode in FsAclPermission. (cnauroth)
+
 Release 2.4.0 - 2014-04-07 
 
   INCOMPATIBLE CHANGES

+ 12 - 0
hadoop-hdfs-project/hadoop-hdfs/src/CMakeLists.txt

@@ -62,6 +62,9 @@ endfunction()
 INCLUDE(CheckCSourceCompiles)
 CHECK_C_SOURCE_COMPILES("int main(void) { static __thread int i = 0; return 0; }" HAVE_BETTER_TLS)
 
+# Check to see if we have Intel SSE intrinsics.
+CHECK_C_SOURCE_COMPILES("#include <emmintrin.h>\nint main(void) { __m128d sum0 = _mm_set_pd(0.0,0.0); return 0; }" HAVE_INTEL_SSE_INTRINSICS)
+
 # 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)
@@ -170,6 +173,15 @@ target_link_libraries(test_libhdfs_zerocopy
     pthread
 )
 
+add_executable(test_libhdfs_vecsum
+    main/native/libhdfs/test/vecsum.c
+)
+target_link_libraries(test_libhdfs_vecsum
+    hdfs
+    pthread
+    rt
+)
+
 IF(REQUIRE_LIBWEBHDFS)
     add_subdirectory(contrib/libwebhdfs)
 ENDIF(REQUIRE_LIBWEBHDFS)

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/config.h.cmake

@@ -22,4 +22,6 @@
 
 #cmakedefine HAVE_BETTER_TLS
 
+#cmakedefine HAVE_INTEL_SSE_INTRINSICS
+
 #endif

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs

@@ -49,6 +49,7 @@ function print_usage(){
   echo "  balancer             run a cluster balancing utility"
   echo "  jmxget               get JMX exported values from NameNode or DataNode."
   echo "  oiv                  apply the offline fsimage viewer to an fsimage"
+  echo "  oiv_legacy           apply the offline fsimage viewer to an legacy fsimage"
   echo "  oev                  apply the offline edits viewer to an edits file"
   echo "  fetchdt              fetch a delegation token from the NameNode"
   echo "  getconf              get config values from configuration"
@@ -161,6 +162,8 @@ elif [ "$COMMAND" = "jmxget" ] ; then
   CLASS=org.apache.hadoop.hdfs.tools.JMXGet
 elif [ "$COMMAND" = "oiv" ] ; then
   CLASS=org.apache.hadoop.hdfs.tools.offlineImageViewer.OfflineImageViewerPB
+elif [ "$COMMAND" = "oiv_legacy" ] ; then
+  CLASS=org.apache.hadoop.hdfs.tools.offlineImageViewer.OfflineImageViewer
 elif [ "$COMMAND" = "oev" ] ; then
   CLASS=org.apache.hadoop.hdfs.tools.offlineEditsViewer.OfflineEditsViewer
 elif [ "$COMMAND" = "fetchdt" ] ; then

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

@@ -497,6 +497,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_SECONDARY_NAMENODE_KERBEROS_INTERNAL_SPNEGO_PRINCIPAL_KEY = "dfs.secondary.namenode.kerberos.internal.spnego.principal";
   public static final String  DFS_NAMENODE_NAME_CACHE_THRESHOLD_KEY = "dfs.namenode.name.cache.threshold";
   public static final int     DFS_NAMENODE_NAME_CACHE_THRESHOLD_DEFAULT = 10;
+  public static final String  DFS_NAMENODE_LEGACY_OIV_IMAGE_DIR_KEY = "dfs.namenode.legacy-oiv-image.dir";
   
   public static final String  DFS_NAMESERVICES = "dfs.nameservices";
   public static final String  DFS_NAMESERVICE_ID = "dfs.nameservice.id";

+ 68 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveIterator.java

@@ -23,6 +23,10 @@ import java.io.IOException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.BatchedRemoteIterator;
+import org.apache.hadoop.fs.InvalidRequestException;
+import org.apache.hadoop.ipc.RemoteException;
+
+import com.google.common.base.Preconditions;
 
 /**
  * CacheDirectiveIterator is a remote iterator that iterates cache directives.
@@ -33,7 +37,7 @@ import org.apache.hadoop.fs.BatchedRemoteIterator;
 public class CacheDirectiveIterator
     extends BatchedRemoteIterator<Long, CacheDirectiveEntry> {
 
-  private final CacheDirectiveInfo filter;
+  private CacheDirectiveInfo filter;
   private final ClientProtocol namenode;
 
   public CacheDirectiveIterator(ClientProtocol namenode,
@@ -43,10 +47,72 @@ public class CacheDirectiveIterator
     this.filter = filter;
   }
 
+  private static CacheDirectiveInfo removeIdFromFilter(CacheDirectiveInfo filter) {
+    CacheDirectiveInfo.Builder builder = new CacheDirectiveInfo.Builder(filter);
+    builder.setId(null);
+    return builder.build();
+  }
+
+  /**
+   * Used for compatibility when communicating with a server version that
+   * does not support filtering directives by ID.
+   */
+  private static class SingleEntry implements
+      BatchedEntries<CacheDirectiveEntry> {
+
+    private final CacheDirectiveEntry entry;
+
+    public SingleEntry(final CacheDirectiveEntry entry) {
+      this.entry = entry;
+    }
+
+    @Override
+    public CacheDirectiveEntry get(int i) {
+      if (i > 0) {
+        return null;
+      }
+      return entry;
+    }
+
+    @Override
+    public int size() {
+      return 1;
+    }
+
+    @Override
+    public boolean hasMore() {
+      return false;
+    }
+  }
+
   @Override
   public BatchedEntries<CacheDirectiveEntry> makeRequest(Long prevKey)
       throws IOException {
-    return namenode.listCacheDirectives(prevKey, filter);
+    BatchedEntries<CacheDirectiveEntry> entries = null;
+    try {
+      entries = namenode.listCacheDirectives(prevKey, filter);
+    } catch (IOException e) {
+      if (e.getMessage().contains("Filtering by ID is unsupported")) {
+        // Retry case for old servers, do the filtering client-side
+        long id = filter.getId();
+        filter = removeIdFromFilter(filter);
+        // Using id - 1 as prevId should get us a window containing the id
+        // This is somewhat brittle, since it depends on directives being
+        // returned in order of ascending ID.
+        entries = namenode.listCacheDirectives(id - 1, filter);
+        for (int i=0; i<entries.size(); i++) {
+          CacheDirectiveEntry entry = entries.get(i);
+          if (entry.getInfo().getId().equals((Long)id)) {
+            return new SingleEntry(entry);
+          }
+        }
+        throw new RemoteException(InvalidRequestException.class.getName(),
+            "Did not find requested id " + id);
+      }
+      throw e;
+    }
+    Preconditions.checkNotNull(entries);
+    return entries;
   }
 
   @Override

+ 77 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/FsAclPermission.java

@@ -0,0 +1,77 @@
+/**
+ * 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.protocol;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+/**
+ * HDFS permission subclass used to indicate an ACL is present.  The ACL bit is
+ * not visible directly to users of {@link FsPermission} serialization.  This is
+ * done for backwards compatibility in case any existing clients assume the
+ * value of FsPermission is in a particular range.
+ */
+@InterfaceAudience.Private
+public class FsAclPermission extends FsPermission {
+  private final static short ACL_BIT = 1 << 12;
+  private final boolean aclBit;
+
+  /**
+   * Constructs a new FsAclPermission based on the given FsPermission.
+   *
+   * @param perm FsPermission containing permission bits
+   */
+  public FsAclPermission(FsPermission perm) {
+    super(perm.toShort());
+    aclBit = true;
+  }
+
+  /**
+   * Creates a new FsAclPermission by calling the base class constructor.
+   *
+   * @param perm short containing permission bits
+   */
+  public FsAclPermission(short perm) {
+    super(perm);
+    aclBit = (perm & ACL_BIT) != 0;
+  }
+
+  @Override
+  public short toExtendedShort() {
+    return (short)(toShort() | (aclBit ? ACL_BIT : 0));
+  }
+
+  @Override
+  public boolean getAclBit() {
+    return aclBit;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    // This intentionally delegates to the base class.  This is only overridden
+    // to suppress a FindBugs warning.
+    return super.equals(o);
+  }
+
+  @Override
+  public int hashCode() {
+    // This intentionally delegates to the base class.  This is only overridden
+    // to suppress a FindBugs warning.
+    return super.hashCode();
+  }
+}

+ 3 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java

@@ -57,6 +57,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.hdfs.protocol.DatanodeLocalInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.FsAclPermission;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
@@ -1186,13 +1187,11 @@ public class PBHelper {
   }
   
   public static FsPermissionProto convert(FsPermission p) {
-    if (p == null) return null;
-    return FsPermissionProto.newBuilder().setPerm(p.toShort()).build();
+    return FsPermissionProto.newBuilder().setPerm(p.toExtendedShort()).build();
   }
   
   public static FsPermission convert(FsPermissionProto p) {
-    if (p == null) return null;
-    return new FsPermission((short)p.getPerm());
+    return new FsAclPermission((short)p.getPerm());
   }
   
   

+ 75 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSecretManager.java

@@ -18,9 +18,16 @@
 
 package org.apache.hadoop.hdfs.security.token.delegation;
 
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import com.google.protobuf.ByteString;
+import java.io.DataInput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map.Entry;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -43,13 +50,9 @@ import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager;
 import org.apache.hadoop.security.token.delegation.DelegationKey;
 
-import java.io.DataInput;
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.net.InetSocketAddress;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map.Entry;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.protobuf.ByteString;
 
 /**
  * A HDFS specific delegation token secret manager.
@@ -211,6 +214,18 @@ public class DelegationTokenSecretManager
     }
   }
 
+  /**
+   * Store the current state of the SecretManager for persistence
+   *
+   * @param out Output stream for writing into fsimage.
+   * @param sdPath String storage directory path
+   * @throws IOException
+   */
+  public synchronized void saveSecretManagerStateCompat(DataOutputStream out,
+      String sdPath) throws IOException {
+    serializerCompat.save(out, sdPath);
+  }
+
   public synchronized SecretManagerState saveSecretManagerState() {
     SecretManagerSection s = SecretManagerSection.newBuilder()
         .setCurrentId(currentId)
@@ -406,6 +421,56 @@ public class DelegationTokenSecretManager
       loadCurrentTokens(in);
     }
 
+    private void save(DataOutputStream out, String sdPath) throws IOException {
+      out.writeInt(currentId);
+      saveAllKeys(out, sdPath);
+      out.writeInt(delegationTokenSequenceNumber);
+      saveCurrentTokens(out, sdPath);
+    }
+
+    /**
+     * Private helper methods to save delegation keys and tokens in fsimage
+     */
+    private synchronized void saveCurrentTokens(DataOutputStream out,
+        String sdPath) throws IOException {
+      StartupProgress prog = NameNode.getStartupProgress();
+      Step step = new Step(StepType.DELEGATION_TOKENS, sdPath);
+      prog.beginStep(Phase.SAVING_CHECKPOINT, step);
+      prog.setTotal(Phase.SAVING_CHECKPOINT, step, currentTokens.size());
+      Counter counter = prog.getCounter(Phase.SAVING_CHECKPOINT, step);
+      out.writeInt(currentTokens.size());
+      Iterator<DelegationTokenIdentifier> iter = currentTokens.keySet()
+          .iterator();
+      while (iter.hasNext()) {
+        DelegationTokenIdentifier id = iter.next();
+        id.write(out);
+        DelegationTokenInformation info = currentTokens.get(id);
+        out.writeLong(info.getRenewDate());
+        counter.increment();
+      }
+      prog.endStep(Phase.SAVING_CHECKPOINT, step);
+    }
+
+    /*
+     * Save the current state of allKeys
+     */
+    private synchronized void saveAllKeys(DataOutputStream out, String sdPath)
+        throws IOException {
+      StartupProgress prog = NameNode.getStartupProgress();
+      Step step = new Step(StepType.DELEGATION_KEYS, sdPath);
+      prog.beginStep(Phase.SAVING_CHECKPOINT, step);
+      prog.setTotal(Phase.SAVING_CHECKPOINT, step, currentTokens.size());
+      Counter counter = prog.getCounter(Phase.SAVING_CHECKPOINT, step);
+      out.writeInt(allKeys.size());
+      Iterator<Integer> iter = allKeys.keySet().iterator();
+      while (iter.hasNext()) {
+        Integer key = iter.next();
+        allKeys.get(key).write(out);
+        counter.increment();
+      }
+      prog.endStep(Phase.SAVING_CHECKPOINT, step);
+    }
+
     /**
      * Private helper methods to load Delegation tokens from fsimage
      */

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java

@@ -170,7 +170,7 @@ class NameNodeConnector {
   }
 
   /* The idea for making sure that there is no more than one balancer
-   * running in an HDFS is to create a file in the HDFS, writes the IP address
+   * running in an HDFS is to create a file in the HDFS, writes the hostname
    * of the machine on which the balancer is running to the file, but did not
    * close the file until the balancer exits. 
    * This prevents the second balancer from running because it can not

+ 15 - 17
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -265,7 +265,8 @@ public class BlockManager {
     final long pendingPeriod = conf.getLong(
         DFSConfigKeys.DFS_NAMENODE_STARTUP_DELAY_BLOCK_DELETION_MS_KEY,
         DFSConfigKeys.DFS_NAMENODE_STARTUP_DELAY_BLOCK_DELETION_MS_DEFAULT);
-    invalidateBlocks = new InvalidateBlocks(datanodeManager, pendingPeriod);
+    invalidateBlocks = new InvalidateBlocks(
+        datanodeManager.blockInvalidateLimit, pendingPeriod);
 
     // Compute the map capacity by allocating 2% of total memory
     blocksMap = new BlocksMap(
@@ -701,7 +702,7 @@ public class BlockManager {
 
     // remove this block from the list of pending blocks to be deleted. 
     for (DatanodeStorageInfo storage : targets) {
-      invalidateBlocks.remove(storage.getStorageID(), oldBlock);
+      invalidateBlocks.remove(storage.getDatanodeDescriptor(), oldBlock);
     }
     
     // Adjust safe-mode totals, since under-construction blocks don't
@@ -726,7 +727,7 @@ public class BlockManager {
     for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
       final String storageID = storage.getStorageID();
       // filter invalidate replicas
-      if(!invalidateBlocks.contains(storageID, block)) {
+      if(!invalidateBlocks.contains(storage.getDatanodeDescriptor(), block)) {
         locations.add(storage);
       }
     }
@@ -1016,7 +1017,7 @@ public class BlockManager {
     pendingDNMessages.removeAllMessagesForDatanode(node);
 
     node.resetBlocks();
-    invalidateBlocks.remove(node.getDatanodeUuid());
+    invalidateBlocks.remove(node);
     
     // If the DN hasn't block-reported since the most recent
     // failover, then we may have been holding up on processing
@@ -1184,7 +1185,7 @@ public class BlockManager {
    * @return total number of block for deletion
    */
   int computeInvalidateWork(int nodesToProcess) {
-    final List<String> nodes = invalidateBlocks.getStorageIDs();
+    final List<DatanodeInfo> nodes = invalidateBlocks.getDatanodes();
     Collections.shuffle(nodes);
 
     nodesToProcess = Math.min(nodes.size(), nodesToProcess);
@@ -1973,7 +1974,7 @@ public class BlockManager {
     }
 
     // Ignore replicas already scheduled to be removed from the DN
-    if(invalidateBlocks.contains(dn.getDatanodeUuid(), block)) {
+    if(invalidateBlocks.contains(dn, block)) {
       /*
        * TODO: following assertion is incorrect, see HDFS-2668 assert
        * storedBlock.findDatanode(dn) < 0 : "Block " + block +
@@ -3199,9 +3200,8 @@ public class BlockManager {
    *
    * @return number of blocks scheduled for removal during this iteration.
    */
-  private int invalidateWorkForOneNode(String nodeId) {
+  private int invalidateWorkForOneNode(DatanodeInfo dn) {
     final List<Block> toInvalidate;
-    final DatanodeDescriptor dn;
     
     namesystem.writeLock();
     try {
@@ -3210,15 +3210,13 @@ public class BlockManager {
         LOG.debug("In safemode, not computing replication work");
         return 0;
       }
-      // get blocks to invalidate for the nodeId
-      assert nodeId != null;
-      dn = datanodeManager.getDatanode(nodeId);
-      if (dn == null) {
-        invalidateBlocks.remove(nodeId);
-        return 0;
-      }
-      toInvalidate = invalidateBlocks.invalidateWork(nodeId, dn);
-      if (toInvalidate == null) {
+      try {
+        toInvalidate = invalidateBlocks.invalidateWork(datanodeManager.getDatanode(dn));
+        
+        if (toInvalidate == null) {
+          return 0;
+        }
+      } catch(UnregisteredNodeException une) {
         return 0;
       }
     } finally {

+ 23 - 27
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/InvalidateBlocks.java

@@ -44,13 +44,13 @@ import com.google.common.annotations.VisibleForTesting;
  */
 @InterfaceAudience.Private
 class InvalidateBlocks {
-  /** Mapping: StorageID -> Collection of Blocks */
-  private final Map<String, LightWeightHashSet<Block>> node2blocks =
-      new TreeMap<String, LightWeightHashSet<Block>>();
+  /** Mapping: DatanodeInfo -> Collection of Blocks */
+  private final Map<DatanodeInfo, LightWeightHashSet<Block>> node2blocks =
+      new TreeMap<DatanodeInfo, LightWeightHashSet<Block>>();
   /** The total number of blocks in the map. */
   private long numBlocks = 0L;
 
-  private final DatanodeManager datanodeManager;
+  private final int blockInvalidateLimit;
 
   /**
    * The period of pending time for block invalidation since the NameNode
@@ -60,8 +60,8 @@ class InvalidateBlocks {
   /** the startup time */
   private final long startupTime = Time.monotonicNow();
 
-  InvalidateBlocks(final DatanodeManager datanodeManager, long pendingPeriodInMs) {
-    this.datanodeManager = datanodeManager;
+  InvalidateBlocks(final int blockInvalidateLimit, long pendingPeriodInMs) {
+    this.blockInvalidateLimit = blockInvalidateLimit;
     this.pendingPeriodInMs = pendingPeriodInMs;
     printBlockDeletionTime(BlockManager.LOG);
   }
@@ -86,12 +86,9 @@ class InvalidateBlocks {
    * invalidation. Blocks are compared including their generation stamps:
    * if a block is pending invalidation but with a different generation stamp,
    * returns false.
-   * @param storageID the storage to check
-   * @param the block to look for
-   * 
    */
-  synchronized boolean contains(final String storageID, final Block block) {
-    final LightWeightHashSet<Block> s = node2blocks.get(storageID);
+  synchronized boolean contains(final DatanodeInfo dn, final Block block) {
+    final LightWeightHashSet<Block> s = node2blocks.get(dn);
     if (s == null) {
       return false; // no invalidate blocks for this storage ID
     }
@@ -106,10 +103,10 @@ class InvalidateBlocks {
    */
   synchronized void add(final Block block, final DatanodeInfo datanode,
       final boolean log) {
-    LightWeightHashSet<Block> set = node2blocks.get(datanode.getDatanodeUuid());
+    LightWeightHashSet<Block> set = node2blocks.get(datanode);
     if (set == null) {
       set = new LightWeightHashSet<Block>();
-      node2blocks.put(datanode.getDatanodeUuid(), set);
+      node2blocks.put(datanode, set);
     }
     if (set.add(block)) {
       numBlocks++;
@@ -121,20 +118,20 @@ class InvalidateBlocks {
   }
 
   /** Remove a storage from the invalidatesSet */
-  synchronized void remove(final String storageID) {
-    final LightWeightHashSet<Block> blocks = node2blocks.remove(storageID);
+  synchronized void remove(final DatanodeInfo dn) {
+    final LightWeightHashSet<Block> blocks = node2blocks.remove(dn);
     if (blocks != null) {
       numBlocks -= blocks.size();
     }
   }
 
   /** Remove the block from the specified storage. */
-  synchronized void remove(final String storageID, final Block block) {
-    final LightWeightHashSet<Block> v = node2blocks.get(storageID);
+  synchronized void remove(final DatanodeInfo dn, final Block block) {
+    final LightWeightHashSet<Block> v = node2blocks.get(dn);
     if (v != null && v.remove(block)) {
       numBlocks--;
       if (v.isEmpty()) {
-        node2blocks.remove(storageID);
+        node2blocks.remove(dn);
       }
     }
   }
@@ -148,18 +145,18 @@ class InvalidateBlocks {
       return;
     }
 
-    for(Map.Entry<String,LightWeightHashSet<Block>> entry : node2blocks.entrySet()) {
+    for(Map.Entry<DatanodeInfo, LightWeightHashSet<Block>> entry : node2blocks.entrySet()) {
       final LightWeightHashSet<Block> blocks = entry.getValue();
       if (blocks.size() > 0) {
-        out.println(datanodeManager.getDatanode(entry.getKey()));
+        out.println(entry.getKey());
         out.println(blocks);
       }
     }
   }
 
   /** @return a list of the storage IDs. */
-  synchronized List<String> getStorageIDs() {
-    return new ArrayList<String>(node2blocks.keySet());
+  synchronized List<DatanodeInfo> getDatanodes() {
+    return new ArrayList<DatanodeInfo>(node2blocks.keySet());
   }
 
   /**
@@ -170,8 +167,7 @@ class InvalidateBlocks {
     return pendingPeriodInMs - (Time.monotonicNow() - startupTime);
   }
 
-  synchronized List<Block> invalidateWork(
-      final String storageId, final DatanodeDescriptor dn) {
+  synchronized List<Block> invalidateWork(final DatanodeDescriptor dn) {
     final long delay = getInvalidationDelay();
     if (delay > 0) {
       if (BlockManager.LOG.isDebugEnabled()) {
@@ -181,18 +177,18 @@ class InvalidateBlocks {
       }
       return null;
     }
-    final LightWeightHashSet<Block> set = node2blocks.get(storageId);
+    final LightWeightHashSet<Block> set = node2blocks.get(dn);
     if (set == null) {
       return null;
     }
 
     // # blocks that can be sent in one message is limited
-    final int limit = datanodeManager.blockInvalidateLimit;
+    final int limit = blockInvalidateLimit;
     final List<Block> toInvalidate = set.pollN(limit);
 
     // If we send everything in this message, remove this node entry
     if (set.isEmpty()) {
-      remove(storageId);
+      remove(dn);
     }
 
     dn.addBlocksToBeInvalidated(toInvalidate);

+ 8 - 43
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclStorage.java

@@ -27,8 +27,10 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclEntryScope;
 import org.apache.hadoop.fs.permission.AclEntryType;
+import org.apache.hadoop.fs.permission.AclUtil;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.permission.ScopedAclEntries;
 import org.apache.hadoop.hdfs.protocol.AclException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 
@@ -90,7 +92,7 @@ final class AclStorage {
     FsPermission childPerm = child.getFsPermission();
 
     // Copy each default ACL entry from parent to new child's access ACL.
-    boolean parentDefaultIsMinimal = isMinimalAcl(parentDefaultEntries);
+    boolean parentDefaultIsMinimal = AclUtil.isMinimalAcl(parentDefaultEntries);
     for (AclEntry entry: parentDefaultEntries) {
       AclEntryType type = entry.getType();
       String name = entry.getName();
@@ -127,7 +129,7 @@ final class AclStorage {
       Collections.<AclEntry>emptyList();
 
     final FsPermission newPerm;
-    if (!isMinimalAcl(accessEntries) || !defaultEntries.isEmpty()) {
+    if (!AclUtil.isMinimalAcl(accessEntries) || !defaultEntries.isEmpty()) {
       // Save the new ACL to the child.
       child.addAclFeature(createAclFeature(accessEntries, defaultEntries));
       newPerm = createFsPermissionForExtendedAcl(accessEntries, childPerm);
@@ -172,7 +174,7 @@ final class AclStorage {
     FsPermission perm = inode.getFsPermission();
     AclFeature f = inode.getAclFeature();
     if (f == null) {
-      return getMinimalAcl(perm);
+      return AclUtil.getMinimalAcl(perm);
     }
 
     final List<AclEntry> existingAcl;
@@ -208,7 +210,7 @@ final class AclStorage {
     } else {
       // It's possible that there is a default ACL but no access ACL. In this
       // case, add the minimal access ACL implied by the permission bits.
-      existingAcl.addAll(getMinimalAcl(perm));
+      existingAcl.addAll(AclUtil.getMinimalAcl(perm));
     }
 
     // Add all default entries after the access entries.
@@ -267,7 +269,7 @@ final class AclStorage {
     assert newAcl.size() >= 3;
     FsPermission perm = inode.getFsPermission();
     final FsPermission newPerm;
-    if (!isMinimalAcl(newAcl)) {
+    if (!AclUtil.isMinimalAcl(newAcl)) {
       // This is an extended ACL.  Split entries into access vs. default.
       ScopedAclEntries scoped = new ScopedAclEntries(newAcl);
       List<AclEntry> accessEntries = scoped.getAccessEntries();
@@ -321,7 +323,7 @@ final class AclStorage {
     // For the access ACL, the feature only needs to hold the named user and
     // group entries.  For a correctly sorted ACL, these will be in a
     // predictable range.
-    if (!isMinimalAcl(accessEntries)) {
+    if (!AclUtil.isMinimalAcl(accessEntries)) {
       featureEntries.addAll(
         accessEntries.subList(1, accessEntries.size() - 2));
     }
@@ -366,41 +368,4 @@ final class AclStorage {
       accessEntries.get(2).getPermission(),
       existingPerm.getStickyBit());
   }
-
-  /**
-   * Translates the given permission bits to the equivalent minimal ACL.
-   *
-   * @param perm FsPermission to translate
-   * @return List<AclEntry> containing exactly 3 entries representing the owner,
-   *   group and other permissions
-   */
-  private static List<AclEntry> getMinimalAcl(FsPermission perm) {
-    return Lists.newArrayList(
-      new AclEntry.Builder()
-        .setScope(AclEntryScope.ACCESS)
-        .setType(AclEntryType.USER)
-        .setPermission(perm.getUserAction())
-        .build(),
-      new AclEntry.Builder()
-        .setScope(AclEntryScope.ACCESS)
-        .setType(AclEntryType.GROUP)
-        .setPermission(perm.getGroupAction())
-        .build(),
-      new AclEntry.Builder()
-        .setScope(AclEntryScope.ACCESS)
-        .setType(AclEntryType.OTHER)
-        .setPermission(perm.getOtherAction())
-        .build());
-  }
-
-  /**
-   * Checks if the given entries represent a minimal ACL (contains exactly 3
-   * entries).
-   *
-   * @param entries List<AclEntry> entries to check
-   * @return boolean true if the entries represent a minimal ACL
-   */
-  private static boolean isMinimalAcl(List<AclEntry> entries) {
-    return entries.size() == 3;
-  }
 }

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclTransformation.java

@@ -40,6 +40,7 @@ import org.apache.hadoop.fs.permission.AclEntryScope;
 import org.apache.hadoop.fs.permission.AclEntryType;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.permission.ScopedAclEntries;
 import org.apache.hadoop.hdfs.protocol.AclException;
 
 /**

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

@@ -27,6 +27,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_PATH_BASED_CACHE
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS_DEFAULT;
 
 import java.io.DataInput;
+import java.io.DataOutputStream;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -61,10 +62,10 @@ import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo.Expiration;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveStats;
 import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.CacheReplicationMonitor;
@@ -690,15 +691,25 @@ public final class CacheManager {
     assert namesystem.hasReadLock();
     final int NUM_PRE_ALLOCATED_ENTRIES = 16;
     String filterPath = null;
-    if (filter.getId() != null) {
-      throw new IOException("Filtering by ID is unsupported.");
-    }
     if (filter.getPath() != null) {
       filterPath = validatePath(filter);
     }
     if (filter.getReplication() != null) {
-      throw new IOException("Filtering by replication is unsupported.");
+      throw new InvalidRequestException(
+          "Filtering by replication is unsupported.");
     }
+
+    // Querying for a single ID
+    final Long id = filter.getId();
+    if (id != null) {
+      if (!directivesById.containsKey(id)) {
+        throw new InvalidRequestException("Did not find requested id " + id);
+      }
+      // Since we use a tailMap on directivesById, setting prev to id-1 gets
+      // us the directive with the id (if present)
+      prevId = id - 1;
+    }
+
     ArrayList<CacheDirectiveEntry> replies =
         new ArrayList<CacheDirectiveEntry>(NUM_PRE_ALLOCATED_ENTRIES);
     int numReplies = 0;
@@ -710,6 +721,14 @@ public final class CacheManager {
       }
       CacheDirective curDirective = cur.getValue();
       CacheDirectiveInfo info = cur.getValue().toInfo();
+
+      // If the requested ID is present, it should be the first item.
+      // Hitting this case means the ID is not present, or we're on the second
+      // item and should break out.
+      if (id != null &&
+          !(info.getId().equals(id))) {
+        break;
+      }
       if (filter.getPool() != null && 
           !info.getPool().equals(filter.getPool())) {
         continue;
@@ -953,6 +972,18 @@ public final class CacheManager {
     }
   }
 
+  /**
+   * Saves the current state of the CacheManager to the DataOutput. Used
+   * to persist CacheManager state in the FSImage.
+   * @param out DataOutput to persist state
+   * @param sdPath path of the storage directory
+   * @throws IOException
+   */
+  public void saveStateCompat(DataOutputStream out, String sdPath)
+      throws IOException {
+    serializerCompat.save(out, sdPath);
+  }
+
   public PersistState saveState() throws IOException {
     ArrayList<CachePoolInfoProto> pools = Lists
         .newArrayListWithCapacity(cachePools.size());
@@ -1072,6 +1103,12 @@ public final class CacheManager {
   }
 
   private final class SerializerCompat {
+    private void save(DataOutputStream out, String sdPath) throws IOException {
+      out.writeLong(nextDirectiveId);
+      savePools(out, sdPath);
+      saveDirectives(out, sdPath);
+    }
+
     private void load(DataInput in) throws IOException {
       nextDirectiveId = in.readLong();
       // pools need to be loaded first since directives point to their parent pool
@@ -1079,6 +1116,42 @@ public final class CacheManager {
       loadDirectives(in);
     }
 
+    /**
+     * Save cache pools to fsimage
+     */
+    private void savePools(DataOutputStream out,
+        String sdPath) throws IOException {
+      StartupProgress prog = NameNode.getStartupProgress();
+      Step step = new Step(StepType.CACHE_POOLS, sdPath);
+      prog.beginStep(Phase.SAVING_CHECKPOINT, step);
+      prog.setTotal(Phase.SAVING_CHECKPOINT, step, cachePools.size());
+      Counter counter = prog.getCounter(Phase.SAVING_CHECKPOINT, step);
+      out.writeInt(cachePools.size());
+      for (CachePool pool: cachePools.values()) {
+        FSImageSerialization.writeCachePoolInfo(out, pool.getInfo(true));
+        counter.increment();
+      }
+      prog.endStep(Phase.SAVING_CHECKPOINT, step);
+    }
+
+    /*
+     * Save cache entries to fsimage
+     */
+    private void saveDirectives(DataOutputStream out, String sdPath)
+        throws IOException {
+      StartupProgress prog = NameNode.getStartupProgress();
+      Step step = new Step(StepType.CACHE_ENTRIES, sdPath);
+      prog.beginStep(Phase.SAVING_CHECKPOINT, step);
+      prog.setTotal(Phase.SAVING_CHECKPOINT, step, directivesById.size());
+      Counter counter = prog.getCounter(Phase.SAVING_CHECKPOINT, step);
+      out.writeInt(directivesById.size());
+      for (CacheDirective directive : directivesById.values()) {
+        FSImageSerialization.writeCacheDirectiveInfo(out, directive.toInfo());
+        counter.increment();
+      }
+      prog.endStep(Phase.SAVING_CHECKPOINT, step);
+    }
+
     /**
      * Load cache pools from fsimage
      */

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CheckpointConf.java

@@ -41,6 +41,9 @@ public class CheckpointConf {
 
   /** maxium number of retries when merge errors occur */
   private final int maxRetriesOnMergeError;
+
+  /** The output dir for legacy OIV image */
+  private final String legacyOivImageDir;
   
   public CheckpointConf(Configuration conf) {
     checkpointCheckPeriod = conf.getLong(
@@ -53,6 +56,7 @@ public class CheckpointConf {
                                   DFS_NAMENODE_CHECKPOINT_TXNS_DEFAULT);
     maxRetriesOnMergeError = conf.getInt(DFS_NAMENODE_CHECKPOINT_MAX_RETRIES_KEY,
                                   DFS_NAMENODE_CHECKPOINT_MAX_RETRIES_DEFAULT);
+    legacyOivImageDir = conf.get(DFS_NAMENODE_LEGACY_OIV_IMAGE_DIR_KEY);
     warnForDeprecatedConfigs(conf);
   }
   
@@ -83,4 +87,8 @@ public class CheckpointConf {
   public int getMaxRetriesOnMergeError() {
     return maxRetriesOnMergeError;
   }
+
+  public String getLegacyOivImageDir() {
+    return legacyOivImageDir;
+  }
 }

+ 20 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -53,6 +53,7 @@ import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.FSLimitException.MaxDirectoryItemsExceededException;
 import org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooLongException;
+import org.apache.hadoop.hdfs.protocol.FsAclPermission;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
@@ -2592,7 +2593,7 @@ public class FSDirectory implements Closeable {
         blocksize,
         node.getModificationTime(snapshot),
         node.getAccessTime(snapshot),
-        node.getFsPermission(snapshot),
+        getPermissionForFileStatus(node, snapshot),
         node.getUserName(snapshot),
         node.getGroupName(snapshot),
         node.isSymlink() ? node.asSymlink().getSymlink() : null,
@@ -2634,7 +2635,8 @@ public class FSDirectory implements Closeable {
     HdfsLocatedFileStatus status =
         new HdfsLocatedFileStatus(size, node.isDirectory(), replication,
           blocksize, node.getModificationTime(snapshot),
-          node.getAccessTime(snapshot), node.getFsPermission(snapshot),
+          node.getAccessTime(snapshot),
+          getPermissionForFileStatus(node, snapshot),
           node.getUserName(snapshot), node.getGroupName(snapshot),
           node.isSymlink() ? node.asSymlink().getSymlink() : null, path,
           node.getId(), loc, childrenNum);
@@ -2648,6 +2650,22 @@ public class FSDirectory implements Closeable {
     return status;
   }
 
+  /**
+   * Returns an inode's FsPermission for use in an outbound FileStatus.  If the
+   * inode has an ACL, then this method will convert to a FsAclPermission.
+   *
+   * @param node INode to check
+   * @param snapshot int snapshot ID
+   * @return FsPermission from inode, with ACL bit on if the inode has an ACL
+   */
+  private static FsPermission getPermissionForFileStatus(INode node,
+      int snapshot) {
+    FsPermission perm = node.getFsPermission(snapshot);
+    if (node.getAclFeature(snapshot) != null) {
+      perm = new FsAclPermission(perm);
+    }
+    return perm;
+  }
     
   /**
    * Add the given symbolic link to the fs. Record it in the edits log.

+ 19 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java

@@ -934,6 +934,25 @@ public class FSImage implements Closeable {
     storage.setMostRecentCheckpointInfo(txid, Time.now());
   }
 
+  /**
+   * Save FSimage in the legacy format. This is not for NN consumption,
+   * but for tools like OIV.
+   */
+  public void saveLegacyOIVImage(FSNamesystem source, String targetDir,
+      Canceler canceler) throws IOException {
+    FSImageCompression compression =
+        FSImageCompression.createCompression(conf);
+    long txid = getLastAppliedOrWrittenTxId();
+    SaveNamespaceContext ctx = new SaveNamespaceContext(source, txid,
+        canceler);
+    FSImageFormat.Saver saver = new FSImageFormat.Saver(ctx);
+    String imageFileName = NNStorage.getLegacyOIVImageFileName(txid);
+    File imageFile = new File(targetDir, imageFileName);
+    saver.save(imageFile, compression);
+    archivalManager.purgeOldLegacyOIVImages(targetDir, txid);
+  }
+
+
   /**
    * FSImageSaver is being run in a separate thread when saving
    * FSImage. There is one thread per each copy of the image.

+ 381 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java

@@ -21,14 +21,20 @@ import static org.apache.hadoop.util.Time.now;
 
 import java.io.DataInput;
 import java.io.DataInputStream;
+import java.io.DataOutputStream;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
 import java.io.IOException;
 import java.security.DigestInputStream;
+import java.security.DigestOutputStream;
 import java.security.MessageDigest;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
 
@@ -50,6 +56,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
@@ -60,6 +67,7 @@ import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress.Counter;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.StepType;
+import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.io.Text;
@@ -69,8 +77,105 @@ import com.google.common.base.Preconditions;
 import com.google.common.annotations.VisibleForTesting;
 
 /**
- * This class loads and stores the FSImage of the NameNode. The file
- * src/main/proto/fsimage.proto describes the on-disk layout of the FSImage.
+ * Contains inner classes for reading or writing the on-disk format for
+ * FSImages.
+ *
+ * In particular, the format of the FSImage looks like:
+ * <pre>
+ * FSImage {
+ *   layoutVersion: int, namespaceID: int, numberItemsInFSDirectoryTree: long,
+ *   namesystemGenerationStampV1: long, namesystemGenerationStampV2: long,
+ *   generationStampAtBlockIdSwitch:long, lastAllocatedBlockId:
+ *   long transactionID: long, snapshotCounter: int, numberOfSnapshots: int,
+ *   numOfSnapshottableDirs: int,
+ *   {FSDirectoryTree, FilesUnderConstruction, SecretManagerState} (can be compressed)
+ * }
+ *
+ * FSDirectoryTree (if {@link Feature#FSIMAGE_NAME_OPTIMIZATION} is supported) {
+ *   INodeInfo of root, numberOfChildren of root: int
+ *   [list of INodeInfo of root's children],
+ *   [list of INodeDirectoryInfo of root's directory children]
+ * }
+ *
+ * FSDirectoryTree (if {@link Feature#FSIMAGE_NAME_OPTIMIZATION} not supported){
+ *   [list of INodeInfo of INodes in topological order]
+ * }
+ *
+ * INodeInfo {
+ *   {
+ *     localName: short + byte[]
+ *   } when {@link Feature#FSIMAGE_NAME_OPTIMIZATION} is supported
+ *   or
+ *   {
+ *     fullPath: byte[]
+ *   } when {@link Feature#FSIMAGE_NAME_OPTIMIZATION} is not supported
+ *   replicationFactor: short, modificationTime: long,
+ *   accessTime: long, preferredBlockSize: long,
+ *   numberOfBlocks: int (-1 for INodeDirectory, -2 for INodeSymLink),
+ *   {
+ *     nsQuota: long, dsQuota: long,
+ *     {
+ *       isINodeSnapshottable: byte,
+ *       isINodeWithSnapshot: byte (if isINodeSnapshottable is false)
+ *     } (when {@link Feature#SNAPSHOT} is supported),
+ *     fsPermission: short, PermissionStatus
+ *   } for INodeDirectory
+ *   or
+ *   {
+ *     symlinkString, fsPermission: short, PermissionStatus
+ *   } for INodeSymlink
+ *   or
+ *   {
+ *     [list of BlockInfo]
+ *     [list of FileDiff]
+ *     {
+ *       isINodeFileUnderConstructionSnapshot: byte,
+ *       {clientName: short + byte[], clientMachine: short + byte[]} (when
+ *       isINodeFileUnderConstructionSnapshot is true),
+ *     } (when {@link Feature#SNAPSHOT} is supported and writing snapshotINode),
+ *     fsPermission: short, PermissionStatus
+ *   } for INodeFile
+ * }
+ *
+ * INodeDirectoryInfo {
+ *   fullPath of the directory: short + byte[],
+ *   numberOfChildren: int, [list of INodeInfo of children INode],
+ *   {
+ *     numberOfSnapshots: int,
+ *     [list of Snapshot] (when NumberOfSnapshots is positive),
+ *     numberOfDirectoryDiffs: int,
+ *     [list of DirectoryDiff] (NumberOfDirectoryDiffs is positive),
+ *     number of children that are directories,
+ *     [list of INodeDirectoryInfo of the directory children] (includes
+ *     snapshot copies of deleted sub-directories)
+ *   } (when {@link Feature#SNAPSHOT} is supported),
+ * }
+ *
+ * Snapshot {
+ *   snapshotID: int, root of Snapshot: INodeDirectoryInfo (its local name is
+ *   the name of the snapshot)
+ * }
+ *
+ * DirectoryDiff {
+ *   full path of the root of the associated Snapshot: short + byte[],
+ *   childrenSize: int,
+ *   isSnapshotRoot: byte,
+ *   snapshotINodeIsNotNull: byte (when isSnapshotRoot is false),
+ *   snapshotINode: INodeDirectory (when SnapshotINodeIsNotNull is true), Diff
+ * }
+ *
+ * Diff {
+ *   createdListSize: int, [Local name of INode in created list],
+ *   deletedListSize: int, [INode in deleted list: INodeInfo]
+ * }
+ *
+ * FileDiff {
+ *   full path of the root of the associated Snapshot: short + byte[],
+ *   fileSize: long,
+ *   snapshotINodeIsNotNull: byte,
+ *   snapshotINode: INodeFile (when SnapshotINodeIsNotNull is true), Diff
+ * }
+ * </pre>
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
@@ -580,6 +685,11 @@ public class FSImageFormat {
       }
     }
 
+    /** @return The FSDirectory of the namesystem where the fsimage is loaded */
+    public FSDirectory getFSDirectoryInLoading() {
+      return namesystem.dir;
+    }
+
     public INode loadINodeWithLocalName(boolean isSnapshotINode, DataInput in,
         boolean updateINodeMap) throws IOException {
       return loadINodeWithLocalName(isSnapshotINode, in, updateINodeMap, null);
@@ -1009,7 +1119,7 @@ public class FSImageFormat {
       + " option to automatically rename these paths during upgrade.";
 
   /**
-   * Same as {@link #renameReservedPathsOnUpgrade}, but for a single
+   * Same as {@link #renameReservedPathsOnUpgrade(String)}, but for a single
    * byte array path component.
    */
   private static byte[] renameReservedComponentOnUpgrade(byte[] component,
@@ -1029,7 +1139,7 @@ public class FSImageFormat {
   }
 
   /**
-   * Same as {@link #renameReservedPathsOnUpgrade}, but for a single
+   * Same as {@link #renameReservedPathsOnUpgrade(String)}, but for a single
    * byte array path component.
    */
   private static byte[] renameReservedRootComponentOnUpgrade(byte[] component,
@@ -1050,4 +1160,271 @@ public class FSImageFormat {
     }
     return component;
   }
+
+  /**
+   * A one-shot class responsible for writing an image file.
+   * The write() function should be called once, after which the getter
+   * functions may be used to retrieve information about the file that was written.
+   *
+   * This is replaced by the PB-based FSImage. The class is to maintain
+   * compatibility for the external fsimage tool.
+   */
+  @Deprecated
+  static class Saver {
+    private static final int LAYOUT_VERSION = -51;
+    private final SaveNamespaceContext context;
+    /** Set to true once an image has been written */
+    private boolean saved = false;
+
+    /** The MD5 checksum of the file that was written */
+    private MD5Hash savedDigest;
+    private final ReferenceMap referenceMap = new ReferenceMap();
+
+    private final Map<Long, INodeFile> snapshotUCMap =
+        new HashMap<Long, INodeFile>();
+
+    /** @throws IllegalStateException if the instance has not yet saved an image */
+    private void checkSaved() {
+      if (!saved) {
+        throw new IllegalStateException("FSImageSaver has not saved an image");
+      }
+    }
+
+    /** @throws IllegalStateException if the instance has already saved an image */
+    private void checkNotSaved() {
+      if (saved) {
+        throw new IllegalStateException("FSImageSaver has already saved an image");
+      }
+    }
+
+
+    Saver(SaveNamespaceContext context) {
+      this.context = context;
+    }
+
+    /**
+     * Return the MD5 checksum of the image file that was saved.
+     */
+    MD5Hash getSavedDigest() {
+      checkSaved();
+      return savedDigest;
+    }
+
+    void save(File newFile, FSImageCompression compression) throws IOException {
+      checkNotSaved();
+
+      final FSNamesystem sourceNamesystem = context.getSourceNamesystem();
+      final INodeDirectory rootDir = sourceNamesystem.dir.rootDir;
+      final long numINodes = rootDir.getDirectoryWithQuotaFeature()
+          .getSpaceConsumed().get(Quota.NAMESPACE);
+      String sdPath = newFile.getParentFile().getParentFile().getAbsolutePath();
+      Step step = new Step(StepType.INODES, sdPath);
+      StartupProgress prog = NameNode.getStartupProgress();
+      prog.beginStep(Phase.SAVING_CHECKPOINT, step);
+      prog.setTotal(Phase.SAVING_CHECKPOINT, step, numINodes);
+      Counter counter = prog.getCounter(Phase.SAVING_CHECKPOINT, step);
+      long startTime = now();
+      //
+      // Write out data
+      //
+      MessageDigest digester = MD5Hash.getDigester();
+      FileOutputStream fout = new FileOutputStream(newFile);
+      DigestOutputStream fos = new DigestOutputStream(fout, digester);
+      DataOutputStream out = new DataOutputStream(fos);
+      try {
+        out.writeInt(LAYOUT_VERSION);
+        LayoutFlags.write(out);
+        // We use the non-locked version of getNamespaceInfo here since
+        // the coordinating thread of saveNamespace already has read-locked
+        // the namespace for us. If we attempt to take another readlock
+        // from the actual saver thread, there's a potential of a
+        // fairness-related deadlock. See the comments on HDFS-2223.
+        out.writeInt(sourceNamesystem.unprotectedGetNamespaceInfo()
+            .getNamespaceID());
+        out.writeLong(numINodes);
+        out.writeLong(sourceNamesystem.getGenerationStampV1());
+        out.writeLong(sourceNamesystem.getGenerationStampV2());
+        out.writeLong(sourceNamesystem.getGenerationStampAtblockIdSwitch());
+        out.writeLong(sourceNamesystem.getLastAllocatedBlockId());
+        out.writeLong(context.getTxId());
+        out.writeLong(sourceNamesystem.getLastInodeId());
+
+
+        sourceNamesystem.getSnapshotManager().write(out);
+
+        // write compression info and set up compressed stream
+        out = compression.writeHeaderAndWrapStream(fos);
+        LOG.info("Saving image file " + newFile +
+                 " using " + compression);
+
+        // save the root
+        saveINode2Image(rootDir, out, false, referenceMap, counter);
+        // save the rest of the nodes
+        saveImage(rootDir, out, true, false, counter);
+        prog.endStep(Phase.SAVING_CHECKPOINT, step);
+        // Now that the step is finished, set counter equal to total to adjust
+        // for possible under-counting due to reference inodes.
+        prog.setCount(Phase.SAVING_CHECKPOINT, step, numINodes);
+        // save files under construction
+        // TODO: for HDFS-5428, since we cannot break the compatibility of
+        // fsimage, we store part of the under-construction files that are only
+        // in snapshots in this "under-construction-file" section. As a
+        // temporary solution, we use "/.reserved/.inodes/<inodeid>" as their
+        // paths, so that when loading fsimage we do not put them into the lease
+        // map. In the future, we can remove this hack when we can bump the
+        // layout version.
+        sourceNamesystem.saveFilesUnderConstruction(out, snapshotUCMap);
+
+        context.checkCancelled();
+        sourceNamesystem.saveSecretManagerStateCompat(out, sdPath);
+        context.checkCancelled();
+        sourceNamesystem.getCacheManager().saveStateCompat(out, sdPath);
+        context.checkCancelled();
+        out.flush();
+        context.checkCancelled();
+        fout.getChannel().force(true);
+      } finally {
+        out.close();
+      }
+
+      saved = true;
+      // set md5 of the saved image
+      savedDigest = new MD5Hash(digester.digest());
+
+      LOG.info("Image file " + newFile + " of size " + newFile.length() +
+          " bytes saved in " + (now() - startTime)/1000 + " seconds.");
+    }
+
+    /**
+     * Save children INodes.
+     * @param children The list of children INodes
+     * @param out The DataOutputStream to write
+     * @param inSnapshot Whether the parent directory or its ancestor is in
+     *                   the deleted list of some snapshot (caused by rename or
+     *                   deletion)
+     * @param counter Counter to increment for namenode startup progress
+     * @return Number of children that are directory
+     */
+    private int saveChildren(ReadOnlyList<INode> children,
+        DataOutputStream out, boolean inSnapshot, Counter counter)
+        throws IOException {
+      // Write normal children INode.
+      out.writeInt(children.size());
+      int dirNum = 0;
+      int i = 0;
+      for(INode child : children) {
+        // print all children first
+        // TODO: for HDFS-5428, we cannot change the format/content of fsimage
+        // here, thus even if the parent directory is in snapshot, we still
+        // do not handle INodeUC as those stored in deleted list
+        saveINode2Image(child, out, false, referenceMap, counter);
+        if (child.isDirectory()) {
+          dirNum++;
+        } else if (inSnapshot && child.isFile()
+            && child.asFile().isUnderConstruction()) {
+          this.snapshotUCMap.put(child.getId(), child.asFile());
+        }
+        if (i++ % 50 == 0) {
+          context.checkCancelled();
+        }
+      }
+      return dirNum;
+    }
+
+    /**
+     * Save file tree image starting from the given root.
+     * This is a recursive procedure, which first saves all children and
+     * snapshot diffs of a current directory and then moves inside the
+     * sub-directories.
+     *
+     * @param current The current node
+     * @param out The DataoutputStream to write the image
+     * @param toSaveSubtree Whether or not to save the subtree to fsimage. For
+     *                      reference node, its subtree may already have been
+     *                      saved before.
+     * @param inSnapshot Whether the current directory is in snapshot
+     * @param counter Counter to increment for namenode startup progress
+     */
+    private void saveImage(INodeDirectory current, DataOutputStream out,
+        boolean toSaveSubtree, boolean inSnapshot, Counter counter)
+        throws IOException {
+      // write the inode id of the directory
+      out.writeLong(current.getId());
+
+      if (!toSaveSubtree) {
+        return;
+      }
+
+      final ReadOnlyList<INode> children = current
+          .getChildrenList(Snapshot.CURRENT_STATE_ID);
+      int dirNum = 0;
+      List<INodeDirectory> snapshotDirs = null;
+      DirectoryWithSnapshotFeature sf = current.getDirectoryWithSnapshotFeature();
+      if (sf != null) {
+        snapshotDirs = new ArrayList<INodeDirectory>();
+        sf.getSnapshotDirectory(snapshotDirs);
+        dirNum += snapshotDirs.size();
+      }
+
+      // 2. Write INodeDirectorySnapshottable#snapshotsByNames to record all
+      // Snapshots
+      if (current instanceof INodeDirectorySnapshottable) {
+        INodeDirectorySnapshottable snapshottableNode =
+            (INodeDirectorySnapshottable) current;
+        SnapshotFSImageFormat.saveSnapshots(snapshottableNode, out);
+      } else {
+        out.writeInt(-1); // # of snapshots
+      }
+
+      // 3. Write children INode
+      dirNum += saveChildren(children, out, inSnapshot, counter);
+
+      // 4. Write DirectoryDiff lists, if there is any.
+      SnapshotFSImageFormat.saveDirectoryDiffList(current, out, referenceMap);
+
+      // Write sub-tree of sub-directories, including possible snapshots of
+      // deleted sub-directories
+      out.writeInt(dirNum); // the number of sub-directories
+      for(INode child : children) {
+        if(!child.isDirectory()) {
+          continue;
+        }
+        // make sure we only save the subtree under a reference node once
+        boolean toSave = child.isReference() ?
+            referenceMap.toProcessSubtree(child.getId()) : true;
+        saveImage(child.asDirectory(), out, toSave, inSnapshot, counter);
+      }
+      if (snapshotDirs != null) {
+        for (INodeDirectory subDir : snapshotDirs) {
+          // make sure we only save the subtree under a reference node once
+          boolean toSave = subDir.getParentReference() != null ?
+              referenceMap.toProcessSubtree(subDir.getId()) : true;
+          saveImage(subDir, out, toSave, true, counter);
+        }
+      }
+    }
+
+    /**
+     * Saves inode and increments progress counter.
+     *
+     * @param inode INode to save
+     * @param out DataOutputStream to receive inode
+     * @param writeUnderConstruction boolean true if this is under construction
+     * @param referenceMap ReferenceMap containing reference inodes
+     * @param counter Counter to increment for namenode startup progress
+     * @throws IOException thrown if there is an I/O error
+     */
+    private void saveINode2Image(INode inode, DataOutputStream out,
+        boolean writeUnderConstruction, ReferenceMap referenceMap,
+        Counter counter) throws IOException {
+      FSImageSerialization.saveINode2Image(inode, out, writeUnderConstruction,
+        referenceMap);
+      // Intentionally do not increment counter for reference inodes, because it
+      // is too difficult at this point to assess whether or not this is a
+      // reference that counts toward quota.
+      if (!(inode instanceof INodeReference)) {
+        counter.increment();
+      }
+    }
+  }
 }

+ 213 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java

@@ -17,6 +17,11 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.Path;
@@ -31,20 +36,21 @@ import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
 import org.apache.hadoop.hdfs.util.XMLUtils;
 import org.apache.hadoop.hdfs.util.XMLUtils.InvalidXmlException;
 import org.apache.hadoop.hdfs.util.XMLUtils.Stanza;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.ShortWritable;
+import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.WritableUtils;
 import org.xml.sax.ContentHandler;
 import org.xml.sax.SAXException;
 
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.DataOutputStream;
-import java.io.IOException;
+import com.google.common.base.Preconditions;
 
 /**
  * Static utility functions for serializing various pieces of data in the correct
@@ -82,6 +88,26 @@ public class FSImageSerialization {
     final ShortWritable U_SHORT = new ShortWritable();
     final IntWritable U_INT = new IntWritable();
     final LongWritable U_LONG = new LongWritable();
+    final FsPermission FILE_PERM = new FsPermission((short) 0);
+  }
+
+  private static void writePermissionStatus(INodeAttributes inode,
+      DataOutput out) throws IOException {
+    final FsPermission p = TL_DATA.get().FILE_PERM;
+    p.fromShort(inode.getFsPermissionShort());
+    PermissionStatus.write(out, inode.getUserName(), inode.getGroupName(), p);
+  }
+
+  private static void writeBlocks(final Block[] blocks,
+      final DataOutput out) throws IOException {
+    if (blocks == null) {
+      out.writeInt(0);
+    } else {
+      out.writeInt(blocks.length);
+      for (Block blk : blocks) {
+        blk.write(out);
+      }
+    }
   }
 
   // Helper function that reads in an INodeUnderConstruction
@@ -127,6 +153,183 @@ public class FSImageSerialization {
     return file;
   }
 
+  // Helper function that writes an INodeUnderConstruction
+  // into the input stream
+  //
+  static void writeINodeUnderConstruction(DataOutputStream out, INodeFile cons,
+      String path) throws IOException {
+    writeString(path, out);
+    out.writeLong(cons.getId());
+    out.writeShort(cons.getFileReplication());
+    out.writeLong(cons.getModificationTime());
+    out.writeLong(cons.getPreferredBlockSize());
+
+    writeBlocks(cons.getBlocks(), out);
+    cons.getPermissionStatus().write(out);
+
+    FileUnderConstructionFeature uc = cons.getFileUnderConstructionFeature();
+    writeString(uc.getClientName(), out);
+    writeString(uc.getClientMachine(), out);
+
+    out.writeInt(0); //  do not store locations of last block
+  }
+
+  /**
+   * Serialize a {@link INodeFile} node
+   * @param node The node to write
+   * @param out The {@link DataOutputStream} where the fields are written
+   * @param writeBlock Whether to write block information
+   */
+  public static void writeINodeFile(INodeFile file, DataOutput out,
+      boolean writeUnderConstruction) throws IOException {
+    writeLocalName(file, out);
+    out.writeLong(file.getId());
+    out.writeShort(file.getFileReplication());
+    out.writeLong(file.getModificationTime());
+    out.writeLong(file.getAccessTime());
+    out.writeLong(file.getPreferredBlockSize());
+
+    writeBlocks(file.getBlocks(), out);
+    SnapshotFSImageFormat.saveFileDiffList(file, out);
+
+    if (writeUnderConstruction) {
+      if (file.isUnderConstruction()) {
+        out.writeBoolean(true);
+        final FileUnderConstructionFeature uc = file.getFileUnderConstructionFeature();
+        writeString(uc.getClientName(), out);
+        writeString(uc.getClientMachine(), out);
+      } else {
+        out.writeBoolean(false);
+      }
+    }
+
+    writePermissionStatus(file, out);
+  }
+
+  /** Serialize an {@link INodeFileAttributes}. */
+  public static void writeINodeFileAttributes(INodeFileAttributes file,
+      DataOutput out) throws IOException {
+    writeLocalName(file, out);
+    writePermissionStatus(file, out);
+    out.writeLong(file.getModificationTime());
+    out.writeLong(file.getAccessTime());
+
+    out.writeShort(file.getFileReplication());
+    out.writeLong(file.getPreferredBlockSize());
+  }
+
+  private static void writeQuota(Quota.Counts quota, DataOutput out)
+      throws IOException {
+    out.writeLong(quota.get(Quota.NAMESPACE));
+    out.writeLong(quota.get(Quota.DISKSPACE));
+  }
+
+  /**
+   * Serialize a {@link INodeDirectory}
+   * @param node The node to write
+   * @param out The {@link DataOutput} where the fields are written
+   */
+  public static void writeINodeDirectory(INodeDirectory node, DataOutput out)
+      throws IOException {
+    writeLocalName(node, out);
+    out.writeLong(node.getId());
+    out.writeShort(0);  // replication
+    out.writeLong(node.getModificationTime());
+    out.writeLong(0);   // access time
+    out.writeLong(0);   // preferred block size
+    out.writeInt(-1);   // # of blocks
+
+    writeQuota(node.getQuotaCounts(), out);
+
+    if (node instanceof INodeDirectorySnapshottable) {
+      out.writeBoolean(true);
+    } else {
+      out.writeBoolean(false);
+      out.writeBoolean(node.isWithSnapshot());
+    }
+
+    writePermissionStatus(node, out);
+  }
+
+  /**
+   * Serialize a {@link INodeDirectory}
+   * @param a The node to write
+   * @param out The {@link DataOutput} where the fields are written
+   */
+  public static void writeINodeDirectoryAttributes(
+      INodeDirectoryAttributes a, DataOutput out) throws IOException {
+    writeLocalName(a, out);
+    writePermissionStatus(a, out);
+    out.writeLong(a.getModificationTime());
+    writeQuota(a.getQuotaCounts(), out);
+  }
+
+  /**
+   * Serialize a {@link INodeSymlink} node
+   * @param node The node to write
+   * @param out The {@link DataOutput} where the fields are written
+   */
+  private static void writeINodeSymlink(INodeSymlink node, DataOutput out)
+      throws IOException {
+    writeLocalName(node, out);
+    out.writeLong(node.getId());
+    out.writeShort(0);  // replication
+    out.writeLong(0);   // modification time
+    out.writeLong(0);   // access time
+    out.writeLong(0);   // preferred block size
+    out.writeInt(-2);   // # of blocks
+
+    Text.writeString(out, node.getSymlinkString());
+    writePermissionStatus(node, out);
+  }
+
+  /** Serialize a {@link INodeReference} node */
+  private static void writeINodeReference(INodeReference ref, DataOutput out,
+      boolean writeUnderConstruction, ReferenceMap referenceMap
+      ) throws IOException {
+    writeLocalName(ref, out);
+    out.writeLong(ref.getId());
+    out.writeShort(0);  // replication
+    out.writeLong(0);   // modification time
+    out.writeLong(0);   // access time
+    out.writeLong(0);   // preferred block size
+    out.writeInt(-3);   // # of blocks
+
+    final boolean isWithName = ref instanceof INodeReference.WithName;
+    out.writeBoolean(isWithName);
+
+    if (!isWithName) {
+      Preconditions.checkState(ref instanceof INodeReference.DstReference);
+      // dst snapshot id
+      out.writeInt(((INodeReference.DstReference) ref).getDstSnapshotId());
+    } else {
+      out.writeInt(((INodeReference.WithName) ref).getLastSnapshotId());
+    }
+
+    final INodeReference.WithCount withCount
+        = (INodeReference.WithCount)ref.getReferredINode();
+    referenceMap.writeINodeReferenceWithCount(withCount, out,
+        writeUnderConstruction);
+  }
+
+  /**
+   * Save one inode's attributes to the image.
+   */
+  public static void saveINode2Image(INode node, DataOutput out,
+      boolean writeUnderConstruction, ReferenceMap referenceMap)
+      throws IOException {
+    if (node.isReference()) {
+      writeINodeReference(node.asReference(), out, writeUnderConstruction,
+          referenceMap);
+    } else if (node.isDirectory()) {
+      writeINodeDirectory(node.asDirectory(), out);
+    } else if (node.isSymlink()) {
+      writeINodeSymlink(node.asSymlink(), out);
+    } else if (node.isFile()) {
+      writeINodeFile(node.asFile(), out, writeUnderConstruction);
+    }
+  }
+
   // This should be reverted to package private once the ImageLoader
   // code is moved into this package. This method should not be called
   // by other code.
@@ -226,6 +429,12 @@ public class FSImageSerialization {
     in.readFully(createdNodeName);
     return createdNodeName;
   }
+
+  private static void writeLocalName(INodeAttributes inode, DataOutput out)
+      throws IOException {
+    final byte[] name = inode.getLocalNameBytes();
+    writeBytes(name, out);
+  }
   
   public static void writeBytes(byte[] data, DataOutput out)
       throws IOException {

+ 62 - 19
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -85,17 +85,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
 import static org.apache.hadoop.util.Time.now;
 
-import java.io.BufferedWriter;
-import java.io.ByteArrayInputStream;
-import java.io.DataInput;
-import java.io.DataInputStream;
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStreamWriter;
-import java.io.PrintWriter;
-import java.io.StringWriter;
+import java.io.*;
 import java.lang.management.ManagementFactory;
 import java.net.InetAddress;
 import java.net.URI;
@@ -6091,6 +6081,42 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     leaseManager.changeLease(src, dst);
   }
 
+  /**
+   * Serializes leases.
+   */
+  void saveFilesUnderConstruction(DataOutputStream out,
+      Map<Long, INodeFile> snapshotUCMap) throws IOException {
+    // This is run by an inferior thread of saveNamespace, which holds a read
+    // lock on our behalf. If we took the read lock here, we could block
+    // for fairness if a writer is waiting on the lock.
+    synchronized (leaseManager) {
+      Map<String, INodeFile> nodes = leaseManager.getINodesUnderConstruction();
+      for (Map.Entry<String, INodeFile> entry : nodes.entrySet()) {
+        // TODO: for HDFS-5428, because of rename operations, some
+        // under-construction files that are
+        // in the current fs directory can also be captured in the
+        // snapshotUCMap. We should remove them from the snapshotUCMap.
+        snapshotUCMap.remove(entry.getValue().getId());
+      }
+
+      out.writeInt(nodes.size() + snapshotUCMap.size()); // write the size
+      for (Map.Entry<String, INodeFile> entry : nodes.entrySet()) {
+        FSImageSerialization.writeINodeUnderConstruction(
+            out, entry.getValue(), entry.getKey());
+      }
+      for (Map.Entry<Long, INodeFile> entry : snapshotUCMap.entrySet()) {
+        // for those snapshot INodeFileUC, we use "/.reserved/.inodes/<inodeid>"
+        // as their paths
+        StringBuilder b = new StringBuilder();
+        b.append(FSDirectory.DOT_RESERVED_PATH_PREFIX)
+            .append(Path.SEPARATOR).append(FSDirectory.DOT_INODES_STRING)
+            .append(Path.SEPARATOR).append(entry.getValue().getId());
+        FSImageSerialization.writeINodeUnderConstruction(
+            out, entry.getValue(), b.toString());
+      }
+    }
+  }
+
   /**
    * @return all the under-construction files in the lease map
    */
@@ -6377,6 +6403,15 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     getEditLog().logSync();
   }
 
+  /**
+   * @param out save state of the secret manager
+   * @param sdPath String storage directory path
+   */
+  void saveSecretManagerStateCompat(DataOutputStream out, String sdPath)
+      throws IOException {
+    dtSecretManager.saveSecretManagerStateCompat(out, sdPath);
+  }
+
   SecretManagerState saveSecretManagerState() {
     return dtSecretManager.saveSecretManagerState();
   }
@@ -7403,6 +7438,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       cacheManager.waitForRescanIfNeeded();
     }
     writeLock();
+    String effectiveDirectiveStr = null;
     Long result = null;
     try {
       checkOperation(OperationCategory.WRITE);
@@ -7414,11 +7450,12 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         throw new IOException("addDirective: you cannot specify an ID " +
             "for this operation.");
       }
-      CacheDirectiveInfo effectiveDirective = 
+      CacheDirectiveInfo effectiveDirective =
           cacheManager.addDirective(directive, pc, flags);
       getEditLog().logAddCacheDirectiveInfo(effectiveDirective,
           cacheEntry != null);
       result = effectiveDirective.getId();
+      effectiveDirectiveStr = effectiveDirective.toString();
       success = true;
     } finally {
       writeUnlock();
@@ -7426,7 +7463,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         getEditLog().logSync();
       }
       if (isAuditEnabled() && isExternalInvocation()) {
-        logAuditEvent(success, "addCacheDirective", null, null, null);
+        logAuditEvent(success, "addCacheDirective", effectiveDirectiveStr, null, null);
       }
       RetryCache.setState(cacheEntry, success, result);
     }
@@ -7463,7 +7500,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         getEditLog().logSync();
       }
       if (isAuditEnabled() && isExternalInvocation()) {
-        logAuditEvent(success, "modifyCacheDirective", null, null, null);
+        String idStr = "{id: " + directive.getId().toString() + "}";
+        logAuditEvent(success, "modifyCacheDirective", idStr, directive.toString(), null);
       }
       RetryCache.setState(cacheEntry, success);
     }
@@ -7491,7 +7529,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     } finally {
       writeUnlock();
       if (isAuditEnabled() && isExternalInvocation()) {
-        logAuditEvent(success, "removeCacheDirective", null, null,
+        String idStr = "{id: " + id.toString() + "}";
+        logAuditEvent(success, "removeCacheDirective", idStr, null,
             null);
       }
       RetryCache.setState(cacheEntry, success);
@@ -7516,7 +7555,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     } finally {
       readUnlock();
       if (isAuditEnabled() && isExternalInvocation()) {
-        logAuditEvent(success, "listCacheDirectives", null, null,
+        logAuditEvent(success, "listCacheDirectives", filter.toString(), null,
             null);
       }
     }
@@ -7533,6 +7572,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
     writeLock();
     boolean success = false;
+    String poolInfoStr = null;
     try {
       checkOperation(OperationCategory.WRITE);
       if (isInSafeMode()) {
@@ -7543,12 +7583,13 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         pc.checkSuperuserPrivilege();
       }
       CachePoolInfo info = cacheManager.addCachePool(req);
+      poolInfoStr = info.toString();
       getEditLog().logAddCachePool(info, cacheEntry != null);
       success = true;
     } finally {
       writeUnlock();
       if (isAuditEnabled() && isExternalInvocation()) {
-        logAuditEvent(success, "addCachePool", req.getPoolName(), null, null);
+        logAuditEvent(success, "addCachePool", poolInfoStr, null, null);
       }
       RetryCache.setState(cacheEntry, success);
     }
@@ -7581,7 +7622,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     } finally {
       writeUnlock();
       if (isAuditEnabled() && isExternalInvocation()) {
-        logAuditEvent(success, "modifyCachePool", req.getPoolName(), null, null);
+        String poolNameStr = "{poolName: " + req.getPoolName() + "}";
+        logAuditEvent(success, "modifyCachePool", poolNameStr, req.toString(), null);
       }
       RetryCache.setState(cacheEntry, success);
     }
@@ -7614,7 +7656,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     } finally {
       writeUnlock();
       if (isAuditEnabled() && isExternalInvocation()) {
-        logAuditEvent(success, "removeCachePool", cachePoolName, null, null);
+        String poolNameStr = "{poolName: " + cachePoolName + "}";
+        logAuditEvent(success, "removeCachePool", poolNameStr, null, null);
       }
       RetryCache.setState(cacheEntry, success);
     }

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

@@ -39,7 +39,7 @@ import com.google.common.base.Preconditions;
  * snapshots and it is renamed/moved to other locations.
  * 
  * For example,
- * (1) Support we have /abc/foo, say the inode of foo is inode(id=1000,name=foo)
+ * (1) Suppose we have /abc/foo, say the inode of foo is inode(id=1000,name=foo)
  * (2) create snapshot s0 for /abc
  * (3) mv /abc/foo /xyz/bar, i.e. inode(id=1000,name=...) is renamed from "foo"
  *     to "bar" and its parent becomes /xyz.

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

@@ -77,7 +77,8 @@ public class NNStorage extends Storage implements Closeable,
     IMAGE_ROLLBACK("fsimage_rollback"),
     EDITS_NEW ("edits.new"), // from "old" pre-HDFS-1073 format
     EDITS_INPROGRESS ("edits_inprogress"),
-    EDITS_TMP ("edits_tmp");
+    EDITS_TMP ("edits_tmp"),
+    IMAGE_LEGACY_OIV ("fsimage_legacy_oiv");  // For pre-PB format
 
     private String fileName = null;
     private NameNodeFile(String name) { this.fileName = name; }
@@ -693,6 +694,10 @@ public class NNStorage extends Storage implements Closeable,
     return getNameNodeFileName(NameNodeFile.IMAGE_ROLLBACK, txid);
   }
 
+  public static String getLegacyOIVImageFileName(long txid) {
+    return getNameNodeFileName(NameNodeFile.IMAGE_LEGACY_OIV, txid);
+  }
+
   private static String getNameNodeFileName(NameNodeFile nnf, long txid) {
     return String.format("%s_%019d", nnf.getName(), txid);
   }

+ 56 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorageRetentionManager.java

@@ -18,11 +18,13 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.File;
+import java.io.FilenameFilter;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.EnumSet;
+import java.util.Iterator;
 import java.util.List;
 import java.util.TreeSet;
 
@@ -233,4 +235,58 @@ public class NNStorageRetentionManager {
       }      
     }
   }
+
+  /**
+   * Delete old OIV fsimages. Since the target dir is not a full blown
+   * storage directory, we simply list and keep the latest ones. For the
+   * same reason, no storage inspector is used.
+   */
+  void purgeOldLegacyOIVImages(String dir, long txid) {
+    File oivImageDir = new File(dir);
+    final String oivImagePrefix = NameNodeFile.IMAGE_LEGACY_OIV.getName();
+    String filesInStorage[];
+
+    // Get the listing
+    filesInStorage = oivImageDir.list(new FilenameFilter() {
+      @Override
+      public boolean accept(File dir, String name) {
+        return name.matches(oivImagePrefix + "_(\\d+)");
+      }
+    });
+
+    // Check whether there is any work to do.
+    if (filesInStorage.length <= numCheckpointsToRetain) {
+      return;
+    }
+
+    // Create a sorted list of txids from the file names.
+    TreeSet<Long> sortedTxIds = new TreeSet<Long>();
+    for (String fName : filesInStorage) {
+      // Extract the transaction id from the file name.
+      long fTxId;
+      try {
+        fTxId = Long.parseLong(fName.substring(oivImagePrefix.length() + 1));
+      } catch (NumberFormatException nfe) {
+        // This should not happen since we have already filtered it.
+        // Log and continue.
+        LOG.warn("Invalid file name. Skipping " + fName);
+        continue;
+      }
+      sortedTxIds.add(Long.valueOf(fTxId));
+    }
+
+    int numFilesToDelete = sortedTxIds.size() - numCheckpointsToRetain;
+    Iterator<Long> iter = sortedTxIds.iterator();
+    while (numFilesToDelete > 0 && iter.hasNext()) {
+      long txIdVal = iter.next().longValue();
+      String fileName = NNStorage.getLegacyOIVImageFileName(txIdVal);
+      LOG.info("Deleting " + fileName);
+      File fileToDelete = new File(oivImageDir, fileName);
+      if (!fileToDelete.delete()) {
+        // deletion failed.
+        LOG.warn("Failed to delete image file: " + fileToDelete);
+      }
+      numFilesToDelete--;
+    }
+  }
 }

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

@@ -1677,7 +1677,11 @@ public class NameNode implements NameNodeStatusMXBean {
   public boolean isStandbyState() {
     return (state.equals(STANDBY_STATE));
   }
-
+  
+  public boolean isActiveState() {
+    return (state.equals(ACTIVE_STATE));
+  }
+  
   /**
    * Check that a request to change this node's HA state is valid.
    * In particular, verifies that, if auto failover is enabled, non-forced

+ 16 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java

@@ -62,6 +62,7 @@ import org.apache.hadoop.hdfs.server.namenode.NNStorageRetentionManager.StorageP
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
+import org.apache.hadoop.hdfs.util.Canceler;
 import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.io.MD5Hash;
@@ -125,6 +126,7 @@ public class SecondaryNameNode implements Runnable,
 
   private Thread checkpointThread;
   private ObjectName nameNodeStatusBeanName;
+  private String legacyOivImageDir;
 
   @Override
   public String toString() {
@@ -289,6 +291,9 @@ public class SecondaryNameNode implements Runnable,
           NetUtils.getHostPortString(httpsAddress));
     }
 
+    legacyOivImageDir = conf.get(
+        DFSConfigKeys.DFS_NAMENODE_LEGACY_OIV_IMAGE_DIR_KEY);
+
     LOG.info("Checkpoint Period   :" + checkpointConf.getPeriod() + " secs "
         + "(" + checkpointConf.getPeriod() / 60 + " min)");
     LOG.info("Log Size Trigger    :" + checkpointConf.getTxnCount() + " txns");
@@ -497,6 +502,7 @@ public class SecondaryNameNode implements Runnable,
    * @return if the image is fetched from primary or not
    */
   @VisibleForTesting
+  @SuppressWarnings("deprecated")
   public boolean doCheckpoint() throws IOException {
     checkpointImage.ensureCurrentDirExists();
     NNStorage dstStorage = checkpointImage.getStorage();
@@ -559,11 +565,18 @@ public class SecondaryNameNode implements Runnable,
 
     LOG.warn("Checkpoint done. New Image Size: " 
              + dstStorage.getFsImageName(txid).length());
-    
+
+    if (legacyOivImageDir != null && !legacyOivImageDir.isEmpty()) {
+      try {
+        checkpointImage.saveLegacyOIVImage(namesystem, legacyOivImageDir,
+            new Canceler());
+      } catch (IOException e) {
+        LOG.warn("Failed to write legacy OIV image: ", e);
+      }
+    }
     return loadImage;
   }
-  
-  
+
   /**
    * @param opts The parameters passed to this program.
    * @exception Exception if the filesystem does not exist.

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java

@@ -183,6 +183,12 @@ public class StandbyCheckpointer {
       txid = img.getStorage().getMostRecentCheckpointTxId();
       assert txid == thisCheckpointTxId : "expected to save checkpoint at txid=" +
         thisCheckpointTxId + " but instead saved at txid=" + txid;
+
+      // Save the legacy OIV image, if the output dir is defined.
+      String outputDir = checkpointConf.getLegacyOivImageDir();
+      if (outputDir != null && !outputDir.isEmpty()) {
+        img.saveLegacyOIVImage(namesystem, outputDir, canceler);
+      }
     } finally {
       namesystem.longReadUnlock();
     }

+ 13 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiff.java

@@ -17,13 +17,17 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
-import com.google.common.base.Preconditions;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeAttributes;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
 
-import java.util.List;
+import com.google.common.base.Preconditions;
 
 /**
  * The difference of an inode between in two snapshots.
@@ -128,4 +132,11 @@ abstract class AbstractINodeDiff<N extends INode,
     return getClass().getSimpleName() + ": " + this.getSnapshotId() + " (post="
         + (posteriorDiff == null? null: posteriorDiff.getSnapshotId()) + ")";
   }
+
+  void writeSnapshot(DataOutput out) throws IOException {
+    out.writeInt(snapshotId);
+  }
+
+  abstract void write(DataOutput out, ReferenceMap referenceMap
+      ) throws IOException;
 }

+ 52 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java

@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
+import java.io.DataOutput;
+import java.io.IOException;
 import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -32,6 +34,7 @@ import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
 import org.apache.hadoop.hdfs.server.namenode.Content;
 import org.apache.hadoop.hdfs.server.namenode.ContentSummaryComputationContext;
+import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
@@ -39,6 +42,7 @@ import org.apache.hadoop.hdfs.server.namenode.INodeDirectoryAttributes;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
 import org.apache.hadoop.hdfs.util.Diff;
 import org.apache.hadoop.hdfs.util.Diff.Container;
 import org.apache.hadoop.hdfs.util.Diff.ListType;
@@ -120,6 +124,35 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
       return counts;
     }
 
+    /** Serialize {@link #created} */
+    private void writeCreated(DataOutput out) throws IOException {
+      final List<INode> created = getList(ListType.CREATED);
+      out.writeInt(created.size());
+      for (INode node : created) {
+        // For INode in created list, we only need to record its local name
+        byte[] name = node.getLocalNameBytes();
+        out.writeShort(name.length);
+        out.write(name);
+      }
+    }
+
+    /** Serialize {@link #deleted} */
+    private void writeDeleted(DataOutput out,
+        ReferenceMap referenceMap) throws IOException {
+      final List<INode> deleted = getList(ListType.DELETED);
+      out.writeInt(deleted.size());
+      for (INode node : deleted) {
+        FSImageSerialization.saveINode2Image(node, out, true, referenceMap);
+      }
+    }
+
+    /** Serialize to out */
+    private void write(DataOutput out, ReferenceMap referenceMap
+        ) throws IOException {
+      writeCreated(out);
+      writeDeleted(out, referenceMap);
+    }
+
     /** Get the list of INodeDirectory contained in the deleted list */
     private void getDirsInDeleted(List<INodeDirectory> dirList) {
       for (INode node : getList(ListType.DELETED)) {
@@ -314,6 +347,25 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
       return childrenSize;
     }
 
+    @Override
+    void write(DataOutput out, ReferenceMap referenceMap) throws IOException {
+      writeSnapshot(out);
+      out.writeInt(childrenSize);
+
+      // Write snapshotINode
+      out.writeBoolean(isSnapshotRoot);
+      if (!isSnapshotRoot) {
+        if (snapshotINode != null) {
+          out.writeBoolean(true);
+          FSImageSerialization.writeINodeDirectoryAttributes(snapshotINode, out);
+        } else {
+          out.writeBoolean(false);
+        }
+      }
+      // Write diff. Node need to write poseriorDiff, since diffs is a list.
+      diff.write(out, referenceMap);
+    }
+
     @Override
     Quota.Counts destroyDiffAndCollectBlocks(INodeDirectory currentINode,
         BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) {

+ 20 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiff.java

@@ -17,13 +17,17 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeFileAttributes;
 import org.apache.hadoop.hdfs.server.namenode.Quota;
-
-import java.util.List;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
 
 /**
  * The difference of an {@link INodeFile} between two snapshots.
@@ -66,6 +70,20 @@ public class FileDiff extends
         + (snapshotINode == null? "?": snapshotINode.getFileReplication());
   }
 
+  @Override
+  void write(DataOutput out, ReferenceMap referenceMap) throws IOException {
+    writeSnapshot(out);
+    out.writeLong(fileSize);
+
+    // write snapshotINode
+    if (snapshotINode != null) {
+      out.writeBoolean(true);
+      FSImageSerialization.writeINodeFileAttributes(snapshotINode, out);
+    } else {
+      out.writeBoolean(false);
+    }
+  }
+
   @Override
   Quota.Counts destroyDiffAndCollectBlocks(INodeFile currentINode,
       BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) {

+ 9 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/Snapshot.java

@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
 import java.io.DataInput;
+import java.io.DataOutput;
 import java.io.IOException;
 import java.text.SimpleDateFormat;
 import java.util.Arrays;
@@ -30,6 +31,7 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.namenode.AclFeature;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormat;
+import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
@@ -214,4 +216,11 @@ public class Snapshot implements Comparable<byte[]> {
   public String toString() {
     return getClass().getSimpleName() + "." + root.getLocalName() + "(id=" + id + ")";
   }
+
+  /** Serialize the fields to out */
+  void write(DataOutput out) throws IOException {
+    out.writeInt(id);
+    // write root
+    FSImageSerialization.writeINodeDirectory(root, out);
+  }
 }

+ 72 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotFSImageFormat.java

@@ -29,21 +29,75 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormat;
 import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
 import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.namenode.INodeAttributes;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectoryAttributes;
+import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeFileAttributes;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiffList;
 import org.apache.hadoop.hdfs.tools.snapshot.SnapshotDiff;
 import org.apache.hadoop.hdfs.util.Diff.ListType;
-import org.apache.hadoop.hdfs.server.namenode.FSImageFormat.Loader;
+import org.apache.hadoop.hdfs.util.ReadOnlyList;
 
 /**
  * A helper class defining static methods for reading/writing snapshot related
  * information from/to FSImage.
  */
 public class SnapshotFSImageFormat {
+  /**
+   * Save snapshots and snapshot quota for a snapshottable directory.
+   * @param current The directory that the snapshots belongs to.
+   * @param out The {@link DataOutput} to write.
+   * @throws IOException
+   */
+  public static void saveSnapshots(INodeDirectorySnapshottable current,
+      DataOutput out) throws IOException {
+    // list of snapshots in snapshotsByNames
+    ReadOnlyList<Snapshot> snapshots = current.getSnapshotsByNames();
+    out.writeInt(snapshots.size());
+    for (Snapshot s : snapshots) {
+      // write the snapshot id
+      out.writeInt(s.getId());
+    }
+    // snapshot quota
+    out.writeInt(current.getSnapshotQuota());
+  }
+
+  /**
+   * Save SnapshotDiff list for an INodeDirectoryWithSnapshot.
+   * @param sNode The directory that the SnapshotDiff list belongs to.
+   * @param out The {@link DataOutput} to write.
+   */
+  private static <N extends INode, A extends INodeAttributes, D extends AbstractINodeDiff<N, A, D>>
+      void saveINodeDiffs(final AbstractINodeDiffList<N, A, D> diffs,
+      final DataOutput out, ReferenceMap referenceMap) throws IOException {
+    // Record the diffs in reversed order, so that we can find the correct
+    // reference for INodes in the created list when loading the FSImage
+    if (diffs == null) {
+      out.writeInt(-1); // no diffs
+    } else {
+      final List<D> list = diffs.asList();
+      final int size = list.size();
+      out.writeInt(size);
+      for (int i = size - 1; i >= 0; i--) {
+        list.get(i).write(out, referenceMap);
+      }
+    }
+  }
+
+  public static void saveDirectoryDiffList(final INodeDirectory dir,
+      final DataOutput out, final ReferenceMap referenceMap
+      ) throws IOException {
+    saveINodeDiffs(dir.getDiffs(), out, referenceMap);
+  }
+
+  public static void saveFileDiffList(final INodeFile file,
+      final DataOutput out) throws IOException {
+    saveINodeDiffs(file.getDiffs(), out, null);
+  }
+
   public static FileDiffList loadFileDiffList(DataInput in,
       FSImageFormat.Loader loader) throws IOException {
     final int size = in.readInt();
@@ -264,6 +318,23 @@ public class SnapshotFSImageFormat {
      * Used to record whether the subtree of the reference node has been saved 
      */
     private final Map<Long, Long> dirMap = new HashMap<Long, Long>();
+
+    public void writeINodeReferenceWithCount(
+        INodeReference.WithCount withCount, DataOutput out,
+        boolean writeUnderConstruction) throws IOException {
+      final INode referred = withCount.getReferredINode();
+      final long id = withCount.getId();
+      final boolean firstReferred = !referenceMap.containsKey(id);
+      out.writeBoolean(firstReferred);
+
+      if (firstReferred) {
+        FSImageSerialization.saveINode2Image(referred, out,
+            writeUnderConstruction, this);
+        referenceMap.put(id, withCount);
+      } else {
+        out.writeLong(id);
+      }
+    }
     
     public boolean toProcessSubtree(long id) {
       if (dirMap.containsKey(id)) {

+ 17 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java

@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
 import java.io.DataInput;
+import java.io.DataOutput;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -287,6 +288,22 @@ public class SnapshotManager implements SnapshotStatsMXBean {
     return snapshottables.values().toArray(
         new INodeDirectorySnapshottable[snapshottables.size()]);
   }
+
+  /**
+   * Write {@link #snapshotCounter}, {@link #numSnapshots},
+   * and all snapshots to the DataOutput.
+   */
+  public void write(DataOutput out) throws IOException {
+    out.writeInt(snapshotCounter);
+    out.writeInt(numSnapshots.get());
+
+    // write all snapshots.
+    for(INodeDirectorySnapshottable snapshottableDir : snapshottables.values()) {
+      for(Snapshot s : snapshottableDir.getSnapshotsByNames()) {
+        s.write(out);
+      }
+    }
+  }
   
   /**
    * Read values of {@link #snapshotCounter}, {@link #numSnapshots}, and

+ 8 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CacheAdmin.java

@@ -503,19 +503,21 @@ public class CacheAdmin extends Configured implements Tool {
 
     @Override
     public String getShortUsage() {
-      return "[" + getName() + " [-stats] [-path <path>] [-pool <pool>]]\n";
+      return "[" + getName()
+          + " [-stats] [-path <path>] [-pool <pool>] [-id <id>]\n";
     }
 
     @Override
     public String getLongUsage() {
       TableListing listing = getOptionDescriptionListing();
+      listing.addRow("-stats", "List path-based cache directive statistics.");
       listing.addRow("<path>", "List only " +
           "cache directives with this path. " +
           "Note that if there is a cache directive for <path> " +
           "in a cache pool that we don't have read access for, it " + 
           "will not be listed.");
       listing.addRow("<pool>", "List only path cache directives in that pool.");
-      listing.addRow("-stats", "List path-based cache directive statistics.");
+      listing.addRow("<id>", "List the cache directive with this id.");
       return getShortUsage() + "\n" +
         "List cache directives.\n\n" +
         listing.toString();
@@ -534,6 +536,10 @@ public class CacheAdmin extends Configured implements Tool {
         builder.setPool(poolFilter);
       }
       boolean printStats = StringUtils.popOption("-stats", args);
+      String idFilter = StringUtils.popOptionWithArgument("-id", args);
+      if (idFilter != null) {
+        builder.setId(Long.parseLong(idFilter));
+      }
       if (!args.isEmpty()) {
         System.err.println("Can't understand argument: " + args.get(0));
         return 1;

+ 11 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSHAAdmin.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.tools;
 
 import java.io.PrintStream;
 import java.util.Arrays;
+import java.util.Collection;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -27,6 +28,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.ha.HAAdmin;
 import org.apache.hadoop.ha.HAServiceTarget;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.util.ToolRunner;
 
@@ -117,7 +119,15 @@ public class DFSHAAdmin extends HAAdmin {
     
     return super.runCmd(argv);
   }
-
+  
+  /**
+   * returns the list of all namenode ids for the given configuration 
+   */
+  @Override
+  protected Collection<String> getTargetIds(String namenodeToActivate) {
+    return DFSUtil.getNameNodeIds(getConf(), (nameserviceId != null)? nameserviceId : DFSUtil.getNamenodeNameServiceId(getConf()));
+  }
+  
   public static void main(String[] argv) throws Exception {
     int res = ToolRunner.run(new DFSHAAdmin(), argv);
     System.exit(res);

+ 172 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/DelimitedImageVisitor.java

@@ -0,0 +1,172 @@
+/**
+ * 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.tools.offlineImageViewer;
+
+import java.io.IOException;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedList;
+
+/**
+ * A DelimitedImageVisitor generates a text representation of the fsimage,
+ * with each element separated by a delimiter string.  All of the elements
+ * common to both inodes and inodes-under-construction are included. When 
+ * processing an fsimage with a layout version that did not include an 
+ * element, such as AccessTime, the output file will include a column
+ * for the value, but no value will be included.
+ * 
+ * Individual block information for each file is not currently included.
+ * 
+ * The default delimiter is tab, as this is an unlikely value to be included
+ * an inode path or other text metadata.  The delimiter value can be via the
+ * constructor.
+ */
+class DelimitedImageVisitor extends TextWriterImageVisitor {
+  private static final String defaultDelimiter = "\t"; 
+  
+  final private LinkedList<ImageElement> elemQ = new LinkedList<ImageElement>();
+  private long fileSize = 0l;
+  // Elements of fsimage we're interested in tracking
+  private final Collection<ImageElement> elementsToTrack;
+  // Values for each of the elements in elementsToTrack
+  private final AbstractMap<ImageElement, String> elements = 
+                                            new HashMap<ImageElement, String>();
+  private final String delimiter;
+
+  {
+    elementsToTrack = new ArrayList<ImageElement>();
+    
+    // This collection determines what elements are tracked and the order
+    // in which they are output
+    Collections.addAll(elementsToTrack,  ImageElement.INODE_PATH,
+                                         ImageElement.REPLICATION,
+                                         ImageElement.MODIFICATION_TIME,
+                                         ImageElement.ACCESS_TIME,
+                                         ImageElement.BLOCK_SIZE,
+                                         ImageElement.NUM_BLOCKS,
+                                         ImageElement.NUM_BYTES,
+                                         ImageElement.NS_QUOTA,
+                                         ImageElement.DS_QUOTA,
+                                         ImageElement.PERMISSION_STRING,
+                                         ImageElement.USER_NAME,
+                                         ImageElement.GROUP_NAME);
+  }
+  
+  public DelimitedImageVisitor(String filename) throws IOException {
+    this(filename, false);
+  }
+
+  public DelimitedImageVisitor(String outputFile, boolean printToScreen) 
+                                                           throws IOException {
+    this(outputFile, printToScreen, defaultDelimiter);
+  }
+  
+  public DelimitedImageVisitor(String outputFile, boolean printToScreen, 
+                               String delimiter) throws IOException {
+    super(outputFile, printToScreen);
+    this.delimiter = delimiter;
+    reset();
+  }
+
+  /**
+   * Reset the values of the elements we're tracking in order to handle
+   * the next file
+   */
+  private void reset() {
+    elements.clear();
+    for(ImageElement e : elementsToTrack) 
+      elements.put(e, null);
+    
+    fileSize = 0l;
+  }
+  
+  @Override
+  void leaveEnclosingElement() throws IOException {
+    ImageElement elem = elemQ.pop();
+
+    // If we're done with an inode, write out our results and start over
+    if(elem == ImageElement.INODE || 
+       elem == ImageElement.INODE_UNDER_CONSTRUCTION) {
+      writeLine();
+      write("\n");
+      reset();
+    }
+  }
+
+  /**
+   * Iterate through all the elements we're tracking and, if a value was
+   * recorded for it, write it out.
+   */
+  private void writeLine() throws IOException {
+    Iterator<ImageElement> it = elementsToTrack.iterator();
+    
+    while(it.hasNext()) {
+      ImageElement e = it.next();
+      
+      String v = null;
+      if(e == ImageElement.NUM_BYTES)
+        v = String.valueOf(fileSize);
+      else
+        v = elements.get(e);
+      
+      if(v != null)
+        write(v);
+      
+      if(it.hasNext())
+        write(delimiter);
+    }
+  }
+
+  @Override
+  void visit(ImageElement element, String value) throws IOException {
+    // Explicitly label the root path
+    if(element == ImageElement.INODE_PATH && value.equals(""))
+      value = "/";
+    
+    // Special case of file size, which is sum of the num bytes in each block
+    if(element == ImageElement.NUM_BYTES)
+      fileSize += Long.valueOf(value);
+    
+    if(elements.containsKey(element) && element != ImageElement.NUM_BYTES)
+      elements.put(element, value);
+    
+  }
+
+  @Override
+  void visitEnclosingElement(ImageElement element) throws IOException {
+    elemQ.push(element);
+  }
+
+  @Override
+  void visitEnclosingElement(ImageElement element, ImageElement key,
+      String value) throws IOException {
+    // Special case as numBlocks is an attribute of the blocks element
+    if(key == ImageElement.NUM_BLOCKS 
+        && elements.containsKey(ImageElement.NUM_BLOCKS))
+      elements.put(key, value);
+    
+    elemQ.push(element);
+  }
+  
+  @Override
+  void start() throws IOException { /* Nothing to do */ }
+}

+ 36 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/DepthCounter.java

@@ -0,0 +1,36 @@
+/**
+ * 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.tools.offlineImageViewer;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Utility class for tracking descent into the structure of the
+ * Visitor class (ImageVisitor, EditsVisitor etc.)
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class DepthCounter {
+  private int depth = 0;
+
+  public void incLevel() { depth++; }
+  public void decLevel() { if(depth >= 1) depth--; }
+  public int  getLevel() { return depth; }
+}
+

+ 193 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FileDistributionVisitor.java

@@ -0,0 +1,193 @@
+/**
+ * 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.tools.offlineImageViewer;
+
+import java.io.IOException;
+import java.util.LinkedList;
+
+/**
+ * File size distribution visitor.
+ * 
+ * <h3>Description.</h3>
+ * This is the tool for analyzing file sizes in the namespace image.
+ * In order to run the tool one should define a range of integers
+ * <tt>[0, maxSize]</tt> by specifying <tt>maxSize</tt> and a <tt>step</tt>.
+ * The range of integers is divided into segments of size <tt>step</tt>: 
+ * <tt>[0, s<sub>1</sub>, ..., s<sub>n-1</sub>, maxSize]</tt>,
+ * and the visitor calculates how many files in the system fall into 
+ * each segment <tt>[s<sub>i-1</sub>, s<sub>i</sub>)</tt>. 
+ * Note that files larger than <tt>maxSize</tt> always fall into 
+ * the very last segment.
+ * 
+ * <h3>Input.</h3>
+ * <ul>
+ * <li><tt>filename</tt> specifies the location of the image file;</li>
+ * <li><tt>maxSize</tt> determines the range <tt>[0, maxSize]</tt> of files
+ * sizes considered by the visitor;</li>
+ * <li><tt>step</tt> the range is divided into segments of size step.</li>
+ * </ul>
+ *
+ * <h3>Output.</h3>
+ * The output file is formatted as a tab separated two column table:
+ * Size and NumFiles. Where Size represents the start of the segment,
+ * and numFiles is the number of files form the image which size falls in 
+ * this segment.
+ */
+class FileDistributionVisitor extends TextWriterImageVisitor {
+  final private LinkedList<ImageElement> elemS = new LinkedList<ImageElement>();
+
+  private final static long MAX_SIZE_DEFAULT = 0x2000000000L;   // 1/8 TB = 2^37
+  private final static int INTERVAL_DEFAULT = 0x200000;         // 2 MB = 2^21
+
+  private int[] distribution;
+  private long maxSize;
+  private int step;
+
+  private int totalFiles;
+  private int totalDirectories;
+  private int totalBlocks;
+  private long totalSpace;
+  private long maxFileSize;
+
+  private FileContext current;
+
+  private boolean inInode = false;
+
+  /**
+   * File or directory information.
+   */
+  private static class FileContext {
+    String path;
+    long fileSize;
+    int numBlocks;
+    int replication;
+  }
+
+  public FileDistributionVisitor(String filename,
+                                 long maxSize,
+                                 int step) throws IOException {
+    super(filename, false);
+    this.maxSize = (maxSize == 0 ? MAX_SIZE_DEFAULT : maxSize);
+    this.step = (step == 0 ? INTERVAL_DEFAULT : step);
+    long numIntervals = this.maxSize / this.step;
+    if(numIntervals >= Integer.MAX_VALUE)
+      throw new IOException("Too many distribution intervals " + numIntervals);
+    this.distribution = new int[1 + (int)(numIntervals)];
+    this.totalFiles = 0;
+    this.totalDirectories = 0;
+    this.totalBlocks = 0;
+    this.totalSpace = 0;
+    this.maxFileSize = 0;
+  }
+
+  @Override
+  void start() throws IOException {}
+
+  @Override
+  void finish() throws IOException {
+    output();
+    super.finish();
+  }
+
+  @Override
+  void finishAbnormally() throws IOException {
+    System.out.println("*** Image processing finished abnormally.  Ending ***");
+    output();
+    super.finishAbnormally();
+  }
+
+  private void output() throws IOException {
+    // write the distribution into the output file
+    write("Size\tNumFiles\n");
+    for(int i = 0; i < distribution.length; i++)
+      write(((long)i * step) + "\t" + distribution[i] + "\n");
+    System.out.println("totalFiles = " + totalFiles);
+    System.out.println("totalDirectories = " + totalDirectories);
+    System.out.println("totalBlocks = " + totalBlocks);
+    System.out.println("totalSpace = " + totalSpace);
+    System.out.println("maxFileSize = " + maxFileSize);
+  }
+
+  @Override
+  void leaveEnclosingElement() throws IOException {
+    ImageElement elem = elemS.pop();
+
+    if(elem != ImageElement.INODE &&
+       elem != ImageElement.INODE_UNDER_CONSTRUCTION)
+      return;
+    inInode = false;
+    if(current.numBlocks < 0) {
+      totalDirectories ++;
+      return;
+    }
+    totalFiles++;
+    totalBlocks += current.numBlocks;
+    totalSpace += current.fileSize * current.replication;
+    if(maxFileSize < current.fileSize)
+      maxFileSize = current.fileSize;
+    int high;
+    if(current.fileSize > maxSize)
+      high = distribution.length-1;
+    else
+      high = (int)Math.ceil((double)current.fileSize / step);
+    distribution[high]++;
+    if(totalFiles % 1000000 == 1)
+      System.out.println("Files processed: " + totalFiles
+          + "  Current: " + current.path);
+  }
+
+  @Override
+  void visit(ImageElement element, String value) throws IOException {
+    if(inInode) {
+      switch(element) {
+      case INODE_PATH:
+        current.path = (value.equals("") ? "/" : value);
+        break;
+      case REPLICATION:
+        current.replication = Integer.valueOf(value);
+        break;
+      case NUM_BYTES:
+        current.fileSize += Long.valueOf(value);
+        break;
+      default:
+        break;
+      }
+    }
+  }
+
+  @Override
+  void visitEnclosingElement(ImageElement element) throws IOException {
+    elemS.push(element);
+    if(element == ImageElement.INODE ||
+       element == ImageElement.INODE_UNDER_CONSTRUCTION) {
+      current = new FileContext();
+      inInode = true;
+    }
+  }
+
+  @Override
+  void visitEnclosingElement(ImageElement element,
+      ImageElement key, String value) throws IOException {
+    elemS.push(element);
+    if(element == ImageElement.INODE ||
+       element == ImageElement.INODE_UNDER_CONSTRUCTION)
+      inInode = true;
+    else if(element == ImageElement.BLOCKS)
+      current.numBlocks = Integer.parseInt(value);
+  }
+}

+ 83 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoader.java

@@ -0,0 +1,83 @@
+/**
+ * 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.tools.offlineImageViewer;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * An ImageLoader can accept a DataInputStream to an Hadoop FSImage file
+ * and walk over its structure using the supplied ImageVisitor.
+ *
+ * Each implementation of ImageLoader is designed to rapidly process an
+ * image file.  As long as minor changes are made from one layout version
+ * to another, it is acceptable to tweak one implementation to read the next.
+ * However, if the layout version changes enough that it would make a
+ * processor slow or difficult to read, another processor should be created.
+ * This allows each processor to quickly read an image without getting
+ * bogged down in dealing with significant differences between layout versions.
+ */
+interface ImageLoader {
+
+  /**
+   * @param in DataInputStream pointing to an Hadoop FSImage file
+   * @param v Visit to apply to the FSImage file
+   * @param enumerateBlocks Should visitor visit each of the file blocks?
+   */
+  public void loadImage(DataInputStream in, ImageVisitor v,
+      boolean enumerateBlocks) throws IOException;
+
+  /**
+   * Can this processor handle the specified version of FSImage file?
+   *
+   * @param version FSImage version file
+   * @return True if this instance can process the file
+   */
+  public boolean canLoadVersion(int version);
+
+  /**
+   * Factory for obtaining version of image loader that can read
+   * a particular image format.
+   */
+  @InterfaceAudience.Private
+  public class LoaderFactory {
+    // Java doesn't support static methods on interfaces, which necessitates
+    // this factory class
+
+    /**
+     * Find an image loader capable of interpreting the specified
+     * layout version number.  If none, return null;
+     *
+     * @param version fsimage layout version number to be processed
+     * @return ImageLoader that can interpret specified version, or null
+     */
+    static public ImageLoader getLoader(int version) {
+      // Easy to add more image processors as they are written
+      ImageLoader[] loaders = { new ImageLoaderCurrent() };
+
+      for (ImageLoader l : loaders) {
+        if (l.canLoadVersion(version))
+          return l;
+      }
+
+      return null;
+    }
+  }
+}

+ 821 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java

@@ -0,0 +1,821 @@
+/**
+ * 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.tools.offlineImageViewer;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
+import org.apache.hadoop.hdfs.protocol.LayoutFlags;
+import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
+import org.apache.hadoop.hdfs.server.namenode.INodeId;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion;
+import org.apache.hadoop.hdfs.tools.offlineImageViewer.ImageVisitor.ImageElement;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.apache.hadoop.security.token.delegation.DelegationKey;
+
+/**
+ * ImageLoaderCurrent processes Hadoop FSImage files and walks over
+ * them using a provided ImageVisitor, calling the visitor at each element
+ * enumerated below.
+ *
+ * The only difference between v18 and v19 was the utilization of the
+ * stickybit.  Therefore, the same viewer can reader either format.
+ *
+ * Versions -19 fsimage layout (with changes from -16 up):
+ * Image version (int)
+ * Namepsace ID (int)
+ * NumFiles (long)
+ * Generation stamp (long)
+ * INodes (count = NumFiles)
+ *  INode
+ *    Path (String)
+ *    Replication (short)
+ *    Modification Time (long as date)
+ *    Access Time (long) // added in -16
+ *    Block size (long)
+ *    Num blocks (int)
+ *    Blocks (count = Num blocks)
+ *      Block
+ *        Block ID (long)
+ *        Num bytes (long)
+ *        Generation stamp (long)
+ *    Namespace Quota (long)
+ *    Diskspace Quota (long) // added in -18
+ *    Permissions
+ *      Username (String)
+ *      Groupname (String)
+ *      OctalPerms (short -> String)  // Modified in -19
+ *    Symlink (String) // added in -23
+ * NumINodesUnderConstruction (int)
+ * INodesUnderConstruction (count = NumINodesUnderConstruction)
+ *  INodeUnderConstruction
+ *    Path (bytes as string)
+ *    Replication (short)
+ *    Modification time (long as date)
+ *    Preferred block size (long)
+ *    Num blocks (int)
+ *    Blocks
+ *      Block
+ *        Block ID (long)
+ *        Num bytes (long)
+ *        Generation stamp (long)
+ *    Permissions
+ *      Username (String)
+ *      Groupname (String)
+ *      OctalPerms (short -> String)
+ *    Client Name (String)
+ *    Client Machine (String)
+ *    NumLocations (int)
+ *    DatanodeDescriptors (count = numLocations) // not loaded into memory
+ *      short                                    // but still in file
+ *      long
+ *      string
+ *      long
+ *      int
+ *      string
+ *      string
+ *      enum
+ *    CurrentDelegationKeyId (int)
+ *    NumDelegationKeys (int)
+ *      DelegationKeys (count = NumDelegationKeys)
+ *        DelegationKeyLength (vint)
+ *        DelegationKey (bytes)
+ *    DelegationTokenSequenceNumber (int)
+ *    NumDelegationTokens (int)
+ *    DelegationTokens (count = NumDelegationTokens)
+ *      DelegationTokenIdentifier
+ *        owner (String)
+ *        renewer (String)
+ *        realUser (String)
+ *        issueDate (vlong)
+ *        maxDate (vlong)
+ *        sequenceNumber (vint)
+ *        masterKeyId (vint)
+ *      expiryTime (long)     
+ *
+ */
+class ImageLoaderCurrent implements ImageLoader {
+  protected final DateFormat dateFormat = 
+                                      new SimpleDateFormat("yyyy-MM-dd HH:mm");
+  private static int[] versions = { -16, -17, -18, -19, -20, -21, -22, -23,
+      -24, -25, -26, -27, -28, -30, -31, -32, -33, -34, -35, -36, -37, -38, -39,
+      -40, -41, -42, -43, -44, -45, -46, -47, -48, -49, -50, -51 };
+  private int imageVersion = 0;
+  
+  private final Map<Long, Boolean> subtreeMap = new HashMap<Long, Boolean>();
+  private final Map<Long, String> dirNodeMap = new HashMap<Long, String>();
+
+  /* (non-Javadoc)
+   * @see ImageLoader#canProcessVersion(int)
+   */
+  @Override
+  public boolean canLoadVersion(int version) {
+    for(int v : versions)
+      if(v == version) return true;
+
+    return false;
+  }
+
+  /* (non-Javadoc)
+   * @see ImageLoader#processImage(java.io.DataInputStream, ImageVisitor, boolean)
+   */
+  @Override
+  public void loadImage(DataInputStream in, ImageVisitor v,
+      boolean skipBlocks) throws IOException {
+    boolean done = false;
+    try {
+      v.start();
+      v.visitEnclosingElement(ImageElement.FS_IMAGE);
+
+      imageVersion = in.readInt();
+      if( !canLoadVersion(imageVersion))
+        throw new IOException("Cannot process fslayout version " + imageVersion);
+      if (NameNodeLayoutVersion.supports(Feature.ADD_LAYOUT_FLAGS, imageVersion)) {
+        LayoutFlags.read(in);
+      }
+
+      v.visit(ImageElement.IMAGE_VERSION, imageVersion);
+      v.visit(ImageElement.NAMESPACE_ID, in.readInt());
+
+      long numInodes = in.readLong();
+
+      v.visit(ImageElement.GENERATION_STAMP, in.readLong());
+
+      if (NameNodeLayoutVersion.supports(Feature.SEQUENTIAL_BLOCK_ID, imageVersion)) {
+        v.visit(ImageElement.GENERATION_STAMP_V2, in.readLong());
+        v.visit(ImageElement.GENERATION_STAMP_V1_LIMIT, in.readLong());
+        v.visit(ImageElement.LAST_ALLOCATED_BLOCK_ID, in.readLong());
+      }
+
+      if (NameNodeLayoutVersion.supports(Feature.STORED_TXIDS, imageVersion)) {
+        v.visit(ImageElement.TRANSACTION_ID, in.readLong());
+      }
+      
+      if (NameNodeLayoutVersion.supports(Feature.ADD_INODE_ID, imageVersion)) {
+        v.visit(ImageElement.LAST_INODE_ID, in.readLong());
+      }
+      
+      boolean supportSnapshot = NameNodeLayoutVersion.supports(Feature.SNAPSHOT,
+          imageVersion);
+      if (supportSnapshot) {
+        v.visit(ImageElement.SNAPSHOT_COUNTER, in.readInt());
+        int numSnapshots = in.readInt();
+        v.visit(ImageElement.NUM_SNAPSHOTS_TOTAL, numSnapshots);
+        for (int i = 0; i < numSnapshots; i++) {
+          processSnapshot(in, v);
+        }
+      }
+      
+      if (NameNodeLayoutVersion.supports(Feature.FSIMAGE_COMPRESSION, imageVersion)) {
+        boolean isCompressed = in.readBoolean();
+        v.visit(ImageElement.IS_COMPRESSED, String.valueOf(isCompressed));
+        if (isCompressed) {
+          String codecClassName = Text.readString(in);
+          v.visit(ImageElement.COMPRESS_CODEC, codecClassName);
+          CompressionCodecFactory codecFac = new CompressionCodecFactory(
+              new Configuration());
+          CompressionCodec codec = codecFac.getCodecByClassName(codecClassName);
+          if (codec == null) {
+            throw new IOException("Image compression codec not supported: "
+                + codecClassName);
+          }
+          in = new DataInputStream(codec.createInputStream(in));
+        }
+      }
+      processINodes(in, v, numInodes, skipBlocks, supportSnapshot);
+      subtreeMap.clear();
+      dirNodeMap.clear();
+
+      processINodesUC(in, v, skipBlocks);
+
+      if (NameNodeLayoutVersion.supports(Feature.DELEGATION_TOKEN, imageVersion)) {
+        processDelegationTokens(in, v);
+      }
+      
+      if (NameNodeLayoutVersion.supports(Feature.CACHING, imageVersion)) {
+        processCacheManagerState(in, v);
+      }
+      v.leaveEnclosingElement(); // FSImage
+      done = true;
+    } finally {
+      if (done) {
+        v.finish();
+      } else {
+        v.finishAbnormally();
+      }
+    }
+  }
+
+  /**
+   * Process CacheManager state from the fsimage.
+   */
+  private void processCacheManagerState(DataInputStream in, ImageVisitor v)
+      throws IOException {
+    v.visit(ImageElement.CACHE_NEXT_ENTRY_ID, in.readLong());
+    final int numPools = in.readInt();
+    for (int i=0; i<numPools; i++) {
+      v.visit(ImageElement.CACHE_POOL_NAME, Text.readString(in));
+      processCachePoolPermission(in, v);
+      v.visit(ImageElement.CACHE_POOL_WEIGHT, in.readInt());
+    }
+    final int numEntries = in.readInt();
+    for (int i=0; i<numEntries; i++) {
+      v.visit(ImageElement.CACHE_ENTRY_PATH, Text.readString(in));
+      v.visit(ImageElement.CACHE_ENTRY_REPLICATION, in.readShort());
+      v.visit(ImageElement.CACHE_ENTRY_POOL_NAME, Text.readString(in));
+    }
+  }
+  /**
+   * Process the Delegation Token related section in fsimage.
+   * 
+   * @param in DataInputStream to process
+   * @param v Visitor to walk over records
+   */
+  private void processDelegationTokens(DataInputStream in, ImageVisitor v)
+      throws IOException {
+    v.visit(ImageElement.CURRENT_DELEGATION_KEY_ID, in.readInt());
+    int numDKeys = in.readInt();
+    v.visitEnclosingElement(ImageElement.DELEGATION_KEYS,
+        ImageElement.NUM_DELEGATION_KEYS, numDKeys);
+    for(int i =0; i < numDKeys; i++) {
+      DelegationKey key = new DelegationKey();
+      key.readFields(in);
+      v.visit(ImageElement.DELEGATION_KEY, key.toString());
+    }
+    v.leaveEnclosingElement();
+    v.visit(ImageElement.DELEGATION_TOKEN_SEQUENCE_NUMBER, in.readInt());
+    int numDTokens = in.readInt();
+    v.visitEnclosingElement(ImageElement.DELEGATION_TOKENS,
+        ImageElement.NUM_DELEGATION_TOKENS, numDTokens);
+    for(int i=0; i<numDTokens; i++){
+      DelegationTokenIdentifier id = new  DelegationTokenIdentifier();
+      id.readFields(in);
+      long expiryTime = in.readLong();
+      v.visitEnclosingElement(ImageElement.DELEGATION_TOKEN_IDENTIFIER);
+      v.visit(ImageElement.DELEGATION_TOKEN_IDENTIFIER_KIND,
+          id.getKind().toString());
+      v.visit(ImageElement.DELEGATION_TOKEN_IDENTIFIER_SEQNO,
+          id.getSequenceNumber());
+      v.visit(ImageElement.DELEGATION_TOKEN_IDENTIFIER_OWNER,
+          id.getOwner().toString());
+      v.visit(ImageElement.DELEGATION_TOKEN_IDENTIFIER_RENEWER,
+          id.getRenewer().toString());
+      v.visit(ImageElement.DELEGATION_TOKEN_IDENTIFIER_REALUSER,
+          id.getRealUser().toString());
+      v.visit(ImageElement.DELEGATION_TOKEN_IDENTIFIER_ISSUE_DATE,
+          id.getIssueDate());
+      v.visit(ImageElement.DELEGATION_TOKEN_IDENTIFIER_MAX_DATE,
+          id.getMaxDate());
+      v.visit(ImageElement.DELEGATION_TOKEN_IDENTIFIER_EXPIRY_TIME,
+          expiryTime);
+      v.visit(ImageElement.DELEGATION_TOKEN_IDENTIFIER_MASTER_KEY_ID,
+          id.getMasterKeyId());
+      v.leaveEnclosingElement(); // DELEGATION_TOKEN_IDENTIFIER
+    }
+    v.leaveEnclosingElement(); // DELEGATION_TOKENS
+  }
+
+  /**
+   * Process the INodes under construction section of the fsimage.
+   *
+   * @param in DataInputStream to process
+   * @param v Visitor to walk over inodes
+   * @param skipBlocks Walk over each block?
+   */
+  private void processINodesUC(DataInputStream in, ImageVisitor v,
+      boolean skipBlocks) throws IOException {
+    int numINUC = in.readInt();
+
+    v.visitEnclosingElement(ImageElement.INODES_UNDER_CONSTRUCTION,
+                           ImageElement.NUM_INODES_UNDER_CONSTRUCTION, numINUC);
+
+    for(int i = 0; i < numINUC; i++) {
+      v.visitEnclosingElement(ImageElement.INODE_UNDER_CONSTRUCTION);
+      byte [] name = FSImageSerialization.readBytes(in);
+      String n = new String(name, "UTF8");
+      v.visit(ImageElement.INODE_PATH, n);
+      
+      if (NameNodeLayoutVersion.supports(Feature.ADD_INODE_ID, imageVersion)) {
+        long inodeId = in.readLong();
+        v.visit(ImageElement.INODE_ID, inodeId);
+      }
+      
+      v.visit(ImageElement.REPLICATION, in.readShort());
+      v.visit(ImageElement.MODIFICATION_TIME, formatDate(in.readLong()));
+
+      v.visit(ImageElement.PREFERRED_BLOCK_SIZE, in.readLong());
+      int numBlocks = in.readInt();
+      processBlocks(in, v, numBlocks, skipBlocks);
+
+      processPermission(in, v);
+      v.visit(ImageElement.CLIENT_NAME, FSImageSerialization.readString(in));
+      v.visit(ImageElement.CLIENT_MACHINE, FSImageSerialization.readString(in));
+
+      // Skip over the datanode descriptors, which are still stored in the
+      // file but are not used by the datanode or loaded into memory
+      int numLocs = in.readInt();
+      for(int j = 0; j < numLocs; j++) {
+        in.readShort();
+        in.readLong();
+        in.readLong();
+        in.readLong();
+        in.readInt();
+        FSImageSerialization.readString(in);
+        FSImageSerialization.readString(in);
+        WritableUtils.readEnum(in, AdminStates.class);
+      }
+
+      v.leaveEnclosingElement(); // INodeUnderConstruction
+    }
+
+    v.leaveEnclosingElement(); // INodesUnderConstruction
+  }
+
+  /**
+   * Process the blocks section of the fsimage.
+   *
+   * @param in Datastream to process
+   * @param v Visitor to walk over inodes
+   * @param skipBlocks Walk over each block?
+   */
+  private void processBlocks(DataInputStream in, ImageVisitor v,
+      int numBlocks, boolean skipBlocks) throws IOException {
+    v.visitEnclosingElement(ImageElement.BLOCKS,
+                            ImageElement.NUM_BLOCKS, numBlocks);
+    
+    // directory or symlink or reference node, no blocks to process    
+    if(numBlocks < 0) { 
+      v.leaveEnclosingElement(); // Blocks
+      return;
+    }
+    
+    if(skipBlocks) {
+      int bytesToSkip = ((Long.SIZE * 3 /* fields */) / 8 /*bits*/) * numBlocks;
+      if(in.skipBytes(bytesToSkip) != bytesToSkip)
+        throw new IOException("Error skipping over blocks");
+      
+    } else {
+      for(int j = 0; j < numBlocks; j++) {
+        v.visitEnclosingElement(ImageElement.BLOCK);
+        v.visit(ImageElement.BLOCK_ID, in.readLong());
+        v.visit(ImageElement.NUM_BYTES, in.readLong());
+        v.visit(ImageElement.GENERATION_STAMP, in.readLong());
+        v.leaveEnclosingElement(); // Block
+      }
+    }
+    v.leaveEnclosingElement(); // Blocks
+  }
+
+  /**
+   * Extract the INode permissions stored in the fsimage file.
+   *
+   * @param in Datastream to process
+   * @param v Visitor to walk over inodes
+   */
+  private void processPermission(DataInputStream in, ImageVisitor v)
+      throws IOException {
+    v.visitEnclosingElement(ImageElement.PERMISSIONS);
+    v.visit(ImageElement.USER_NAME, Text.readString(in));
+    v.visit(ImageElement.GROUP_NAME, Text.readString(in));
+    FsPermission fsp = new FsPermission(in.readShort());
+    v.visit(ImageElement.PERMISSION_STRING, fsp.toString());
+    v.leaveEnclosingElement(); // Permissions
+  }
+
+  /**
+   * Extract CachePool permissions stored in the fsimage file.
+   *
+   * @param in Datastream to process
+   * @param v Visitor to walk over inodes
+   */
+  private void processCachePoolPermission(DataInputStream in, ImageVisitor v)
+      throws IOException {
+    v.visitEnclosingElement(ImageElement.PERMISSIONS);
+    v.visit(ImageElement.CACHE_POOL_OWNER_NAME, Text.readString(in));
+    v.visit(ImageElement.CACHE_POOL_GROUP_NAME, Text.readString(in));
+    FsPermission fsp = new FsPermission(in.readShort());
+    v.visit(ImageElement.CACHE_POOL_PERMISSION_STRING, fsp.toString());
+    v.leaveEnclosingElement(); // Permissions
+  }
+
+  /**
+   * Process the INode records stored in the fsimage.
+   *
+   * @param in Datastream to process
+   * @param v Visitor to walk over INodes
+   * @param numInodes Number of INodes stored in file
+   * @param skipBlocks Process all the blocks within the INode?
+   * @param supportSnapshot Whether or not the imageVersion supports snapshot
+   * @throws VisitException
+   * @throws IOException
+   */
+  private void processINodes(DataInputStream in, ImageVisitor v,
+      long numInodes, boolean skipBlocks, boolean supportSnapshot)
+      throws IOException {
+    v.visitEnclosingElement(ImageElement.INODES,
+        ImageElement.NUM_INODES, numInodes);
+    
+    if (NameNodeLayoutVersion.supports(Feature.FSIMAGE_NAME_OPTIMIZATION, imageVersion)) {
+      if (!supportSnapshot) {
+        processLocalNameINodes(in, v, numInodes, skipBlocks);
+      } else {
+        processLocalNameINodesWithSnapshot(in, v, skipBlocks);
+      }
+    } else { // full path name
+      processFullNameINodes(in, v, numInodes, skipBlocks);
+    }
+
+    
+    v.leaveEnclosingElement(); // INodes
+  }
+  
+  /**
+   * Process image with full path name
+   * 
+   * @param in image stream
+   * @param v visitor
+   * @param numInodes number of indoes to read
+   * @param skipBlocks skip blocks or not
+   * @throws IOException if there is any error occurs
+   */
+  private void processLocalNameINodes(DataInputStream in, ImageVisitor v,
+      long numInodes, boolean skipBlocks) throws IOException {
+    // process root
+    processINode(in, v, skipBlocks, "", false);
+    numInodes--;
+    while (numInodes > 0) {
+      numInodes -= processDirectory(in, v, skipBlocks);
+    }
+  }
+  
+  private int processDirectory(DataInputStream in, ImageVisitor v,
+     boolean skipBlocks) throws IOException {
+    String parentName = FSImageSerialization.readString(in);
+    return processChildren(in, v, skipBlocks, parentName);
+  }
+  
+  /**
+   * Process image with local path name and snapshot support
+   * 
+   * @param in image stream
+   * @param v visitor
+   * @param skipBlocks skip blocks or not
+   */
+  private void processLocalNameINodesWithSnapshot(DataInputStream in,
+      ImageVisitor v, boolean skipBlocks) throws IOException {
+    // process root
+    processINode(in, v, skipBlocks, "", false);
+    processDirectoryWithSnapshot(in, v, skipBlocks);
+  }
+  
+  /**
+   * Process directories when snapshot is supported.
+   */
+  private void processDirectoryWithSnapshot(DataInputStream in, ImageVisitor v,
+      boolean skipBlocks) throws IOException {
+    // 1. load dir node id
+    long inodeId = in.readLong();
+    
+    String dirName = dirNodeMap.remove(inodeId);
+    Boolean visitedRef = subtreeMap.get(inodeId);
+    if (visitedRef != null) {
+      if (visitedRef.booleanValue()) { // the subtree has been visited
+        return;
+      } else { // first time to visit
+        subtreeMap.put(inodeId, true);
+      }
+    } // else the dir is not linked by a RefNode, thus cannot be revisited
+    
+    // 2. load possible snapshots
+    processSnapshots(in, v, dirName);
+    // 3. load children nodes
+    processChildren(in, v, skipBlocks, dirName);
+    // 4. load possible directory diff list
+    processDirectoryDiffList(in, v, dirName);
+    // recursively process sub-directories
+    final int numSubTree = in.readInt();
+    for (int i = 0; i < numSubTree; i++) {
+      processDirectoryWithSnapshot(in, v, skipBlocks);
+    }
+  }
+  
+  /**
+   * Process snapshots of a snapshottable directory
+   */
+  private void processSnapshots(DataInputStream in, ImageVisitor v,
+      String rootName) throws IOException {
+    final int numSnapshots = in.readInt();
+    if (numSnapshots >= 0) {
+      v.visitEnclosingElement(ImageElement.SNAPSHOTS,
+          ImageElement.NUM_SNAPSHOTS, numSnapshots);
+      for (int i = 0; i < numSnapshots; i++) {
+        // process snapshot
+        v.visitEnclosingElement(ImageElement.SNAPSHOT);
+        v.visit(ImageElement.SNAPSHOT_ID, in.readInt());
+        v.leaveEnclosingElement();
+      }
+      v.visit(ImageElement.SNAPSHOT_QUOTA, in.readInt());
+      v.leaveEnclosingElement();
+    }
+  }
+  
+  private void processSnapshot(DataInputStream in, ImageVisitor v)
+      throws IOException {
+    v.visitEnclosingElement(ImageElement.SNAPSHOT);
+    v.visit(ImageElement.SNAPSHOT_ID, in.readInt());
+    // process root of snapshot
+    v.visitEnclosingElement(ImageElement.SNAPSHOT_ROOT);
+    processINode(in, v, true, "", false);
+    v.leaveEnclosingElement();
+    v.leaveEnclosingElement();
+  }
+  
+  private void processDirectoryDiffList(DataInputStream in, ImageVisitor v,
+      String currentINodeName) throws IOException {
+    final int numDirDiff = in.readInt();
+    if (numDirDiff >= 0) {
+      v.visitEnclosingElement(ImageElement.SNAPSHOT_DIR_DIFFS,
+          ImageElement.NUM_SNAPSHOT_DIR_DIFF, numDirDiff);
+      for (int i = 0; i < numDirDiff; i++) {
+        // process directory diffs in reverse chronological oder
+        processDirectoryDiff(in, v, currentINodeName); 
+      }
+      v.leaveEnclosingElement();
+    }
+  }
+  
+  private void processDirectoryDiff(DataInputStream in, ImageVisitor v,
+      String currentINodeName) throws IOException {
+    v.visitEnclosingElement(ImageElement.SNAPSHOT_DIR_DIFF);
+    int snapshotId = in.readInt();
+    v.visit(ImageElement.SNAPSHOT_DIFF_SNAPSHOTID, snapshotId);
+    v.visit(ImageElement.SNAPSHOT_DIR_DIFF_CHILDREN_SIZE, in.readInt());
+    
+    // process snapshotINode
+    boolean useRoot = in.readBoolean();
+    if (!useRoot) {
+      if (in.readBoolean()) {
+        v.visitEnclosingElement(ImageElement.SNAPSHOT_INODE_DIRECTORY_ATTRIBUTES);
+        if (NameNodeLayoutVersion.supports(Feature.OPTIMIZE_SNAPSHOT_INODES, imageVersion)) {
+          processINodeDirectoryAttributes(in, v, currentINodeName);
+        } else {
+          processINode(in, v, true, currentINodeName, true);
+        }
+        v.leaveEnclosingElement();
+      }
+    }
+    
+    // process createdList
+    int createdSize = in.readInt();
+    v.visitEnclosingElement(ImageElement.SNAPSHOT_DIR_DIFF_CREATEDLIST,
+        ImageElement.SNAPSHOT_DIR_DIFF_CREATEDLIST_SIZE, createdSize);
+    for (int i = 0; i < createdSize; i++) {
+      String createdNode = FSImageSerialization.readString(in);
+      v.visit(ImageElement.SNAPSHOT_DIR_DIFF_CREATED_INODE, createdNode);
+    }
+    v.leaveEnclosingElement();
+    
+    // process deletedList
+    int deletedSize = in.readInt();
+    v.visitEnclosingElement(ImageElement.SNAPSHOT_DIR_DIFF_DELETEDLIST,
+        ImageElement.SNAPSHOT_DIR_DIFF_DELETEDLIST_SIZE, deletedSize);
+    for (int i = 0; i < deletedSize; i++) {
+      v.visitEnclosingElement(ImageElement.SNAPSHOT_DIR_DIFF_DELETED_INODE);
+      processINode(in, v, false, currentINodeName, true);
+      v.leaveEnclosingElement();
+    }
+    v.leaveEnclosingElement();
+    v.leaveEnclosingElement();
+  }
+
+  private void processINodeDirectoryAttributes(DataInputStream in, ImageVisitor v,
+      String parentName) throws IOException {
+    final String pathName = readINodePath(in, parentName);
+    v.visit(ImageElement.INODE_PATH, pathName);
+    processPermission(in, v);
+    v.visit(ImageElement.MODIFICATION_TIME, formatDate(in.readLong()));
+
+    v.visit(ImageElement.NS_QUOTA, in.readLong());
+    v.visit(ImageElement.DS_QUOTA, in.readLong());
+  }
+
+  /** Process children under a directory */
+  private int processChildren(DataInputStream in, ImageVisitor v,
+      boolean skipBlocks, String parentName) throws IOException {
+    int numChildren = in.readInt();
+    for (int i = 0; i < numChildren; i++) {
+      processINode(in, v, skipBlocks, parentName, false);
+    }
+    return numChildren;
+  }
+  
+  /**
+   * Process image with full path name
+   * 
+   * @param in image stream
+   * @param v visitor
+   * @param numInodes number of indoes to read
+   * @param skipBlocks skip blocks or not
+   * @throws IOException if there is any error occurs
+   */
+  private void processFullNameINodes(DataInputStream in, ImageVisitor v,
+      long numInodes, boolean skipBlocks) throws IOException {
+    for(long i = 0; i < numInodes; i++) {
+      processINode(in, v, skipBlocks, null, false);
+    }
+  }
+ 
+  private String readINodePath(DataInputStream in, String parentName)
+      throws IOException {
+    String pathName = FSImageSerialization.readString(in);
+    if (parentName != null) {  // local name
+      pathName = "/" + pathName;
+      if (!"/".equals(parentName)) { // children of non-root directory
+        pathName = parentName + pathName;
+      }
+    }
+    return pathName;
+  }
+
+  /**
+   * Process an INode
+   * 
+   * @param in image stream
+   * @param v visitor
+   * @param skipBlocks skip blocks or not
+   * @param parentName the name of its parent node
+   * @param isSnapshotCopy whether or not the inode is a snapshot copy
+   * @throws IOException
+   */
+  private void processINode(DataInputStream in, ImageVisitor v,
+      boolean skipBlocks, String parentName, boolean isSnapshotCopy)
+      throws IOException {
+    boolean supportSnapshot = 
+        NameNodeLayoutVersion.supports(Feature.SNAPSHOT, imageVersion);
+    boolean supportInodeId = 
+        NameNodeLayoutVersion.supports(Feature.ADD_INODE_ID, imageVersion);
+    
+    v.visitEnclosingElement(ImageElement.INODE);
+    final String pathName = readINodePath(in, parentName);
+    v.visit(ImageElement.INODE_PATH, pathName);
+
+    long inodeId = INodeId.GRANDFATHER_INODE_ID;
+    if (supportInodeId) {
+      inodeId = in.readLong();
+      v.visit(ImageElement.INODE_ID, inodeId);
+    }
+    v.visit(ImageElement.REPLICATION, in.readShort());
+    v.visit(ImageElement.MODIFICATION_TIME, formatDate(in.readLong()));
+    if(NameNodeLayoutVersion.supports(Feature.FILE_ACCESS_TIME, imageVersion))
+      v.visit(ImageElement.ACCESS_TIME, formatDate(in.readLong()));
+    v.visit(ImageElement.BLOCK_SIZE, in.readLong());
+    int numBlocks = in.readInt();
+
+    processBlocks(in, v, numBlocks, skipBlocks);
+    
+    if (numBlocks >= 0) { // File
+      if (supportSnapshot) {
+        // make sure subtreeMap only contains entry for directory
+        subtreeMap.remove(inodeId);
+        // process file diffs
+        processFileDiffList(in, v, parentName);
+        if (isSnapshotCopy) {
+          boolean underConstruction = in.readBoolean();
+          if (underConstruction) {
+            v.visit(ImageElement.CLIENT_NAME,
+                FSImageSerialization.readString(in));
+            v.visit(ImageElement.CLIENT_MACHINE,
+                FSImageSerialization.readString(in));
+          }
+        }
+      }
+      processPermission(in, v);
+    } else if (numBlocks == -1) { // Directory
+      if (supportSnapshot && supportInodeId) {
+        dirNodeMap.put(inodeId, pathName);
+      }
+      v.visit(ImageElement.NS_QUOTA, numBlocks == -1 ? in.readLong() : -1);
+      if (NameNodeLayoutVersion.supports(Feature.DISKSPACE_QUOTA, imageVersion))
+        v.visit(ImageElement.DS_QUOTA, numBlocks == -1 ? in.readLong() : -1);
+      if (supportSnapshot) {
+        boolean snapshottable = in.readBoolean();
+        if (!snapshottable) {
+          boolean withSnapshot = in.readBoolean();
+          v.visit(ImageElement.IS_WITHSNAPSHOT_DIR, Boolean.toString(withSnapshot));
+        } else {
+          v.visit(ImageElement.IS_SNAPSHOTTABLE_DIR, Boolean.toString(snapshottable));
+        }
+      }
+      processPermission(in, v);
+    } else if (numBlocks == -2) {
+      v.visit(ImageElement.SYMLINK, Text.readString(in));
+      processPermission(in, v);
+    } else if (numBlocks == -3) { // reference node
+      final boolean isWithName = in.readBoolean();
+      int snapshotId = in.readInt();
+      if (isWithName) {
+        v.visit(ImageElement.SNAPSHOT_LAST_SNAPSHOT_ID, snapshotId);
+      } else {
+        v.visit(ImageElement.SNAPSHOT_DST_SNAPSHOT_ID, snapshotId);
+      }
+      
+      final boolean firstReferred = in.readBoolean();
+      if (firstReferred) {
+        // if a subtree is linked by multiple "parents", the corresponding dir
+        // must be referred by a reference node. we put the reference node into
+        // the subtreeMap here and let its value be false. when we later visit
+        // the subtree for the first time, we change the value to true.
+        subtreeMap.put(inodeId, false);
+        v.visitEnclosingElement(ImageElement.SNAPSHOT_REF_INODE);
+        processINode(in, v, skipBlocks, parentName, isSnapshotCopy);
+        v.leaveEnclosingElement();  // referred inode    
+      } else {
+        v.visit(ImageElement.SNAPSHOT_REF_INODE_ID, in.readLong());
+      }
+    }
+
+    v.leaveEnclosingElement(); // INode
+  }
+
+  private void processINodeFileAttributes(DataInputStream in, ImageVisitor v,
+      String parentName) throws IOException {
+    final String pathName = readINodePath(in, parentName);
+    v.visit(ImageElement.INODE_PATH, pathName);
+    processPermission(in, v);
+    v.visit(ImageElement.MODIFICATION_TIME, formatDate(in.readLong()));
+    if(NameNodeLayoutVersion.supports(Feature.FILE_ACCESS_TIME, imageVersion)) {
+      v.visit(ImageElement.ACCESS_TIME, formatDate(in.readLong()));
+    }
+
+    v.visit(ImageElement.REPLICATION, in.readShort());
+    v.visit(ImageElement.BLOCK_SIZE, in.readLong());
+  }
+  
+  private void processFileDiffList(DataInputStream in, ImageVisitor v,
+      String currentINodeName) throws IOException {
+    final int size = in.readInt();
+    if (size >= 0) {
+      v.visitEnclosingElement(ImageElement.SNAPSHOT_FILE_DIFFS,
+          ImageElement.NUM_SNAPSHOT_FILE_DIFF, size);
+      for (int i = 0; i < size; i++) {
+        processFileDiff(in, v, currentINodeName);
+      }
+      v.leaveEnclosingElement();
+    }
+  }
+  
+  private void processFileDiff(DataInputStream in, ImageVisitor v,
+      String currentINodeName) throws IOException {
+    int snapshotId = in.readInt();
+    v.visitEnclosingElement(ImageElement.SNAPSHOT_FILE_DIFF,
+        ImageElement.SNAPSHOT_DIFF_SNAPSHOTID, snapshotId);
+    v.visit(ImageElement.SNAPSHOT_FILE_SIZE, in.readLong());
+    if (in.readBoolean()) {
+      v.visitEnclosingElement(ImageElement.SNAPSHOT_INODE_FILE_ATTRIBUTES);
+      if (NameNodeLayoutVersion.supports(Feature.OPTIMIZE_SNAPSHOT_INODES, imageVersion)) {
+        processINodeFileAttributes(in, v, currentINodeName);
+      } else {
+        processINode(in, v, true, currentINodeName, true);
+      }
+      v.leaveEnclosingElement();
+    }
+    v.leaveEnclosingElement();
+  }
+  
+  /**
+   * Helper method to format dates during processing.
+   * @param date Date as read from image file
+   * @return String version of date format
+   */
+  private String formatDate(long date) {
+    return dateFormat.format(new Date(date));
+  }
+}

+ 212 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageVisitor.java

@@ -0,0 +1,212 @@
+/**
+ * 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.tools.offlineImageViewer;
+
+import java.io.IOException;
+
+/**
+ * An implementation of ImageVisitor can traverse the structure of an
+ * Hadoop fsimage and respond to each of the structures within the file.
+ */
+abstract class ImageVisitor {
+
+  /**
+   * Structural elements of an FSImage that may be encountered within the
+   * file. ImageVisitors are able to handle processing any of these elements.
+   */
+  public enum ImageElement {
+    FS_IMAGE,
+    IMAGE_VERSION,
+    NAMESPACE_ID,
+    IS_COMPRESSED,
+    COMPRESS_CODEC,
+    LAYOUT_VERSION,
+    NUM_INODES,
+    GENERATION_STAMP,
+    GENERATION_STAMP_V2,
+    GENERATION_STAMP_V1_LIMIT,
+    LAST_ALLOCATED_BLOCK_ID,
+    INODES,
+    INODE,
+    INODE_PATH,
+    REPLICATION,
+    MODIFICATION_TIME,
+    ACCESS_TIME,
+    BLOCK_SIZE,
+    NUM_BLOCKS,
+    BLOCKS,
+    BLOCK,
+    BLOCK_ID,
+    NUM_BYTES,
+    NS_QUOTA,
+    DS_QUOTA,
+    PERMISSIONS,
+    SYMLINK,
+    NUM_INODES_UNDER_CONSTRUCTION,
+    INODES_UNDER_CONSTRUCTION,
+    INODE_UNDER_CONSTRUCTION,
+    PREFERRED_BLOCK_SIZE,
+    CLIENT_NAME,
+    CLIENT_MACHINE,
+    USER_NAME,
+    GROUP_NAME,
+    PERMISSION_STRING,
+    CURRENT_DELEGATION_KEY_ID,
+    NUM_DELEGATION_KEYS,
+    DELEGATION_KEYS,
+    DELEGATION_KEY,
+    DELEGATION_TOKEN_SEQUENCE_NUMBER,
+    NUM_DELEGATION_TOKENS,
+    DELEGATION_TOKENS,
+    DELEGATION_TOKEN_IDENTIFIER,
+    DELEGATION_TOKEN_IDENTIFIER_KIND,
+    DELEGATION_TOKEN_IDENTIFIER_SEQNO,
+    DELEGATION_TOKEN_IDENTIFIER_OWNER,
+    DELEGATION_TOKEN_IDENTIFIER_RENEWER,
+    DELEGATION_TOKEN_IDENTIFIER_REALUSER,
+    DELEGATION_TOKEN_IDENTIFIER_ISSUE_DATE,
+    DELEGATION_TOKEN_IDENTIFIER_MAX_DATE,
+    DELEGATION_TOKEN_IDENTIFIER_EXPIRY_TIME,
+    DELEGATION_TOKEN_IDENTIFIER_MASTER_KEY_ID,
+    TRANSACTION_ID,
+    LAST_INODE_ID,
+    INODE_ID,
+
+    SNAPSHOT_COUNTER,
+    NUM_SNAPSHOTS_TOTAL,
+    NUM_SNAPSHOTS,
+    SNAPSHOTS,
+    SNAPSHOT,
+    SNAPSHOT_ID,
+    SNAPSHOT_ROOT,
+    SNAPSHOT_QUOTA,
+    NUM_SNAPSHOT_DIR_DIFF,
+    SNAPSHOT_DIR_DIFFS,
+    SNAPSHOT_DIR_DIFF,
+    SNAPSHOT_DIFF_SNAPSHOTID,
+    SNAPSHOT_DIR_DIFF_CHILDREN_SIZE,
+    SNAPSHOT_INODE_FILE_ATTRIBUTES,
+    SNAPSHOT_INODE_DIRECTORY_ATTRIBUTES,
+    SNAPSHOT_DIR_DIFF_CREATEDLIST,
+    SNAPSHOT_DIR_DIFF_CREATEDLIST_SIZE,
+    SNAPSHOT_DIR_DIFF_CREATED_INODE,
+    SNAPSHOT_DIR_DIFF_DELETEDLIST,
+    SNAPSHOT_DIR_DIFF_DELETEDLIST_SIZE,
+    SNAPSHOT_DIR_DIFF_DELETED_INODE,
+    IS_SNAPSHOTTABLE_DIR,
+    IS_WITHSNAPSHOT_DIR,
+    SNAPSHOT_FILE_DIFFS,
+    SNAPSHOT_FILE_DIFF,
+    NUM_SNAPSHOT_FILE_DIFF,
+    SNAPSHOT_FILE_SIZE,
+    SNAPSHOT_DST_SNAPSHOT_ID,
+    SNAPSHOT_LAST_SNAPSHOT_ID,
+    SNAPSHOT_REF_INODE_ID,
+    SNAPSHOT_REF_INODE,
+
+    CACHE_NEXT_ENTRY_ID,
+    CACHE_NUM_POOLS,
+    CACHE_POOL_NAME,
+    CACHE_POOL_OWNER_NAME,
+    CACHE_POOL_GROUP_NAME,
+    CACHE_POOL_PERMISSION_STRING,
+    CACHE_POOL_WEIGHT,
+    CACHE_NUM_ENTRIES,
+    CACHE_ENTRY_PATH,
+    CACHE_ENTRY_REPLICATION,
+    CACHE_ENTRY_POOL_NAME
+  }
+  
+  /**
+   * Begin visiting the fsimage structure.  Opportunity to perform
+   * any initialization necessary for the implementing visitor.
+   */
+  abstract void start() throws IOException;
+
+  /**
+   * Finish visiting the fsimage structure.  Opportunity to perform any
+   * clean up necessary for the implementing visitor.
+   */
+  abstract void finish() throws IOException;
+
+  /**
+   * Finish visiting the fsimage structure after an error has occurred
+   * during the processing.  Opportunity to perform any clean up necessary
+   * for the implementing visitor.
+   */
+  abstract void finishAbnormally() throws IOException;
+
+  /**
+   * Visit non enclosing element of fsimage with specified value.
+   *
+   * @param element FSImage element
+   * @param value Element's value
+   */
+  abstract void visit(ImageElement element, String value) throws IOException;
+
+  // Convenience methods to automatically convert numeric value types to strings
+  void visit(ImageElement element, int value) throws IOException {
+    visit(element, Integer.toString(value));
+  }
+
+  void visit(ImageElement element, long value) throws IOException {
+    visit(element, Long.toString(value));
+  }
+
+  /**
+   * Begin visiting an element that encloses another element, such as
+   * the beginning of the list of blocks that comprise a file.
+   *
+   * @param element Element being visited
+   */
+  abstract void visitEnclosingElement(ImageElement element)
+     throws IOException;
+
+  /**
+   * Begin visiting an element that encloses another element, such as
+   * the beginning of the list of blocks that comprise a file.
+   *
+   * Also provide an additional key and value for the element, such as the
+   * number items within the element.
+   *
+   * @param element Element being visited
+   * @param key Key describing the element being visited
+   * @param value Value associated with element being visited
+   */
+  abstract void visitEnclosingElement(ImageElement element,
+      ImageElement key, String value) throws IOException;
+
+  // Convenience methods to automatically convert value types to strings
+  void visitEnclosingElement(ImageElement element,
+      ImageElement key, int value)
+     throws IOException {
+    visitEnclosingElement(element, key, Integer.toString(value));
+  }
+
+  void visitEnclosingElement(ImageElement element,
+      ImageElement key, long value)
+     throws IOException {
+    visitEnclosingElement(element, key, Long.toString(value));
+  }
+
+  /**
+   * Leave current enclosing element.  Called, for instance, at the end of
+   * processing the blocks that compromise a file.
+   */
+  abstract void leaveEnclosingElement() throws IOException;
+}

+ 111 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/IndentedImageVisitor.java

@@ -0,0 +1,111 @@
+/**
+ * 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.tools.offlineImageViewer;
+
+import java.io.IOException;
+import java.util.Date;
+
+/**
+ * IndentedImageVisitor walks over an FSImage and displays its structure 
+ * using indenting to organize sections within the image file.
+ */
+class IndentedImageVisitor extends TextWriterImageVisitor {
+  
+  public IndentedImageVisitor(String filename) throws IOException {
+    super(filename);
+  }
+
+  public IndentedImageVisitor(String filename, boolean printToScreen) throws IOException {
+    super(filename, printToScreen);
+  }
+
+  final private DepthCounter dc = new DepthCounter();// to track leading spacing
+
+  @Override
+  void start() throws IOException {}
+
+  @Override
+  void finish() throws IOException { super.finish(); }
+
+  @Override
+  void finishAbnormally() throws IOException {
+    System.out.println("*** Image processing finished abnormally.  Ending ***");
+    super.finishAbnormally();
+  }
+
+  @Override
+  void leaveEnclosingElement() throws IOException {
+    dc.decLevel();
+  }
+
+  @Override
+  void visit(ImageElement element, String value) throws IOException {
+    printIndents();
+    write(element + " = " + value + "\n");
+  }
+
+  @Override
+  void visit(ImageElement element, long value) throws IOException {
+    if ((element == ImageElement.DELEGATION_TOKEN_IDENTIFIER_EXPIRY_TIME) || 
+        (element == ImageElement.DELEGATION_TOKEN_IDENTIFIER_ISSUE_DATE) || 
+        (element == ImageElement.DELEGATION_TOKEN_IDENTIFIER_MAX_DATE)) {
+      visit(element, new Date(value).toString());
+    } else {
+      visit(element, Long.toString(value));
+    }
+  }
+  
+  @Override
+  void visitEnclosingElement(ImageElement element) throws IOException {
+    printIndents();
+    write(element + "\n");
+    dc.incLevel();
+  }
+
+  // Print element, along with associated key/value pair, in brackets
+  @Override
+  void visitEnclosingElement(ImageElement element,
+      ImageElement key, String value)
+      throws IOException {
+    printIndents();
+    write(element + " [" + key + " = " + value + "]\n");
+    dc.incLevel();
+  }
+
+  /**
+  * Print an appropriate number of spaces for the current level.
+  * FsImages can potentially be millions of lines long, so caching can
+  * significantly speed up output.
+  */
+  final private static String [] indents = { "",
+                                             "  ",
+                                             "    ",
+                                             "      ",
+                                             "        ",
+                                             "          ",
+                                             "            "};
+  private void printIndents() throws IOException {
+    try {
+      write(indents[dc.getLevel()]);
+    } catch (IndexOutOfBoundsException e) {
+      // There's no reason in an fsimage would need a deeper indent
+      for(int i = 0; i < dc.getLevel(); i++)
+        write(" ");
+    }
+   }
+}

+ 178 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/LsImageVisitor.java

@@ -0,0 +1,178 @@
+/**
+ * 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.tools.offlineImageViewer;
+
+import java.io.IOException;
+import java.util.Formatter;
+import java.util.LinkedList;
+
+/**
+ * LsImageVisitor displays the blocks of the namespace in a format very similar
+ * to the output of ls/lsr.  Entries are marked as directories or not,
+ * permissions listed, replication, username and groupname, along with size,
+ * modification date and full path.
+ *
+ * Note: A significant difference between the output of the lsr command
+ * and this image visitor is that this class cannot sort the file entries;
+ * they are listed in the order they are stored within the fsimage file. 
+ * Therefore, the output of this class cannot be directly compared to the
+ * output of the lsr command.
+ */
+class LsImageVisitor extends TextWriterImageVisitor {
+  final private LinkedList<ImageElement> elemQ = new LinkedList<ImageElement>();
+
+  private int numBlocks;
+  private String perms;
+  private int replication;
+  private String username;
+  private String group;
+  private long filesize;
+  private String modTime;
+  private String path;
+  private String linkTarget;
+
+  private boolean inInode = false;
+  final private StringBuilder sb = new StringBuilder();
+  final private Formatter formatter = new Formatter(sb);
+
+  public LsImageVisitor(String filename) throws IOException {
+    super(filename);
+  }
+
+  public LsImageVisitor(String filename, boolean printToScreen) throws IOException {
+    super(filename, printToScreen);
+  }
+
+  /**
+   * Start a new line of output, reset values.
+   */
+  private void newLine() {
+    numBlocks = 0;
+    perms = username = group = path = linkTarget = "";
+    filesize = 0l;
+    replication = 0;
+
+    inInode = true;
+  }
+
+  /**
+   * All the values have been gathered.  Print them to the console in an
+   * ls-style format.
+   */
+  private final static int widthRepl = 2;  
+  private final static int widthUser = 8; 
+  private final static int widthGroup = 10; 
+  private final static int widthSize = 10;
+  private final static int widthMod = 10;
+  private final static String lsStr = " %" + widthRepl + "s %" + widthUser + 
+                                       "s %" + widthGroup + "s %" + widthSize +
+                                       "d %" + widthMod + "s %s";
+  private void printLine() throws IOException {
+    sb.append(numBlocks < 0 ? "d" : "-");
+    sb.append(perms);
+
+    if (0 != linkTarget.length()) {
+      path = path + " -> " + linkTarget; 
+    }
+    formatter.format(lsStr, replication > 0 ? replication : "-",
+                           username, group, filesize, modTime, path);
+    sb.append("\n");
+
+    write(sb.toString());
+    sb.setLength(0); // clear string builder
+
+    inInode = false;
+  }
+
+  @Override
+  void start() throws IOException {}
+
+  @Override
+  void finish() throws IOException {
+    super.finish();
+  }
+
+  @Override
+  void finishAbnormally() throws IOException {
+    System.out.println("Input ended unexpectedly.");
+    super.finishAbnormally();
+  }
+
+  @Override
+  void leaveEnclosingElement() throws IOException {
+    ImageElement elem = elemQ.pop();
+
+    if(elem == ImageElement.INODE)
+      printLine();
+  }
+
+  // Maintain state of location within the image tree and record
+  // values needed to display the inode in ls-style format.
+  @Override
+  void visit(ImageElement element, String value) throws IOException {
+    if(inInode) {
+      switch(element) {
+      case INODE_PATH:
+        if(value.equals("")) path = "/";
+        else path = value;
+        break;
+      case PERMISSION_STRING:
+        perms = value;
+        break;
+      case REPLICATION:
+        replication = Integer.valueOf(value);
+        break;
+      case USER_NAME:
+        username = value;
+        break;
+      case GROUP_NAME:
+        group = value;
+        break;
+      case NUM_BYTES:
+        filesize += Long.valueOf(value);
+        break;
+      case MODIFICATION_TIME:
+        modTime = value;
+        break;
+      case SYMLINK:
+        linkTarget = value;
+        break;
+      default:
+        // This is OK.  We're not looking for all the values.
+        break;
+      }
+    }
+  }
+
+  @Override
+  void visitEnclosingElement(ImageElement element) throws IOException {
+    elemQ.push(element);
+    if(element == ImageElement.INODE)
+      newLine();
+  }
+
+  @Override
+  void visitEnclosingElement(ImageElement element,
+      ImageElement key, String value) throws IOException {
+    elemQ.push(element);
+    if(element == ImageElement.INODE)
+      newLine();
+    else if (element == ImageElement.BLOCKS)
+      numBlocks = Integer.valueOf(value);
+  }
+}

+ 118 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/NameDistributionVisitor.java

@@ -0,0 +1,118 @@
+/**
+ * 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.tools.offlineImageViewer;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * File name distribution visitor. 
+ * <p>
+ * It analyzes file names in fsimage and prints the following information: 
+ * <li>Number of unique file names</li> 
+ * <li>Number file names and the corresponding number range of files that use 
+ * these same names</li>
+ * <li>Heap saved if the file name objects are reused</li>
+ */
+@InterfaceAudience.Private
+public class NameDistributionVisitor extends TextWriterImageVisitor {
+  HashMap<String, Integer> counts = new HashMap<String, Integer>();
+
+  public NameDistributionVisitor(String filename, boolean printToScreen)
+      throws IOException {
+    super(filename, printToScreen);
+  }
+
+  @Override
+  void finish() throws IOException {
+    final int BYTEARRAY_OVERHEAD = 24;
+
+    write("Total unique file names " + counts.size());
+    // Columns: Frequency of file occurrence, savings in heap, total files using
+    // the name and number of file names
+    final long stats[][] = { { 100000, 0, 0, 0 },
+                             { 10000, 0, 0, 0 },
+                             { 1000, 0, 0, 0 },
+                             { 100, 0, 0, 0 },
+                             { 10, 0, 0, 0 },
+                             { 5, 0, 0, 0 },
+                             { 4, 0, 0, 0 },
+                             { 3, 0, 0, 0 },
+                             { 2, 0, 0, 0 }};
+
+    int highbound = Integer.MIN_VALUE;
+    for (Entry<String, Integer> entry : counts.entrySet()) {
+      highbound = Math.max(highbound, entry.getValue());
+      for (int i = 0; i < stats.length; i++) {
+        if (entry.getValue() >= stats[i][0]) {
+          stats[i][1] += (BYTEARRAY_OVERHEAD + entry.getKey().length())
+              * (entry.getValue() - 1);
+          stats[i][2] += entry.getValue();
+          stats[i][3]++;
+          break;
+        }
+      }
+    }
+
+    long lowbound = 0;
+    long totalsavings = 0;
+    for (long[] stat : stats) {
+      lowbound = stat[0];
+      totalsavings += stat[1];
+      String range = lowbound == highbound ? " " + lowbound :
+          " between " + lowbound + "-" + highbound;
+      write("\n" + stat[3] + " names are used by " + stat[2] + " files"
+          + range + " times. Heap savings ~" + stat[1] + " bytes.");
+      highbound = (int) stat[0] - 1;
+    }
+    write("\n\nTotal saved heap ~" + totalsavings + "bytes.\n");
+    super.finish();
+  }
+
+  @Override
+  void visit(ImageElement element, String value) throws IOException {
+    if (element == ImageElement.INODE_PATH) {
+      String filename = value.substring(value.lastIndexOf("/") + 1);
+      if (counts.containsKey(filename)) {
+        counts.put(filename, counts.get(filename) + 1);
+      } else {
+        counts.put(filename, 1);
+      }
+    }
+  }
+
+  @Override
+  void leaveEnclosingElement() throws IOException {
+  }
+
+  @Override
+  void start() throws IOException {
+  }
+
+  @Override
+  void visitEnclosingElement(ImageElement element) throws IOException {
+  }
+
+  @Override
+  void visitEnclosingElement(ImageElement element, ImageElement key,
+      String value) throws IOException {
+  }
+}

+ 274 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewer.java

@@ -0,0 +1,274 @@
+/**
+ * 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.tools.offlineImageViewer;
+
+import java.io.BufferedInputStream;
+import java.io.DataInputStream;
+import java.io.EOFException;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.cli.PosixParser;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader.PositionTrackingInputStream;
+
+/**
+ * OfflineImageViewer to dump the contents of an Hadoop image file to XML
+ * or the console.  Main entry point into utility, either via the
+ * command line or programatically.
+ */
+@InterfaceAudience.Private
+public class OfflineImageViewer {
+  public static final Log LOG = LogFactory.getLog(OfflineImageViewer.class);
+  
+  private final static String usage = 
+    "Usage: bin/hdfs oiv_legacy [OPTIONS] -i INPUTFILE -o OUTPUTFILE\n" +
+    "Offline Image Viewer\n" + 
+    "View a Hadoop fsimage INPUTFILE using the specified PROCESSOR,\n" +
+    "saving the results in OUTPUTFILE.\n" +
+    "\n" +
+    "The oiv utility will attempt to parse correctly formed image files\n" +
+    "and will abort fail with mal-formed image files.\n" +
+    "\n" +
+    "The tool works offline and does not require a running cluster in\n" +
+    "order to process an image file.\n" +
+    "\n" +
+    "The following image processors are available:\n" +
+    "  * Ls: The default image processor generates an lsr-style listing\n" +
+    "    of the files in the namespace, with the same fields in the same\n" +
+    "    order.  Note that in order to correctly determine file sizes,\n" +
+    "    this formatter cannot skip blocks and will override the\n" +
+    "    -skipBlocks option.\n" +
+    "  * Indented: This processor enumerates over all of the elements in\n" +
+    "    the fsimage file, using levels of indentation to delineate\n" +
+    "    sections within the file.\n" +
+    "  * Delimited: Generate a text file with all of the elements common\n" +
+    "    to both inodes and inodes-under-construction, separated by a\n" +
+    "    delimiter. The default delimiter is \u0001, though this may be\n" +
+    "    changed via the -delimiter argument. This processor also overrides\n" +
+    "    the -skipBlocks option for the same reason as the Ls processor\n" +
+    "  * XML: This processor creates an XML document with all elements of\n" +
+    "    the fsimage enumerated, suitable for further analysis by XML\n" +
+    "    tools.\n" +
+    "  * FileDistribution: This processor analyzes the file size\n" +
+    "    distribution in the image.\n" +
+    "    -maxSize specifies the range [0, maxSize] of file sizes to be\n" +
+    "     analyzed (128GB by default).\n" +
+    "    -step defines the granularity of the distribution. (2MB by default)\n" +
+    "  * NameDistribution: This processor analyzes the file names\n" +
+    "    in the image and prints total number of file names and how frequently\n" +
+    "    file names are reused.\n" +
+    "\n" + 
+    "Required command line arguments:\n" +
+    "-i,--inputFile <arg>   FSImage file to process.\n" +
+    "-o,--outputFile <arg>  Name of output file. If the specified\n" +
+    "                       file exists, it will be overwritten.\n" +
+    "\n" + 
+    "Optional command line arguments:\n" +
+    "-p,--processor <arg>   Select which type of processor to apply\n" +
+    "                       against image file." +
+    " (Ls|XML|Delimited|Indented|FileDistribution).\n" +
+    "-h,--help              Display usage information and exit\n" +
+    "-printToScreen         For processors that write to a file, also\n" +
+    "                       output to screen. On large image files this\n" +
+    "                       will dramatically increase processing time.\n" +
+    "-skipBlocks            Skip inodes' blocks information. May\n" +
+    "                       significantly decrease output.\n" +
+    "                       (default = false).\n" +
+    "-delimiter <arg>       Delimiting string to use with Delimited processor\n";
+
+  private final boolean skipBlocks;
+  private final String inputFile;
+  private final ImageVisitor processor;
+  
+  public OfflineImageViewer(String inputFile, ImageVisitor processor, 
+             boolean skipBlocks) {
+    this.inputFile = inputFile;
+    this.processor = processor;
+    this.skipBlocks = skipBlocks;
+  }
+
+  /**
+   * Process image file.
+   */
+  public void go() throws IOException  {
+    DataInputStream in = null;
+    PositionTrackingInputStream tracker = null;
+    ImageLoader fsip = null;
+    boolean done = false;
+    try {
+      tracker = new PositionTrackingInputStream(new BufferedInputStream(
+               new FileInputStream(new File(inputFile))));
+      in = new DataInputStream(tracker);
+
+      int imageVersionFile = findImageVersion(in);
+
+      fsip = ImageLoader.LoaderFactory.getLoader(imageVersionFile);
+
+      if(fsip == null) 
+        throw new IOException("No image processor to read version " +
+            imageVersionFile + " is available.");
+      fsip.loadImage(in, processor, skipBlocks);
+      done = true;
+    } finally {
+      if (!done) {
+        LOG.error("image loading failed at offset " + tracker.getPos());
+      }
+      IOUtils.cleanup(LOG, in, tracker);
+    }
+  }
+
+  /**
+   * Check an fsimage datainputstream's version number.
+   *
+   * The datainput stream is returned at the same point as it was passed in;
+   * this method has no effect on the datainputstream's read pointer.
+   *
+   * @param in Datainputstream of fsimage
+   * @return Filesystem layout version of fsimage represented by stream
+   * @throws IOException If problem reading from in
+   */
+  private int findImageVersion(DataInputStream in) throws IOException {
+    in.mark(42); // arbitrary amount, resetting immediately
+
+    int version = in.readInt();
+    in.reset();
+
+    return version;
+  }
+  
+  /**
+   * Build command-line options and descriptions
+   */
+  public static Options buildOptions() {
+    Options options = new Options();
+
+    // Build in/output file arguments, which are required, but there is no 
+    // addOption method that can specify this
+    OptionBuilder.isRequired();
+    OptionBuilder.hasArgs();
+    OptionBuilder.withLongOpt("outputFile");
+    options.addOption(OptionBuilder.create("o"));
+    
+    OptionBuilder.isRequired();
+    OptionBuilder.hasArgs();
+    OptionBuilder.withLongOpt("inputFile");
+    options.addOption(OptionBuilder.create("i"));
+    
+    options.addOption("p", "processor", true, "");
+    options.addOption("h", "help", false, "");
+    options.addOption("skipBlocks", false, "");
+    options.addOption("printToScreen", false, "");
+    options.addOption("delimiter", true, "");
+
+    return options;
+  }
+  
+  /**
+   * Entry point to command-line-driven operation.  User may specify
+   * options and start fsimage viewer from the command line.  Program
+   * will process image file and exit cleanly or, if an error is
+   * encountered, inform user and exit.
+   *
+   * @param args Command line options
+   * @throws IOException 
+   */
+  public static void main(String[] args) throws IOException {
+    Options options = buildOptions();
+    if(args.length == 0) {
+      printUsage();
+      return;
+    }
+    
+    CommandLineParser parser = new PosixParser();
+    CommandLine cmd;
+
+    try {
+      cmd = parser.parse(options, args);
+    } catch (ParseException e) {
+      System.out.println("Error parsing command-line options: ");
+      printUsage();
+      return;
+    }
+
+    if(cmd.hasOption("h")) { // print help and exit
+      printUsage();
+      return;
+    }
+
+    boolean skipBlocks = cmd.hasOption("skipBlocks");
+    boolean printToScreen = cmd.hasOption("printToScreen");
+    String inputFile = cmd.getOptionValue("i");
+    String processor = cmd.getOptionValue("p", "Ls");
+    String outputFile = cmd.getOptionValue("o");
+    String delimiter = cmd.getOptionValue("delimiter");
+    
+    if( !(delimiter == null || processor.equals("Delimited")) ) {
+      System.out.println("Can only specify -delimiter with Delimited processor");
+      printUsage();
+      return;
+    }
+    
+    ImageVisitor v;
+    if(processor.equals("Indented")) {
+      v = new IndentedImageVisitor(outputFile, printToScreen);
+    } else if (processor.equals("XML")) {
+      v = new XmlImageVisitor(outputFile, printToScreen);
+    } else if (processor.equals("Delimited")) {
+      v = delimiter == null ?  
+                 new DelimitedImageVisitor(outputFile, printToScreen) :
+                 new DelimitedImageVisitor(outputFile, printToScreen, delimiter);
+      skipBlocks = false;
+    } else if (processor.equals("FileDistribution")) {
+      long maxSize = Long.parseLong(cmd.getOptionValue("maxSize", "0"));
+      int step = Integer.parseInt(cmd.getOptionValue("step", "0"));
+      v = new FileDistributionVisitor(outputFile, maxSize, step);
+    } else if (processor.equals("NameDistribution")) {
+      v = new NameDistributionVisitor(outputFile, printToScreen);
+    } else {
+      v = new LsImageVisitor(outputFile, printToScreen);
+      skipBlocks = false;
+    }
+    
+    try {
+      OfflineImageViewer d = new OfflineImageViewer(inputFile, v, skipBlocks);
+      d.go();
+    } catch (EOFException e) {
+      System.err.println("Input file ended unexpectedly.  Exiting");
+    } catch(IOException e) {
+      System.err.println("Encountered exception.  Exiting: " + e.getMessage());
+    }
+  }
+
+  /**
+   * Print application usage instructions.
+   */
+  private static void printUsage() {
+    System.out.println(usage);
+  }
+}

+ 109 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TextWriterImageVisitor.java

@@ -0,0 +1,109 @@
+/**
+ * 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.tools.offlineImageViewer;
+
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+
+import com.google.common.base.Charsets;
+
+/**
+ * TextWriterImageProcessor mixes in the ability for ImageVisitor
+ * implementations to easily write their output to a text file.
+ *
+ * Implementing classes should be sure to call the super methods for the
+ * constructors, finish and finishAbnormally methods, in order that the
+ * underlying file may be opened and closed correctly.
+ *
+ * Note, this class does not add newlines to text written to file or (if
+ * enabled) screen.  This is the implementing class' responsibility.
+ */
+abstract class TextWriterImageVisitor extends ImageVisitor {
+  private boolean printToScreen = false;
+  private boolean okToWrite = false;
+  final private OutputStreamWriter fw;
+
+  /**
+   * Create a processor that writes to the file named.
+   *
+   * @param filename Name of file to write output to
+   */
+  public TextWriterImageVisitor(String filename) throws IOException {
+    this(filename, false);
+  }
+
+  /**
+   * Create a processor that writes to the file named and may or may not
+   * also output to the screen, as specified.
+   *
+   * @param filename Name of file to write output to
+   * @param printToScreen Mirror output to screen?
+   */
+  public TextWriterImageVisitor(String filename, boolean printToScreen)
+         throws IOException {
+    super();
+    this.printToScreen = printToScreen;
+    fw = new OutputStreamWriter(new FileOutputStream(filename), Charsets.UTF_8);
+    okToWrite = true;
+  }
+  
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.hdfs.tools.offlineImageViewer.ImageVisitor#finish()
+   */
+  @Override
+  void finish() throws IOException {
+    close();
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hadoop.hdfs.tools.offlineImageViewer.ImageVisitor#finishAbnormally()
+   */
+  @Override
+  void finishAbnormally() throws IOException {
+    close();
+  }
+
+  /**
+   * Close output stream and prevent further writing
+   */
+  private void close() throws IOException {
+    fw.close();
+    okToWrite = false;
+  }
+
+  /**
+   * Write parameter to output file (and possibly screen).
+   *
+   * @param toWrite Text to write to file
+   */
+  protected void write(String toWrite) throws IOException  {
+    if(!okToWrite)
+      throw new IOException("file not open for writing.");
+
+    if(printToScreen)
+      System.out.print(toWrite);
+
+    try {
+      fw.write(toWrite);
+    } catch (IOException e) {
+      okToWrite = false;
+      throw e;
+    }
+  }
+}

+ 88 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/XmlImageVisitor.java

@@ -0,0 +1,88 @@
+/**
+ * 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.tools.offlineImageViewer;
+
+import java.io.IOException;
+import java.util.LinkedList;
+
+/**
+ * An XmlImageVisitor walks over an fsimage structure and writes out
+ * an equivalent XML document that contains the fsimage's components.
+ */
+public class XmlImageVisitor extends TextWriterImageVisitor {
+  final private LinkedList<ImageElement> tagQ =
+                                          new LinkedList<ImageElement>();
+
+  public XmlImageVisitor(String filename) throws IOException {
+    super(filename, false);
+  }
+
+  public XmlImageVisitor(String filename, boolean printToScreen)
+       throws IOException {
+    super(filename, printToScreen);
+  }
+
+  @Override
+  void finish() throws IOException {
+    super.finish();
+  }
+
+  @Override
+  void finishAbnormally() throws IOException {
+    write("\n<!-- Error processing image file.  Exiting -->\n");
+    super.finishAbnormally();
+  }
+
+  @Override
+  void leaveEnclosingElement() throws IOException {
+    if(tagQ.size() == 0)
+      throw new IOException("Tried to exit non-existent enclosing element " +
+                "in FSImage file");
+
+    ImageElement element = tagQ.pop();
+    write("</" + element.toString() + ">\n");
+  }
+
+  @Override
+  void start() throws IOException {
+    write("<?xml version=\"1.0\" ?>\n");
+  }
+
+  @Override
+  void visit(ImageElement element, String value) throws IOException {
+    writeTag(element.toString(), value);
+  }
+
+  @Override
+  void visitEnclosingElement(ImageElement element) throws IOException {
+    write("<" + element.toString() + ">\n");
+    tagQ.push(element);
+  }
+
+  @Override
+  void visitEnclosingElement(ImageElement element,
+      ImageElement key, String value)
+       throws IOException {
+    write("<" + element.toString() + " " + key + "=\"" + value +"\">\n");
+    tagQ.push(element);
+  }
+
+  private void writeTag(String tag, String value) throws IOException {
+    write("<" + tag + ">" + value + "</" + tag + ">\n");
+  }
+}

+ 10 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java

@@ -176,8 +176,9 @@ public class JsonUtil {
   }
 
   /** Convert a string to a FsPermission object. */
-  private static FsPermission toFsPermission(final String s) {
-    return new FsPermission(Short.parseShort(s, 8));
+  private static FsPermission toFsPermission(final String s, Boolean aclBit) {
+    FsPermission perm = new FsPermission(Short.parseShort(s, 8));
+    return (aclBit != null && aclBit) ? new FsAclPermission(perm) : perm;
   }
 
   static enum PathType {
@@ -204,7 +205,11 @@ public class JsonUtil {
     m.put("length", status.getLen());
     m.put("owner", status.getOwner());
     m.put("group", status.getGroup());
-    m.put("permission", toString(status.getPermission()));
+    FsPermission perm = status.getPermission();
+    m.put("permission", toString(perm));
+    if (perm.getAclBit()) {
+      m.put("aclBit", true);
+    }
     m.put("accessTime", status.getAccessTime());
     m.put("modificationTime", status.getModificationTime());
     m.put("blockSize", status.getBlockSize());
@@ -230,7 +235,8 @@ public class JsonUtil {
     final long len = (Long) m.get("length");
     final String owner = (String) m.get("owner");
     final String group = (String) m.get("group");
-    final FsPermission permission = toFsPermission((String) m.get("permission"));
+    final FsPermission permission = toFsPermission((String) m.get("permission"),
+      (Boolean)m.get("aclBit"));
     final long aTime = (Long) m.get("accessTime");
     final long mTime = (Long) m.get("modificationTime");
     final long blockSize = (Long) m.get("blockSize");

+ 804 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/test/vecsum.c

@@ -0,0 +1,804 @@
+/**
+ * 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.
+ */
+
+#include <errno.h>
+#include <fcntl.h>
+#include <malloc.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+#include <sys/mman.h>
+#include <sys/resource.h>
+#include <sys/stat.h>
+#include <sys/time.h>
+#include <sys/types.h>
+#include <time.h>
+#include <unistd.h>
+
+#include "config.h"
+#include "hdfs.h"
+
+#define VECSUM_CHUNK_SIZE (8 * 1024 * 1024)
+#define ZCR_READ_CHUNK_SIZE (1024 * 1024 * 8)
+#define NORMAL_READ_CHUNK_SIZE (8 * 1024 * 1024)
+#define DOUBLES_PER_LOOP_ITER 16
+
+static double timespec_to_double(const struct timespec *ts)
+{
+    double sec = ts->tv_sec;
+    double nsec = ts->tv_nsec;
+    return sec + (nsec / 1000000000L);
+}
+
+struct stopwatch {
+    struct timespec start;
+    struct timespec stop;
+    struct rusage rusage;
+};
+
+static struct stopwatch *stopwatch_create(void)
+{
+    struct stopwatch *watch;
+
+    watch = calloc(1, sizeof(struct stopwatch));
+    if (!watch) {
+        fprintf(stderr, "failed to allocate memory for stopwatch\n");
+        goto error;
+    }
+    if (clock_gettime(CLOCK_MONOTONIC, &watch->start)) {
+        int err = errno;
+        fprintf(stderr, "clock_gettime(CLOCK_MONOTONIC) failed with "
+            "error %d (%s)\n", err, strerror(err));
+        goto error;
+    }
+    if (getrusage(RUSAGE_THREAD, &watch->rusage) < 0) {
+        int err = errno;
+        fprintf(stderr, "getrusage failed: error %d (%s)\n",
+            err, strerror(err));
+        goto error;
+    }
+    return watch;
+
+error:
+    free(watch);
+    return NULL;
+}
+
+static void stopwatch_stop(struct stopwatch *watch,
+        long long bytes_read)
+{
+    double elapsed, rate;
+
+    if (clock_gettime(CLOCK_MONOTONIC, &watch->stop)) {
+        int err = errno;
+        fprintf(stderr, "clock_gettime(CLOCK_MONOTONIC) failed with "
+            "error %d (%s)\n", err, strerror(err));
+        goto done;
+    }
+    elapsed = timespec_to_double(&watch->stop) -
+        timespec_to_double(&watch->start);
+    rate = (bytes_read / elapsed) / (1024 * 1024 * 1024);
+    printf("stopwatch: took %.5g seconds to read %lld bytes, "
+        "for %.5g GB/s\n", elapsed, bytes_read, rate);
+    printf("stopwatch:  %.5g seconds\n", elapsed);
+done:
+    free(watch);
+}
+
+enum vecsum_type {
+    VECSUM_LOCAL = 0,
+    VECSUM_LIBHDFS,
+    VECSUM_ZCR,
+};
+
+#define VECSUM_TYPE_VALID_VALUES "libhdfs, zcr, or local"
+
+int parse_vecsum_type(const char *str)
+{
+    if (strcasecmp(str, "local") == 0)
+        return VECSUM_LOCAL;
+    else if (strcasecmp(str, "libhdfs") == 0)
+        return VECSUM_LIBHDFS;
+    else if (strcasecmp(str, "zcr") == 0)
+        return VECSUM_ZCR;
+    else
+        return -1;
+}
+
+struct options {
+    // The path to read.
+    const char *path;
+
+    // Length of the file.
+    long long length;
+
+    // The number of times to read the path.
+    int passes;
+
+    // Type of vecsum to do
+    enum vecsum_type ty;
+
+    // RPC address to use for HDFS
+    const char *rpc_address;
+};
+
+static struct options *options_create(void)
+{
+    struct options *opts = NULL;
+    const char *pass_str;
+    const char *ty_str;
+    const char *length_str;
+    int ty;
+
+    opts = calloc(1, sizeof(struct options));
+    if (!opts) {
+        fprintf(stderr, "failed to calloc options\n");
+        goto error;
+    }
+    opts->path = getenv("VECSUM_PATH");
+    if (!opts->path) {
+        fprintf(stderr, "You must set the VECSUM_PATH environment "
+            "variable to the path of the file to read.\n");
+        goto error;
+    }
+    length_str = getenv("VECSUM_LENGTH");
+    if (!length_str) {
+        length_str = "2147483648";
+    }
+    opts->length = atoll(length_str);
+    if (!opts->length) {
+        fprintf(stderr, "Can't parse VECSUM_LENGTH of '%s'.\n",
+                length_str);
+        goto error;
+    }
+    if (opts->length % VECSUM_CHUNK_SIZE) {
+        fprintf(stderr, "VECSUM_LENGTH must be a multiple of '%lld'.  The "
+                "currently specified length of '%lld' is not.\n",
+                (long long)VECSUM_CHUNK_SIZE, (long long)opts->length);
+        goto error;
+    }
+    pass_str = getenv("VECSUM_PASSES");
+    if (!pass_str) {
+        fprintf(stderr, "You must set the VECSUM_PASSES environment "
+            "variable to the number of passes to make.\n");
+        goto error;
+    }
+    opts->passes = atoi(pass_str);
+    if (opts->passes <= 0) {
+        fprintf(stderr, "Invalid value for the VECSUM_PASSES "
+            "environment variable.  You must set this to a "
+            "number greater than 0.\n");
+        goto error;
+    }
+    ty_str = getenv("VECSUM_TYPE");
+    if (!ty_str) {
+        fprintf(stderr, "You must set the VECSUM_TYPE environment "
+            "variable to " VECSUM_TYPE_VALID_VALUES "\n");
+        goto error;
+    }
+    ty = parse_vecsum_type(ty_str);
+    if (ty < 0) {
+        fprintf(stderr, "Invalid VECSUM_TYPE environment variable.  "
+            "Valid values are " VECSUM_TYPE_VALID_VALUES "\n");
+        goto error;
+    }
+    opts->ty = ty;
+    opts->rpc_address = getenv("VECSUM_RPC_ADDRESS");
+    if (!opts->rpc_address) {
+        opts->rpc_address = "default";
+    }
+    return opts;
+error:
+    free(opts);
+    return NULL;
+}
+
+static int test_file_chunk_setup(double **chunk)
+{
+    int i;
+    double *c, val;
+
+    c = malloc(VECSUM_CHUNK_SIZE);
+    if (!c) {
+        fprintf(stderr, "test_file_create: failed to malloc "
+                "a buffer of size '%lld'\n",
+                (long long) VECSUM_CHUNK_SIZE);
+        return EIO;
+    }
+    val = 0.0;
+    for (i = 0; i < VECSUM_CHUNK_SIZE / sizeof(double); i++) {
+        c[i] = val;
+        val += 0.5;
+    }
+    *chunk = c;
+    return 0;
+}
+
+static void options_free(struct options *opts)
+{
+    free(opts);
+}
+
+struct local_data {
+    int fd;
+    double *mmap;
+    long long length;
+};
+
+static int local_data_create_file(struct local_data *cdata,
+                                  const struct options *opts)
+{
+    int ret = EIO;
+    int dup_fd = -1;
+    FILE *fp = NULL;
+    double *chunk = NULL;
+    long long offset = 0;
+
+    dup_fd = dup(cdata->fd);
+    if (dup_fd < 0) {
+        ret = errno;
+        fprintf(stderr, "local_data_create_file: dup failed: %s (%d)\n",
+                strerror(ret), ret);
+        goto done;
+    }
+    fp = fdopen(dup_fd, "w");
+    if (!fp) {
+        ret = errno;
+        fprintf(stderr, "local_data_create_file: fdopen failed: %s (%d)\n",
+                strerror(ret), ret);
+        goto done;
+    }
+    ret = test_file_chunk_setup(&chunk);
+    if (ret)
+        goto done;
+    while (offset < opts->length) {
+        if (fwrite(chunk, VECSUM_CHUNK_SIZE, 1, fp) != 1) {
+            fprintf(stderr, "local_data_create_file: failed to write to "
+                    "the local file '%s' at offset %lld\n",
+                    opts->path, offset);
+            ret = EIO;
+            goto done;
+        }
+        offset += VECSUM_CHUNK_SIZE;
+    }
+    fprintf(stderr, "local_data_create_file: successfully re-wrote %s as "
+            "a file of length %lld\n", opts->path, opts->length);
+    ret = 0;
+
+done:
+    if (dup_fd >= 0) {
+        close(dup_fd);
+    }
+    if (fp) {
+        fclose(fp);
+    }
+    free(chunk);
+    return ret;
+}
+
+static struct local_data *local_data_create(const struct options *opts)
+{
+    struct local_data *cdata = NULL;
+    struct stat st_buf;
+
+    cdata = malloc(sizeof(*cdata));
+    if (!cdata) {
+        fprintf(stderr, "Failed to allocate local test data.\n");
+        goto error;
+    }
+    cdata->fd = -1;
+    cdata->mmap = MAP_FAILED;
+    cdata->length = opts->length;
+
+    cdata->fd = open(opts->path, O_RDWR | O_CREAT, 0777);
+    if (cdata->fd < 0) {
+        int err = errno;
+        fprintf(stderr, "local_data_create: failed to open %s "
+            "for read/write: error %d (%s)\n", opts->path, err, strerror(err));
+        goto error;
+    }
+    if (fstat(cdata->fd, &st_buf)) {
+        int err = errno;
+        fprintf(stderr, "local_data_create: fstat(%s) failed: "
+            "error %d (%s)\n", opts->path, err, strerror(err));
+        goto error;
+    }
+    if (st_buf.st_size != opts->length) {
+        int err;
+        fprintf(stderr, "local_data_create: current size of %s is %lld, but "
+                "we want %lld.  Re-writing the file.\n",
+                opts->path, (long long)st_buf.st_size,
+                (long long)opts->length);
+        err = local_data_create_file(cdata, opts);
+        if (err)
+            goto error;
+    }
+    cdata->mmap = mmap(NULL, cdata->length, PROT_READ,
+                       MAP_PRIVATE, cdata->fd, 0);
+    if (cdata->mmap == MAP_FAILED) {
+        int err = errno;
+        fprintf(stderr, "local_data_create: mmap(%s) failed: "
+            "error %d (%s)\n", opts->path, err, strerror(err));
+        goto error;
+    }
+    return cdata;
+
+error:
+    if (cdata) {
+        if (cdata->fd >= 0) {
+            close(cdata->fd);
+        }
+        free(cdata);
+    }
+    return NULL;
+}
+
+static void local_data_free(struct local_data *cdata)
+{
+    close(cdata->fd);
+    munmap(cdata->mmap, cdata->length);
+}
+
+struct libhdfs_data {
+    hdfsFS fs;
+    hdfsFile file;
+    long long length;
+    double *buf;
+};
+
+static void libhdfs_data_free(struct libhdfs_data *ldata)
+{
+    if (ldata->fs) {
+        free(ldata->buf);
+        if (ldata->file) {
+            hdfsCloseFile(ldata->fs, ldata->file);
+        }
+        hdfsDisconnect(ldata->fs);
+    }
+    free(ldata);
+}
+
+static int libhdfs_data_create_file(struct libhdfs_data *ldata,
+                                    const struct options *opts)
+{
+    int ret;
+    double *chunk = NULL;
+    long long offset = 0;
+
+    ldata->file = hdfsOpenFile(ldata->fs, opts->path, O_WRONLY, 0, 1, 0);
+    if (!ldata->file) {
+        ret = errno;
+        fprintf(stderr, "libhdfs_data_create_file: hdfsOpenFile(%s, "
+            "O_WRONLY) failed: error %d (%s)\n", opts->path, ret,
+            strerror(ret));
+        goto done;
+    }
+    ret = test_file_chunk_setup(&chunk);
+    if (ret)
+        goto done;
+    while (offset < opts->length) {
+        ret = hdfsWrite(ldata->fs, ldata->file, chunk, VECSUM_CHUNK_SIZE);
+        if (ret < 0) {
+            ret = errno;
+            fprintf(stderr, "libhdfs_data_create_file: got error %d (%s) at "
+                    "offset %lld of %s\n", ret, strerror(ret),
+                    offset, opts->path);
+            goto done;
+        } else if (ret < VECSUM_CHUNK_SIZE) {
+            fprintf(stderr, "libhdfs_data_create_file: got short write "
+                    "of %d at offset %lld of %s\n", ret, offset, opts->path);
+            goto done;
+        }
+        offset += VECSUM_CHUNK_SIZE;
+    }
+    ret = 0;
+done:
+    free(chunk);
+    if (ldata->file) {
+        if (hdfsCloseFile(ldata->fs, ldata->file)) {
+            fprintf(stderr, "libhdfs_data_create_file: hdfsCloseFile error.");
+            ret = EIO;
+        }
+        ldata->file = NULL;
+    }
+    return ret;
+}
+
+static struct libhdfs_data *libhdfs_data_create(const struct options *opts)
+{
+    struct libhdfs_data *ldata = NULL;
+    struct hdfsBuilder *builder = NULL;
+    hdfsFileInfo *pinfo = NULL;
+
+    ldata = calloc(1, sizeof(struct libhdfs_data));
+    if (!ldata) {
+        fprintf(stderr, "Failed to allocate libhdfs test data.\n");
+        goto error;
+    }
+    builder = hdfsNewBuilder();
+    if (!builder) {
+        fprintf(stderr, "Failed to create builder.\n");
+        goto error;
+    }
+    hdfsBuilderSetNameNode(builder, opts->rpc_address);
+    hdfsBuilderConfSetStr(builder,
+        "dfs.client.read.shortcircuit.skip.checksum", "true");
+    ldata->fs = hdfsBuilderConnect(builder);
+    if (!ldata->fs) {
+        fprintf(stderr, "Could not connect to default namenode!\n");
+        goto error;
+    }
+    pinfo = hdfsGetPathInfo(ldata->fs, opts->path);
+    if (!pinfo) {
+        int err = errno;
+        fprintf(stderr, "hdfsGetPathInfo(%s) failed: error %d (%s).  "
+                "Attempting to re-create file.\n",
+            opts->path, err, strerror(err));
+        if (libhdfs_data_create_file(ldata, opts))
+            goto error;
+    } else if (pinfo->mSize != opts->length) {
+        fprintf(stderr, "hdfsGetPathInfo(%s) failed: length was %lld, "
+                "but we want length %lld.  Attempting to re-create file.\n",
+                opts->path, (long long)pinfo->mSize, (long long)opts->length);
+        if (libhdfs_data_create_file(ldata, opts))
+            goto error;
+    }
+    ldata->file = hdfsOpenFile(ldata->fs, opts->path, O_RDONLY, 0, 0, 0);
+    if (!ldata->file) {
+        int err = errno;
+        fprintf(stderr, "hdfsOpenFile(%s) failed: error %d (%s)\n",
+            opts->path, err, strerror(err));
+        goto error;
+    }
+    ldata->length = opts->length;
+    return ldata;
+
+error:
+    if (pinfo)
+        hdfsFreeFileInfo(pinfo, 1);
+    if (ldata)
+        libhdfs_data_free(ldata);
+    return NULL;
+}
+
+static int check_byte_size(int byte_size, const char *const str)
+{
+    if (byte_size % sizeof(double)) {
+        fprintf(stderr, "%s is not a multiple "
+            "of sizeof(double)\n", str);
+        return EINVAL;
+    }
+    if ((byte_size / sizeof(double)) % DOUBLES_PER_LOOP_ITER) {
+        fprintf(stderr, "The number of doubles contained in "
+            "%s is not a multiple of DOUBLES_PER_LOOP_ITER\n",
+            str);
+        return EINVAL;
+    }
+    return 0;
+}
+
+#ifdef HAVE_INTEL_SSE_INTRINSICS
+
+#include <emmintrin.h>
+
+static double vecsum(const double *buf, int num_doubles)
+{
+    int i;
+    double hi, lo;
+    __m128d x0, x1, x2, x3, x4, x5, x6, x7;
+    __m128d sum0 = _mm_set_pd(0.0,0.0);
+    __m128d sum1 = _mm_set_pd(0.0,0.0);
+    __m128d sum2 = _mm_set_pd(0.0,0.0);
+    __m128d sum3 = _mm_set_pd(0.0,0.0);
+    __m128d sum4 = _mm_set_pd(0.0,0.0);
+    __m128d sum5 = _mm_set_pd(0.0,0.0);
+    __m128d sum6 = _mm_set_pd(0.0,0.0);
+    __m128d sum7 = _mm_set_pd(0.0,0.0);
+    for (i = 0; i < num_doubles; i+=DOUBLES_PER_LOOP_ITER) {
+        x0 = _mm_load_pd(buf + i + 0);
+        x1 = _mm_load_pd(buf + i + 2);
+        x2 = _mm_load_pd(buf + i + 4);
+        x3 = _mm_load_pd(buf + i + 6);
+        x4 = _mm_load_pd(buf + i + 8);
+        x5 = _mm_load_pd(buf + i + 10);
+        x6 = _mm_load_pd(buf + i + 12);
+        x7 = _mm_load_pd(buf + i + 14);
+        sum0 = _mm_add_pd(sum0, x0);
+        sum1 = _mm_add_pd(sum1, x1);
+        sum2 = _mm_add_pd(sum2, x2);
+        sum3 = _mm_add_pd(sum3, x3);
+        sum4 = _mm_add_pd(sum4, x4);
+        sum5 = _mm_add_pd(sum5, x5);
+        sum6 = _mm_add_pd(sum6, x6);
+        sum7 = _mm_add_pd(sum7, x7);
+    }
+    x0 = _mm_add_pd(sum0, sum1);
+    x1 = _mm_add_pd(sum2, sum3);
+    x2 = _mm_add_pd(sum4, sum5);
+    x3 = _mm_add_pd(sum6, sum7);
+    x4 = _mm_add_pd(x0, x1);
+    x5 = _mm_add_pd(x2, x3);
+    x6 = _mm_add_pd(x4, x5);
+    _mm_storeh_pd(&hi, x6);
+    _mm_storel_pd(&lo, x6);
+    return hi + lo;
+}
+
+#else
+
+static double vecsum(const double *buf, int num_doubles)
+{
+    int i;
+    double sum = 0.0;
+    for (i = 0; i < num_doubles; i++) {
+        sum += buf[i];
+    }
+    return sum;
+}
+
+#endif
+
+static int vecsum_zcr_loop(int pass, struct libhdfs_data *ldata,
+        struct hadoopRzOptions *zopts,
+        const struct options *opts)
+{
+    int32_t len;
+    double sum = 0.0;
+    const double *buf;
+    struct hadoopRzBuffer *rzbuf = NULL;
+    int ret;
+
+    while (1) {
+        rzbuf = hadoopReadZero(ldata->file, zopts, ZCR_READ_CHUNK_SIZE);
+        if (!rzbuf) {
+            ret = errno;
+            fprintf(stderr, "hadoopReadZero failed with error "
+                "code %d (%s)\n", ret, strerror(ret));
+            goto done;
+        }
+        buf = hadoopRzBufferGet(rzbuf);
+        if (!buf) break;
+        len = hadoopRzBufferLength(rzbuf);
+        if (len < ZCR_READ_CHUNK_SIZE) {
+            fprintf(stderr, "hadoopReadZero got a partial read "
+                "of length %d\n", len);
+            ret = EINVAL;
+            goto done;
+        }
+        sum += vecsum(buf,
+            ZCR_READ_CHUNK_SIZE / sizeof(double));
+        hadoopRzBufferFree(ldata->file, rzbuf);
+    }
+    printf("finished zcr pass %d.  sum = %g\n", pass, sum);
+    ret = 0;
+
+done:
+    if (rzbuf)
+        hadoopRzBufferFree(ldata->file, rzbuf);
+    return ret;
+}
+
+static int vecsum_zcr(struct libhdfs_data *ldata,
+        const struct options *opts)
+{
+    int ret, pass;
+    struct hadoopRzOptions *zopts = NULL;
+
+    zopts = hadoopRzOptionsAlloc();
+    if (!zopts) {
+        fprintf(stderr, "hadoopRzOptionsAlloc failed.\n");
+        ret = ENOMEM;
+        goto done;
+    }
+    if (hadoopRzOptionsSetSkipChecksum(zopts, 1)) {
+        ret = errno;
+        perror("hadoopRzOptionsSetSkipChecksum failed: ");
+        goto done;
+    }
+    if (hadoopRzOptionsSetByteBufferPool(zopts, NULL)) {
+        ret = errno;
+        perror("hadoopRzOptionsSetByteBufferPool failed: ");
+        goto done;
+    }
+    for (pass = 0; pass < opts->passes; ++pass) {
+        ret = vecsum_zcr_loop(pass, ldata, zopts, opts);
+        if (ret) {
+            fprintf(stderr, "vecsum_zcr_loop pass %d failed "
+                "with error %d\n", pass, ret);
+            goto done;
+        }
+        hdfsSeek(ldata->fs, ldata->file, 0);
+    }
+    ret = 0;
+done:
+    if (zopts)
+        hadoopRzOptionsFree(zopts);
+    return ret;
+}
+
+tSize hdfsReadFully(hdfsFS fs, hdfsFile f, void* buffer, tSize length)
+{
+    uint8_t *buf = buffer;
+    tSize ret, nread = 0;
+
+    while (length > 0) {
+        ret = hdfsRead(fs, f, buf, length);
+        if (ret < 0) {
+            if (errno != EINTR) {
+                return -1;
+            }
+        }
+        if (ret == 0) {
+            break;
+        }
+        nread += ret;
+        length -= ret;
+        buf += ret;
+    }
+    return nread;
+}
+
+static int vecsum_normal_loop(int pass, const struct libhdfs_data *ldata,
+            const struct options *opts)
+{
+    double sum = 0.0;
+
+    while (1) {
+        int res = hdfsReadFully(ldata->fs, ldata->file, ldata->buf,
+                NORMAL_READ_CHUNK_SIZE);
+        if (res == 0) // EOF
+            break;
+        if (res < 0) {
+            int err = errno;
+            fprintf(stderr, "hdfsRead failed with error %d (%s)\n",
+                err, strerror(err));
+            return err;
+        }
+        if (res < NORMAL_READ_CHUNK_SIZE) {
+            fprintf(stderr, "hdfsRead got a partial read of "
+                "length %d\n", res);
+            return EINVAL;
+        }
+        sum += vecsum(ldata->buf,
+                  NORMAL_READ_CHUNK_SIZE / sizeof(double));
+    }
+    printf("finished normal pass %d.  sum = %g\n", pass, sum);
+    return 0;
+}
+
+static int vecsum_libhdfs(struct libhdfs_data *ldata,
+            const struct options *opts)
+{
+    int pass;
+
+    ldata->buf = malloc(NORMAL_READ_CHUNK_SIZE);
+    if (!ldata->buf) {
+        fprintf(stderr, "failed to malloc buffer of size %d\n",
+            NORMAL_READ_CHUNK_SIZE);
+        return ENOMEM;
+    }
+    for (pass = 0; pass < opts->passes; ++pass) {
+        int ret = vecsum_normal_loop(pass, ldata, opts);
+        if (ret) {
+            fprintf(stderr, "vecsum_normal_loop pass %d failed "
+                "with error %d\n", pass, ret);
+            return ret;
+        }
+        hdfsSeek(ldata->fs, ldata->file, 0);
+    }
+    return 0;
+}
+
+static void vecsum_local(struct local_data *cdata, const struct options *opts)
+{
+    int pass;
+
+    for (pass = 0; pass < opts->passes; pass++) {
+        double sum = vecsum(cdata->mmap, cdata->length / sizeof(double));
+        printf("finished vecsum_local pass %d.  sum = %g\n", pass, sum);
+    }
+}
+
+static long long vecsum_length(const struct options *opts,
+                const struct libhdfs_data *ldata)
+{
+    if (opts->ty == VECSUM_LOCAL) {
+        struct stat st_buf = { 0 };
+        if (stat(opts->path, &st_buf)) {
+            int err = errno;
+            fprintf(stderr, "vecsum_length: stat(%s) failed: "
+                "error %d (%s)\n", opts->path, err, strerror(err));
+            return -EIO;
+        }
+        return st_buf.st_size;
+    } else {
+        return ldata->length;
+    }
+}
+
+/*
+ * vecsum is a microbenchmark which measures the speed of various ways of
+ * reading from HDFS.  It creates a file containing floating-point 'doubles',
+ * and computes the sum of all the doubles several times.  For some CPUs,
+ * assembly optimizations are used for the summation (SSE, etc).
+ */
+int main(void)
+{
+    int ret = 1;
+    struct options *opts = NULL;
+    struct local_data *cdata = NULL;
+    struct libhdfs_data *ldata = NULL;
+    struct stopwatch *watch = NULL;
+
+    if (check_byte_size(VECSUM_CHUNK_SIZE, "VECSUM_CHUNK_SIZE") ||
+        check_byte_size(ZCR_READ_CHUNK_SIZE,
+                "ZCR_READ_CHUNK_SIZE") ||
+        check_byte_size(NORMAL_READ_CHUNK_SIZE,
+                "NORMAL_READ_CHUNK_SIZE")) {
+        goto done;
+    }
+    opts = options_create();
+    if (!opts)
+        goto done;
+    if (opts->ty == VECSUM_LOCAL) {
+        cdata = local_data_create(opts);
+        if (!cdata)
+            goto done;
+    } else {
+        ldata = libhdfs_data_create(opts);
+        if (!ldata)
+            goto done;
+    }
+    watch = stopwatch_create();
+    if (!watch)
+        goto done;
+    switch (opts->ty) {
+    case VECSUM_LOCAL:
+        vecsum_local(cdata, opts);
+        ret = 0;
+        break;
+    case VECSUM_LIBHDFS:
+        ret = vecsum_libhdfs(ldata, opts);
+        break;
+    case VECSUM_ZCR:
+        ret = vecsum_zcr(ldata, opts);
+        break;
+    }
+    if (ret) {
+        fprintf(stderr, "vecsum failed with error %d\n", ret);
+        goto done;
+    }
+    ret = 0;
+done:
+    fprintf(stderr, "cleaning up...\n");
+    if (watch && (ret == 0)) {
+        long long length = vecsum_length(opts, ldata);
+        if (length >= 0) {
+            stopwatch_stop(watch, length * opts->passes);
+        }
+    }
+    if (cdata)
+        local_data_free(cdata);
+    if (ldata)
+        libhdfs_data_free(ldata);
+    if (opts)
+        options_free(opts);
+    return ret;
+}
+
+// vim: ts=4:sw=4:tw=79:et

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/test_libhdfs_threaded.c

@@ -122,7 +122,7 @@ struct tlhPaths {
 
 static int setupPaths(const struct tlhThreadInfo *ti, struct tlhPaths *paths)
 {
-    memset(paths, sizeof(*paths), 0);
+    memset(paths, 0, sizeof(*paths));
     if (snprintf(paths->prefix, sizeof(paths->prefix), "/tlhData%04d",
                  ti->threadIdx) >= sizeof(paths->prefix)) {
         return ENAMETOOLONG;

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

@@ -91,7 +91,7 @@
         <tbody>
           {#FileStatus}
           <tr>
-            <td>{type|helper_to_directory}{permission|helper_to_permission}</td>
+            <td>{type|helper_to_directory}{permission|helper_to_permission}{aclBit|helper_to_acl_bit}</td>
             <td>{owner}</td>
             <td>{group}</td>
             <td>{length|fmt_bytes}</td>

+ 7 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dfs-dust.js

@@ -63,7 +63,8 @@
 
     'helper_to_permission': function (v) {
       var symbols = [ '---', '--x', '-w-', '-wx', 'r--', 'r-x', 'rw-', 'rwx' ];
-      var sticky = v > 1000;
+      var vInt = parseInt(v, 8);
+      var sticky = (vInt & (1 << 9)) != 0;
 
       var res = "";
       for (var i = 0; i < 3; ++i) {
@@ -72,7 +73,7 @@
       }
 
       if (sticky) {
-        var otherExec = ((v % 10) & 1) == 1;
+        var otherExec = (vInt & 1) == 1;
         res = res.substr(0, res.length - 1) + (otherExec ? 't' : 'T');
       }
 
@@ -81,6 +82,10 @@
 
     'helper_to_directory' : function (v) {
       return v === 'DIRECTORY' ? 'd' : '-';
+    },
+
+    'helper_to_acl_bit': function (v) {
+      return v ? '+' : "";
     }
   };
   $.extend(dust.filters, filters);

+ 30 - 20
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithNodeGroup.java

@@ -22,8 +22,9 @@ import static org.junit.Assert.assertEquals;
 import java.io.IOException;
 import java.net.URI;
 import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
 import java.util.concurrent.TimeoutException;
 
 import org.apache.commons.logging.Log;
@@ -39,6 +40,9 @@ import org.apache.hadoop.hdfs.MiniDFSClusterWithNodeGroup;
 import org.apache.hadoop.hdfs.NameNodeProxies;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyWithNodeGroup;
 import org.apache.hadoop.net.NetworkTopology;
@@ -53,7 +57,7 @@ public class TestBalancerWithNodeGroup {
   private static final Log LOG = LogFactory.getLog(
   "org.apache.hadoop.hdfs.TestBalancerWithNodeGroup");
   
-  final private static long CAPACITY = 6000L;
+  final private static long CAPACITY = 5000L;
   final private static String RACK0 = "/rack0";
   final private static String RACK1 = "/rack1";
   final private static String NODEGROUP0 = "/nodegroup0";
@@ -77,6 +81,7 @@ public class TestBalancerWithNodeGroup {
   static Configuration createConf() {
     Configuration conf = new HdfsConfiguration();
     TestBalancer.initConf(conf);
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE);
     conf.set(CommonConfigurationKeysPublic.NET_TOPOLOGY_IMPL_KEY, 
         NetworkTopologyWithNodeGroup.class.getName());
     conf.set(DFSConfigKeys.DFS_BLOCK_REPLICATOR_CLASSNAME_KEY, 
@@ -191,6 +196,19 @@ public class TestBalancerWithNodeGroup {
     LOG.info("Rebalancing with default factor.");
   }
 
+  private Set<ExtendedBlock> getBlocksOnRack(List<LocatedBlock> blks, String rack) {
+    Set<ExtendedBlock> ret = new HashSet<ExtendedBlock>();
+    for (LocatedBlock blk : blks) {
+      for (DatanodeInfo di : blk.getLocations()) {
+        if (rack.equals(NetworkTopology.getFirstHalf(di.getNetworkLocation()))) {
+          ret.add(blk.getBlock());
+          break;
+        }
+      }
+    }
+    return ret;
+  }
+
   /**
    * Create a cluster with even distribution, and a new empty node is added to
    * the cluster, then test rack locality for balancer policy. 
@@ -220,9 +238,14 @@ public class TestBalancerWithNodeGroup {
 
       // fill up the cluster to be 30% full
       long totalUsedSpace = totalCapacity * 3 / 10;
-      TestBalancer.createFile(cluster, filePath, totalUsedSpace / numOfDatanodes,
+      long length = totalUsedSpace / numOfDatanodes;
+      TestBalancer.createFile(cluster, filePath, length,
           (short) numOfDatanodes, 0);
       
+      LocatedBlocks lbs = client.getBlockLocations(filePath.toUri().getPath(), 0,
+          length);
+      Set<ExtendedBlock> before = getBlocksOnRack(lbs.getLocatedBlocks(), RACK0);
+
       long newCapacity = CAPACITY;
       String newRack = RACK1;
       String newNodeGroup = NODEGROUP2;
@@ -235,22 +258,9 @@ public class TestBalancerWithNodeGroup {
       // run balancer and validate results
       runBalancerCanFinish(conf, totalUsedSpace, totalCapacity);
       
-      DatanodeInfo[] datanodeReport = 
-              client.getDatanodeReport(DatanodeReportType.ALL);
-      
-      Map<String, Integer> rackToUsedCapacity = new HashMap<String, Integer>();
-      for (DatanodeInfo datanode: datanodeReport) {
-        String rack = NetworkTopology.getFirstHalf(datanode.getNetworkLocation());
-        int usedCapacity = (int) datanode.getDfsUsed();
-         
-        if (rackToUsedCapacity.get(rack) != null) {
-          rackToUsedCapacity.put(rack, usedCapacity + rackToUsedCapacity.get(rack));
-        } else {
-          rackToUsedCapacity.put(rack, usedCapacity);
-        }
-      }
-      assertEquals(rackToUsedCapacity.size(), 2);
-      assertEquals(rackToUsedCapacity.get(RACK0), rackToUsedCapacity.get(RACK1));
+      lbs = client.getBlockLocations(filePath.toUri().getPath(), 0, length);
+      Set<ExtendedBlock> after = getBlocksOnRack(lbs.getLocatedBlocks(), RACK0);
+      assertEquals(before, after);
       
     } finally {
       cluster.shutdown();

+ 5 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/AclTestHelpers.java

@@ -27,6 +27,7 @@ import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclEntryScope;
 import org.apache.hadoop.fs.permission.AclEntryType;
 import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -150,6 +151,9 @@ public final class AclTestHelpers {
    */
   public static void assertPermission(FileSystem fs, Path pathToCheck,
       short perm) throws IOException {
-    assertEquals(perm, fs.getFileStatus(pathToCheck).getPermission().toShort());
+    short filteredPerm = (short)(perm & 01777);
+    FsPermission fsPermission = fs.getFileStatus(pathToCheck).getPermission();
+    assertEquals(filteredPerm, fsPermission.toShort());
+    assertEquals(((perm & (1 << 12)) != 0), fsPermission.getAclBit());
   }
 }

+ 52 - 34
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSAclBaseTest.java

@@ -38,6 +38,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.AclException;
+import org.apache.hadoop.hdfs.protocol.FsAclPermission;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -118,7 +119,7 @@ public abstract class FSAclBaseTest {
       aclEntry(DEFAULT, GROUP, READ_EXECUTE),
       aclEntry(DEFAULT, MASK, READ_EXECUTE),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission((short)0750);
+    assertPermission((short)010750);
     assertAclFeature(true);
   }
 
@@ -140,7 +141,7 @@ public abstract class FSAclBaseTest {
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "foo", READ_EXECUTE),
       aclEntry(ACCESS, GROUP, READ_EXECUTE) }, returned);
-    assertPermission((short)0750);
+    assertPermission((short)010750);
     assertAclFeature(true);
   }
 
@@ -161,7 +162,7 @@ public abstract class FSAclBaseTest {
       aclEntry(DEFAULT, GROUP, READ_EXECUTE),
       aclEntry(DEFAULT, MASK, READ_EXECUTE),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission((short)0750);
+    assertPermission((short)010750);
     assertAclFeature(true);
   }
 
@@ -177,7 +178,7 @@ public abstract class FSAclBaseTest {
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "foo", READ_WRITE),
       aclEntry(ACCESS, GROUP, READ) }, returned);
-    assertPermission((short)0660);
+    assertPermission((short)010660);
     assertAclFeature(true);
   }
 
@@ -195,7 +196,7 @@ public abstract class FSAclBaseTest {
       aclEntry(DEFAULT, USER, ALL),
       aclEntry(DEFAULT, GROUP, READ_EXECUTE),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission((short)0750);
+    assertPermission((short)010750);
     assertAclFeature(true);
   }
 
@@ -212,7 +213,7 @@ public abstract class FSAclBaseTest {
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "foo", ALL),
       aclEntry(ACCESS, GROUP, READ) }, returned);
-    assertPermission((short)0600);
+    assertPermission((short)010600);
     assertAclFeature(true);
   }
 
@@ -240,7 +241,7 @@ public abstract class FSAclBaseTest {
       aclEntry(DEFAULT, GROUP, READ_EXECUTE),
       aclEntry(DEFAULT, MASK, READ_EXECUTE),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission((short)01750);
+    assertPermission((short)011750);
     assertAclFeature(true);
   }
 
@@ -286,7 +287,7 @@ public abstract class FSAclBaseTest {
       aclEntry(DEFAULT, GROUP, READ_EXECUTE),
       aclEntry(DEFAULT, MASK, READ_EXECUTE),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission((short)0750);
+    assertPermission((short)010750);
     assertAclFeature(true);
   }
 
@@ -309,7 +310,7 @@ public abstract class FSAclBaseTest {
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "bar", READ_WRITE),
       aclEntry(ACCESS, GROUP, READ_WRITE) }, returned);
-    assertPermission((short)0760);
+    assertPermission((short)010760);
     assertAclFeature(true);
   }
 
@@ -334,7 +335,7 @@ public abstract class FSAclBaseTest {
       aclEntry(DEFAULT, GROUP, READ_EXECUTE),
       aclEntry(DEFAULT, MASK, READ_EXECUTE),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission((short)0750);
+    assertPermission((short)010750);
     assertAclFeature(true);
   }
 
@@ -382,7 +383,7 @@ public abstract class FSAclBaseTest {
       aclEntry(DEFAULT, USER, ALL),
       aclEntry(DEFAULT, GROUP, READ_EXECUTE),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission((short)0750);
+    assertPermission((short)010750);
     assertAclFeature(true);
   }
 
@@ -408,7 +409,7 @@ public abstract class FSAclBaseTest {
       aclEntry(DEFAULT, GROUP, READ_EXECUTE),
       aclEntry(DEFAULT, MASK, READ_EXECUTE),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission((short)01750);
+    assertPermission((short)011750);
     assertAclFeature(true);
   }
 
@@ -436,7 +437,7 @@ public abstract class FSAclBaseTest {
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "foo", ALL),
       aclEntry(ACCESS, GROUP, READ_EXECUTE) }, returned);
-    assertPermission((short)0770);
+    assertPermission((short)010770);
     assertAclFeature(true);
   }
 
@@ -456,7 +457,7 @@ public abstract class FSAclBaseTest {
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "foo", ALL),
       aclEntry(ACCESS, GROUP, READ_EXECUTE) }, returned);
-    assertPermission((short)0770);
+    assertPermission((short)010770);
     assertAclFeature(true);
   }
 
@@ -501,7 +502,7 @@ public abstract class FSAclBaseTest {
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "foo", ALL),
       aclEntry(ACCESS, GROUP, READ_EXECUTE) }, returned);
-    assertPermission((short)01770);
+    assertPermission((short)011770);
     assertAclFeature(true);
   }
 
@@ -602,7 +603,7 @@ public abstract class FSAclBaseTest {
       aclEntry(DEFAULT, GROUP, READ_EXECUTE),
       aclEntry(DEFAULT, MASK, ALL),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission((short)0770);
+    assertPermission((short)010770);
     assertAclFeature(true);
   }
 
@@ -621,7 +622,7 @@ public abstract class FSAclBaseTest {
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "foo", READ),
       aclEntry(ACCESS, GROUP, READ) }, returned);
-    assertPermission((short)0640);
+    assertPermission((short)010640);
     assertAclFeature(true);
   }
 
@@ -639,7 +640,7 @@ public abstract class FSAclBaseTest {
       aclEntry(DEFAULT, GROUP, READ_EXECUTE),
       aclEntry(DEFAULT, MASK, ALL),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission((short)0750);
+    assertPermission((short)010750);
     assertAclFeature(true);
   }
 
@@ -679,7 +680,7 @@ public abstract class FSAclBaseTest {
       aclEntry(DEFAULT, USER, ALL),
       aclEntry(DEFAULT, GROUP, READ_EXECUTE),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission((short)0750);
+    assertPermission((short)010750);
     assertAclFeature(true);
   }
 
@@ -699,7 +700,7 @@ public abstract class FSAclBaseTest {
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "foo", READ),
       aclEntry(ACCESS, GROUP, READ) }, returned);
-    assertPermission((short)0670);
+    assertPermission((short)010670);
     assertAclFeature(true);
   }
 
@@ -723,7 +724,7 @@ public abstract class FSAclBaseTest {
       aclEntry(DEFAULT, GROUP, READ_EXECUTE),
       aclEntry(DEFAULT, MASK, ALL),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission((short)01770);
+    assertPermission((short)011770);
     assertAclFeature(true);
   }
 
@@ -768,7 +769,7 @@ public abstract class FSAclBaseTest {
       aclEntry(DEFAULT, GROUP, READ_EXECUTE),
       aclEntry(DEFAULT, MASK, ALL),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission((short)0700);
+    assertPermission((short)010700);
     assertAclFeature(true);
   }
 
@@ -788,7 +789,7 @@ public abstract class FSAclBaseTest {
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "foo", READ),
       aclEntry(ACCESS, GROUP, READ) }, returned);
-    assertPermission((short)0600);
+    assertPermission((short)010600);
     assertAclFeature(true);
   }
 
@@ -810,10 +811,27 @@ public abstract class FSAclBaseTest {
       aclEntry(DEFAULT, GROUP, READ_EXECUTE),
       aclEntry(DEFAULT, MASK, ALL),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission((short)0700);
+    assertPermission((short)010700);
     assertAclFeature(true);
   }
 
+  @Test
+  public void testSetPermissionCannotSetAclBit() throws IOException {
+    FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short)0750));
+    fs.setPermission(path, FsPermission.createImmutable((short)0700));
+    assertPermission((short)0700);
+    fs.setPermission(path,
+      new FsAclPermission(FsPermission.createImmutable((short)0755)));
+    INode inode = cluster.getNamesystem().getFSDirectory().getNode(
+      path.toUri().getPath(), false);
+    assertNotNull(inode);
+    FsPermission perm = inode.getFsPermission();
+    assertNotNull(perm);
+    assertEquals(0755, perm.toShort());
+    assertEquals(0755, perm.toExtendedShort());
+    assertAclFeature(false);
+  }
+
   @Test
   public void testDefaultAclNewFile() throws Exception {
     FileSystem.mkdirs(fs, path, FsPermission.createImmutable((short)0750));
@@ -827,7 +845,7 @@ public abstract class FSAclBaseTest {
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "foo", ALL),
       aclEntry(ACCESS, GROUP, READ_EXECUTE) }, returned);
-    assertPermission(filePath, (short)0640);
+    assertPermission(filePath, (short)010640);
     assertAclFeature(filePath, true);
   }
 
@@ -881,7 +899,7 @@ public abstract class FSAclBaseTest {
       aclEntry(DEFAULT, GROUP, READ_EXECUTE),
       aclEntry(DEFAULT, MASK, ALL),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission(dirPath, (short)0750);
+    assertPermission(dirPath, (short)010750);
     assertAclFeature(dirPath, true);
   }
 
@@ -916,7 +934,7 @@ public abstract class FSAclBaseTest {
       aclEntry(DEFAULT, USER, ALL),
       aclEntry(DEFAULT, GROUP, READ_EXECUTE),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission(dirPath, (short)0750);
+    assertPermission(dirPath, (short)010750);
     assertAclFeature(dirPath, true);
   }
 
@@ -940,7 +958,7 @@ public abstract class FSAclBaseTest {
     AclStatus s = fs.getAclStatus(dirPath);
     AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission(dirPath, (short)0750);
+    assertPermission(dirPath, (short)010750);
     assertAclFeature(dirPath, true);
     expected = new AclEntry[] {
       aclEntry(ACCESS, USER, "foo", ALL),
@@ -948,7 +966,7 @@ public abstract class FSAclBaseTest {
     s = fs.getAclStatus(filePath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission(filePath, (short)0640);
+    assertPermission(filePath, (short)010640);
     assertAclFeature(filePath, true);
   }
 
@@ -972,12 +990,12 @@ public abstract class FSAclBaseTest {
     AclStatus s = fs.getAclStatus(dirPath);
     AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission(dirPath, (short)0750);
+    assertPermission(dirPath, (short)010750);
     assertAclFeature(dirPath, true);
     s = fs.getAclStatus(subdirPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission(subdirPath, (short)0750);
+    assertPermission(subdirPath, (short)010750);
     assertAclFeature(subdirPath, true);
   }
 
@@ -1004,7 +1022,7 @@ public abstract class FSAclBaseTest {
     AclStatus s = fs.getAclStatus(dirPath);
     AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission(dirPath, (short)0750);
+    assertPermission(dirPath, (short)010750);
     assertAclFeature(dirPath, true);
     expected = new AclEntry[] { };
     s = fs.getAclStatus(linkPath);
@@ -1037,7 +1055,7 @@ public abstract class FSAclBaseTest {
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "foo", ALL),
       aclEntry(ACCESS, GROUP, READ_EXECUTE) }, returned);
-    assertPermission(filePath, (short)0740);
+    assertPermission(filePath, (short)010740);
     assertAclFeature(filePath, true);
   }
 
@@ -1059,7 +1077,7 @@ public abstract class FSAclBaseTest {
       aclEntry(DEFAULT, GROUP, READ_EXECUTE),
       aclEntry(DEFAULT, MASK, ALL),
       aclEntry(DEFAULT, OTHER, READ_EXECUTE) }, returned);
-    assertPermission(dirPath, (short)0740);
+    assertPermission(dirPath, (short)010740);
     assertAclFeature(dirPath, true);
   }
 

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java

@@ -477,6 +477,12 @@ public class TestCacheDirectives {
     iter = dfs.listCacheDirectives(
         new CacheDirectiveInfo.Builder().setPool("pool2").build());
     validateListAll(iter, betaId);
+    iter = dfs.listCacheDirectives(
+        new CacheDirectiveInfo.Builder().setId(alphaId2).build());
+    validateListAll(iter, alphaId2);
+    iter = dfs.listCacheDirectives(
+        new CacheDirectiveInfo.Builder().setId(relativeId).build());
+    validateListAll(iter, relativeId);
 
     dfs.removeCacheDirective(betaId);
     iter = dfs.listCacheDirectives(

+ 43 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java

@@ -27,6 +27,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -43,6 +44,7 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 
+import com.google.common.io.Files;
 import org.apache.commons.cli.ParseException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -87,7 +89,6 @@ import org.apache.hadoop.util.ExitUtil.ExitException;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.log4j.Level;
 import org.junit.After;
-import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.ArgumentMatcher;
@@ -1084,7 +1085,7 @@ public class TestCheckpoint {
       
       FSDirectory secondaryFsDir = secondary.getFSNamesystem().dir;
       INode rootInMap = secondaryFsDir.getInode(secondaryFsDir.rootDir.getId());
-      Assert.assertSame(rootInMap, secondaryFsDir.rootDir);
+      assertSame(rootInMap, secondaryFsDir.rootDir);
       
       fileSys.delete(tmpDir, true);
       fileSys.mkdirs(tmpDir);
@@ -2404,6 +2405,46 @@ public class TestCheckpoint {
     }
   }
 
+  @Test
+  public void testLegacyOivImage() throws Exception {
+    MiniDFSCluster cluster = null;
+    SecondaryNameNode secondary = null;
+    File tmpDir = Files.createTempDir();
+    Configuration conf = new HdfsConfiguration();
+    conf.set(DFSConfigKeys.DFS_NAMENODE_LEGACY_OIV_IMAGE_DIR_KEY,
+        tmpDir.getAbsolutePath());
+    conf.set(DFSConfigKeys.DFS_NAMENODE_NUM_CHECKPOINTS_RETAINED_KEY,
+        "2");
+
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
+              .format(true).build();
+
+      secondary = startSecondaryNameNode(conf);
+
+      // Checkpoint once
+      secondary.doCheckpoint();
+      String files1[] = tmpDir.list();
+      assertEquals("Only one file is expected", 1, files1.length);
+
+      // Perform more checkpointngs and check whether retention management
+      // is working.
+      secondary.doCheckpoint();
+      secondary.doCheckpoint();
+      String files2[] = tmpDir.list();
+      assertEquals("Two files are expected", 2, files2.length);
+
+      // Verify that the first file is deleted.
+      for (String fName : files2) {
+        assertFalse(fName.equals(files1[0]));
+      }
+    } finally {
+      cleanup(secondary);
+      cleanup(cluster);
+      tmpDir.delete();
+    }
+  }
+
   private static void cleanup(SecondaryNameNode snn) {
     if (snn != null) {
       try {

+ 6 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithAcl.java

@@ -142,7 +142,7 @@ public class TestFSImageWithAcl {
     AclEntry[] subdirReturned = fs.getAclStatus(subdirPath).getEntries()
       .toArray(new AclEntry[0]);
     Assert.assertArrayEquals(subdirExpected, subdirReturned);
-    assertPermission(fs, subdirPath, (short)0755);
+    assertPermission(fs, subdirPath, (short)010755);
 
     restart(fs, persistNamespace);
 
@@ -152,7 +152,7 @@ public class TestFSImageWithAcl {
     subdirReturned = fs.getAclStatus(subdirPath).getEntries()
       .toArray(new AclEntry[0]);
     Assert.assertArrayEquals(subdirExpected, subdirReturned);
-    assertPermission(fs, subdirPath, (short)0755);
+    assertPermission(fs, subdirPath, (short)010755);
 
     aclSpec = Lists.newArrayList(aclEntry(DEFAULT, USER, "foo", READ_WRITE));
     fs.modifyAclEntries(dirPath, aclSpec);
@@ -163,7 +163,7 @@ public class TestFSImageWithAcl {
     subdirReturned = fs.getAclStatus(subdirPath).getEntries()
       .toArray(new AclEntry[0]);
     Assert.assertArrayEquals(subdirExpected, subdirReturned);
-    assertPermission(fs, subdirPath, (short)0755);
+    assertPermission(fs, subdirPath, (short)010755);
 
     restart(fs, persistNamespace);
 
@@ -173,7 +173,7 @@ public class TestFSImageWithAcl {
     subdirReturned = fs.getAclStatus(subdirPath).getEntries()
       .toArray(new AclEntry[0]);
     Assert.assertArrayEquals(subdirExpected, subdirReturned);
-    assertPermission(fs, subdirPath, (short)0755);
+    assertPermission(fs, subdirPath, (short)010755);
 
     fs.removeAcl(dirPath);
 
@@ -183,7 +183,7 @@ public class TestFSImageWithAcl {
     subdirReturned = fs.getAclStatus(subdirPath).getEntries()
       .toArray(new AclEntry[0]);
     Assert.assertArrayEquals(subdirExpected, subdirReturned);
-    assertPermission(fs, subdirPath, (short)0755);
+    assertPermission(fs, subdirPath, (short)010755);
 
     restart(fs, persistNamespace);
 
@@ -193,7 +193,7 @@ public class TestFSImageWithAcl {
     subdirReturned = fs.getAclStatus(subdirPath).getEntries()
       .toArray(new AclEntry[0]);
     Assert.assertArrayEquals(subdirExpected, subdirReturned);
-    assertPermission(fs, subdirPath, (short)0755);
+    assertPermission(fs, subdirPath, (short)010755);
   }
 
   @Test

+ 9 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java

@@ -66,24 +66,28 @@ import com.google.common.base.Supplier;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
-
+import com.google.common.io.Files;
 
 public class TestStandbyCheckpoints {
   private static final int NUM_DIRS_IN_LOG = 200000;
   protected MiniDFSCluster cluster;
   protected NameNode nn0, nn1;
   protected FileSystem fs;
+  protected File tmpOivImgDir;
   
   private static final Log LOG = LogFactory.getLog(TestStandbyCheckpoints.class);
 
   @SuppressWarnings("rawtypes")
   @Before
   public void setupCluster() throws Exception {
+    tmpOivImgDir = Files.createTempDir();
     Configuration conf = new Configuration();
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_CHECK_PERIOD_KEY, 1);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_KEY, 5);
     conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
-    
+    conf.set(DFSConfigKeys.DFS_NAMENODE_LEGACY_OIV_IMAGE_DIR_KEY,
+        tmpOivImgDir.getAbsolutePath());
+
     // Dial down the retention of extra edits and checkpoints. This is to
     // help catch regressions of HDFS-4238 (SBN should not purge shared edits)
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_NUM_CHECKPOINTS_RETAINED_KEY, 1);
@@ -129,6 +133,9 @@ public class TestStandbyCheckpoints {
     // Once the standby catches up, it should notice that it needs to
     // do a checkpoint and save one to its local directories.
     HATestUtil.waitForCheckpoint(cluster, 1, ImmutableList.of(12));
+
+    // It should have saved the oiv image too.
+    assertEquals("One file is expected", 1, tmpOivImgDir.list().length);
     
     // It should also upload it back to the active.
     HATestUtil.waitForCheckpoint(cluster, 0, ImmutableList.of(12));

+ 28 - 28
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestAclWithSnapshot.java

@@ -119,14 +119,14 @@ public class TestAclWithSnapshot {
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "bruce", READ_EXECUTE),
       aclEntry(ACCESS, GROUP, NONE) }, returned);
-    assertPermission((short)0750, path);
+    assertPermission((short)010750, path);
 
     s = hdfs.getAclStatus(snapshotPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "bruce", READ_EXECUTE),
       aclEntry(ACCESS, GROUP, NONE) }, returned);
-    assertPermission((short)0750, snapshotPath);
+    assertPermission((short)010750, snapshotPath);
 
     assertDirPermissionGranted(fsAsBruce, BRUCE, snapshotPath);
     assertDirPermissionDenied(fsAsDiana, DIANA, snapshotPath);
@@ -153,14 +153,14 @@ public class TestAclWithSnapshot {
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "diana", READ_EXECUTE),
       aclEntry(ACCESS, GROUP, NONE) }, returned);
-    assertPermission((short)0550, path);
+    assertPermission((short)010550, path);
 
     s = hdfs.getAclStatus(snapshotPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "bruce", READ_EXECUTE),
       aclEntry(ACCESS, GROUP, NONE) }, returned);
-    assertPermission((short)0750, snapshotPath);
+    assertPermission((short)010750, snapshotPath);
 
     assertDirPermissionDenied(fsAsBruce, BRUCE, path);
     assertDirPermissionGranted(fsAsDiana, DIANA, path);
@@ -202,24 +202,24 @@ public class TestAclWithSnapshot {
     AclStatus s = hdfs.getAclStatus(filePath);
     AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission((short)0550, filePath);
+    assertPermission((short)010550, filePath);
 
     s = hdfs.getAclStatus(subdirPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission((short)0550, subdirPath);
+    assertPermission((short)010550, subdirPath);
 
     s = hdfs.getAclStatus(fileSnapshotPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission((short)0550, fileSnapshotPath);
+    assertPermission((short)010550, fileSnapshotPath);
     assertFilePermissionGranted(fsAsBruce, BRUCE, fileSnapshotPath);
     assertFilePermissionDenied(fsAsDiana, DIANA, fileSnapshotPath);
 
     s = hdfs.getAclStatus(subdirSnapshotPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission((short)0550, subdirSnapshotPath);
+    assertPermission((short)010550, subdirSnapshotPath);
     assertDirPermissionGranted(fsAsBruce, BRUCE, subdirSnapshotPath);
     assertDirPermissionDenied(fsAsDiana, DIANA, subdirSnapshotPath);
 
@@ -251,14 +251,14 @@ public class TestAclWithSnapshot {
     AclStatus s = hdfs.getAclStatus(filePath);
     AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission((short)0570, filePath);
+    assertPermission((short)010570, filePath);
     assertFilePermissionDenied(fsAsBruce, BRUCE, filePath);
     assertFilePermissionGranted(fsAsDiana, DIANA, filePath);
 
     s = hdfs.getAclStatus(subdirPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission((short)0570, subdirPath);
+    assertPermission((short)010570, subdirPath);
     assertDirPermissionDenied(fsAsBruce, BRUCE, subdirPath);
     assertDirPermissionGranted(fsAsDiana, DIANA, subdirPath);
 
@@ -268,14 +268,14 @@ public class TestAclWithSnapshot {
     s = hdfs.getAclStatus(fileSnapshotPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission((short)0550, fileSnapshotPath);
+    assertPermission((short)010550, fileSnapshotPath);
     assertFilePermissionGranted(fsAsBruce, BRUCE, fileSnapshotPath);
     assertFilePermissionDenied(fsAsDiana, DIANA, fileSnapshotPath);
 
     s = hdfs.getAclStatus(subdirSnapshotPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission((short)0550, subdirSnapshotPath);
+    assertPermission((short)010550, subdirSnapshotPath);
     assertDirPermissionGranted(fsAsBruce, BRUCE, subdirSnapshotPath);
     assertDirPermissionDenied(fsAsDiana, DIANA, subdirSnapshotPath);
   }
@@ -302,14 +302,14 @@ public class TestAclWithSnapshot {
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "bruce", READ_EXECUTE),
       aclEntry(ACCESS, GROUP, NONE) }, returned);
-    assertPermission((short)0750, path);
+    assertPermission((short)010750, path);
 
     s = hdfs.getAclStatus(snapshotPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "bruce", READ_EXECUTE),
       aclEntry(ACCESS, GROUP, NONE) }, returned);
-    assertPermission((short)0750, snapshotPath);
+    assertPermission((short)010750, snapshotPath);
 
     assertDirPermissionGranted(fsAsBruce, BRUCE, snapshotPath);
     assertDirPermissionDenied(fsAsDiana, DIANA, snapshotPath);
@@ -336,7 +336,7 @@ public class TestAclWithSnapshot {
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "bruce", READ_EXECUTE),
       aclEntry(ACCESS, GROUP, NONE) }, returned);
-    assertPermission((short)0750, snapshotPath);
+    assertPermission((short)010750, snapshotPath);
 
     assertDirPermissionDenied(fsAsBruce, BRUCE, path);
     assertDirPermissionDenied(fsAsDiana, DIANA, path);
@@ -378,24 +378,24 @@ public class TestAclWithSnapshot {
     AclStatus s = hdfs.getAclStatus(filePath);
     AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission((short)0550, filePath);
+    assertPermission((short)010550, filePath);
 
     s = hdfs.getAclStatus(subdirPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission((short)0550, subdirPath);
+    assertPermission((short)010550, subdirPath);
 
     s = hdfs.getAclStatus(fileSnapshotPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission((short)0550, fileSnapshotPath);
+    assertPermission((short)010550, fileSnapshotPath);
     assertFilePermissionGranted(fsAsBruce, BRUCE, fileSnapshotPath);
     assertFilePermissionDenied(fsAsDiana, DIANA, fileSnapshotPath);
 
     s = hdfs.getAclStatus(subdirSnapshotPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission((short)0550, subdirSnapshotPath);
+    assertPermission((short)010550, subdirSnapshotPath);
     assertDirPermissionGranted(fsAsBruce, BRUCE, subdirSnapshotPath);
     assertDirPermissionDenied(fsAsDiana, DIANA, subdirSnapshotPath);
 
@@ -437,14 +437,14 @@ public class TestAclWithSnapshot {
     s = hdfs.getAclStatus(fileSnapshotPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission((short)0550, fileSnapshotPath);
+    assertPermission((short)010550, fileSnapshotPath);
     assertFilePermissionGranted(fsAsBruce, BRUCE, fileSnapshotPath);
     assertFilePermissionDenied(fsAsDiana, DIANA, fileSnapshotPath);
 
     s = hdfs.getAclStatus(subdirSnapshotPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission((short)0550, subdirSnapshotPath);
+    assertPermission((short)010550, subdirSnapshotPath);
     assertDirPermissionGranted(fsAsBruce, BRUCE, subdirSnapshotPath);
     assertDirPermissionDenied(fsAsDiana, DIANA, subdirSnapshotPath);
   }
@@ -470,7 +470,7 @@ public class TestAclWithSnapshot {
     AclStatus s = hdfs.getAclStatus(path);
     AclEntry[] returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission((short)0770, path);
+    assertPermission((short)010770, path);
     assertDirPermissionGranted(fsAsBruce, BRUCE, path);
     assertDirPermissionGranted(fsAsDiana, DIANA, path);
   }
@@ -514,7 +514,7 @@ public class TestAclWithSnapshot {
       aclEntry(DEFAULT, GROUP, NONE),
       aclEntry(DEFAULT, MASK, READ_EXECUTE),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission((short)0700, path);
+    assertPermission((short)010700, path);
 
     s = hdfs.getAclStatus(snapshotPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
@@ -524,7 +524,7 @@ public class TestAclWithSnapshot {
       aclEntry(DEFAULT, GROUP, NONE),
       aclEntry(DEFAULT, MASK, READ_EXECUTE),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission((short)0700, snapshotPath);
+    assertPermission((short)010700, snapshotPath);
 
     assertDirPermissionDenied(fsAsBruce, BRUCE, snapshotPath);
   }
@@ -596,14 +596,14 @@ public class TestAclWithSnapshot {
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "bruce", READ_WRITE),
       aclEntry(ACCESS, GROUP, NONE) }, returned);
-    assertPermission((short)0660, filePath);
+    assertPermission((short)010660, filePath);
 
     s = hdfs.getAclStatus(fileSnapshotPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "bruce", READ_WRITE),
       aclEntry(ACCESS, GROUP, NONE) }, returned);
-    assertPermission((short)0660, filePath);
+    assertPermission((short)010660, filePath);
 
     aclSpec = Lists.newArrayList(
       aclEntry(ACCESS, USER, "bruce", READ));
@@ -632,14 +632,14 @@ public class TestAclWithSnapshot {
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "bruce", READ_WRITE),
       aclEntry(ACCESS, GROUP, NONE) }, returned);
-    assertPermission((short)0660, filePath);
+    assertPermission((short)010660, filePath);
 
     s = hdfs.getAclStatus(fileSnapshotPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "bruce", READ_WRITE),
       aclEntry(ACCESS, GROUP, NONE) }, returned);
-    assertPermission((short)0660, filePath);
+    assertPermission((short)010660, filePath);
 
     aclSpec = Lists.newArrayList(
       aclEntry(ACCESS, USER, "bruce", READ));

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdmin.java

@@ -176,6 +176,7 @@ public class TestDFSHAAdmin {
   
   @Test
   public void testTransitionToActive() throws Exception {
+    Mockito.doReturn(STANDBY_READY_RESULT).when(mockProtocol).getServiceStatus();
     assertEquals(0, runTool("-transitionToActive", "nn1"));
     Mockito.verify(mockProtocol).transitionToActive(
         reqInfoCaptor.capture());

+ 65 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdminMiniCluster.java

@@ -31,6 +31,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ha.HAAdmin;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
@@ -204,6 +205,70 @@ public class TestDFSHAAdminMiniCluster {
     assertEquals(0, runTool("-checkHealth", "nn2"));
   }
   
+  /**
+   * Test case to check whether both the name node is active or not
+   * @throws Exception
+   */
+  @Test
+  public void testTransitionToActiveWhenOtherNamenodeisActive() 
+      throws Exception {
+    NameNode nn1 = cluster.getNameNode(0);
+    NameNode nn2 = cluster.getNameNode(1);
+    if(nn1.getState() != null && !nn1.getState().
+        equals(HAServiceState.STANDBY.name()) ) {
+      cluster.transitionToStandby(0);
+    }
+    if(nn2.getState() != null && !nn2.getState().
+        equals(HAServiceState.STANDBY.name()) ) {
+      cluster.transitionToStandby(1);
+    }
+    //Making sure both the namenode are in standby state
+    assertTrue(nn1.isStandbyState());
+    assertTrue(nn2.isStandbyState());
+    // Triggering the transition for both namenode to Active
+    runTool("-transitionToActive", "nn1");
+    runTool("-transitionToActive", "nn2");
+
+    assertFalse("Both namenodes cannot be active", nn1.isActiveState() 
+        && nn2.isActiveState());
+   
+    /*  This test case doesn't allow nn2 to transition to Active even with
+        forceActive switch since nn1 is already active  */
+    if(nn1.getState() != null && !nn1.getState().
+        equals(HAServiceState.STANDBY.name()) ) {
+      cluster.transitionToStandby(0);
+    }
+    if(nn2.getState() != null && !nn2.getState().
+        equals(HAServiceState.STANDBY.name()) ) {
+      cluster.transitionToStandby(1);
+    }
+    //Making sure both the namenode are in standby state
+    assertTrue(nn1.isStandbyState());
+    assertTrue(nn2.isStandbyState());
+    
+    runTool("-transitionToActive", "nn1");
+    runTool("-transitionToActive", "nn2","--forceactive");
+    
+    assertFalse("Both namenodes cannot be active even though with forceActive",
+        nn1.isActiveState() && nn2.isActiveState());
+
+    /*  In this test case, we have deliberately shut down nn1 and this will
+        cause HAAAdmin#isOtherTargetNodeActive to throw an Exception 
+        and transitionToActive for nn2 with  forceActive switch will succeed 
+        even with Exception  */
+    cluster.shutdownNameNode(0);
+    if(nn2.getState() != null && !nn2.getState().
+        equals(HAServiceState.STANDBY.name()) ) {
+      cluster.transitionToStandby(1);
+    }
+    //Making sure both the namenode (nn2) is in standby state
+    assertTrue(nn2.isStandbyState());
+    assertFalse(cluster.isNameNodeUp(0));
+    
+    runTool("-transitionToActive", "nn2", "--forceactive");
+    assertTrue("Namenode nn2 should be active", nn2.isActiveState());
+  }
+  
   private int runTool(String ... args) throws Exception {
     errOutBytes.reset();
     LOG.info("Running: DFSHAAdmin " + Joiner.on(" ").join(args));

+ 24 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testCacheAdminConf.xml

@@ -519,5 +519,29 @@
         </comparator>
       </comparators>
     </test>
+
+    <test> <!--Tested -->
+      <description>Testing listing a single cache directive</description>
+      <test-commands>
+        <cache-admin-command>-addPool pool1</cache-admin-command>
+        <cache-admin-command>-addDirective -path /foo -pool pool1 -ttl 2d</cache-admin-command>
+        <cache-admin-command>-addDirective -path /bar -pool pool1 -ttl 24h</cache-admin-command>
+        <cache-admin-command>-addDirective -path /baz -replication 2 -pool pool1 -ttl 60m</cache-admin-command>
+        <cache-admin-command>-listDirectives -stats -id 30</cache-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <cache-admin-command>-removePool pool1</cache-admin-command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Found 1 entry</expected-output>
+        </comparator>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>30 pool1      1</expected-output>
+        </comparator>
+      </comparators>
+    </test>
   </tests>
 </configuration>

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

@@ -194,6 +194,11 @@ Release 2.5.0 - UNRELEASED
     MAPREDUCE-5652. NM Recovery. ShuffleHandler should handle NM restarts.
     (Jason Lowe via kasha)
 
+    MAPREDUCE-5861. finishedSubMaps field in LocalContainerLauncher does not 
+    need to be volatile. (Tsuyoshi OZAWA via junping_du)
+
+    MAPREDUCE-5809. Enhance distcp to support preserving HDFS ACLs. (cnauroth)
+
   OPTIMIZATIONS
 
   BUG FIXES 
@@ -273,6 +278,9 @@ Release 2.4.1 - UNRELEASED
     MAPREDUCE-5835. Killing Task might cause the job to go to ERROR state
     (Ming Ma via jlowe)
 
+    MAPREDUCE-5821. Avoid unintentional reallocation of byte arrays in segments
+    during merge. (Todd Lipcon via cdouglas)
+
 Release 2.4.0 - 2014-04-07 
 
   INCOMPATIBLE CHANGES

+ 4 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/LocalContainerLauncher.java

@@ -177,8 +177,10 @@ public class LocalContainerLauncher extends AbstractService implements
    */
   private class EventHandler implements Runnable {
 
-    private volatile boolean doneWithMaps = false;
-    private volatile int finishedSubMaps = 0;
+    // doneWithMaps and finishedSubMaps are accessed from only
+    // one thread. Therefore, no need to make them volatile.
+    private boolean doneWithMaps = false;
+    private int finishedSubMaps = 0;
 
     private final Map<TaskAttemptId,Future<?>> futures =
         new ConcurrentHashMap<TaskAttemptId,Future<?>>();

+ 5 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Merger.java

@@ -537,6 +537,8 @@ public class Merger {
         }
       }
       minSegment = top();
+      long startPos = minSegment.getPosition();
+      key = minSegment.getKey();
       if (!minSegment.inMemory()) {
         //When we load the value from an inmemory segment, we reset
         //the "value" DIB in this class to the inmem segment's byte[].
@@ -547,11 +549,11 @@ public class Merger {
         //segment, we reset the "value" DIB to the byte[] in that (so 
         //we reuse the disk segment DIB whenever we consider
         //a disk segment).
+        minSegment.getValue(diskIFileValue);
         value.reset(diskIFileValue.getData(), diskIFileValue.getLength());
+      } else {
+        minSegment.getValue(value);
       }
-      long startPos = minSegment.getPosition();
-      key = minSegment.getKey();
-      minSegment.getValue(value);
       long endPos = minSegment.getPosition();
       totalBytesProcessed += endPos - startPos;
       mergeProgress.set(totalBytesProcessed * progPerByte);

+ 0 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/MapredAppMasterRest.apt.vm → hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/apt/MapredAppMasterRest.apt.vm

@@ -18,8 +18,6 @@
 
 MapReduce Application Master REST API's.
 
-  \[ {{{./index.html}Go Back}} \]
-
 %{toc|section=1|fromDepth=0|toDepth=2}
 
 * Overview

+ 4 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/HistoryServerRest.apt.vm → hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/site/apt/HistoryServerRest.apt.vm

@@ -11,20 +11,18 @@
 ~~ limitations under the License. See accompanying LICENSE file.
 
   ---
-  History Server REST API's.
+  MapReduce History Server REST API's.
   ---
   ---
   ${maven.build.timestamp}
 
-History Server REST API's.
-
-  \[ {{{./index.html}Go Back}} \]
+MapReduce History Server REST API's.
 
 %{toc|section=1|fromDepth=0|toDepth=3}
 
 * Overview
 
-  The history server REST API's allow the user to get status on finished applications. Currently it only supports MapReduce and provides information on finished jobs.
+  The history server REST API's allow the user to get status on finished applications.
 
 * History Server Information API
 
@@ -2671,4 +2669,4 @@ History Server REST API's.
     </counter>
   </taskAttemptCounterGroup>
 </jobTaskAttemptCounters>
-+---+
++---+

+ 5 - 2
hadoop-project/src/site/site.xml

@@ -99,6 +99,11 @@
       <item name="DistCp" href="hadoop-mapreduce-client/hadoop-mapreduce-client-core/DistCp.html"/>
     </menu>
 
+    <menu name="MapReduce REST APIs" inherit="top">
+      <item name="MR Application Master" href="hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapredAppMasterRest.html"/>
+      <item name="MR History Server" href="hadoop-mapreduce-client/hadoop-mapreduce-client-hs/HistoryServerRest.html"/>
+    </menu>
+
     <menu name="YARN" inherit="top">
       <item name="YARN Architecture" href="hadoop-yarn/hadoop-yarn-site/YARN.html"/>
       <item name="Capacity Scheduler" href="hadoop-yarn/hadoop-yarn-site/CapacityScheduler.html"/>
@@ -116,8 +121,6 @@
       <item name="Introduction" href="hadoop-yarn/hadoop-yarn-site/WebServicesIntro.html"/>
       <item name="Resource Manager" href="hadoop-yarn/hadoop-yarn-site/ResourceManagerRest.html"/>
       <item name="Node Manager" href="hadoop-yarn/hadoop-yarn-site/NodeManagerRest.html"/>
-      <item name="MR Application Master" href="hadoop-yarn/hadoop-yarn-site/MapredAppMasterRest.html"/>
-      <item name="History Server" href="hadoop-yarn/hadoop-yarn-site/HistoryServerRest.html"/>
     </menu>
     
     <menu name="Auth" inherit="top">

+ 27 - 7
hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/CopyListing.java

@@ -22,7 +22,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.IOUtils;
@@ -31,11 +30,15 @@ import org.apache.hadoop.security.Credentials;
 
 import java.io.IOException;
 import java.lang.reflect.Constructor;
+import java.net.URI;
+import java.util.Set;
+
+import com.google.common.collect.Sets;
 
 /**
  * The CopyListing abstraction is responsible for how the list of
  * sources and targets is constructed, for DistCp's copy function.
- * The copy-listing should be a SequenceFile<Text, FileStatus>,
+ * The copy-listing should be a SequenceFile<Text, CopyListingFileStatus>,
  * located at the path specified to buildListing(),
  * each entry being a pair of (Source relative path, source file status),
  * all the paths being fully qualified.
@@ -85,7 +88,7 @@ public abstract class CopyListing extends Configured {
     config.setLong(DistCpConstants.CONF_LABEL_TOTAL_BYTES_TO_BE_COPIED, getBytesToCopy());
     config.setLong(DistCpConstants.CONF_LABEL_TOTAL_NUMBER_OF_RECORDS, getNumberOfPaths());
 
-    checkForDuplicates(pathToListFile);
+    validateFinalListing(pathToListFile, options);
   }
 
   /**
@@ -124,13 +127,15 @@ public abstract class CopyListing extends Configured {
   protected abstract long getNumberOfPaths();
 
   /**
-   * Validate the final resulting path listing to see if there are any duplicate entries
+   * Validate the final resulting path listing.  Checks if there are duplicate
+   * entries.  If preserving ACLs, checks that file system can support ACLs.
    *
    * @param pathToListFile - path listing build by doBuildListing
+   * @param options - Input options to distcp
    * @throws IOException - Any issues while checking for duplicates and throws
    * @throws DuplicateFileException - if there are duplicates
    */
-  private void checkForDuplicates(Path pathToListFile)
+  private void validateFinalListing(Path pathToListFile, DistCpOptions options)
       throws DuplicateFileException, IOException {
 
     Configuration config = getConf();
@@ -142,17 +147,26 @@ public abstract class CopyListing extends Configured {
                           config, SequenceFile.Reader.file(sortedList));
     try {
       Text lastKey = new Text("*"); //source relative path can never hold *
-      FileStatus lastFileStatus = new FileStatus();
+      CopyListingFileStatus lastFileStatus = new CopyListingFileStatus();
 
       Text currentKey = new Text();
+      Set<URI> aclSupportCheckFsSet = Sets.newHashSet();
       while (reader.next(currentKey)) {
         if (currentKey.equals(lastKey)) {
-          FileStatus currentFileStatus = new FileStatus();
+          CopyListingFileStatus currentFileStatus = new CopyListingFileStatus();
           reader.getCurrentValue(currentFileStatus);
           throw new DuplicateFileException("File " + lastFileStatus.getPath() + " and " +
               currentFileStatus.getPath() + " would cause duplicates. Aborting");
         }
         reader.getCurrentValue(lastFileStatus);
+        if (options.shouldPreserve(DistCpOptions.FileAttribute.ACL)) {
+          FileSystem lastFs = lastFileStatus.getPath().getFileSystem(config);
+          URI lastFsUri = lastFs.getUri();
+          if (!aclSupportCheckFsSet.contains(lastFsUri)) {
+            DistCpUtils.checkFileSystemAclSupport(lastFs);
+            aclSupportCheckFsSet.add(lastFsUri);
+          }
+        }
         lastKey.set(currentKey);
       }
     } finally {
@@ -236,4 +250,10 @@ public abstract class CopyListing extends Configured {
       super(message);
     }
   }
+
+  public static class AclsNotSupportedException extends RuntimeException {
+    public AclsNotSupportedException(String message) {
+      super(message);
+    }
+  }
 }

+ 153 - 0
hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/CopyListingFileStatus.java

@@ -0,0 +1,153 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.tools;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.AclEntryType;
+import org.apache.hadoop.fs.permission.AclEntryScope;
+import org.apache.hadoop.fs.permission.AclUtil;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.io.WritableUtils;
+
+import com.google.common.base.Objects;
+import com.google.common.collect.Lists;
+
+/**
+ * CopyListingFileStatus is a specialized subclass of {@link FileStatus} for
+ * attaching additional data members useful to distcp.  This class does not
+ * override {@link FileStatus#compareTo}, because the additional data members
+ * are not relevant to sort order.
+ */
+@InterfaceAudience.Private
+public final class CopyListingFileStatus extends FileStatus {
+
+  private static final byte NO_ACL_ENTRIES = -1;
+
+  // Retain static arrays of enum values to prevent repeated allocation of new
+  // arrays during deserialization.
+  private static final AclEntryType[] ACL_ENTRY_TYPES = AclEntryType.values();
+  private static final AclEntryScope[] ACL_ENTRY_SCOPES = AclEntryScope.values();
+  private static final FsAction[] FS_ACTIONS = FsAction.values();
+
+  private List<AclEntry> aclEntries;
+
+  /**
+   * Default constructor.
+   */
+  public CopyListingFileStatus() {
+  }
+
+  /**
+   * Creates a new CopyListingFileStatus by copying the members of the given
+   * FileStatus.
+   *
+   * @param fileStatus FileStatus to copy
+   */
+  public CopyListingFileStatus(FileStatus fileStatus) throws IOException {
+    super(fileStatus);
+  }
+
+  /**
+   * Returns the full logical ACL.
+   *
+   * @return List<AclEntry> containing full logical ACL
+   */
+  public List<AclEntry> getAclEntries() {
+    return AclUtil.getAclFromPermAndEntries(getPermission(),
+      aclEntries != null ? aclEntries : Collections.<AclEntry>emptyList());
+  }
+
+  /**
+   * Sets optional ACL entries.
+   *
+   * @param aclEntries List<AclEntry> containing all ACL entries
+   */
+  public void setAclEntries(List<AclEntry> aclEntries) {
+    this.aclEntries = aclEntries;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    super.write(out);
+    if (aclEntries != null) {
+      // byte is sufficient, because 32 ACL entries is the max enforced by HDFS.
+      out.writeByte(aclEntries.size());
+      for (AclEntry entry: aclEntries) {
+        out.writeByte(entry.getScope().ordinal());
+        out.writeByte(entry.getType().ordinal());
+        WritableUtils.writeString(out, entry.getName());
+        out.writeByte(entry.getPermission().ordinal());
+      }
+    } else {
+      out.writeByte(NO_ACL_ENTRIES);
+    }
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    super.readFields(in);
+    byte aclEntriesSize = in.readByte();
+    if (aclEntriesSize != NO_ACL_ENTRIES) {
+      aclEntries = Lists.newArrayListWithCapacity(aclEntriesSize);
+      for (int i = 0; i < aclEntriesSize; ++i) {
+        aclEntries.add(new AclEntry.Builder()
+          .setScope(ACL_ENTRY_SCOPES[in.readByte()])
+          .setType(ACL_ENTRY_TYPES[in.readByte()])
+          .setName(WritableUtils.readString(in))
+          .setPermission(FS_ACTIONS[in.readByte()])
+          .build());
+      }
+    } else {
+      aclEntries = null;
+    }
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (!super.equals(o)) {
+      return false;
+    }
+    if (getClass() != o.getClass()) {
+      return false;
+    }
+    CopyListingFileStatus other = (CopyListingFileStatus)o;
+    return Objects.equal(aclEntries, other.aclEntries);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(super.hashCode(), aclEntries);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder(super.toString());
+    sb.append('{');
+    sb.append("aclEntries = " + aclEntries);
+    sb.append('}');
+    return sb.toString();
+  }
+}

+ 6 - 1
hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCp.java

@@ -125,6 +125,9 @@ public class DistCp extends Configured implements Tool {
     } catch (DuplicateFileException e) {
       LOG.error("Duplicate files in input path: ", e);
       return DistCpConstants.DUPLICATE_INPUT;
+    } catch (AclsNotSupportedException e) {
+      LOG.error("ACLs not supported on at least one file system: ", e);
+      return DistCpConstants.ACLS_NOT_SUPPORTED;
     } catch (Exception e) {
       LOG.error("Exception encountered ", e);
       return DistCpConstants.UNKNOWN_ERROR;
@@ -298,7 +301,9 @@ public class DistCp extends Configured implements Tool {
     FileSystem targetFS = targetPath.getFileSystem(configuration);
     targetPath = targetPath.makeQualified(targetFS.getUri(),
                                           targetFS.getWorkingDirectory());
-
+    if (inputOptions.shouldPreserve(DistCpOptions.FileAttribute.ACL)) {
+      DistCpUtils.checkFileSystemAclSupport(targetFS);
+    }
     if (inputOptions.shouldAtomicCommit()) {
       Path workDir = inputOptions.getAtomicWorkPath();
       if (workDir == null) {

+ 1 - 0
hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java

@@ -115,6 +115,7 @@ public class DistCpConstants {
   public static final int SUCCESS = 0;
   public static final int INVALID_ARGUMENT = -1;
   public static final int DUPLICATE_INPUT = -2;
+  public static final int ACLS_NOT_SUPPORTED = -3;
   public static final int UNKNOWN_ERROR = -999;
   
   /**

+ 4 - 2
hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptionSwitch.java

@@ -45,8 +45,10 @@ public enum DistCpOptionSwitch {
    *
    */
   PRESERVE_STATUS(DistCpConstants.CONF_LABEL_PRESERVE_STATUS,
-      new Option("p", true, "preserve status (rbugpc)" +
-          "(replication, block-size, user, group, permission, checksum-type)")),
+      new Option("p", true, "preserve status (rbugpca)(replication, " +
+          "block-size, user, group, permission, checksum-type, ACL).  If " +
+          "-p is specified with no <arg>, then preserves replication, block " +
+          "size, user, group, permission and checksum type.")),
 
   /**
    * Update target location by copying only files that are missing

+ 1 - 1
hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java

@@ -65,7 +65,7 @@ public class DistCpOptions {
   private boolean targetPathExists = true;
   
   public static enum FileAttribute{
-    REPLICATION, BLOCKSIZE, USER, GROUP, PERMISSION, CHECKSUMTYPE;
+    REPLICATION, BLOCKSIZE, USER, GROUP, PERMISSION, CHECKSUMTYPE, ACL;
 
     public static FileAttribute getAttribute(char symbol) {
       for (FileAttribute attribute : values()) {

+ 24 - 35
hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/SimpleCopyListing.java

@@ -23,11 +23,12 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.tools.DistCpOptions.FileAttribute;
 import org.apache.hadoop.tools.util.DistCpUtils;
 import org.apache.hadoop.mapreduce.security.TokenCache;
 import org.apache.hadoop.security.Credentials;
@@ -35,6 +36,7 @@ import org.apache.hadoop.security.Credentials;
 import com.google.common.annotations.VisibleForTesting;
 
 import java.io.*;
+import java.util.List;
 import java.util.Stack;
 
 /**
@@ -139,28 +141,34 @@ public class SimpleCopyListing extends CopyListing {
 
         FileStatus rootStatus = sourceFS.getFileStatus(path);
         Path sourcePathRoot = computeSourceRootPath(rootStatus, options);
-        boolean localFile = (rootStatus.getClass() != FileStatus.class);
 
         FileStatus[] sourceFiles = sourceFS.listStatus(path);
         boolean explore = (sourceFiles != null && sourceFiles.length > 0);
         if (!explore || rootStatus.isDirectory()) {
-          writeToFileListingRoot(fileListWriter, rootStatus, sourcePathRoot,
-              localFile, options);
+          CopyListingFileStatus rootCopyListingStatus =
+            DistCpUtils.toCopyListingFileStatus(sourceFS, rootStatus,
+              options.shouldPreserve(FileAttribute.ACL));
+          writeToFileListingRoot(fileListWriter, rootCopyListingStatus,
+              sourcePathRoot, options);
         }
         if (explore) {
           for (FileStatus sourceStatus: sourceFiles) {
             if (LOG.isDebugEnabled()) {
               LOG.debug("Recording source-path: " + sourceStatus.getPath() + " for copy.");
             }
-            writeToFileListing(fileListWriter, sourceStatus, sourcePathRoot,
-                localFile, options);
+            CopyListingFileStatus sourceCopyListingStatus =
+              DistCpUtils.toCopyListingFileStatus(sourceFS, sourceStatus,
+                options.shouldPreserve(FileAttribute.ACL) &&
+                sourceStatus.isDirectory());
+            writeToFileListing(fileListWriter, sourceCopyListingStatus,
+                sourcePathRoot, options);
 
             if (isDirectoryAndNotEmpty(sourceFS, sourceStatus)) {
               if (LOG.isDebugEnabled()) {
                 LOG.debug("Traversing non-empty source dir: " + sourceStatus.getPath());
               }
               traverseNonEmptyDirectory(fileListWriter, sourceStatus, sourcePathRoot,
-                  localFile, options);
+                  options);
             }
           }
         }
@@ -233,7 +241,7 @@ public class SimpleCopyListing extends CopyListing {
     return SequenceFile.createWriter(getConf(),
             SequenceFile.Writer.file(pathToListFile),
             SequenceFile.Writer.keyClass(Text.class),
-            SequenceFile.Writer.valueClass(FileStatus.class),
+            SequenceFile.Writer.valueClass(CopyListingFileStatus.class),
             SequenceFile.Writer.compression(SequenceFile.CompressionType.NONE));
   }
 
@@ -250,7 +258,6 @@ public class SimpleCopyListing extends CopyListing {
   private void traverseNonEmptyDirectory(SequenceFile.Writer fileListWriter,
                                          FileStatus sourceStatus,
                                          Path sourcePathRoot,
-                                         boolean localFile,
                                          DistCpOptions options)
                                          throws IOException {
     FileSystem sourceFS = sourcePathRoot.getFileSystem(getConf());
@@ -262,8 +269,11 @@ public class SimpleCopyListing extends CopyListing {
         if (LOG.isDebugEnabled())
           LOG.debug("Recording source-path: "
                     + sourceStatus.getPath() + " for copy.");
-        writeToFileListing(fileListWriter, child, sourcePathRoot,
-             localFile, options);
+        CopyListingFileStatus childCopyListingStatus =
+          DistCpUtils.toCopyListingFileStatus(sourceFS, child,
+            options.shouldPreserve(FileAttribute.ACL) && child.isDirectory());
+        writeToFileListing(fileListWriter, childCopyListingStatus,
+             sourcePathRoot, options);
         if (isDirectoryAndNotEmpty(sourceFS, child)) {
           if (LOG.isDebugEnabled())
             LOG.debug("Traversing non-empty source dir: "
@@ -275,8 +285,7 @@ public class SimpleCopyListing extends CopyListing {
   }
   
   private void writeToFileListingRoot(SequenceFile.Writer fileListWriter,
-      FileStatus fileStatus, Path sourcePathRoot,
-      boolean localFile,
+      CopyListingFileStatus fileStatus, Path sourcePathRoot,
       DistCpOptions options) throws IOException {
     boolean syncOrOverwrite = options.shouldSyncFolder() ||
         options.shouldOverwrite();
@@ -288,14 +297,12 @@ public class SimpleCopyListing extends CopyListing {
       }      
       return;
     }
-    writeToFileListing(fileListWriter, fileStatus, sourcePathRoot, localFile,
-        options);
+    writeToFileListing(fileListWriter, fileStatus, sourcePathRoot, options);
   }
 
   private void writeToFileListing(SequenceFile.Writer fileListWriter,
-                                  FileStatus fileStatus,
+                                  CopyListingFileStatus fileStatus,
                                   Path sourcePathRoot,
-                                  boolean localFile,
                                   DistCpOptions options) throws IOException {
     if (LOG.isDebugEnabled()) {
       LOG.debug("REL PATH: " + DistCpUtils.getRelativePath(sourcePathRoot,
@@ -303,9 +310,6 @@ public class SimpleCopyListing extends CopyListing {
     }
 
     FileStatus status = fileStatus;
-    if (localFile) {
-      status = getFileStatus(fileStatus);
-    }
 
     if (!shouldCopy(fileStatus.getPath(), options)) {
       return;
@@ -320,19 +324,4 @@ public class SimpleCopyListing extends CopyListing {
     }
     totalPaths++;
   }
-
-  private static final ByteArrayOutputStream buffer = new ByteArrayOutputStream(64);
-  private DataInputBuffer in = new DataInputBuffer();
-  
-  private FileStatus getFileStatus(FileStatus fileStatus) throws IOException {
-    FileStatus status = new FileStatus();
-
-    buffer.reset();
-    DataOutputStream out = new DataOutputStream(buffer);
-    fileStatus.write(out);
-
-    in.reset(buffer.toByteArray(), 0, buffer.size());
-    status.readFields(in);
-    return status;
-  }
 }

+ 3 - 3
hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyCommitter.java

@@ -178,7 +178,7 @@ public class CopyCommitter extends FileOutputCommitter {
 
     long preservedEntries = 0;
     try {
-      FileStatus srcFileStatus = new FileStatus();
+      CopyListingFileStatus srcFileStatus = new CopyListingFileStatus();
       Text srcRelPath = new Text();
 
       // Iterate over every source path that was copied.
@@ -246,9 +246,9 @@ public class CopyCommitter extends FileOutputCommitter {
     // Delete all from target that doesn't also exist on source.
     long deletedEntries = 0;
     try {
-      FileStatus srcFileStatus = new FileStatus();
+      CopyListingFileStatus srcFileStatus = new CopyListingFileStatus();
       Text srcRelPath = new Text();
-      FileStatus trgtFileStatus = new FileStatus();
+      CopyListingFileStatus trgtFileStatus = new CopyListingFileStatus();
       Text trgtRelPath = new Text();
 
       FileSystem targetFS = targetFinalPath.getFileSystem(conf);

+ 10 - 5
hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java

@@ -24,9 +24,11 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.tools.CopyListingFileStatus;
 import org.apache.hadoop.tools.DistCpConstants;
 import org.apache.hadoop.tools.DistCpOptionSwitch;
 import org.apache.hadoop.tools.DistCpOptions;
@@ -37,12 +39,13 @@ import org.apache.hadoop.util.StringUtils;
 import java.io.*;
 import java.util.EnumSet;
 import java.util.Arrays;
+import java.util.List;
 
 /**
  * Mapper class that executes the DistCp copy operation.
  * Implements the o.a.h.mapreduce.Mapper<> interface.
  */
-public class CopyMapper extends Mapper<Text, FileStatus, Text, Text> {
+public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text> {
 
   /**
    * Hadoop counters for the DistCp CopyMapper.
@@ -172,8 +175,8 @@ public class CopyMapper extends Mapper<Text, FileStatus, Text, Text> {
    * @throws IOException
    */
   @Override
-  public void map(Text relPath, FileStatus sourceFileStatus, Context context)
-          throws IOException, InterruptedException {
+  public void map(Text relPath, CopyListingFileStatus sourceFileStatus,
+          Context context) throws IOException, InterruptedException {
     Path sourcePath = sourceFileStatus.getPath();
 
     if (LOG.isDebugEnabled())
@@ -191,11 +194,13 @@ public class CopyMapper extends Mapper<Text, FileStatus, Text, Text> {
     LOG.info(description);
 
     try {
-      FileStatus sourceCurrStatus;
+      CopyListingFileStatus sourceCurrStatus;
       FileSystem sourceFS;
       try {
         sourceFS = sourcePath.getFileSystem(conf);
-        sourceCurrStatus = sourceFS.getFileStatus(sourcePath);
+        sourceCurrStatus = DistCpUtils.toCopyListingFileStatus(sourceFS,
+          sourceFS.getFileStatus(sourcePath),
+          fileAttributes.contains(FileAttribute.ACL));
       } catch (FileNotFoundException e) {
         throw new IOException(new RetriableFileCopyCommand.CopyReadException(e));
       }

+ 8 - 7
hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/UniformSizeInputFormat.java

@@ -23,11 +23,11 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.tools.CopyListingFileStatus;
 import org.apache.hadoop.tools.DistCpConstants;
 import org.apache.hadoop.tools.util.DistCpUtils;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.mapreduce.*;
 import org.apache.hadoop.mapreduce.lib.input.SequenceFileRecordReader;
 import org.apache.hadoop.mapreduce.lib.input.FileSplit;
@@ -44,7 +44,8 @@ import java.util.ArrayList;
  * that the total-number of bytes to be copied for each input split is
  * uniform.
  */
-public class UniformSizeInputFormat extends InputFormat<Text, FileStatus> {
+public class UniformSizeInputFormat
+    extends InputFormat<Text, CopyListingFileStatus> {
   private static final Log LOG
                 = LogFactory.getLog(UniformSizeInputFormat.class);
 
@@ -76,7 +77,7 @@ public class UniformSizeInputFormat extends InputFormat<Text, FileStatus> {
     List<InputSplit> splits = new ArrayList<InputSplit>(numSplits);
     long nBytesPerSplit = (long) Math.ceil(totalSizeBytes * 1.0 / numSplits);
 
-    FileStatus srcFileStatus = new FileStatus();
+    CopyListingFileStatus srcFileStatus = new CopyListingFileStatus();
     Text srcRelPath = new Text();
     long currentSplitSize = 0;
     long lastSplitStart = 0;
@@ -161,9 +162,9 @@ public class UniformSizeInputFormat extends InputFormat<Text, FileStatus> {
    * @throws InterruptedException
    */
   @Override
-  public RecordReader<Text, FileStatus> createRecordReader(InputSplit split,
-                                                     TaskAttemptContext context)
-                                      throws IOException, InterruptedException {
-    return new SequenceFileRecordReader<Text, FileStatus>();
+  public RecordReader<Text, CopyListingFileStatus> createRecordReader(
+      InputSplit split, TaskAttemptContext context)
+      throws IOException, InterruptedException {
+    return new SequenceFileRecordReader<Text, CopyListingFileStatus>();
   }
 }

+ 3 - 2
hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/lib/DynamicInputChunk.java

@@ -26,6 +26,7 @@ import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.tools.CopyListingFileStatus;
 import org.apache.hadoop.tools.DistCpConstants;
 import org.apache.hadoop.tools.util.DistCpUtils;
 import org.apache.hadoop.mapreduce.lib.input.SequenceFileRecordReader;
@@ -90,7 +91,7 @@ class DynamicInputChunk<K, V> {
   private void openForWrite() throws IOException {
     writer = SequenceFile.createWriter(
             chunkFilePath.getFileSystem(configuration), configuration,
-            chunkFilePath, Text.class, FileStatus.class,
+            chunkFilePath, Text.class, CopyListingFileStatus.class,
             SequenceFile.CompressionType.NONE);
 
   }
@@ -117,7 +118,7 @@ class DynamicInputChunk<K, V> {
    * @param value Corresponding value from the listing file.
    * @throws IOException Exception onf failure to write to the file.
    */
-  public void write(Text key, FileStatus value) throws IOException {
+  public void write(Text key, CopyListingFileStatus value) throws IOException {
     writer.append(key, value);
   }
 

+ 2 - 2
hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/lib/DynamicInputFormat.java

@@ -29,7 +29,7 @@ import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.tools.CopyListingFileStatus;
 
 import java.util.List;
 import java.util.ArrayList;
@@ -133,7 +133,7 @@ public class DynamicInputFormat<K, V> extends InputFormat<K, V> {
     
     List<DynamicInputChunk> chunksFinal = new ArrayList<DynamicInputChunk>();
 
-    FileStatus fileStatus = new FileStatus();
+    CopyListingFileStatus fileStatus = new CopyListingFileStatus();
     Text relPath = new Text();
     int recordCounter = 0;
     int chunkCount = 0;

+ 80 - 3
hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/DistCpUtils.java

@@ -25,15 +25,21 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileChecksum;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.AclUtil;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.tools.CopyListingFileStatus;
 import org.apache.hadoop.tools.DistCpOptions.FileAttribute;
 import org.apache.hadoop.tools.mapred.UniformSizeInputFormat;
+import org.apache.hadoop.tools.CopyListing.AclsNotSupportedException;
 import org.apache.hadoop.tools.DistCpOptions;
 import org.apache.hadoop.mapreduce.InputFormat;
 
 import java.io.IOException;
 import java.util.EnumSet;
+import java.util.List;
 import java.util.Locale;
 import java.text.DecimalFormat;
 import java.net.URI;
@@ -181,7 +187,7 @@ public class DistCpUtils {
    *                       change or any transient error)
    */
   public static void preserve(FileSystem targetFS, Path path,
-                              FileStatus srcFileStatus,
+                              CopyListingFileStatus srcFileStatus,
                               EnumSet<FileAttribute> attributes) throws IOException {
 
     FileStatus targetFileStatus = targetFS.getFileStatus(path);
@@ -189,7 +195,18 @@ public class DistCpUtils {
     String user = targetFileStatus.getOwner();
     boolean chown = false;
 
-    if (attributes.contains(FileAttribute.PERMISSION) &&
+    if (attributes.contains(FileAttribute.ACL)) {
+      List<AclEntry> srcAcl = srcFileStatus.getAclEntries();
+      List<AclEntry> targetAcl = getAcl(targetFS, targetFileStatus);
+      if (!srcAcl.equals(targetAcl)) {
+        targetFS.setAcl(path, srcAcl);
+      }
+      // setAcl can't preserve sticky bit, so also call setPermission if needed.
+      if (srcFileStatus.getPermission().getStickyBit() !=
+          targetFileStatus.getPermission().getStickyBit()) {
+        targetFS.setPermission(path, srcFileStatus.getPermission());
+      }
+    } else if (attributes.contains(FileAttribute.PERMISSION) &&
       !srcFileStatus.getPermission().equals(targetFileStatus.getPermission())) {
       targetFS.setPermission(path, srcFileStatus.getPermission());
     }
@@ -216,6 +233,46 @@ public class DistCpUtils {
     }
   }
 
+  /**
+   * Returns a file's full logical ACL.
+   *
+   * @param fileSystem FileSystem containing the file
+   * @param fileStatus FileStatus of file
+   * @return List<AclEntry> containing full logical ACL
+   * @throws IOException if there is an I/O error
+   */
+  public static List<AclEntry> getAcl(FileSystem fileSystem,
+      FileStatus fileStatus) throws IOException {
+    List<AclEntry> entries = fileSystem.getAclStatus(fileStatus.getPath())
+      .getEntries();
+    return AclUtil.getAclFromPermAndEntries(fileStatus.getPermission(), entries);
+  }
+
+  /**
+   * Converts a FileStatus to a CopyListingFileStatus.  If preserving ACLs,
+   * populates the CopyListingFileStatus with the ACLs.
+   *
+   * @param fileSystem FileSystem containing the file
+   * @param fileStatus FileStatus of file
+   * @param preserveAcls boolean true if preserving ACLs
+   * @throws IOException if there is an I/O error
+   */
+  public static CopyListingFileStatus toCopyListingFileStatus(
+      FileSystem fileSystem, FileStatus fileStatus, boolean preserveAcls)
+      throws IOException {
+    CopyListingFileStatus copyListingFileStatus =
+      new CopyListingFileStatus(fileStatus);
+    if (preserveAcls) {
+      FsPermission perm = fileStatus.getPermission();
+      if (perm.getAclBit()) {
+        List<AclEntry> aclEntries = fileSystem.getAclStatus(
+          fileStatus.getPath()).getEntries();
+        copyListingFileStatus.setAclEntries(aclEntries);
+      }
+    }
+    return copyListingFileStatus;
+  }
+
   /**
    * Sort sequence file containing FileStatus and Text as key and value respecitvely
    *
@@ -227,7 +284,8 @@ public class DistCpUtils {
    */
   public static Path sortListing(FileSystem fs, Configuration conf, Path sourceListing)
       throws IOException {
-    SequenceFile.Sorter sorter = new SequenceFile.Sorter(fs, Text.class, FileStatus.class, conf);
+    SequenceFile.Sorter sorter = new SequenceFile.Sorter(fs, Text.class,
+      CopyListingFileStatus.class, conf);
     Path output = new Path(sourceListing.toString() +  "_sorted");
 
     if (fs.exists(output)) {
@@ -238,6 +296,25 @@ public class DistCpUtils {
     return output;
   }
 
+  /**
+   * Determines if a file system supports ACLs by running a canary getAclStatus
+   * request on the file system root.  This method is used before distcp job
+   * submission to fail fast if the user requested preserving ACLs, but the file
+   * system cannot support ACLs.
+   *
+   * @param fs FileSystem to check
+   * @throws AclsNotSupportedException if fs does not support ACLs
+   */
+  public static void checkFileSystemAclSupport(FileSystem fs)
+      throws AclsNotSupportedException {
+    try {
+      fs.getAclStatus(new Path(Path.SEPARATOR));
+    } catch (Exception e) {
+      throw new AclsNotSupportedException("ACLs not supported for file system: "
+        + fs.getUri());
+    }
+  }
+
   /**
    * String utility to convert a number-of-bytes to human readable format.
    */

+ 11 - 11
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/StubContext.java

@@ -23,7 +23,6 @@ import org.apache.hadoop.mapreduce.task.MapContextImpl;
 import org.apache.hadoop.mapreduce.lib.map.WrappedMapper;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.conf.Configuration;
 
 import java.util.List;
@@ -33,18 +32,19 @@ import java.io.IOException;
 public class StubContext {
 
   private StubStatusReporter reporter = new StubStatusReporter();
-  private RecordReader<Text, FileStatus> reader;
+  private RecordReader<Text, CopyListingFileStatus> reader;
   private StubInMemoryWriter writer = new StubInMemoryWriter();
-  private Mapper<Text, FileStatus, Text, Text>.Context mapperContext;
+  private Mapper<Text, CopyListingFileStatus, Text, Text>.Context mapperContext;
 
-  public StubContext(Configuration conf, RecordReader<Text, FileStatus> reader,
-                     int taskId) throws IOException, InterruptedException {
+  public StubContext(Configuration conf,
+      RecordReader<Text, CopyListingFileStatus> reader, int taskId)
+      throws IOException, InterruptedException {
 
-    WrappedMapper<Text, FileStatus, Text, Text> wrappedMapper
-            = new WrappedMapper<Text, FileStatus, Text, Text>();
+    WrappedMapper<Text, CopyListingFileStatus, Text, Text> wrappedMapper
+            = new WrappedMapper<Text, CopyListingFileStatus, Text, Text>();
 
-    MapContextImpl<Text, FileStatus, Text, Text> contextImpl
-            = new MapContextImpl<Text, FileStatus, Text, Text>(conf,
+    MapContextImpl<Text, CopyListingFileStatus, Text, Text> contextImpl
+            = new MapContextImpl<Text, CopyListingFileStatus, Text, Text>(conf,
             getTaskAttemptID(taskId), reader, writer,
             null, reporter, null);
 
@@ -52,7 +52,7 @@ public class StubContext {
     this.mapperContext = wrappedMapper.getMapContext(contextImpl);
   }
 
-  public Mapper<Text, FileStatus, Text, Text>.Context getContext() {
+  public Mapper<Text, CopyListingFileStatus, Text, Text>.Context getContext() {
     return mapperContext;
   }
 
@@ -60,7 +60,7 @@ public class StubContext {
     return reporter;
   }
 
-  public RecordReader<Text, FileStatus> getReader() {
+  public RecordReader<Text, CopyListingFileStatus> getReader() {
     return reader;
   }
 

+ 2 - 3
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestCopyListing.java

@@ -24,7 +24,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
 import org.apache.hadoop.tools.util.TestDistCpUtils;
@@ -106,7 +105,7 @@ public class TestCopyListing extends SimpleCopyListing {
     Assert.assertEquals(listing.getNumberOfPaths(), 3);
     SequenceFile.Reader reader = new SequenceFile.Reader(getConf(),
         SequenceFile.Reader.file(listingFile));
-    FileStatus fileStatus = new FileStatus();
+    CopyListingFileStatus fileStatus = new CopyListingFileStatus();
     Text relativePath = new Text();
     Assert.assertTrue(reader.next(relativePath, fileStatus));
     Assert.assertEquals(relativePath.toString(), "/1");
@@ -274,7 +273,7 @@ public class TestCopyListing extends SimpleCopyListing {
 
       reader = new SequenceFile.Reader(getConf(), SequenceFile.Reader.file(listFile));
 
-      FileStatus fileStatus = new FileStatus();
+      CopyListingFileStatus fileStatus = new CopyListingFileStatus();
       Text relativePath = new Text();
       Assert.assertTrue(reader.next(relativePath, fileStatus));
       Assert.assertTrue(relativePath.toString().equals(""));

+ 329 - 0
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpWithAcls.java

@@ -0,0 +1,329 @@
+/**
+ * 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.apache.hadoop.fs.permission.AclEntryScope.*;
+import static org.apache.hadoop.fs.permission.AclEntryType.*;
+import static org.apache.hadoop.fs.permission.FsAction.*;
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.Arrays;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.AclEntryScope;
+import org.apache.hadoop.fs.permission.AclEntryType;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.ToolRunner;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Tests distcp in combination with HDFS ACLs.
+ */
+public class TestDistCpWithAcls {
+
+  private static MiniDFSCluster cluster;
+  private static Configuration conf;
+  private static FileSystem fs;
+
+  @BeforeClass
+  public static void init() throws Exception {
+    initCluster(true, true);
+    // Create this directory structure:
+    // /src
+    //   /dir1
+    //     /subdir1
+    //   /dir2
+    //     /dir2/file2
+    //     /dir2/file3
+    //   /dir3sticky
+    //   /file1    
+    fs.mkdirs(new Path("/src/dir1/subdir1"));
+    fs.mkdirs(new Path("/src/dir2"));
+    fs.create(new Path("/src/dir2/file2")).close();
+    fs.create(new Path("/src/dir2/file3")).close();
+    fs.mkdirs(new Path("/src/dir3sticky"));
+    fs.create(new Path("/src/file1")).close();
+
+    // Set a mix of ACLs and plain permissions throughout the tree.
+    fs.modifyAclEntries(new Path("/src/dir1"), Arrays.asList(
+      aclEntry(DEFAULT, USER, "bruce", ALL)));
+
+    fs.modifyAclEntries(new Path("/src/dir2/file2"), Arrays.asList(
+      aclEntry(ACCESS, GROUP, "sales", NONE)));
+
+    fs.setPermission(new Path("/src/dir2/file3"),
+      new FsPermission((short)0660));
+
+    fs.modifyAclEntries(new Path("/src/file1"), Arrays.asList(
+      aclEntry(ACCESS, USER, "diana", READ)));
+
+    fs.setPermission(new Path("/src/dir3sticky"),
+      new FsPermission((short)01777));
+  }
+
+  @AfterClass
+  public static void shutdown() {
+    IOUtils.cleanup(null, fs);
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testPreserveAcls() throws Exception {
+    assertRunDistCp(DistCpConstants.SUCCESS, "/dstPreserveAcls");
+
+    assertAclEntries("/dstPreserveAcls/dir1", new AclEntry[] {
+      aclEntry(DEFAULT, USER, ALL),
+      aclEntry(DEFAULT, USER, "bruce", ALL),
+      aclEntry(DEFAULT, GROUP, READ_EXECUTE),
+      aclEntry(DEFAULT, MASK, ALL),
+      aclEntry(DEFAULT, OTHER, READ_EXECUTE) } );
+    assertPermission("/dstPreserveAcls/dir1", (short)0755);
+
+    assertAclEntries("/dstPreserveAcls/dir1/subdir1", new AclEntry[] { });
+    assertPermission("/dstPreserveAcls/dir1/subdir1", (short)0755);
+
+    assertAclEntries("/dstPreserveAcls/dir2", new AclEntry[] { });
+    assertPermission("/dstPreserveAcls/dir2", (short)0755);
+
+    assertAclEntries("/dstPreserveAcls/dir2/file2", new AclEntry[] {
+      aclEntry(ACCESS, GROUP, READ),
+      aclEntry(ACCESS, GROUP, "sales", NONE) } );
+    assertPermission("/dstPreserveAcls/dir2/file2", (short)0644);
+
+    assertAclEntries("/dstPreserveAcls/dir2/file3", new AclEntry[] { });
+    assertPermission("/dstPreserveAcls/dir2/file3", (short)0660);
+
+    assertAclEntries("/dstPreserveAcls/dir3sticky", new AclEntry[] { });
+    assertPermission("/dstPreserveAcls/dir3sticky", (short)01777);
+
+    assertAclEntries("/dstPreserveAcls/file1", new AclEntry[] {
+      aclEntry(ACCESS, USER, "diana", READ),
+      aclEntry(ACCESS, GROUP, READ) } );
+    assertPermission("/dstPreserveAcls/file1", (short)0644);
+  }
+
+  @Test
+  public void testAclsNotEnabled() throws Exception {
+    try {
+      restart(false);
+      assertRunDistCp(DistCpConstants.ACLS_NOT_SUPPORTED, "/dstAclsNotEnabled");
+    } finally {
+      restart(true);
+    }
+  }
+
+  @Test
+  public void testAclsNotImplemented() throws Exception {
+    assertRunDistCp(DistCpConstants.ACLS_NOT_SUPPORTED,
+      "stubfs://dstAclsNotImplemented");
+  }
+
+  /**
+   * Stub FileSystem implementation used for testing the case of attempting
+   * distcp with ACLs preserved on a file system that does not support ACLs.
+   * The base class implementation throws UnsupportedOperationException for the
+   * ACL methods, so we don't need to override them.
+   */
+  public static class StubFileSystem extends FileSystem {
+
+    @Override
+    public FSDataOutputStream append(Path f, int bufferSize,
+        Progressable progress) throws IOException {
+      return null;
+    }
+
+    @Override
+    public FSDataOutputStream create(Path f, FsPermission permission,
+        boolean overwrite, int bufferSize, short replication, long blockSize,
+        Progressable progress) throws IOException {
+      return null;
+    }
+
+    @Override
+    public boolean delete(Path f, boolean recursive) throws IOException {
+      return false;
+    }
+
+    @Override
+    public FileStatus getFileStatus(Path f) throws IOException {
+      return null;
+    }
+
+    @Override
+    public URI getUri() {
+      return URI.create("stubfs:///");
+    }
+
+    @Override
+    public Path getWorkingDirectory() {
+      return new Path(Path.SEPARATOR);
+    }
+
+    @Override
+    public FileStatus[] listStatus(Path f) throws IOException {
+      return null;
+    }
+
+    @Override
+    public boolean mkdirs(Path f, FsPermission permission)
+        throws IOException {
+      return false;
+    }
+
+    @Override
+    public FSDataInputStream open(Path f, int bufferSize) throws IOException {
+      return null;
+    }
+
+    @Override
+    public boolean rename(Path src, Path dst) throws IOException {
+      return false;
+    }
+
+    @Override
+    public void setWorkingDirectory(Path dir) {
+    }
+  }
+
+  /**
+   * Create a new AclEntry with scope, type and permission (no name).
+   *
+   * @param scope AclEntryScope scope of the ACL entry
+   * @param type AclEntryType ACL entry type
+   * @param permission FsAction set of permissions in the ACL entry
+   * @return AclEntry new AclEntry
+   */
+  private static AclEntry aclEntry(AclEntryScope scope, AclEntryType type,
+      FsAction permission) {
+    return new AclEntry.Builder()
+      .setScope(scope)
+      .setType(type)
+      .setPermission(permission)
+      .build();
+  }
+
+  /**
+   * Create a new AclEntry with scope, type, name and permission.
+   *
+   * @param scope AclEntryScope scope of the ACL entry
+   * @param type AclEntryType ACL entry type
+   * @param name String optional ACL entry name
+   * @param permission FsAction set of permissions in the ACL entry
+   * @return AclEntry new AclEntry
+   */
+  private static AclEntry aclEntry(AclEntryScope scope, AclEntryType type,
+      String name, FsAction permission) {
+    return new AclEntry.Builder()
+      .setScope(scope)
+      .setType(type)
+      .setName(name)
+      .setPermission(permission)
+      .build();
+  }
+
+  /**
+   * Asserts the ACL entries returned by getAclStatus for a specific path.
+   *
+   * @param path String path to check
+   * @param entries AclEntry[] expected ACL entries
+   * @throws Exception if there is any error
+   */
+  private static void assertAclEntries(String path, AclEntry[] entries)
+      throws Exception {
+    assertArrayEquals(entries, fs.getAclStatus(new Path(path)).getEntries()
+      .toArray(new AclEntry[0]));
+  }
+
+  /**
+   * Asserts the value of the FsPermission bits on the inode of a specific path.
+   *
+   * @param path String path to check
+   * @param perm short expected permission bits
+   * @throws Exception if there is any error
+   */
+  private static void assertPermission(String path, short perm)
+      throws Exception {
+    assertEquals(perm,
+      fs.getFileStatus(new Path(path)).getPermission().toShort());
+  }
+
+  /**
+   * Runs distcp from /src to specified destination, preserving ACLs.  Asserts
+   * expected exit code.
+   *
+   * @param int exitCode expected exit code
+   * @param dst String distcp destination
+   * @throws Exception if there is any error
+   */
+  private static void assertRunDistCp(int exitCode, String dst)
+      throws Exception {
+    DistCp distCp = new DistCp(conf, null);
+    assertEquals(exitCode, ToolRunner.run(
+      conf, distCp, new String[] { "-pa", "/src", dst }));
+  }
+
+  /**
+   * Initialize the cluster, wait for it to become active, and get FileSystem.
+   *
+   * @param format if true, format the NameNode and DataNodes before starting up
+   * @param aclsEnabled if true, ACL support is enabled
+   * @throws Exception if any step fails
+   */
+  private static void initCluster(boolean format, boolean aclsEnabled)
+      throws Exception {
+    conf = new Configuration();
+    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY, aclsEnabled);
+    conf.set(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY, "stubfs:///");
+    conf.setClass("fs.stubfs.impl", StubFileSystem.class, FileSystem.class);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).format(format)
+      .build();
+    cluster.waitActive();
+    fs = cluster.getFileSystem();
+  }
+
+  /**
+   * Restarts the cluster with ACLs enabled or disabled.
+   *
+   * @param aclsEnabled if true, ACL support is enabled
+   * @throws Exception if any step fails
+   */
+  private static void restart(boolean aclsEnabled) throws Exception {
+    shutdown();
+    initCluster(false, aclsEnabled);
+  }
+}

+ 1 - 2
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestFileBasedCopyListing.java

@@ -23,7 +23,6 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Text;
@@ -531,7 +530,7 @@ public class TestFileBasedCopyListing {
                                             SequenceFile.Reader.file(listFile));
     try {
       Text relPath = new Text();
-      FileStatus fileStatus = new FileStatus();
+      CopyListingFileStatus fileStatus = new CopyListingFileStatus();
       while (reader.next(relPath, fileStatus)) {
         if (fileStatus.isDirectory() && relPath.toString().equals("")) {
           // ignore root with empty relPath, which is an entry to be 

Certains fichiers n'ont pas été affichés car il y a eu trop de fichiers modifiés dans ce diff