فهرست منبع

Merge trunk r1603663 to branch.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/fs-encryption@1603664 13f79535-47bb-0310-9956-ffa450edef68
Andrew Wang 11 سال پیش
والد
کامیت
5e5e1a6202
100فایلهای تغییر یافته به همراه3398 افزوده شده و 421 حذف شده
  1. 18 1
      hadoop-common-project/hadoop-auth/pom.xml
  2. 1 1
      hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/KerberosAuthenticationHandler.java
  3. 27 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  4. 2 0
      hadoop-common-project/hadoop-common/src/main/bin/hadoop
  5. 0 2
      hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh
  6. 0 2
      hadoop-common-project/hadoop-common/src/main/conf/hadoop-policy.xml
  7. 0 2
      hadoop-common-project/hadoop-common/src/main/conf/log4j.properties
  8. 76 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
  9. 3 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/JavaKeyStoreProvider.java
  10. 0 27
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java
  11. 2 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
  12. 17 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandWithDestination.java
  13. 7 5
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java
  14. 1 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java
  15. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
  16. 31 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/GraphiteSink.java
  17. 52 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ProviderUtils.java
  18. 13 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
  19. 124 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/CredentialProvider.java
  20. 76 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/CredentialProviderFactory.java
  21. 423 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/CredentialShell.java
  22. 290 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/JavaKeyStoreProvider.java
  23. 123 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/UserProvider.java
  24. 3 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/ServiceAuthorizationManager.java
  25. 4 3
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/security/JniBasedUnixGroupsNetgroupMapping.c
  26. 17 0
      hadoop-common-project/hadoop-common/src/main/resources/META-INF/services/org.apache.hadoop.security.alias.CredentialProviderFactory
  27. 7 2
      hadoop-common-project/hadoop-common/src/site/apt/CommandsManual.apt.vm
  28. 5 3
      hadoop-common-project/hadoop-common/src/site/apt/FileSystemShell.apt.vm
  29. 5 4
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProvider.java
  30. 2 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderFactory.java
  31. 37 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/impl/TestGraphiteMetrics.java
  32. 14 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
  33. 173 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredShell.java
  34. 51 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredentialProvider.java
  35. 234 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredentialProviderFactory.java
  36. 8 4
      hadoop-common-project/hadoop-common/src/test/resources/testConf.xml
  37. 31 39
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Interface.java
  38. 25 27
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcProgram.java
  39. 26 0
      hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestFrameDecoder.java
  40. 4 3
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/conf/NfsConfigKeys.java
  41. 3 1
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/conf/NfsConfiguration.java
  42. 7 1
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/RpcProgramMountd.java
  43. 2 2
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3.java
  44. 145 89
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
  45. 16 8
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestReaddir.java
  46. 7 7
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestWrites.java
  47. 29 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  48. 1 1
      hadoop-hdfs-project/hadoop-hdfs/pom.xml
  49. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
  50. 11 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  51. 16 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  52. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/site/apt/Federation.apt.vm
  53. 16 0
      hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HdfsNfsGateway.apt.vm
  54. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HdfsUserGuide.apt.vm
  55. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/site/xdoc/HdfsRollingUpgrade.xml
  56. 162 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
  57. 30 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java
  58. 11 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java
  59. 149 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeleteRace.java
  60. 42 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
  61. 10 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java
  62. 3 0
      hadoop-mapreduce-project/CHANGES.txt
  63. 9 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
  64. 9 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRApp.java
  65. 0 2
      hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/protoc/ProtocMojo.java
  66. 0 2
      hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/util/Exec.java
  67. 0 2
      hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/util/FileSetUtils.java
  68. 0 2
      hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/versioninfo/VersionInfoMojo.java
  69. 6 0
      hadoop-project/pom.xml
  70. 0 2
      hadoop-tools/hadoop-distcp/src/main/resources/distcp-default.xml
  71. 0 2
      hadoop-tools/hadoop-distcp/src/test/resources/sslConfig.xml
  72. 14 0
      hadoop-yarn-project/CHANGES.txt
  73. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestResourceTrackerOnHA.java
  74. 25 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RegisterNodeManagerRequest.java
  75. 83 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RegisterNodeManagerRequestPBImpl.java
  76. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
  77. 4 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/api/protocolrecords/TestProtocolRecords.java
  78. 81 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/api/protocolrecords/TestRegisterNodeManagerRequest.java
  79. 171 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DeletionService.java
  80. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
  81. 7 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
  82. 60 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java
  83. 17 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.java
  84. 18 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMStateStoreService.java
  85. 9 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/proto/yarn_server_nodemanager_recovery.proto
  86. 55 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestDeletionService.java
  87. 27 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMMemoryStateStoreService.java
  88. 56 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java
  89. 14 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
  90. 10 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
  91. 3 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppEventType.java
  92. 49 12
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
  93. 11 16
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppRunningOnNodeEvent.java
  94. 0 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttempt.java
  95. 0 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptEventType.java
  96. 4 39
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
  97. 7 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
  98. 28 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
  99. 14 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeStartedEvent.java
  100. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java

+ 18 - 1
hadoop-common-project/hadoop-auth/pom.xml

@@ -100,8 +100,25 @@
     <dependency>
       <groupId>org.apache.directory.server</groupId>
       <artifactId>apacheds-kerberos-codec</artifactId>
-      <version>2.0.0-M15</version>
       <scope>compile</scope>
+        <exclusions>
+          <exclusion>
+            <groupId>org.apache.directory.api</groupId>
+            <artifactId>api-asn1-ber</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.directory.api</groupId>
+            <artifactId>api-i18n</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>org.apache.directory.api</groupId>
+            <artifactId>api-ldap-model</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>net.sf.ehcache</groupId>
+            <artifactId>ehcache-core</artifactId>
+          </exclusion>
+        </exclusions>
     </dependency>
   </dependencies>
 

+ 1 - 1
hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/KerberosAuthenticationHandler.java

@@ -193,7 +193,7 @@ public class KerberosAuthenticationHandler implements AuthenticationHandler {
       
       for (String spnegoPrincipal : spnegoPrincipals) {
         LOG.info("Login using keytab {}, for principal {}",
-            keytab, principal);
+            keytab, spnegoPrincipal);
         final KerberosConfiguration kerberosConfiguration =
             new KerberosConfiguration(keytab, spnegoPrincipal);
         final LoginContext loginContext =

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

@@ -149,6 +149,9 @@ Trunk (Unreleased)
 
     HADOOP-10485. Remove dead classes in hadoop-streaming. (wheat9)
 
+    HADOOP-10607. Create API to separate credential/password storage from
+    applications. (Larry McCay via omalley)
+
   BUG FIXES
 
     HADOOP-9451. Fault single-layer config if node group topology is enabled.
@@ -427,6 +430,15 @@ Release 2.5.0 - UNRELEASED
 
     HADOOP-10688. Expose thread-level FileSystem StatisticsData (Sandy Ryza)
 
+    HADOOP-10657. Have RetryInvocationHandler log failover attempt at INFO
+    level. (Ming Ma via jing9)
+
+    HADOOP-10666. Remove Copyright /d/d/d/d Apache Software Foundation from
+    the source files license header. (Henry Saputra via wang)
+
+    HADOOP-10557. FsShell -cp -pa option for preserving extended ACLs.
+    (Akira Ajisaka via cnauroth)
+
   OPTIMIZATIONS
 
   BUG FIXES 
@@ -550,6 +562,17 @@ Release 2.5.0 - UNRELEASED
     HADOOP-10678. SecurityUtil has unnecessary synchronization on collection
     used for only tests. (Benoy Antony via cnauroth)
 
+    HADOOP-10683. Users authenticated with KERBEROS are recorded as being
+    authenticated with SIMPLE. (Benoy Antony via cnauroth)
+
+    HADOOP-10702. KerberosAuthenticationHandler does not log the principal names
+    correctly. (Benoy Antony via cnauroth)
+
+    HADOOP-10699. Fix build native library on mac osx (Binglin Chang via
+    jlowe)
+
+    HADOOP-10660. GraphiteSink should implement Closeable (Chen He and Ted Yu via raviprak)
+
   BREAKDOWN OF HADOOP-10514 SUBTASKS AND RELATED JIRAS
 
     HADOOP-10520. Extended attributes definition and FileSystem APIs for
@@ -571,6 +594,10 @@ Release 2.5.0 - UNRELEASED
     HADOOP-10561. Copy command with preserve option should handle Xattrs.
     (Yi Liu via cnauroth)
 
+    HADOOP-10590. ServiceAuthorizationManager is not threadsafe. (Benoy Antony via vinayakumarb)
+
+    HADOOP-10711. Cleanup some extra dependencies from hadoop-auth. (rkanter via tucu)
+
 Release 2.4.1 - 2014-06-23 
 
   INCOMPATIBLE CHANGES

+ 2 - 0
hadoop-common-project/hadoop-common/src/main/bin/hadoop

@@ -116,6 +116,8 @@ case $COMMAND in
     elif [ "$COMMAND" = "archive" ] ; then
       CLASS=org.apache.hadoop.tools.HadoopArchives
       CLASSPATH=${CLASSPATH}:${TOOL_PATH}
+    elif [ "$COMMAND" = "credential" ] ; then
+      CLASS=org.apache.hadoop.security.alias.CredentialShell
     elif [[ "$COMMAND" = -*  ]] ; then
         # class and package names cannot begin with a -
         echo "Error: No command named \`$COMMAND' was found. Perhaps you meant \`hadoop ${COMMAND#-}'"

+ 0 - 2
hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh

@@ -1,5 +1,3 @@
-# Copyright 2011 The Apache Software Foundation
-# 
 # 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

+ 0 - 2
hadoop-common-project/hadoop-common/src/main/conf/hadoop-policy.xml

@@ -1,8 +1,6 @@
 <?xml version="1.0"?>
 <?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
 <!--
-
- Copyright 2011 The Apache Software Foundation
  
  Licensed to the Apache Software Foundation (ASF) under one
  or more contributor license agreements.  See the NOTICE file

+ 0 - 2
hadoop-common-project/hadoop-common/src/main/conf/log4j.properties

@@ -1,5 +1,3 @@
-# Copyright 2011 The Apache Software Foundation
-# 
 # 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

+ 76 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java

@@ -78,6 +78,9 @@ import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.alias.CredentialProvider;
+import org.apache.hadoop.security.alias.CredentialProvider.CredentialEntry;
+import org.apache.hadoop.security.alias.CredentialProviderFactory;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringInterner;
 import org.apache.hadoop.util.StringUtils;
@@ -1767,6 +1770,79 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     set(name, StringUtils.arrayToString(values));
   }
 
+  /**
+   * Get the value for a known password configuration element.
+   * In order to enable the elimination of clear text passwords in config,
+   * this method attempts to resolve the property name as an alias through
+   * the CredentialProvider API and conditionally fallsback to config.
+   * @param name property name
+   * @return password
+   */
+  public char[] getPassword(String name) throws IOException {
+    char[] pass = null;
+
+    pass = getPasswordFromCredenitalProviders(name);
+
+    if (pass == null) {
+      pass = getPasswordFromConfig(name);
+    }
+
+    return pass;
+  }
+
+  /**
+   * Try and resolve the provided element name as a credential provider
+   * alias.
+   * @param name alias of the provisioned credential
+   * @return password or null if not found
+   * @throws IOException
+   */
+  protected char[] getPasswordFromCredenitalProviders(String name)
+      throws IOException {
+    char[] pass = null;
+    try {
+      List<CredentialProvider> providers =
+          CredentialProviderFactory.getProviders(this);
+
+      if (providers != null) {
+        for (CredentialProvider provider : providers) {
+          try {
+            CredentialEntry entry = provider.getCredentialEntry(name);
+            if (entry != null) {
+              pass = entry.getCredential();
+              break;
+            }
+          }
+          catch (IOException ioe) {
+            throw new IOException("Can't get key " + name + " from key provider" +
+            		"of type: " + provider.getClass().getName() + ".", ioe);
+          }
+        }
+      }
+    }
+    catch (IOException ioe) {
+      throw new IOException("Configuration problem with provider path.", ioe);
+    }
+
+    return pass;
+  }
+
+  /**
+   * Fallback to clear text passwords in configuration.
+   * @param name
+   * @return clear text password or null
+   */
+  protected char[] getPasswordFromConfig(String name) {
+    char[] pass = null;
+    if (getBoolean(CredentialProvider.CLEAR_TEXT_FALLBACK, true)) {
+      String passStr = get(name);
+      if (passStr != null) {
+        pass = passStr.toCharArray();
+      }
+    }
+    return pass;
+  }
+
   /**
    * Get the socket address for <code>name</code> property as a
    * <code>InetSocketAddress</code>.

+ 3 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/JavaKeyStoreProvider.java

@@ -26,6 +26,8 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.ProviderUtils;
+
 import javax.crypto.spec.SecretKeySpec;
 import java.io.IOException;
 import java.io.InputStream;
@@ -101,7 +103,7 @@ public class JavaKeyStoreProvider extends KeyProvider {
 
   private JavaKeyStoreProvider(URI uri, Configuration conf) throws IOException {
     this.uri = uri;
-    path = unnestUri(uri);
+    path = ProviderUtils.unnestUri(uri);
     fs = path.getFileSystem(conf);
     // Get the password file from the conf, if not present from the user's
     // environment var

+ 0 - 27
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java

@@ -488,33 +488,6 @@ public abstract class KeyProvider {
     return name + "@" + version;
   }
 
-  /**
-   * Convert a nested URI to decode the underlying path. The translation takes
-   * the authority and parses it into the underlying scheme and authority.
-   * For example, "myscheme://hdfs@nn/my/path" is converted to
-   * "hdfs://nn/my/path".
-   * @param nestedUri the URI from the nested URI
-   * @return the unnested path
-   */
-  public static Path unnestUri(URI nestedUri) {
-    String[] parts = nestedUri.getAuthority().split("@", 2);
-    StringBuilder result = new StringBuilder(parts[0]);
-    result.append("://");
-    if (parts.length == 2) {
-      result.append(parts[1]);
-    }
-    result.append(nestedUri.getPath());
-    if (nestedUri.getQuery() != null) {
-      result.append("?");
-      result.append(nestedUri.getQuery());
-    }
-    if (nestedUri.getFragment() != null) {
-      result.append("#");
-      result.append(nestedUri.getFragment());
-    }
-    return new Path(result.toString());
-  }
-
   /**
    * Find the provider with the given key.
    * @param providerList the list of providers

+ 2 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java

@@ -24,6 +24,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProviderFactory;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.ProviderUtils;
 import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.authentication.client.PseudoAuthenticator;
@@ -147,7 +148,7 @@ public class KMSClientProvider extends KeyProvider {
   }
 
   public KMSClientProvider(URI uri, Configuration conf) throws IOException {
-    Path path = unnestUri(uri);
+    Path path = ProviderUtils.unnestUri(uri);
     URL url = path.toUri().toURL();
     kmsUrl = createServiceURL(url);
     if ("https".equalsIgnoreCase(url.getProtocol())) {

+ 17 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandWithDestination.java

@@ -25,6 +25,7 @@ import java.net.URISyntaxException;
 import java.util.EnumSet;
 import java.util.Iterator;
 import java.util.LinkedList;
+import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.NoSuchElementException;
@@ -39,6 +40,9 @@ import org.apache.hadoop.fs.PathIsDirectoryException;
 import org.apache.hadoop.fs.PathIsNotDirectoryException;
 import org.apache.hadoop.fs.PathNotFoundException;
 import org.apache.hadoop.fs.PathOperationException;
+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.IOUtils;
 
 /**
@@ -88,7 +92,7 @@ abstract class CommandWithDestination extends FsCommand {
   }
   
   protected static enum FileAttribute {
-    TIMESTAMPS, OWNERSHIP, PERMISSION, XATTR;
+    TIMESTAMPS, OWNERSHIP, PERMISSION, ACL, XATTR;
 
     public static FileAttribute getAttribute(char symbol) {
       for (FileAttribute attribute : values()) {
@@ -306,11 +310,22 @@ abstract class CommandWithDestination extends FsCommand {
           src.stat.getOwner(),
           src.stat.getGroup());
       }
-      if (shouldPreserve(FileAttribute.PERMISSION)) {
+      if (shouldPreserve(FileAttribute.PERMISSION) ||
+          shouldPreserve(FileAttribute.ACL)) {
         target.fs.setPermission(
           target.path,
           src.stat.getPermission());
       }
+      if (shouldPreserve(FileAttribute.ACL)) {
+        FsPermission perm = src.stat.getPermission();
+        if (perm.getAclBit()) {
+          List<AclEntry> srcEntries =
+              src.fs.getAclStatus(src.path).getEntries();
+          List<AclEntry> srcFullEntries =
+              AclUtil.getAclFromPermAndEntries(perm, srcEntries);
+          target.fs.setAcl(target.path, srcFullEntries);
+        }
+      }
       if (shouldPreserve(FileAttribute.XATTR)) {
         Map<String, byte[]> srcXAttrs = src.fs.getXAttrs(src.path);
         if (srcXAttrs != null) {

+ 7 - 5
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java

@@ -133,16 +133,18 @@ class CopyCommands {
 
   static class Cp extends CommandWithDestination {
     public static final String NAME = "cp";
-    public static final String USAGE = "[-f] [-p | -p[topx]] <src> ... <dst>";
+    public static final String USAGE = "[-f] [-p | -p[topax]] <src> ... <dst>";
     public static final String DESCRIPTION =
       "Copy files that match the file pattern <src> to a " +
       "destination.  When copying multiple files, the destination " +
       "must be a directory. Passing -p preserves status " +
-      "[topx] (timestamps, ownership, permission, XAttr). " +
+      "[topax] (timestamps, ownership, permission, ACLs, XAttr). " +
       "If -p is specified with no <arg>, then preserves " +
-      "timestamps, ownership, permission. Passing -f " +
-      "overwrites the destination if it already exists.\n";
-    
+      "timestamps, ownership, permission. If -pa is specified, " +
+      "then preserves permission also because ACL is a super-set of " +
+      "permission. Passing -f overwrites the destination if it " +
+      "already exists.\n";
+
     @Override
     protected void processOptions(LinkedList<String> args) throws IOException {
       popPreserveOption(args);

+ 1 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java

@@ -136,9 +136,7 @@ public class RetryInvocationHandler<T> implements RpcInvocationHandler {
               msg += " after " + invocationFailoverCount + " fail over attempts"; 
             }
             msg += ". Trying to fail over " + formatSleepMessage(action.delayMillis);
-            if (LOG.isDebugEnabled()) {
-              LOG.debug(msg, e);
-            }
+            LOG.info(msg, e);
           } else {
             if(LOG.isDebugEnabled()) {
               LOG.debug("Exception while invoking " + method.getName()

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java

@@ -1221,7 +1221,7 @@ public abstract class Server {
         ugi.addTokenIdentifier(tokenId);
         return ugi;
       } else {
-        return UserGroupInformation.createRemoteUser(authorizedId);
+        return UserGroupInformation.createRemoteUser(authorizedId, authMethod);
       }
     }
 

+ 31 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/GraphiteSink.java

@@ -18,13 +18,18 @@
 
 package org.apache.hadoop.metrics2.sink;
 
+import java.io.IOException;
 import java.io.OutputStreamWriter;
 import java.io.Writer;
+import java.io.Closeable;
 import java.net.Socket;
 
 import org.apache.commons.configuration.SubsetConfiguration;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.metrics2.AbstractMetric;
 import org.apache.hadoop.metrics2.MetricsException;
 import org.apache.hadoop.metrics2.MetricsRecord;
@@ -36,12 +41,14 @@ import org.apache.hadoop.metrics2.MetricsTag;
  */
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
-public class GraphiteSink implements MetricsSink {
+public class GraphiteSink implements MetricsSink, Closeable {
+    private static final Log LOG = LogFactory.getLog(GraphiteSink.class);
     private static final String SERVER_HOST_KEY = "server_host";
     private static final String SERVER_PORT_KEY = "server_port";
     private static final String METRICS_PREFIX = "metrics_prefix";
     private Writer writer = null;
     private String metricsPrefix = null;
+    private Socket socket = null;
 
     public void setWriter(Writer writer) {
         this.writer = writer;
@@ -60,7 +67,7 @@ public class GraphiteSink implements MetricsSink {
 
         try {
             // Open an connection to Graphite server.
-            Socket socket = new Socket(serverHost, serverPort);
+            socket = new Socket(serverHost, serverPort);
             setWriter(new OutputStreamWriter(socket.getOutputStream()));
         } catch (Exception e) {
             throw new MetricsException("Error creating connection, "
@@ -99,7 +106,11 @@ public class GraphiteSink implements MetricsSink {
         }
 
         try {
-            writer.write(lines.toString());
+            if(writer != null){
+              writer.write(lines.toString());
+            } else {
+              throw new MetricsException("Writer in GraphiteSink is null!");
+            }
         } catch (Exception e) {
             throw new MetricsException("Error sending metrics", e);
         }
@@ -113,4 +124,21 @@ public class GraphiteSink implements MetricsSink {
             throw new MetricsException("Error flushing metrics", e);
         }
     }
+
+    @Override
+    public void close() throws IOException {
+      try {
+        IOUtils.closeStream(writer);
+        writer = null;
+        LOG.info("writer in GraphiteSink is closed!");
+      } catch (Throwable e){
+        throw new MetricsException("Error closing writer", e);
+      } finally {
+        if (socket != null && !socket.isClosed()) {
+          socket.close();
+          socket = null;
+          LOG.info("socket in GraphiteSink is closed!");
+        }
+      }
+    }
 }

+ 52 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ProviderUtils.java

@@ -0,0 +1,52 @@
+/**
+ * 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.security;
+
+import java.net.URI;
+
+import org.apache.hadoop.fs.Path;
+
+public class ProviderUtils {
+  /**
+   * Convert a nested URI to decode the underlying path. The translation takes
+   * the authority and parses it into the underlying scheme and authority.
+   * For example, "myscheme://hdfs@nn/my/path" is converted to
+   * "hdfs://nn/my/path".
+   * @param nestedUri the URI from the nested URI
+   * @return the unnested path
+   */
+  public static Path unnestUri(URI nestedUri) {
+    String[] parts = nestedUri.getAuthority().split("@", 2);
+    StringBuilder result = new StringBuilder(parts[0]);
+    result.append("://");
+    if (parts.length == 2) {
+      result.append(parts[1]);
+    }
+    result.append(nestedUri.getPath());
+    if (nestedUri.getQuery() != null) {
+      result.append("?");
+      result.append(nestedUri.getQuery());
+    }
+    if (nestedUri.getFragment() != null) {
+      result.append("#");
+      result.append(nestedUri.getFragment());
+    }
+    return new Path(result.toString());
+  }
+}

+ 13 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java

@@ -1157,13 +1157,25 @@ public class UserGroupInformation {
   @InterfaceAudience.Public
   @InterfaceStability.Evolving
   public static UserGroupInformation createRemoteUser(String user) {
+    return createRemoteUser(user, AuthMethod.SIMPLE);
+  }
+  
+  /**
+   * Create a user from a login name. It is intended to be used for remote
+   * users in RPC, since it won't have any credentials.
+   * @param user the full user principal name, must not be empty or null
+   * @return the UserGroupInformation for the remote user.
+   */
+  @InterfaceAudience.Public
+  @InterfaceStability.Evolving
+  public static UserGroupInformation createRemoteUser(String user, AuthMethod authMethod) {
     if (user == null || user.isEmpty()) {
       throw new IllegalArgumentException("Null user");
     }
     Subject subject = new Subject();
     subject.getPrincipals().add(new User(user));
     UserGroupInformation result = new UserGroupInformation(subject);
-    result.setAuthenticationMethod(AuthenticationMethod.SIMPLE);
+    result.setAuthenticationMethod(authMethod);
     return result;
   }
 

+ 124 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/CredentialProvider.java

@@ -0,0 +1,124 @@
+/**
+ * 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.security.alias;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A provider of credentials or password for Hadoop applications. Provides an
+ * abstraction to separate credential storage from users of them. It
+ * is intended to support getting or storing passwords in a variety of ways,
+ * including third party bindings.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public abstract class CredentialProvider {
+  public static final String CLEAR_TEXT_FALLBACK 
+    = "hadoop.security.credential.clear-text-fallback";
+
+  /**
+   * The combination of both the alias and the actual credential value.
+   */
+  public static class CredentialEntry {
+    private final String alias;
+    private final char[] credential;
+
+    protected CredentialEntry(String alias,
+                         char[] credential) {
+      this.alias = alias;
+      this.credential = credential;
+    }
+
+    public String getAlias() {
+      return alias;
+    }
+
+    public char[] getCredential() {
+      return credential;
+    }
+
+    public String toString() {
+      StringBuilder buf = new StringBuilder();
+      buf.append("alias(");
+      buf.append(alias);
+      buf.append(")=");
+      if (credential == null) {
+        buf.append("null");
+      } else {
+        for(char c: credential) {
+          buf.append(c);
+        }
+      }
+      return buf.toString();
+    }
+  }
+
+  /**
+   * Indicates whether this provider represents a store
+   * that is intended for transient use - such as the UserProvider
+   * is. These providers are generally used to provide job access to
+   * passwords rather than for long term storage.
+   * @return true if transient, false otherwise
+   */
+  public boolean isTransient() {
+    return false;
+  }
+
+  /**
+   * Ensures that any changes to the credentials are written to persistent store.
+   * @throws IOException
+   */
+  public abstract void flush() throws IOException;
+
+  /**
+   * Get the credential entry for a specific alias.
+   * @param alias the name of a specific credential
+   * @return the credentialEntry
+   * @throws IOException
+   */
+  public abstract CredentialEntry getCredentialEntry(String alias) 
+      throws IOException;
+
+  /**
+   * Get the aliases for all credentials.
+   * @return the list of alias names
+   * @throws IOException
+   */
+  public abstract List<String> getAliases() throws IOException;
+
+  /**
+   * Create a new credential. The given alias must not already exist.
+   * @param name the alias of the credential
+   * @param credential the credential value for the alias.
+   * @throws IOException
+   */
+  public abstract CredentialEntry createCredentialEntry(String name, 
+      char[] credential) throws IOException;
+
+  /**
+   * Delete the given credential.
+   * @param name the alias of the credential to delete
+   * @throws IOException
+   */
+  public abstract void deleteCredentialEntry(String name) throws IOException;
+}

+ 76 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/CredentialProviderFactory.java

@@ -0,0 +1,76 @@
+/**
+ * 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.security.alias;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.ServiceLoader;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * A factory to create a list of CredentialProvider based on the path given in a
+ * Configuration. It uses a service loader interface to find the available
+ * CredentialProviders and create them based on the list of URIs.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public abstract class CredentialProviderFactory {
+  public static final String CREDENTIAL_PROVIDER_PATH =
+      "hadoop.security.credential.provider.path";
+
+  public abstract CredentialProvider createProvider(URI providerName,
+                                             Configuration conf
+                                             ) throws IOException;
+
+  private static final ServiceLoader<CredentialProviderFactory> serviceLoader =
+      ServiceLoader.load(CredentialProviderFactory.class);
+
+  public static List<CredentialProvider> getProviders(Configuration conf
+                                               ) throws IOException {
+    List<CredentialProvider> result = new ArrayList<CredentialProvider>();
+    for(String path: conf.getStringCollection(CREDENTIAL_PROVIDER_PATH)) {
+      try {
+        URI uri = new URI(path);
+        boolean found = false;
+        for(CredentialProviderFactory factory: serviceLoader) {
+          CredentialProvider kp = factory.createProvider(uri, conf);
+          if (kp != null) {
+            result.add(kp);
+            found = true;
+            break;
+          }
+        }
+        if (!found) {
+          throw new IOException("No CredentialProviderFactory for " + uri + " in " +
+              CREDENTIAL_PROVIDER_PATH);
+        }
+      } catch (URISyntaxException error) {
+        throw new IOException("Bad configuration of " + CREDENTIAL_PROVIDER_PATH +
+            " at " + path, error);
+      }
+    }
+    return result;
+  }
+}

+ 423 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/CredentialShell.java

@@ -0,0 +1,423 @@
+/**
+ * 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.security.alias;
+
+import java.io.Console;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.security.InvalidParameterException;
+import java.security.NoSuchAlgorithmException;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+/**
+ * This program is the CLI utility for the CredentialProvider facilities in 
+ * Hadoop.
+ */
+public class CredentialShell extends Configured implements Tool {
+  final static private String USAGE_PREFIX = "Usage: hadoop credential " +
+  		"[generic options]\n";
+  final static private String COMMANDS =
+      "   [--help]\n" +
+      "   [" + CreateCommand.USAGE + "]\n" +
+      "   [" + DeleteCommand.USAGE + "]\n" +
+      "   [" + ListCommand.USAGE + "]\n";
+
+  private boolean interactive = false;
+  private Command command = null;
+
+  /** allows stdout to be captured if necessary */
+  public PrintStream out = System.out;
+  /** allows stderr to be captured if necessary */
+  public PrintStream err = System.err;
+
+  private boolean userSuppliedProvider = false;
+  private String value = null;
+  private PasswordReader passwordReader;
+
+  @Override
+  public int run(String[] args) throws Exception {
+    int exitCode = 0;
+    try {
+      exitCode = init(args);
+      if (exitCode != 0) {
+        return exitCode;
+      }
+      if (command.validate()) {
+          command.execute();
+      } else {
+        exitCode = -1;
+      }
+    } catch (Exception e) {
+      e.printStackTrace(err);
+      return -1;
+    }
+    return exitCode;
+  }
+
+  /**
+   * Parse the command line arguments and initialize the data
+   * <pre>
+   * % hadoop alias create alias [--provider providerPath]
+   * % hadoop alias list [-provider providerPath]
+   * % hadoop alias delete alias [--provider providerPath] [-i]
+   * </pre>
+   * @param args
+   * @return
+   * @throws IOException
+   */
+  private int init(String[] args) throws IOException {
+    for (int i = 0; i < args.length; i++) { // parse command line
+      if (args[i].equals("create")) {
+        String alias = args[++i];
+        command = new CreateCommand(alias);
+        if (alias.equals("--help")) {
+          printCredShellUsage();
+          return -1;
+        }
+      } else if (args[i].equals("delete")) {
+        String alias = args[++i];
+        command = new DeleteCommand(alias);
+        if (alias.equals("--help")) {
+          printCredShellUsage();
+          return -1;
+        }
+      } else if (args[i].equals("list")) {
+        command = new ListCommand();
+      } else if (args[i].equals("--provider")) {
+        userSuppliedProvider = true;
+        getConf().set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, 
+            args[++i]);
+      } else if (args[i].equals("-i") || (args[i].equals("--interactive"))) {
+        interactive = true;
+      } else if (args[i].equals("-v") || (args[i].equals("--value"))) {
+        value = args[++i];
+      } else if (args[i].equals("--help")) {
+        printCredShellUsage();
+        return -1;
+      } else {
+        printCredShellUsage();
+        ToolRunner.printGenericCommandUsage(System.err);
+        return -1;
+      }
+    }
+    return 0;
+  }
+
+  private void printCredShellUsage() {
+    out.println(USAGE_PREFIX + COMMANDS);
+    if (command != null) {
+      out.println(command.getUsage());
+    }
+    else {
+      out.println("=========================================================" +
+      		"======");
+      out.println(CreateCommand.USAGE + ":\n\n" + CreateCommand.DESC);
+      out.println("=========================================================" +
+          "======");
+      out.println(DeleteCommand.USAGE + ":\n\n" + DeleteCommand.DESC);
+      out.println("=========================================================" +
+          "======");
+      out.println(ListCommand.USAGE + ":\n\n" + ListCommand.DESC);
+    }
+  }
+
+  private abstract class Command {
+    protected CredentialProvider provider = null;
+
+    public boolean validate() {
+      return true;
+    }
+
+    protected CredentialProvider getCredentialProvider() {
+      CredentialProvider provider = null;
+      List<CredentialProvider> providers;
+      try {
+        providers = CredentialProviderFactory.getProviders(getConf());
+        if (userSuppliedProvider) {
+          provider = providers.get(0);
+        }
+        else {
+          for (CredentialProvider p : providers) {
+            if (!p.isTransient()) {
+              provider = p;
+              break;
+            }
+          }
+        }
+      } catch (IOException e) {
+        e.printStackTrace(err);
+      }
+      return provider;
+    }
+
+    protected void printProviderWritten() {
+        out.println(provider.getClass().getName() + " has been updated.");
+    }
+
+    protected void warnIfTransientProvider() {
+      if (provider.isTransient()) {
+        out.println("WARNING: you are modifying a transient provider.");
+      }
+    }
+
+    public abstract void execute() throws Exception;
+
+    public abstract String getUsage();
+  }
+
+  private class ListCommand extends Command {
+    public static final String USAGE = "list <alias> [--provider] [--help]";
+    public static final String DESC =
+        "The list subcommand displays the aliases contained within \n" +
+        "a particular provider - as configured in core-site.xml or " +
+        "indicated\nthrough the --provider argument.";
+
+    public boolean validate() {
+      boolean rc = true;
+      provider = getCredentialProvider();
+      if (provider == null) {
+        out.println("There are no non-transient CredentialProviders configured.\n"
+            + "Consider using the --provider option to indicate the provider\n"
+            + "to use. If you want to list a transient provider then you\n"
+            + "you MUST use the --provider argument.");
+        rc = false;
+      }
+      return rc;
+    }
+
+    public void execute() throws IOException {
+      List<String> aliases;
+      try {
+        aliases = provider.getAliases();
+        out.println("Listing aliases for CredentialProvider: " + provider.toString());
+        for (String alias : aliases) {
+          out.println(alias);
+        }
+      } catch (IOException e) {
+        out.println("Cannot list aliases for CredentialProvider: " + provider.toString()
+            + ": " + e.getMessage());
+        throw e;
+      }
+    }
+
+    @Override
+    public String getUsage() {
+      return USAGE + ":\n\n" + DESC;
+    }
+  }
+
+  private class DeleteCommand extends Command {
+    public static final String USAGE = "delete <alias> [--provider] [--help]";
+    public static final String DESC =
+        "The delete subcommand deletes the credenital\n" +
+        "specified as the <alias> argument from within the provider\n" +
+        "indicated through the --provider argument";
+
+    String alias = null;
+    boolean cont = true;
+
+    public DeleteCommand(String alias) {
+      this.alias = alias;
+    }
+
+    @Override
+    public boolean validate() {
+      provider = getCredentialProvider();
+      if (provider == null) {
+        out.println("There are no valid CredentialProviders configured.\n"
+            + "Nothing will be deleted.\n"
+            + "Consider using the --provider option to indicate the provider"
+            + " to use.");
+        return false;
+      }
+      if (alias == null) {
+        out.println("There is no alias specified. Please provide the" +
+            "mandatory <alias>. See the usage description with --help.");
+        return false;
+      }
+      if (interactive) {
+        try {
+          cont = ToolRunner
+              .confirmPrompt("You are about to DELETE the credential: " + 
+                  alias + " from CredentialProvider " + provider.toString() +
+                  ". Continue?:");
+          if (!cont) {
+            out.println("Nothing has been be deleted.");
+          }
+          return cont;
+        } catch (IOException e) {
+          out.println(alias + " will not be deleted.");
+          e.printStackTrace(err);
+        }
+      }
+      return true;
+    }
+
+    public void execute() throws IOException {
+      warnIfTransientProvider();
+      out.println("Deleting credential: " + alias + " from CredentialProvider: "
+          + provider.toString());
+      if (cont) {
+        try {
+          provider.deleteCredentialEntry(alias);
+          out.println(alias + " has been successfully deleted.");
+          provider.flush();
+          printProviderWritten();
+        } catch (IOException e) {
+          out.println(alias + "has NOT been deleted.");
+          throw e;
+        }
+      }
+    }
+
+    @Override
+    public String getUsage() {
+      return USAGE + ":\n\n" + DESC;
+    }
+  }
+
+  private class CreateCommand extends Command {
+    public static final String USAGE = "create <alias> [--provider] [--help]";
+    public static final String DESC =
+        "The create subcommand creates a new credential for the name specified\n" +
+        "as the <alias> argument within the provider indicated through\n" +
+        "the --provider argument.";
+
+    String alias = null;
+
+    public CreateCommand(String alias) {
+      this.alias = alias;
+    }
+
+    public boolean validate() {
+      boolean rc = true;
+      provider = getCredentialProvider();
+      if (provider == null) {
+        out.println("There are no valid CredentialProviders configured." +
+        		"\nCredential will not be created.\n"
+            + "Consider using the --provider option to indicate the provider" +
+            " to use.");
+        rc = false;
+      }
+      if (alias == null) {
+        out.println("There is no alias specified. Please provide the" +
+        		"mandatory <alias>. See the usage description with --help.");
+        rc = false;
+      }
+      return rc;
+    }
+
+    public void execute() throws IOException, NoSuchAlgorithmException {
+      warnIfTransientProvider();
+      try {
+        char[] credential = null;
+        if (value != null) {
+          // testing only
+          credential = value.toCharArray();
+        }
+        else {
+           credential = promptForCredential();
+        }
+        provider.createCredentialEntry(alias, credential);
+        out.println(alias + " has been successfully created.");
+        provider.flush();
+        printProviderWritten();
+      } catch (InvalidParameterException e) {
+        out.println(alias + " has NOT been created. " + e.getMessage());
+        throw e;
+      } catch (IOException e) {
+        out.println(alias + " has NOT been created. " + e.getMessage());
+        throw e;
+      }
+    }
+
+    @Override
+    public String getUsage() {
+      return USAGE + ":\n\n" + DESC;
+    }
+  }
+  
+  protected char[] promptForCredential() throws IOException {
+    PasswordReader c = getPasswordReader();
+    if (c == null) {
+      throw new IOException("No console available for prompting user.");
+    }
+    
+    char[] cred = null;
+
+    boolean noMatch;
+    do {
+      char[] newPassword1 = c.readPassword("Enter password: ");
+      char[] newPassword2 = c.readPassword("Enter password again: ");
+      noMatch = !Arrays.equals(newPassword1, newPassword2);
+      if (noMatch) {
+        Arrays.fill(newPassword1, ' ');
+        c.format("Passwords don't match. Try again.%n");
+      } else {
+        cred = newPassword1;
+      }
+      Arrays.fill(newPassword2, ' ');
+    } while (noMatch);
+    return cred;
+  }
+  
+  public PasswordReader getPasswordReader() {
+    if (passwordReader == null) {
+      passwordReader = new PasswordReader();
+    }
+    return passwordReader;
+  }
+  
+  public void setPasswordReader(PasswordReader reader) {
+    passwordReader = reader;
+  }
+  
+  // to facilitate testing since Console is a final class...
+  public static class PasswordReader {
+    public char[] readPassword(String prompt) {
+      Console console = System.console();
+      char[] pass = console.readPassword(prompt);
+      return pass;
+    }
+
+    public void format(String message) {
+      Console console = System.console();
+      console.format(message);
+    }
+  }
+  
+  
+  /**
+   * Main program.
+   *
+   * @param args
+   *          Command line arguments
+   * @throws Exception
+   */
+  public static void main(String[] args) throws Exception {
+    int res = ToolRunner.run(new Configuration(), new CredentialShell(), args);
+    System.exit(res);
+  }
+}

+ 290 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/JavaKeyStoreProvider.java

@@ -0,0 +1,290 @@
+/**
+ * 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.security.alias;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.ProviderUtils;
+
+import javax.crypto.spec.SecretKeySpec;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URI;
+import java.net.URL;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.UnrecoverableKeyException;
+import java.security.cert.CertificateException;
+import java.util.ArrayList;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+/**
+ * CredentialProvider based on Java's KeyStore file format. The file may be 
+ * stored in any Hadoop FileSystem using the following name mangling:
+ *  jceks://hdfs@nn1.example.com/my/creds.jceks -> hdfs://nn1.example.com/my/creds.jceks
+ *  jceks://file/home/larry/creds.jceks -> file:///home/larry/creds.jceks
+ *
+ * The password for the keystore is taken from the HADOOP_CREDSTORE_PASSWORD
+ * environment variable with a default of 'none'.
+ *
+ * It is expected that for access to credential protected resource to copy the 
+ * creds from the original provider into the job's Credentials object, which is
+ * accessed via the UserProvider. Therefore, this provider won't be directly 
+ * used by MapReduce tasks.
+ */
+@InterfaceAudience.Private
+public class JavaKeyStoreProvider extends CredentialProvider {
+  public static final String SCHEME_NAME = "jceks";
+  public static final String CREDENTIAL_PASSWORD_NAME =
+      "HADOOP_CREDSTORE_PASSWORD";
+  public static final String KEYSTORE_PASSWORD_FILE_KEY =
+      "hadoop.security.credstore.java-keystore-provider.password-file";
+  public static final String KEYSTORE_PASSWORD_DEFAULT = "none";
+
+  private final URI uri;
+  private final Path path;
+  private final FileSystem fs;
+  private final FsPermission permissions;
+  private final KeyStore keyStore;
+  private char[] password = null;
+  private boolean changed = false;
+  private Lock readLock;
+  private Lock writeLock;
+
+  private final Map<String, CredentialEntry> cache = new HashMap<String, CredentialEntry>();
+
+  private JavaKeyStoreProvider(URI uri, Configuration conf) throws IOException {
+    this.uri = uri;
+    path = ProviderUtils.unnestUri(uri);
+    fs = path.getFileSystem(conf);
+    // Get the password from the user's environment
+    if (System.getenv().containsKey(CREDENTIAL_PASSWORD_NAME)) {
+      password = System.getenv(CREDENTIAL_PASSWORD_NAME).toCharArray();
+    }
+    // if not in ENV get check for file
+    if (password == null) {
+      String pwFile = conf.get(KEYSTORE_PASSWORD_FILE_KEY);
+      if (pwFile != null) {
+        ClassLoader cl = Thread.currentThread().getContextClassLoader();
+        URL pwdFile = cl.getResource(pwFile);
+        if (pwdFile != null) {
+          InputStream is = pwdFile.openStream();
+          try {
+            password = IOUtils.toCharArray(is);
+          } finally {
+            is.close();
+          }
+        }
+      }
+    }
+    if (password == null) {
+      password = KEYSTORE_PASSWORD_DEFAULT.toCharArray();
+    }
+    try {
+      keyStore = KeyStore.getInstance(SCHEME_NAME);
+      if (fs.exists(path)) {
+        // save off permissions in case we need to
+        // rewrite the keystore in flush()
+        FileStatus s = fs.getFileStatus(path);
+        permissions = s.getPermission();
+
+        keyStore.load(fs.open(path), password);
+      } else {
+        permissions = new FsPermission("700");
+        // required to create an empty keystore. *sigh*
+        keyStore.load(null, password);
+      }
+    } catch (KeyStoreException e) {
+      throw new IOException("Can't create keystore", e);
+    } catch (NoSuchAlgorithmException e) {
+      throw new IOException("Can't load keystore " + path, e);
+    } catch (CertificateException e) {
+      throw new IOException("Can't load keystore " + path, e);
+    }
+    ReadWriteLock lock = new ReentrantReadWriteLock(true);
+    readLock = lock.readLock();
+    writeLock = lock.writeLock();
+  }
+
+  @Override
+  public CredentialEntry getCredentialEntry(String alias) throws IOException {
+    readLock.lock();
+    try {
+      SecretKeySpec key = null;
+      try {
+        if (cache.containsKey(alias)) {
+          return cache.get(alias);
+        }
+        if (!keyStore.containsAlias(alias)) {
+          return null;
+        }
+        key = (SecretKeySpec) keyStore.getKey(alias, password);
+      } catch (KeyStoreException e) {
+        throw new IOException("Can't get credential " + alias + " from " +
+                              path, e);
+      } catch (NoSuchAlgorithmException e) {
+        throw new IOException("Can't get algorithm for credential " + alias + " from " +
+                              path, e);
+      } catch (UnrecoverableKeyException e) {
+        throw new IOException("Can't recover credential " + alias + " from " + path, e);
+      }
+      return new CredentialEntry(alias, bytesToChars(key.getEncoded()));
+    } 
+    finally {
+      readLock.unlock();
+    }
+  }
+  
+  public static char[] bytesToChars(byte[] bytes) {
+    String pass = new String(bytes);
+    return pass.toCharArray();
+  }
+
+  @Override
+  public List<String> getAliases() throws IOException {
+    readLock.lock();
+    try {
+      ArrayList<String> list = new ArrayList<String>();
+      String alias = null;
+      try {
+        Enumeration<String> e = keyStore.aliases();
+        while (e.hasMoreElements()) {
+           alias = e.nextElement();
+           list.add(alias);
+        }
+      } catch (KeyStoreException e) {
+        throw new IOException("Can't get alias " + alias + " from " + path, e);
+      }
+      return list;
+    }
+    finally {
+      readLock.unlock();
+    }
+  }
+
+  @Override
+  public CredentialEntry createCredentialEntry(String alias, char[] credential)
+      throws IOException {
+    try {
+      if (keyStore.containsAlias(alias) || cache.containsKey(alias)) {
+        throw new IOException("Credential " + alias + " already exists in " + this);
+      }
+    } catch (KeyStoreException e) {
+      throw new IOException("Problem looking up credential " + alias + " in " + this,
+          e);
+    }
+    return innerSetCredential(alias, credential);
+  }
+
+  @Override
+  public void deleteCredentialEntry(String name) throws IOException {
+    writeLock.lock();
+    try {
+      try {
+        if (keyStore.containsAlias(name)) {
+          keyStore.deleteEntry(name);
+        }
+        else {
+          throw new IOException("Credential " + name + " does not exist in " + this);
+        }
+      } catch (KeyStoreException e) {
+        throw new IOException("Problem removing " + name + " from " +
+            this, e);
+      }
+      cache.remove(name);
+      changed = true;
+    }
+    finally {
+      writeLock.unlock();
+    }
+  }
+
+  CredentialEntry innerSetCredential(String alias, char[] material)
+      throws IOException {
+    try {
+      keyStore.setKeyEntry(alias, new SecretKeySpec(
+          new String(material).getBytes("UTF-8"), "AES"),
+          password, null);
+    } catch (KeyStoreException e) {
+      throw new IOException("Can't store credential " + alias + " in " + this,
+          e);
+    }
+    changed = true;
+    return new CredentialEntry(alias, material);
+  }
+
+  @Override
+  public void flush() throws IOException {
+    writeLock.lock();
+    try {
+      if (!changed) {
+        return;
+      }
+      // write out the keystore
+      FSDataOutputStream out = FileSystem.create(fs, path, permissions);
+      try {
+        keyStore.store(out, password);
+      } catch (KeyStoreException e) {
+        throw new IOException("Can't store keystore " + this, e);
+      } catch (NoSuchAlgorithmException e) {
+        throw new IOException("No such algorithm storing keystore " + this, e);
+      } catch (CertificateException e) {
+        throw new IOException("Certificate exception storing keystore " + this,
+            e);
+      }
+      out.close();
+      changed = false;
+    }
+    finally {
+      writeLock.unlock();
+    }
+  }
+
+  @Override
+  public String toString() {
+    return uri.toString();
+  }
+
+  /**
+   * The factory to create JksProviders, which is used by the ServiceLoader.
+   */
+  public static class Factory extends CredentialProviderFactory {
+    @Override
+    public CredentialProvider createProvider(URI providerName,
+                                      Configuration conf) throws IOException {
+      if (SCHEME_NAME.equals(providerName.getScheme())) {
+        return new JavaKeyStoreProvider(providerName, conf);
+      }
+      return null;
+    }
+  }
+}

+ 123 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/UserProvider.java

@@ -0,0 +1,123 @@
+/**
+ * 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.security.alias;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
+
+/**
+ * A CredentialProvider for UGIs. It uses the credentials object associated
+ * with the current user to find credentials. This provider is created using a
+ * URI of "user:///".
+ */
+@InterfaceAudience.Private
+public class UserProvider extends CredentialProvider {
+  public static final String SCHEME_NAME = "user";
+  private final UserGroupInformation user;
+  private final Credentials credentials;
+  private final Map<String, CredentialEntry> cache = new HashMap<String, 
+      CredentialEntry>();
+
+  private UserProvider() throws IOException {
+    user = UserGroupInformation.getCurrentUser();
+    credentials = user.getCredentials();
+  }
+
+  @Override
+  public boolean isTransient() {
+    return true;
+  }
+
+  @Override
+  public CredentialEntry getCredentialEntry(String alias) {
+    byte[] bytes = credentials.getSecretKey(new Text(alias));
+    if (bytes == null) {
+      return null;
+    }
+    return new CredentialEntry(alias, new String(bytes).toCharArray());
+  }
+
+  @Override
+  public CredentialEntry createCredentialEntry(String name, char[] credential) 
+      throws IOException {
+    Text nameT = new Text(name);
+    if (credentials.getSecretKey(nameT) != null) {
+      throw new IOException("Credential " + name + 
+          " already exists in " + this);
+    }
+    credentials.addSecretKey(new Text(name), 
+        new String(credential).getBytes("UTF-8"));
+    return new CredentialEntry(name, credential);
+  }
+
+  @Override
+  public void deleteCredentialEntry(String name) throws IOException {
+    byte[] cred = credentials.getSecretKey(new Text(name));
+    if (cred != null) {
+      credentials.removeSecretKey(new Text(name));
+    }
+    else {
+      throw new IOException("Credential " + name + 
+          " does not exist in " + this);
+    }
+    cache.remove(name);
+  }
+
+  @Override
+  public String toString() {
+    return SCHEME_NAME + ":///";
+  }
+
+  @Override
+  public void flush() {
+    user.addCredentials(credentials);
+  }
+
+  public static class Factory extends CredentialProviderFactory {
+
+    @Override
+    public CredentialProvider createProvider(URI providerName,
+                                      Configuration conf) throws IOException {
+      if (SCHEME_NAME.equals(providerName.getScheme())) {
+        return new UserProvider();
+      }
+      return null;
+    }
+  }
+
+  @Override
+  public List<String> getAliases() throws IOException {
+    List<String> list = new ArrayList<String>();
+    List<Text> aliases = credentials.getAllSecretKeys();
+    for (Text key : aliases) {
+      list.add(key.toString());
+    }
+    return list;
+  }
+}

+ 3 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/ServiceAuthorizationManager.java

@@ -45,7 +45,7 @@ import com.google.common.annotations.VisibleForTesting;
 public class ServiceAuthorizationManager {
   private static final String HADOOP_POLICY_FILE = "hadoop-policy.xml";
 
-  private Map<Class<?>, AccessControlList> protocolToAcl =
+  private volatile Map<Class<?>, AccessControlList> protocolToAcl =
     new IdentityHashMap<Class<?>, AccessControlList>();
   
   /**
@@ -114,7 +114,7 @@ public class ServiceAuthorizationManager {
     AUDITLOG.info(AUTHZ_SUCCESSFUL_FOR + user + " for protocol="+protocol);
   }
 
-  public synchronized void refresh(Configuration conf,
+  public void refresh(Configuration conf,
                                           PolicyProvider provider) {
     // Get the system property 'hadoop.policy.file'
     String policyFile = 
@@ -127,7 +127,7 @@ public class ServiceAuthorizationManager {
   }
 
   @Private
-  public synchronized void refreshWithLoadedConfiguration(Configuration conf,
+  public void refreshWithLoadedConfiguration(Configuration conf,
       PolicyProvider provider) {
     final Map<Class<?>, AccessControlList> newAcls =
         new IdentityHashMap<Class<?>, AccessControlList>();

+ 4 - 3
hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/security/JniBasedUnixGroupsNetgroupMapping.c

@@ -73,7 +73,10 @@ Java_org_apache_hadoop_security_JniBasedUnixGroupsNetgroupMapping_getUsersForNet
   // was successful or not (as long as it was called we need to call
   // endnetgrent)
   setnetgrentCalledFlag = 1;
-#ifndef __FreeBSD__
+#if defined(__FreeBSD__) || defined(__MACH__)
+  setnetgrent(cgroup);
+  {
+#else
   if(setnetgrent(cgroup) == 1) {
 #endif
     current = NULL;
@@ -90,9 +93,7 @@ Java_org_apache_hadoop_security_JniBasedUnixGroupsNetgroupMapping_getUsersForNet
         userListSize++;
       }
     }
-#ifndef __FreeBSD__
   }
-#endif
 
   //--------------------------------------------------
   // build return data (java array)

+ 17 - 0
hadoop-common-project/hadoop-common/src/main/resources/META-INF/services/org.apache.hadoop.security.alias.CredentialProviderFactory

@@ -0,0 +1,17 @@
+# 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.
+
+org.apache.hadoop.security.alias.JavaKeyStoreProvider$Factory
+org.apache.hadoop.security.alias.UserProvider$Factory

+ 7 - 2
hadoop-common-project/hadoop-common/src/site/apt/CommandsManual.apt.vm

@@ -306,9 +306,9 @@ Administration Commands
 
    Runs a cluster balancing utility. An administrator can simply press Ctrl-C
    to stop the rebalancing process. See
-   {{{../hadoop-hdfs/HdfsUserGuide.html#Rebalancer}Rebalancer}} for more details.
+   {{{../hadoop-hdfs/HdfsUserGuide.html#Balancer}Balancer}} for more details.
 
-   Usage: <<<hadoop balancer [-threshold <threshold>]>>>
+   Usage: <<<hadoop balancer [-threshold <threshold>] [-policy <policy>]>>>
 
 *------------------------+-----------------------------------------------------------+
 || COMMAND_OPTION        | Description
@@ -316,6 +316,11 @@ Administration Commands
 | -threshold <threshold> | Percentage of disk capacity. This overwrites the
                          | default threshold.
 *------------------------+-----------------------------------------------------------+
+| -policy <policy>       | <<<datanode>>> (default): Cluster is balanced if each datanode is balanced. \
+                         | <<<blockpool>>>: Cluster is balanced if each block pool in each datanode is balanced.
+*------------------------+-----------------------------------------------------------+
+
+   Note that the <<<blockpool>>> policy is more strict than the <<<datanode>>> policy.
 
 * <<<daemonlog>>>
 

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

@@ -159,7 +159,7 @@ count
 
 cp
 
-   Usage: <<<hdfs dfs -cp [-f] [-p | -p[topx]] URI [URI ...] <dest> >>>
+   Usage: <<<hdfs dfs -cp [-f] [-p | -p[topax]] URI [URI ...] <dest> >>>
 
    Copy files from source to destination. This command allows multiple sources
    as well in which case the destination must be a directory.
@@ -169,8 +169,10 @@ cp
       * The -f option will overwrite the destination if it already exists.
       
       * The -p option will preserve file attributes [topx] (timestamps, 
-        ownership, permission, XAttr). If -p is specified with no <arg>, 
-        then preserves timestamps, ownership, permission.
+        ownership, permission, ACL, XAttr). If -p is specified with no <arg>,
+        then preserves timestamps, ownership, permission. If -pa is specified,
+        then preserves permission also because ACL is a super-set of
+        permission.
 
    Example:
 

+ 5 - 4
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProvider.java

@@ -21,6 +21,7 @@ import org.junit.Assert;
 import org.apache.hadoop.conf.Configuration;
 
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.ProviderUtils;
 import org.junit.Test;
 
 import java.io.IOException;
@@ -133,13 +134,13 @@ public class TestKeyProvider {
   @Test
   public void testUnnestUri() throws Exception {
     assertEquals(new Path("hdfs://nn.example.com/my/path"),
-        KeyProvider.unnestUri(new URI("myscheme://hdfs@nn.example.com/my/path")));
+        ProviderUtils.unnestUri(new URI("myscheme://hdfs@nn.example.com/my/path")));
     assertEquals(new Path("hdfs://nn/my/path?foo=bar&baz=bat#yyy"),
-        KeyProvider.unnestUri(new URI("myscheme://hdfs@nn/my/path?foo=bar&baz=bat#yyy")));
+        ProviderUtils.unnestUri(new URI("myscheme://hdfs@nn/my/path?foo=bar&baz=bat#yyy")));
     assertEquals(new Path("inner://hdfs@nn1.example.com/my/path"),
-        KeyProvider.unnestUri(new URI("outer://inner@hdfs@nn1.example.com/my/path")));
+        ProviderUtils.unnestUri(new URI("outer://inner@hdfs@nn1.example.com/my/path")));
     assertEquals(new Path("user:///"),
-        KeyProvider.unnestUri(new URI("outer://user/")));
+        ProviderUtils.unnestUri(new URI("outer://user/")));
   }
 
   private static class MyKeyProvider extends KeyProvider {

+ 2 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderFactory.java

@@ -31,6 +31,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.ProviderUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.junit.Assert;
 import org.junit.Before;
@@ -213,7 +214,7 @@ public class TestKeyProviderFactory {
     file.delete();
     conf.set(KeyProviderFactory.KEY_PROVIDER_PATH, ourUrl);
     checkSpecificProvider(conf, ourUrl);
-    Path path = KeyProvider.unnestUri(new URI(ourUrl));
+    Path path = ProviderUtils.unnestUri(new URI(ourUrl));
     FileSystem fs = path.getFileSystem(conf);
     FileStatus s = fs.getFileStatus(path);
     assertTrue(s.getPermission().toString().equals("rwx------"));

+ 37 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/impl/TestGraphiteMetrics.java

@@ -22,6 +22,7 @@ import static org.junit.Assert.assertEquals;
 
 import java.io.IOException;
 import java.io.OutputStreamWriter;
+import java.io.Writer;
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
@@ -30,6 +31,7 @@ import java.util.Set;
 import static org.mockito.Mockito.*;
 
 import org.apache.hadoop.metrics2.AbstractMetric;
+import org.apache.hadoop.metrics2.MetricsException;
 import org.apache.hadoop.metrics2.MetricsRecord;
 import org.apache.hadoop.metrics2.MetricsTag;
 import org.apache.hadoop.metrics2.sink.GraphiteSink;
@@ -107,4 +109,39 @@ public class TestGraphiteMetrics {
             result.equals("null.all.Context.Context=all.foo2 2 10\n" + 
             "null.all.Context.Context=all.foo1 1 10\n"));
     }
+    @Test(expected=MetricsException.class)
+    public void testCloseAndWrite() throws IOException {
+      GraphiteSink sink = new GraphiteSink();
+      List<MetricsTag> tags = new ArrayList<MetricsTag>();
+      tags.add(new MetricsTag(MsInfo.Context, "all"));
+      tags.add(new MetricsTag(MsInfo.Hostname, "host"));
+      Set<AbstractMetric> metrics = new HashSet<AbstractMetric>();
+      metrics.add(makeMetric("foo1", 1.25));
+      metrics.add(makeMetric("foo2", 2.25));
+      MetricsRecord record = new MetricsRecordImpl(MsInfo.Context, (long) 10000, tags, metrics);
+
+      OutputStreamWriter writer = mock(OutputStreamWriter.class);
+
+      sink.setWriter(writer);
+      sink.close();
+      sink.putMetrics(record);
+    }
+
+    @Test
+    public void testClose(){
+      GraphiteSink sink = new GraphiteSink();
+      Writer mockWriter = mock(Writer.class);
+      sink.setWriter(mockWriter);
+      try {
+        sink.close();
+      } catch (IOException ioe) {
+        ioe.printStackTrace();
+      }
+
+      try {
+        verify(mockWriter).close();
+      } catch (IOException ioe) {
+        ioe.printStackTrace();
+      }
+    }
 }

+ 14 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java

@@ -20,6 +20,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.security.SaslRpcServer.AuthMethod;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.authentication.util.KerberosName;
 import org.apache.hadoop.security.token.Token;
@@ -31,6 +32,7 @@ import javax.security.auth.Subject;
 import javax.security.auth.kerberos.KerberosPrincipal;
 import javax.security.auth.login.AppConfigurationEntry;
 import javax.security.auth.login.LoginContext;
+
 import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.InputStreamReader;
@@ -151,6 +153,18 @@ public class TestUserGroupInformation {
     assertEquals(AuthenticationMethod.PROXY, ugi.getAuthenticationMethod());
     assertEquals(AuthenticationMethod.SIMPLE, ugi.getRealAuthenticationMethod());
   }
+  
+  @Test (timeout = 30000)
+  public void testCreateRemoteUser() {
+    UserGroupInformation ugi = UserGroupInformation.createRemoteUser("user1");
+    assertEquals(AuthenticationMethod.SIMPLE, ugi.getAuthenticationMethod());
+    assertTrue (ugi.toString().contains("(auth:SIMPLE)"));
+    ugi = UserGroupInformation.createRemoteUser("user1", 
+        AuthMethod.KERBEROS);
+    assertEquals(AuthenticationMethod.KERBEROS, ugi.getAuthenticationMethod());
+    assertTrue (ugi.toString().contains("(auth:KERBEROS)"));
+  }
+  
   /** Test login method */
   @Test (timeout = 30000)
   public void testLogin() throws Exception {

+ 173 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredShell.java

@@ -0,0 +1,173 @@
+/**
+ * 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.security.alias;
+
+import static org.junit.Assert.*;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.alias.CredentialShell.PasswordReader;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestCredShell {
+  private final ByteArrayOutputStream outContent = new ByteArrayOutputStream();
+  private final ByteArrayOutputStream errContent = new ByteArrayOutputStream();
+  private static final File tmpDir =
+      new File(System.getProperty("test.build.data", "/tmp"), "creds");
+  
+  @Before
+  public void setup() throws Exception {
+    System.setOut(new PrintStream(outContent));
+    System.setErr(new PrintStream(errContent));
+  }
+  
+  @Test
+  public void testCredentialSuccessfulLifecycle() throws Exception {
+    outContent.reset();
+    String[] args1 = {"create", "credential1", "--value", "p@ssw0rd", "--provider", 
+        "jceks://file" + tmpDir + "/credstore.jceks"};
+    int rc = 0;
+    CredentialShell cs = new CredentialShell();
+    cs.setConf(new Configuration());
+    rc = cs.run(args1);
+    assertEquals(outContent.toString(), 0, rc);
+    assertTrue(outContent.toString().contains("credential1 has been successfully " +
+    		"created."));
+
+    outContent.reset();
+    String[] args2 = {"list", "--provider", 
+        "jceks://file" + tmpDir + "/credstore.jceks"};
+    rc = cs.run(args2);
+    assertEquals(0, rc);
+    assertTrue(outContent.toString().contains("credential1"));
+
+    outContent.reset();
+    String[] args4 = {"delete", "credential1", "--provider", 
+        "jceks://file" + tmpDir + "/credstore.jceks"};
+    rc = cs.run(args4);
+    assertEquals(0, rc);
+    assertTrue(outContent.toString().contains("credential1 has been successfully " +
+    		"deleted."));
+
+    outContent.reset();
+    String[] args5 = {"list", "--provider", 
+        "jceks://file" + tmpDir + "/credstore.jceks"};
+    rc = cs.run(args5);
+    assertEquals(0, rc);
+    assertFalse(outContent.toString(), outContent.toString().contains("credential1"));
+  }
+
+  @Test
+  public void testInvalidProvider() throws Exception {
+    String[] args1 = {"create", "credential1", "--value", "p@ssw0rd", "--provider", 
+      "sdff://file/tmp/credstore.jceks"};
+    
+    int rc = 0;
+    CredentialShell cs = new CredentialShell();
+    cs.setConf(new Configuration());
+    rc = cs.run(args1);
+    assertEquals(-1, rc);
+    assertTrue(outContent.toString().contains("There are no valid " +
+    		"CredentialProviders configured."));
+  }
+
+  @Test
+  public void testTransientProviderWarning() throws Exception {
+    String[] args1 = {"create", "credential1", "--value", "p@ssw0rd", "--provider", 
+      "user:///"};
+    
+    int rc = 0;
+    CredentialShell cs = new CredentialShell();
+    cs.setConf(new Configuration());
+    rc = cs.run(args1);
+    assertEquals(outContent.toString(), 0, rc);
+    assertTrue(outContent.toString().contains("WARNING: you are modifying a " +
+    		"transient provider."));
+
+    String[] args2 = {"delete", "credential1", "--provider", "user:///"};
+    rc = cs.run(args2);
+    assertEquals(outContent.toString(), 0, rc);
+    assertTrue(outContent.toString().contains("credential1 has been successfully " +
+        "deleted."));
+  }
+  
+  @Test
+  public void testTransientProviderOnlyConfig() throws Exception {
+    String[] args1 = {"create", "credential1"};
+    
+    int rc = 0;
+    CredentialShell cs = new CredentialShell();
+    Configuration config = new Configuration();
+    config.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, "user:///");
+    cs.setConf(config);
+    rc = cs.run(args1);
+    assertEquals(-1, rc);
+    assertTrue(outContent.toString().contains("There are no valid " +
+    		"CredentialProviders configured."));
+  }
+  
+  @Test
+  public void testPromptForCredential() throws Exception {
+    String[] args1 = {"create", "credential1", "--provider", 
+        "jceks://file" + tmpDir + "/credstore.jceks"};
+    ArrayList<String> passwords = new ArrayList<String>();
+    passwords.add("p@ssw0rd");
+    passwords.add("p@ssw0rd");
+    int rc = 0;
+    CredentialShell shell = new CredentialShell();
+    shell.setConf(new Configuration());
+    shell.setPasswordReader(new MockPasswordReader(passwords));
+    rc = shell.run(args1);
+    assertEquals(0, rc);
+    assertTrue(outContent.toString().contains("credential1 has been successfully " +
+        "created."));
+
+    String[] args2 = {"delete", "credential1", "--provider", 
+        "jceks://file" + tmpDir + "/credstore.jceks"};
+    rc = shell.run(args2);
+    assertEquals(0, rc);
+    assertTrue(outContent.toString().contains("credential1 has been successfully " +
+        "deleted."));
+  }
+  
+  public class MockPasswordReader extends CredentialShell.PasswordReader {
+    List<String> passwords = null;
+    
+    public MockPasswordReader(List<String> passwds) {
+      passwords = passwds;
+    }
+
+    @Override
+    public char[] readPassword(String prompt) {
+      if (passwords.size() == 0) return null;
+      String pass = passwords.remove(0);
+      return pass.toCharArray();
+    }
+
+    @Override
+    public void format(String message) {
+      System.out.println(message);
+    }
+  }
+}

+ 51 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredentialProvider.java

@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.security.alias;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.ProviderUtils;
+import org.junit.Test;
+
+import java.net.URI;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertArrayEquals;
+
+public class TestCredentialProvider {
+
+  @Test
+  public void testCredentialEntry() throws Exception {
+    char[] key1 = new char[]{1,2,3,4};
+    CredentialProvider.CredentialEntry obj = 
+        new CredentialProvider.CredentialEntry("cred1", key1);
+    assertEquals("cred1", obj.getAlias());
+    assertArrayEquals(new char[]{1,2,3,4}, obj.getCredential());
+  }
+
+  @Test
+  public void testUnnestUri() throws Exception {
+    assertEquals(new Path("hdfs://nn.example.com/my/path"),
+        ProviderUtils.unnestUri(new URI("myscheme://hdfs@nn.example.com/my/path")));
+    assertEquals(new Path("hdfs://nn/my/path?foo=bar&baz=bat#yyy"),
+        ProviderUtils.unnestUri(new URI("myscheme://hdfs@nn/my/path?foo=bar&baz=bat#yyy")));
+    assertEquals(new Path("inner://hdfs@nn1.example.com/my/path"),
+        ProviderUtils.unnestUri(new URI("outer://inner@hdfs@nn1.example.com/my/path")));
+    assertEquals(new Path("user:///"),
+        ProviderUtils.unnestUri(new URI("outer://user/")));
+  }
+}

+ 234 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredentialProviderFactory.java

@@ -0,0 +1,234 @@
+/**
+ * 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.security.alias;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.util.List;
+import java.util.Random;
+
+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.FsPermission;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.ProviderUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.junit.Test;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestCredentialProviderFactory {
+  
+  private static char[] chars = { 'a', 'b', 'c', 'd', 'e', 'f', 'g',
+  'h', 'j', 'k', 'm', 'n', 'p', 'q', 'r', 's', 't', 'u', 'v', 'w',
+  'x', 'y', 'z', 'A', 'B', 'C', 'D', 'E', 'F', 'G', 'H', 'J', 'K',
+  'M', 'N', 'P', 'Q', 'R', 'S', 'T', 'U', 'V', 'W', 'X', 'Y', 'Z',
+  '2', '3', '4', '5', '6', '7', '8', '9',};
+
+  private static final File tmpDir =
+      new File(System.getProperty("test.build.data", "/tmp"), "creds");
+
+  @Test
+  public void testFactory() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH,
+        UserProvider.SCHEME_NAME + ":///," +
+            JavaKeyStoreProvider.SCHEME_NAME + "://file" + tmpDir + "/test.jks");
+    List<CredentialProvider> providers = 
+        CredentialProviderFactory.getProviders(conf);
+    assertEquals(2, providers.size());
+    assertEquals(UserProvider.class, providers.get(0).getClass());
+    assertEquals(JavaKeyStoreProvider.class, providers.get(1).getClass());
+    assertEquals(UserProvider.SCHEME_NAME +
+        ":///", providers.get(0).toString());
+    assertEquals(JavaKeyStoreProvider.SCHEME_NAME +
+        "://file" + tmpDir + "/test.jks",
+        providers.get(1).toString());
+  }
+
+  @Test
+  public void testFactoryErrors() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, "unknown:///");
+    try {
+      List<CredentialProvider> providers = 
+          CredentialProviderFactory.getProviders(conf);
+      assertTrue("should throw!", false);
+    } catch (IOException e) {
+      assertEquals("No CredentialProviderFactory for unknown:/// in " +
+          CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH,
+          e.getMessage());
+    }
+  }
+
+  @Test
+  public void testUriErrors() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, "unkn@own:/x/y");
+    try {
+      List<CredentialProvider> providers = 
+          CredentialProviderFactory.getProviders(conf);
+      assertTrue("should throw!", false);
+    } catch (IOException e) {
+      assertEquals("Bad configuration of " +
+          CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH +
+          " at unkn@own:/x/y", e.getMessage());
+    }
+  }
+
+  private static char[] generatePassword(int length) {
+    StringBuffer sb = new StringBuffer();
+    Random r = new Random();
+    for (int i = 0; i < length; i++) {
+      sb.append(chars[r.nextInt(chars.length)]);
+    }
+    return sb.toString().toCharArray();
+  }
+  
+  static void checkSpecificProvider(Configuration conf,
+                                   String ourUrl) throws Exception {
+    CredentialProvider provider = 
+        CredentialProviderFactory.getProviders(conf).get(0);
+    char[] passwd = generatePassword(16);
+
+    // ensure that we get nulls when the key isn't there
+    assertEquals(null, provider.getCredentialEntry("no-such-key"));
+    assertEquals(null, provider.getCredentialEntry("key"));
+    // create a new key
+    try {
+      provider.createCredentialEntry("pass", passwd);
+    } catch (Exception e) {
+      e.printStackTrace();
+      throw e;
+    }
+    // make sure we get back the right key
+    assertArrayEquals(passwd, provider.getCredentialEntry("pass").getCredential());
+    // try recreating pass
+    try {
+      provider.createCredentialEntry("pass", passwd);
+      assertTrue("should throw", false);
+    } catch (IOException e) {
+      assertEquals("Credential pass already exists in " + ourUrl, e.getMessage());
+    }
+    provider.deleteCredentialEntry("pass");
+    try {
+      provider.deleteCredentialEntry("pass");
+      assertTrue("should throw", false);
+    } catch (IOException e) {
+      assertEquals("Credential pass does not exist in " + ourUrl, e.getMessage());
+    }
+    char[] passTwo = new char[]{'1', '2', '3'};
+    provider.createCredentialEntry("pass", passwd);
+    provider.createCredentialEntry("pass2", passTwo);
+    assertArrayEquals(passTwo,
+        provider.getCredentialEntry("pass2").getCredential());
+
+    // write them to disk so that configuration.getPassword will find them
+    provider.flush();
+
+    // configuration.getPassword should get this from provider
+    assertArrayEquals(passTwo, conf.getPassword("pass2"));
+
+    // configuration.getPassword should get this from config
+    conf.set("onetwothree", "123");
+    assertArrayEquals(passTwo, conf.getPassword("onetwothree"));
+
+    // configuration.getPassword should NOT get this from config since
+    // we are disabling the fallback to clear text config
+    conf.set(CredentialProvider.CLEAR_TEXT_FALLBACK, "false");
+    assertArrayEquals(null, conf.getPassword("onetwothree"));
+
+    // get a new instance of the provider to ensure it was saved correctly
+    provider = CredentialProviderFactory.getProviders(conf).get(0);
+    assertTrue(provider != null);
+    assertArrayEquals(new char[]{'1', '2', '3'},
+        provider.getCredentialEntry("pass2").getCredential());
+    assertArrayEquals(passwd, provider.getCredentialEntry("pass").getCredential());
+
+    List<String> creds = provider.getAliases();
+    assertTrue("Credentials should have been returned.", creds.size() == 2);
+    assertTrue("Returned Credentials should have included pass.", creds.contains("pass"));
+    assertTrue("Returned Credentials should have included pass2.", creds.contains("pass2"));
+  }
+
+  @Test
+  public void testUserProvider() throws Exception {
+    Configuration conf = new Configuration();
+    final String ourUrl = UserProvider.SCHEME_NAME + ":///";
+    conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, ourUrl);
+    checkSpecificProvider(conf, ourUrl);
+    // see if the credentials are actually in the UGI
+    Credentials credentials =
+        UserGroupInformation.getCurrentUser().getCredentials();
+    assertArrayEquals(new byte[]{'1', '2', '3'},
+        credentials.getSecretKey(new Text("pass2")));
+  }
+
+  @Test
+  public void testJksProvider() throws Exception {
+    Configuration conf = new Configuration();
+    final String ourUrl =
+        JavaKeyStoreProvider.SCHEME_NAME + "://file" + tmpDir + "/test.jks";
+
+    File file = new File(tmpDir, "test.jks");
+    file.delete();
+    conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, ourUrl);
+    checkSpecificProvider(conf, ourUrl);
+    Path path = ProviderUtils.unnestUri(new URI(ourUrl));
+    FileSystem fs = path.getFileSystem(conf);
+    FileStatus s = fs.getFileStatus(path);
+    assertTrue(s.getPermission().toString().equals("rwx------"));
+    assertTrue(file + " should exist", file.isFile());
+
+    // check permission retention after explicit change
+    fs.setPermission(path, new FsPermission("777"));
+    checkPermissionRetention(conf, ourUrl, path);
+  }
+
+  public void checkPermissionRetention(Configuration conf, String ourUrl, 
+      Path path) throws Exception {
+    CredentialProvider provider = CredentialProviderFactory.getProviders(conf).get(0);
+    // let's add a new credential and flush and check that permissions are still set to 777
+    char[] cred = new char[32];
+    for(int i =0; i < cred.length; ++i) {
+      cred[i] = (char) i;
+    }
+    // create a new key
+    try {
+      provider.createCredentialEntry("key5", cred);
+    } catch (Exception e) {
+      e.printStackTrace();
+      throw e;
+    }
+    provider.flush();
+    // get a new instance of the provider to ensure it was saved correctly
+    provider = CredentialProviderFactory.getProviders(conf).get(0);
+    assertArrayEquals(cred, provider.getCredentialEntry("key5").getCredential());
+
+    FileSystem fs = path.getFileSystem(conf);
+    FileStatus s = fs.getFileStatus(path);
+    assertTrue("Permissions should have been retained from the preexisting " +
+    		"keystore.", s.getPermission().toString().equals("rwxrwxrwx"));
+  }
+}

+ 8 - 4
hadoop-common-project/hadoop-common/src/test/resources/testConf.xml

@@ -296,7 +296,7 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-cp \[-f\] \[-p \| -p\[topx\]\] &lt;src&gt; \.\.\. &lt;dst&gt; :\s*</expected-output>
+          <expected-output>^-cp \[-f\] \[-p \| -p\[topax\]\] &lt;src&gt; \.\.\. &lt;dst&gt; :\s*</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -308,15 +308,19 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^( |\t)*\[topx\] \(timestamps, ownership, permission, XAttr\). If -p is specified with no( )*</expected-output>
+          <expected-output>^( |\t)*\[topax\] \(timestamps, ownership, permission, ACLs, XAttr\). If -p is specified( )*</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^( |\t)*&lt;arg&gt;, then preserves timestamps, ownership, permission. Passing -f overwrites( )*</expected-output>
+          <expected-output>^( |\t)*with no &lt;arg&gt;, then preserves timestamps, ownership, permission. If -pa is( )*</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^\s*the destination if it already exists.( )*</expected-output>
+          <expected-output>^( |\t)*specified, then preserves permission also because ACL is a super-set of( )*</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^\s*permission. Passing -f overwrites the destination if it already exists.( )*</expected-output>
         </comparator>
       </comparators>
     </test>

+ 31 - 39
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Interface.java

@@ -17,12 +17,9 @@
  */
 package org.apache.hadoop.nfs.nfs3;
 
-import java.net.InetAddress;
-
 import org.apache.hadoop.nfs.nfs3.response.NFS3Response;
+import org.apache.hadoop.oncrpc.RpcInfo;
 import org.apache.hadoop.oncrpc.XDR;
-import org.apache.hadoop.oncrpc.security.SecurityHandler;
-import org.jboss.netty.channel.Channel;
 
 /**
  * RPC procedures as defined in RFC 1813.
@@ -33,70 +30,65 @@ public interface Nfs3Interface {
   public NFS3Response nullProcedure();
 
   /** GETATTR: Get file attributes */
-  public NFS3Response getattr(XDR xdr, SecurityHandler securityHandler,
-      InetAddress client);
+  public NFS3Response getattr(XDR xdr, RpcInfo info);
 
   /** SETATTR: Set file attributes */
-  public NFS3Response setattr(XDR xdr, SecurityHandler securityHandler,
-      InetAddress client);
+  public NFS3Response setattr(XDR xdr, RpcInfo info);
 
   /** LOOKUP: Lookup filename */
-  public NFS3Response lookup(XDR xdr, SecurityHandler securityHandler,
-      InetAddress client);
+  public NFS3Response lookup(XDR xdr, RpcInfo info);
 
   /** ACCESS: Check access permission */
-  public NFS3Response access(XDR xdr, SecurityHandler securityHandler,
-      InetAddress client);
+  public NFS3Response access(XDR xdr, RpcInfo info);
+
+    /** READLINK: Read from symbolic link */
+  public NFS3Response readlink(XDR xdr, RpcInfo info);
 
   /** READ: Read from file */
-  public NFS3Response read(XDR xdr, SecurityHandler securityHandler,
-      InetAddress client);
+  public NFS3Response read(XDR xdr, RpcInfo info);
 
   /** WRITE: Write to file */
-  public NFS3Response write(XDR xdr, Channel channel, int xid,
-      SecurityHandler securityHandler, InetAddress client);
+  public NFS3Response write(XDR xdr, RpcInfo info);
 
   /** CREATE: Create a file */
-  public NFS3Response create(XDR xdr, SecurityHandler securityHandler,
-      InetAddress client);
+  public NFS3Response create(XDR xdr, RpcInfo info);
 
   /** MKDIR: Create a directory */
-  public NFS3Response mkdir(XDR xdr, SecurityHandler securityHandler,
-      InetAddress client);
+  public NFS3Response mkdir(XDR xdr, RpcInfo info);
+
+  /** SYMLINK: Create a symbolic link */
+  public NFS3Response symlink(XDR xdr, RpcInfo info);
+
+  /** MKNOD: Create a special device */
+  public NFS3Response mknod(XDR xdr, RpcInfo info);
 
   /** REMOVE: Remove a file */
-  public NFS3Response remove(XDR xdr, SecurityHandler securityHandler,
-      InetAddress client);
+  public NFS3Response remove(XDR xdr, RpcInfo info);
 
   /** RMDIR: Remove a directory */
-  public NFS3Response rmdir(XDR xdr, SecurityHandler securityHandler,
-      InetAddress client);
+  public NFS3Response rmdir(XDR xdr, RpcInfo info);
 
   /** RENAME: Rename a file or directory */
-  public NFS3Response rename(XDR xdr, SecurityHandler securityHandler,
-      InetAddress client);
+  public NFS3Response rename(XDR xdr, RpcInfo info);
 
-  /** SYMLINK: Create a symbolic link */
-  public NFS3Response symlink(XDR xdr, SecurityHandler securityHandler,
-      InetAddress client);
+  /** LINK: create link to an object */
+  public NFS3Response link(XDR xdr, RpcInfo info);
 
   /** READDIR: Read From directory */
-  public NFS3Response readdir(XDR xdr, SecurityHandler securityHandler,
-      InetAddress client);
+  public NFS3Response readdir(XDR xdr, RpcInfo info);
 
+  /** READDIRPLUS: Extended read from directory */
+  public NFS3Response readdirplus(XDR xdr, RpcInfo info);
+  
   /** FSSTAT: Get dynamic file system information */
-  public NFS3Response fsstat(XDR xdr, SecurityHandler securityHandler,
-      InetAddress client);
+  public NFS3Response fsstat(XDR xdr, RpcInfo info);
 
   /** FSINFO: Get static file system information */
-  public NFS3Response fsinfo(XDR xdr, SecurityHandler securityHandler,
-      InetAddress client);
+  public NFS3Response fsinfo(XDR xdr, RpcInfo info);
 
   /** PATHCONF: Retrieve POSIX information */
-  public NFS3Response pathconf(XDR xdr, SecurityHandler securityHandler,
-      InetAddress client);
+  public NFS3Response pathconf(XDR xdr, RpcInfo info);
 
   /** COMMIT: Commit cached data on a server to stable storage */
-  public NFS3Response commit(XDR xdr, Channel channel, int xid,
-      SecurityHandler securityHandler, InetAddress client);
+  public NFS3Response commit(XDR xdr, RpcInfo info);
 }

+ 25 - 27
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcProgram.java

@@ -48,7 +48,7 @@ public abstract class RpcProgram extends SimpleChannelUpstreamHandler {
   private final int progNumber;
   private final int lowProgVersion;
   private final int highProgVersion;
-  private final boolean allowInsecurePorts;
+  protected final boolean allowInsecurePorts;
   
   /**
    * If not null, this will be used as the socket to use to connect to the
@@ -146,31 +146,6 @@ public abstract class RpcProgram extends SimpleChannelUpstreamHandler {
     RpcCall call = (RpcCall) info.header();
     
     SocketAddress remoteAddress = info.remoteAddress();
-    if (!allowInsecurePorts) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Will not allow connections from unprivileged ports. " +
-            "Checking for valid client port...");
-      }
-      if (remoteAddress instanceof InetSocketAddress) {
-        InetSocketAddress inetRemoteAddress = (InetSocketAddress) remoteAddress;
-        if (inetRemoteAddress.getPort() > 1023) {
-          LOG.warn("Connection attempted from '" + inetRemoteAddress + "' "
-              + "which is an unprivileged port. Rejecting connection.");
-          sendRejectedReply(call, remoteAddress, ctx);
-          return;
-        } else {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Accepting connection from '" + remoteAddress + "'");
-          }
-        }
-      } else {
-        LOG.warn("Could not determine remote port of socket address '" +
-            remoteAddress + "'. Rejecting connection.");
-        sendRejectedReply(call, remoteAddress, ctx);
-        return;
-      }
-    }
-    
     if (LOG.isTraceEnabled()) {
       LOG.trace(program + " procedure #" + call.getProcedure());
     }
@@ -191,6 +166,29 @@ public abstract class RpcProgram extends SimpleChannelUpstreamHandler {
     handleInternal(ctx, info);
   }
   
+  public boolean doPortMonitoring(SocketAddress remoteAddress) {
+    if (!allowInsecurePorts) {
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Will not allow connections from unprivileged ports. "
+            + "Checking for valid client port...");
+      }
+
+      if (remoteAddress instanceof InetSocketAddress) {
+        InetSocketAddress inetRemoteAddress = (InetSocketAddress) remoteAddress;
+        if (inetRemoteAddress.getPort() > 1023) {
+          LOG.warn("Connection attempted from '" + inetRemoteAddress + "' "
+              + "which is an unprivileged port. Rejecting connection.");
+          return false;
+        }
+      } else {
+        LOG.warn("Could not determine remote port of socket address '"
+            + remoteAddress + "'. Rejecting connection.");
+        return false;
+      }
+    }
+    return true;
+  }
+  
   private void sendAcceptedReply(RpcCall call, SocketAddress remoteAddress,
       AcceptState acceptState, ChannelHandlerContext ctx) {
     RpcAcceptedReply reply = RpcAcceptedReply.getInstance(call.getXid(),
@@ -208,7 +206,7 @@ public abstract class RpcProgram extends SimpleChannelUpstreamHandler {
     RpcUtil.sendRpcResponse(ctx, rsp);
   }
   
-  private static void sendRejectedReply(RpcCall call,
+  protected static void sendRejectedReply(RpcCall call,
       SocketAddress remoteAddress, ChannelHandlerContext ctx) {
     XDR out = new XDR();
     RpcDeniedReply reply = new RpcDeniedReply(call.getXid(),

+ 26 - 0
hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/oncrpc/TestFrameDecoder.java

@@ -66,6 +66,18 @@ public class TestFrameDecoder {
 
     @Override
     protected void handleInternal(ChannelHandlerContext ctx, RpcInfo info) {
+      // This is just like what's done in RpcProgramMountd#handleInternal and
+      // RpcProgramNfs3#handleInternal.
+      RpcCall rpcCall = (RpcCall) info.header();
+      final int procedure = rpcCall.getProcedure();
+      if (procedure != 0) {
+        boolean portMonitorSuccess = doPortMonitoring(info.remoteAddress());
+        if (!portMonitorSuccess) {
+          sendRejectedReply(rpcCall, info.remoteAddress(), ctx);
+          return;
+        }
+      }
+      
       resultSize = info.data().readableBytes();
       RpcAcceptedReply reply = RpcAcceptedReply.getAcceptInstance(1234,
           new VerifierNone());
@@ -190,6 +202,20 @@ public class TestFrameDecoder {
 
     // Verify the server rejected the request.
     assertEquals(0, resultSize);
+    
+    // Ensure that the NULL procedure does in fact succeed.
+    xdrOut = new XDR();
+    createPortmapXDRheader(xdrOut, 0);
+    int headerSize = xdrOut.size();
+    buffer = new byte[bufsize];
+    xdrOut.writeFixedOpaque(buffer);
+    int requestSize = xdrOut.size() - headerSize;
+    
+    // Send the request to the server
+    testRequest(xdrOut, serverPort);
+
+    // Verify the server did not reject the request.
+    assertEquals(requestSize, resultSize);
   }
   
   private static int startRpcServer(boolean allowInsecurePorts) {

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/conf/NfsConfigKeys.java

@@ -51,7 +51,8 @@ public class NfsConfigKeys {
   public static final String DFS_NFS_KEYTAB_FILE_KEY = "nfs.keytab.file";
   public static final String DFS_NFS_KERBEROS_PRINCIPAL_KEY = "nfs.kerberos.principal";
   public static final String DFS_NFS_REGISTRATION_PORT_KEY = "nfs.registration.port";
-  public static final int    DFS_NFS_REGISTRATION_PORT_DEFAULT = 40; // Currently unassigned.
-  public static final String  DFS_NFS_ALLOW_INSECURE_PORTS_KEY = "nfs.allow.insecure.ports";
-  public static final boolean DFS_NFS_ALLOW_INSECURE_PORTS_DEFAULT = true;
+  public static final int DFS_NFS_REGISTRATION_PORT_DEFAULT = 40; // Currently unassigned.
+  public static final String DFS_NFS_PORT_MONITORING_DISABLED_KEY = "nfs.port.monitoring.disabled";
+  public static final boolean DFS_NFS_PORT_MONITORING_DISABLED_DEFAULT = true;
+
 }

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/conf/NfsConfiguration.java

@@ -49,6 +49,8 @@ public class NfsConfiguration extends HdfsConfiguration {
         new DeprecationDelta("dfs.nfs3.stream.timeout",
             NfsConfigKeys.DFS_NFS_STREAM_TIMEOUT_KEY),
         new DeprecationDelta("dfs.nfs3.export.point",
-            NfsConfigKeys.DFS_NFS_EXPORT_POINT_KEY) });
+            NfsConfigKeys.DFS_NFS_EXPORT_POINT_KEY),
+        new DeprecationDelta("nfs.allow.insecure.ports",
+            NfsConfigKeys.DFS_NFS_PORT_MONITORING_DISABLED_KEY) });
   }
 }

+ 7 - 1
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/RpcProgramMountd.java

@@ -194,7 +194,13 @@ public class RpcProgramMountd extends RpcProgram implements MountInterface {
     if (mntproc == MNTPROC.NULL) {
       out = nullOp(out, xid, client);
     } else if (mntproc == MNTPROC.MNT) {
-      out = mnt(xdr, out, xid, client);
+      // Only do port monitoring for MNT
+      if (!doPortMonitoring(info.remoteAddress())) {
+        out = MountResponse.writeMNTResponse(Nfs3Status.NFS3ERR_ACCES, out,
+            xid, null);
+      } else {
+        out = mnt(xdr, out, xid, client);
+      }
     } else if (mntproc == MNTPROC.DUMP) {
       out = dump(out, xid, client);
     } else if (mntproc == MNTPROC.UMNT) {      

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3.java

@@ -61,8 +61,8 @@ public class Nfs3 extends Nfs3Base {
     StringUtils.startupShutdownMessage(Nfs3.class, args, LOG);
     NfsConfiguration conf = new NfsConfiguration();
     boolean allowInsecurePorts = conf.getBoolean(
-        NfsConfigKeys.DFS_NFS_ALLOW_INSECURE_PORTS_KEY,
-        NfsConfigKeys.DFS_NFS_ALLOW_INSECURE_PORTS_DEFAULT);
+        NfsConfigKeys.DFS_NFS_PORT_MONITORING_DISABLED_KEY,
+        NfsConfigKeys.DFS_NFS_PORT_MONITORING_DISABLED_DEFAULT);
     final Nfs3 nfsServer = new Nfs3(conf, registrationSocket,
         allowInsecurePorts);
     nfsServer.startServiceInternal(true);

+ 145 - 89
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java

@@ -23,6 +23,7 @@ import java.io.IOException;
 import java.net.DatagramSocket;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
+import java.net.SocketAddress;
 import java.nio.ByteBuffer;
 import java.util.EnumSet;
 
@@ -230,15 +231,15 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   }
 
   @Override
-  public GETATTR3Response getattr(XDR xdr, SecurityHandler securityHandler,
-      InetAddress client) {
+  public GETATTR3Response getattr(XDR xdr, RpcInfo info) {
     GETATTR3Response response = new GETATTR3Response(Nfs3Status.NFS3_OK);
     
-    if (!checkAccessPrivilege(client, AccessPrivilege.READ_ONLY)) {
+    if (!checkAccessPrivilege(info, AccessPrivilege.READ_ONLY)) {
       response.setStatus(Nfs3Status.NFS3ERR_ACCES);
       return response;
     }
     
+    SecurityHandler securityHandler = getSecurityHandler(info);
     DFSClient dfsClient = clientCache.getDfsClient(securityHandler.getUser());
     if (dfsClient == null) {
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
@@ -322,9 +323,9 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   }
 
   @Override
-  public SETATTR3Response setattr(XDR xdr, SecurityHandler securityHandler,
-      InetAddress client) {
+  public SETATTR3Response setattr(XDR xdr, RpcInfo info) {
     SETATTR3Response response = new SETATTR3Response(Nfs3Status.NFS3_OK);
+    SecurityHandler securityHandler = getSecurityHandler(info);
     DFSClient dfsClient = clientCache.getDfsClient(securityHandler.getUser());
     if (dfsClient == null) {
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
@@ -370,7 +371,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       }
       
       // check the write access privilege
-      if (!checkAccessPrivilege(client, AccessPrivilege.READ_WRITE)) {
+      if (!checkAccessPrivilege(info, AccessPrivilege.READ_WRITE)) {
         return new SETATTR3Response(Nfs3Status.NFS3ERR_ACCES, new WccData(
             preOpWcc, preOpAttr));
       }
@@ -398,15 +399,15 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   }
 
   @Override
-  public LOOKUP3Response lookup(XDR xdr, SecurityHandler securityHandler,
-      InetAddress client) {
+  public LOOKUP3Response lookup(XDR xdr, RpcInfo info) {
     LOOKUP3Response response = new LOOKUP3Response(Nfs3Status.NFS3_OK);
     
-    if (!checkAccessPrivilege(client, AccessPrivilege.READ_ONLY)) {
+    if (!checkAccessPrivilege(info, AccessPrivilege.READ_ONLY)) {
       response.setStatus(Nfs3Status.NFS3ERR_ACCES);
       return response;
     }
     
+    SecurityHandler securityHandler = getSecurityHandler(info);
     DFSClient dfsClient = clientCache.getDfsClient(securityHandler.getUser());
     if (dfsClient == null) {
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
@@ -460,15 +461,15 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   }
   
   @Override
-  public ACCESS3Response access(XDR xdr, SecurityHandler securityHandler,
-      InetAddress client) {
+  public ACCESS3Response access(XDR xdr, RpcInfo info) {
     ACCESS3Response response = new ACCESS3Response(Nfs3Status.NFS3_OK);
     
-    if (!checkAccessPrivilege(client, AccessPrivilege.READ_ONLY)) {
+    if (!checkAccessPrivilege(info, AccessPrivilege.READ_ONLY)) {
       response.setStatus(Nfs3Status.NFS3ERR_ACCES);
       return response;
     }
     
+    SecurityHandler securityHandler = getSecurityHandler(info);
     DFSClient dfsClient = clientCache.getDfsClient(securityHandler.getUser());
     if (dfsClient == null) {
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
@@ -519,15 +520,16 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
     }
   }
 
-  public READLINK3Response readlink(XDR xdr, SecurityHandler securityHandler,
-      InetAddress client) {
+  @Override
+  public READLINK3Response readlink(XDR xdr, RpcInfo info) {
     READLINK3Response response = new READLINK3Response(Nfs3Status.NFS3_OK);
 
-    if (!checkAccessPrivilege(client, AccessPrivilege.READ_ONLY)) {
+    if (!checkAccessPrivilege(info, AccessPrivilege.READ_ONLY)) {
       response.setStatus(Nfs3Status.NFS3ERR_ACCES);
       return response;
     }
 
+    SecurityHandler securityHandler = getSecurityHandler(info);
     DFSClient dfsClient = clientCache.getDfsClient(securityHandler.getUser());
     if (dfsClient == null) {
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
@@ -591,12 +593,19 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   }
 
   @Override
-  public READ3Response read(XDR xdr, SecurityHandler securityHandler,
-      InetAddress client) {
+  public READ3Response read(XDR xdr, RpcInfo info) {
+    SecurityHandler securityHandler = getSecurityHandler(info);
+    SocketAddress remoteAddress = info.remoteAddress();
+    return read(xdr, securityHandler, remoteAddress);
+  }
+  
+  @VisibleForTesting
+  READ3Response read(XDR xdr, SecurityHandler securityHandler,
+      SocketAddress remoteAddress) {
     READ3Response response = new READ3Response(Nfs3Status.NFS3_OK);
     final String userName = securityHandler.getUser();
     
-    if (!checkAccessPrivilege(client, AccessPrivilege.READ_ONLY)) {
+    if (!checkAccessPrivilege(remoteAddress, AccessPrivilege.READ_ONLY)) {
       response.setStatus(Nfs3Status.NFS3ERR_ACCES);
       return response;
     }
@@ -715,8 +724,17 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   }
 
   @Override
-  public WRITE3Response write(XDR xdr, Channel channel, int xid,
-      SecurityHandler securityHandler, InetAddress client) {
+  public WRITE3Response write(XDR xdr, RpcInfo info) {
+    SecurityHandler securityHandler = getSecurityHandler(info);
+    RpcCall rpcCall = (RpcCall) info.header();
+    int xid = rpcCall.getXid();
+    SocketAddress remoteAddress = info.remoteAddress();
+    return write(xdr, info.channel(), xid, securityHandler, remoteAddress);
+  }
+  
+  @VisibleForTesting
+  WRITE3Response write(XDR xdr, Channel channel, int xid,
+      SecurityHandler securityHandler, SocketAddress remoteAddress) {
     WRITE3Response response = new WRITE3Response(Nfs3Status.NFS3_OK);
 
     DFSClient dfsClient = clientCache.getDfsClient(securityHandler.getUser());
@@ -758,7 +776,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
         return new WRITE3Response(Nfs3Status.NFS3ERR_STALE);
       }
       
-      if (!checkAccessPrivilege(client, AccessPrivilege.READ_WRITE)) {
+      if (!checkAccessPrivilege(remoteAddress, AccessPrivilege.READ_WRITE)) {
         return new WRITE3Response(Nfs3Status.NFS3ERR_ACCES, new WccData(
             Nfs3Utils.getWccAttr(preOpAttr), preOpAttr), 0, stableHow,
             Nfs3Constant.WRITE_COMMIT_VERF);
@@ -791,8 +809,15 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   }
 
   @Override
-  public CREATE3Response create(XDR xdr, SecurityHandler securityHandler,
-      InetAddress client) {
+  public CREATE3Response create(XDR xdr, RpcInfo info) {
+    SecurityHandler securityHandler = getSecurityHandler(info);
+    SocketAddress remoteAddress = info.remoteAddress();
+    return create(xdr, securityHandler, remoteAddress);
+  }
+  
+  @VisibleForTesting
+  CREATE3Response create(XDR xdr, SecurityHandler securityHandler,
+      SocketAddress remoteAddress) {
     CREATE3Response response = new CREATE3Response(Nfs3Status.NFS3_OK);
     DFSClient dfsClient = clientCache.getDfsClient(securityHandler.getUser());
     if (dfsClient == null) {
@@ -838,7 +863,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
         return new CREATE3Response(Nfs3Status.NFS3ERR_STALE);
       }
       
-      if (!checkAccessPrivilege(client, AccessPrivilege.READ_WRITE)) {
+      if (!checkAccessPrivilege(remoteAddress, AccessPrivilege.READ_WRITE)) {
         return new CREATE3Response(Nfs3Status.NFS3ERR_ACCES, null,
             preOpDirAttr, new WccData(Nfs3Utils.getWccAttr(preOpDirAttr),
                 preOpDirAttr));
@@ -922,9 +947,9 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   }
 
   @Override
-  public MKDIR3Response mkdir(XDR xdr, SecurityHandler securityHandler,
-      InetAddress client) {
+  public MKDIR3Response mkdir(XDR xdr, RpcInfo info) {
     MKDIR3Response response = new MKDIR3Response(Nfs3Status.NFS3_OK);
+    SecurityHandler securityHandler = getSecurityHandler(info);
     DFSClient dfsClient = clientCache.getDfsClient(securityHandler.getUser());
     if (dfsClient == null) {
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
@@ -960,7 +985,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
         return new MKDIR3Response(Nfs3Status.NFS3ERR_STALE);
       }
 
-      if (!checkAccessPrivilege(client, AccessPrivilege.READ_WRITE)) {
+      if (!checkAccessPrivilege(info, AccessPrivilege.READ_WRITE)) {
         return new MKDIR3Response(Nfs3Status.NFS3ERR_ACCES, null, preOpDirAttr,
             new WccData(Nfs3Utils.getWccAttr(preOpDirAttr), preOpDirAttr));
       }
@@ -1012,15 +1037,15 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
     }
   }
 
-  public READDIR3Response mknod(XDR xdr,
-      SecurityHandler securityHandler, InetAddress client) {
+  @Override
+  public READDIR3Response mknod(XDR xdr, RpcInfo info) {
     return new READDIR3Response(Nfs3Status.NFS3ERR_NOTSUPP);
   }
   
   @Override
-  public REMOVE3Response remove(XDR xdr,
-      SecurityHandler securityHandler, InetAddress client) {
+  public REMOVE3Response remove(XDR xdr, RpcInfo info) {
     REMOVE3Response response = new REMOVE3Response(Nfs3Status.NFS3_OK);
+    SecurityHandler securityHandler = getSecurityHandler(info);
     DFSClient dfsClient = clientCache.getDfsClient(securityHandler.getUser());
     if (dfsClient == null) {
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
@@ -1093,9 +1118,9 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   }
 
   @Override
-  public RMDIR3Response rmdir(XDR xdr, SecurityHandler securityHandler,
-      InetAddress client) {
+  public RMDIR3Response rmdir(XDR xdr, RpcInfo info) {
     RMDIR3Response response = new RMDIR3Response(Nfs3Status.NFS3_OK);
+    SecurityHandler securityHandler = getSecurityHandler(info);
     DFSClient dfsClient = clientCache.getDfsClient(securityHandler.getUser());
     if (dfsClient == null) {
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
@@ -1129,7 +1154,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       
       WccData errWcc = new WccData(Nfs3Utils.getWccAttr(preOpDirAttr),
           preOpDirAttr);
-      if (!checkAccessPrivilege(client, AccessPrivilege.READ_WRITE)) {
+      if (!checkAccessPrivilege(info, AccessPrivilege.READ_WRITE)) {
         return new RMDIR3Response(Nfs3Status.NFS3ERR_ACCES, errWcc); 
       }
 
@@ -1175,9 +1200,9 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   }
 
   @Override
-  public RENAME3Response rename(XDR xdr, SecurityHandler securityHandler,
-      InetAddress client) {
+  public RENAME3Response rename(XDR xdr, RpcInfo info) {
     RENAME3Response response = new RENAME3Response(Nfs3Status.NFS3_OK);
+    SecurityHandler securityHandler = getSecurityHandler(info);
     DFSClient dfsClient = clientCache.getDfsClient(securityHandler.getUser());
     if (dfsClient == null) {
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
@@ -1221,7 +1246,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
         return new RENAME3Response(Nfs3Status.NFS3ERR_STALE);
       }
       
-      if (!checkAccessPrivilege(client, AccessPrivilege.READ_WRITE)) {
+      if (!checkAccessPrivilege(info, AccessPrivilege.READ_WRITE)) {
         WccData fromWcc = new WccData(Nfs3Utils.getWccAttr(fromPreOpAttr),
             fromPreOpAttr);
         WccData toWcc = new WccData(Nfs3Utils.getWccAttr(toPreOpAttr),
@@ -1263,15 +1288,15 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   }
 
   @Override
-  public SYMLINK3Response symlink(XDR xdr, SecurityHandler securityHandler,
-      InetAddress client) {
+  public SYMLINK3Response symlink(XDR xdr, RpcInfo info) {
     SYMLINK3Response response = new SYMLINK3Response(Nfs3Status.NFS3_OK);
 
-    if (!checkAccessPrivilege(client, AccessPrivilege.READ_WRITE)) {
+    if (!checkAccessPrivilege(info, AccessPrivilege.READ_WRITE)) {
       response.setStatus(Nfs3Status.NFS3ERR_ACCES);
       return response;
     }
 
+    SecurityHandler securityHandler = getSecurityHandler(info);
     DFSClient dfsClient = clientCache.getDfsClient(securityHandler.getUser());
     if (dfsClient == null) {
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
@@ -1322,8 +1347,8 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
     }
   }
 
-  public READDIR3Response link(XDR xdr, SecurityHandler securityHandler,
-      InetAddress client) {
+  @Override
+  public READDIR3Response link(XDR xdr, RpcInfo info) {
     return new READDIR3Response(Nfs3Status.NFS3ERR_NOTSUPP);
   }
 
@@ -1351,11 +1376,16 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   }
   
   @Override
+  public READDIR3Response readdir(XDR xdr, RpcInfo info) {
+    SecurityHandler securityHandler = getSecurityHandler(info);
+    SocketAddress remoteAddress = info.remoteAddress();
+    return readdir(xdr, securityHandler, remoteAddress);
+  }
   public READDIR3Response readdir(XDR xdr, SecurityHandler securityHandler,
-      InetAddress client) {
+      SocketAddress remoteAddress) {
     READDIR3Response response = new READDIR3Response(Nfs3Status.NFS3_OK);
     
-    if (!checkAccessPrivilege(client, AccessPrivilege.READ_ONLY)) {
+    if (!checkAccessPrivilege(remoteAddress, AccessPrivilege.READ_ONLY)) {
       response.setStatus(Nfs3Status.NFS3ERR_ACCES);
       return response;
     }
@@ -1491,9 +1521,17 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
         dirStatus.getModificationTime(), dirList);
   }
 
-  public READDIRPLUS3Response readdirplus(XDR xdr,
-      SecurityHandler securityHandler, InetAddress client) {
-    if (!checkAccessPrivilege(client, AccessPrivilege.READ_ONLY)) {
+  @Override
+  public READDIRPLUS3Response readdirplus(XDR xdr, RpcInfo info) {
+    SecurityHandler securityHandler = getSecurityHandler(info);
+    SocketAddress remoteAddress = info.remoteAddress();
+    return readdirplus(xdr, securityHandler, remoteAddress);
+  }
+
+  @VisibleForTesting
+  READDIRPLUS3Response readdirplus(XDR xdr, SecurityHandler securityHandler,
+      SocketAddress remoteAddress) {
+    if (!checkAccessPrivilege(remoteAddress, AccessPrivilege.READ_ONLY)) {
       return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_ACCES);
     }
     
@@ -1643,15 +1681,15 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   }
   
   @Override
-  public FSSTAT3Response fsstat(XDR xdr, SecurityHandler securityHandler,
-      InetAddress client) {
+  public FSSTAT3Response fsstat(XDR xdr, RpcInfo info) {
     FSSTAT3Response response = new FSSTAT3Response(Nfs3Status.NFS3_OK);
     
-    if (!checkAccessPrivilege(client, AccessPrivilege.READ_ONLY)) {
+    if (!checkAccessPrivilege(info, AccessPrivilege.READ_ONLY)) {
       response.setStatus(Nfs3Status.NFS3ERR_ACCES);
       return response;
     }
     
+    SecurityHandler securityHandler = getSecurityHandler(info);
     DFSClient dfsClient = clientCache.getDfsClient(securityHandler.getUser());
     if (dfsClient == null) {
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
@@ -1711,15 +1749,15 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   }
 
   @Override
-  public FSINFO3Response fsinfo(XDR xdr, SecurityHandler securityHandler,
-      InetAddress client) {
+  public FSINFO3Response fsinfo(XDR xdr, RpcInfo info) {
     FSINFO3Response response = new FSINFO3Response(Nfs3Status.NFS3_OK);
     
-    if (!checkAccessPrivilege(client, AccessPrivilege.READ_ONLY)) {
+    if (!checkAccessPrivilege(info, AccessPrivilege.READ_ONLY)) {
       response.setStatus(Nfs3Status.NFS3ERR_ACCES);
       return response;
     }
     
+    SecurityHandler securityHandler = getSecurityHandler(info);
     DFSClient dfsClient = clientCache.getDfsClient(securityHandler.getUser());
     if (dfsClient == null) {
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
@@ -1769,15 +1807,15 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   }
 
   @Override
-  public PATHCONF3Response pathconf(XDR xdr, SecurityHandler securityHandler,
-      InetAddress client) {
+  public PATHCONF3Response pathconf(XDR xdr, RpcInfo info) {
     PATHCONF3Response response = new PATHCONF3Response(Nfs3Status.NFS3_OK);
     
-    if (!checkAccessPrivilege(client, AccessPrivilege.READ_ONLY)) {
+    if (!checkAccessPrivilege(info, AccessPrivilege.READ_ONLY)) {
       response.setStatus(Nfs3Status.NFS3ERR_ACCES);
       return response;
     }
     
+    SecurityHandler securityHandler = getSecurityHandler(info);
     DFSClient dfsClient = clientCache.getDfsClient(securityHandler.getUser());
     if (dfsClient == null) {
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
@@ -1816,9 +1854,11 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
   }
 
   @Override
-  public COMMIT3Response commit(XDR xdr, Channel channel, int xid,
-      SecurityHandler securityHandler, InetAddress client) {
+  public COMMIT3Response commit(XDR xdr, RpcInfo info) {
+    //Channel channel, int xid,
+    //    SecurityHandler securityHandler, InetAddress client) {
     COMMIT3Response response = new COMMIT3Response(Nfs3Status.NFS3_OK);
+    SecurityHandler securityHandler = getSecurityHandler(info);
     DFSClient dfsClient = clientCache.getDfsClient(securityHandler.getUser());
     if (dfsClient == null) {
       response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
@@ -1849,7 +1889,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
         return new COMMIT3Response(Nfs3Status.NFS3ERR_STALE);
       }
       
-      if (!checkAccessPrivilege(client, AccessPrivilege.READ_WRITE)) {
+      if (!checkAccessPrivilege(info, AccessPrivilege.READ_WRITE)) {
         return new COMMIT3Response(Nfs3Status.NFS3ERR_ACCES, new WccData(
             Nfs3Utils.getWccAttr(preOpAttr), preOpAttr),
             Nfs3Constant.WRITE_COMMIT_VERF);
@@ -1859,8 +1899,10 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
           : (request.getOffset() + request.getCount());
       
       // Insert commit as an async request
-      writeManager.handleCommit(dfsClient, handle, commitOffset, channel, xid,
-          preOpAttr);
+      RpcCall rpcCall = (RpcCall) info.header();
+      int xid = rpcCall.getXid();
+      writeManager.handleCommit(dfsClient, handle, commitOffset,
+          info.channel(), xid, preOpAttr);
       return null;
     } catch (IOException e) {
       LOG.warn("Exception ", e);
@@ -1885,11 +1927,16 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       return null;
     }
   }
+
+  private SecurityHandler getSecurityHandler(RpcInfo info) {
+    RpcCall rpcCall = (RpcCall) info.header();
+    return getSecurityHandler(rpcCall.getCredential(), rpcCall.getVerifier());
+  }
   
   @Override
   public void handleInternal(ChannelHandlerContext ctx, RpcInfo info) {
     RpcCall rpcCall = (RpcCall) info.header();
-    final NFSPROC3 nfsproc3 = NFSPROC3.fromValue(rpcCall.getProcedure());
+    final NFSPROC3 nfsproc3 = NFSPROC3.fromValue(rpcCall.getProcedure());    
     int xid = rpcCall.getXid();
     byte[] data = new byte[info.data().readableBytes()];
     info.data().readBytes(data);
@@ -1897,9 +1944,8 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
     XDR out = new XDR();
     InetAddress client = ((InetSocketAddress) info.remoteAddress())
         .getAddress();
-    Channel channel = info.channel();
-
     Credentials credentials = rpcCall.getCredential();
+    
     // Ignore auth only for NFSPROC3_NULL, especially for Linux clients.
     if (nfsproc3 != NFSPROC3.NULL) {
       if (credentials.getFlavor() != AuthFlavor.AUTH_SYS
@@ -1937,27 +1983,24 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       }
     }
     
-    SecurityHandler securityHandler = getSecurityHandler(credentials,
-        rpcCall.getVerifier());
-    
     NFS3Response response = null;
     if (nfsproc3 == NFSPROC3.NULL) {
       response = nullProcedure();
     } else if (nfsproc3 == NFSPROC3.GETATTR) {
-      response = getattr(xdr, securityHandler, client);
+      response = getattr(xdr, info);
     } else if (nfsproc3 == NFSPROC3.SETATTR) {
-      response = setattr(xdr, securityHandler, client);
+      response = setattr(xdr, info);
     } else if (nfsproc3 == NFSPROC3.LOOKUP) {
-      response = lookup(xdr, securityHandler, client);
+      response = lookup(xdr, info);
     } else if (nfsproc3 == NFSPROC3.ACCESS) {
-      response = access(xdr, securityHandler, client);
+      response = access(xdr, info);
     } else if (nfsproc3 == NFSPROC3.READLINK) {
-      response = readlink(xdr, securityHandler, client);
+      response = readlink(xdr, info);
     } else if (nfsproc3 == NFSPROC3.READ) {
       if (LOG.isDebugEnabled()) {
           LOG.debug(Nfs3Utils.READ_RPC_START + xid);
       }    
-      response = read(xdr, securityHandler, client);
+      response = read(xdr, info);
       if (LOG.isDebugEnabled() && (nfsproc3 == NFSPROC3.READ)) {
         LOG.debug(Nfs3Utils.READ_RPC_END + xid);
       }
@@ -1965,36 +2008,36 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       if (LOG.isDebugEnabled()) {
           LOG.debug(Nfs3Utils.WRITE_RPC_START + xid);
       }
-      response = write(xdr, channel, xid, securityHandler, client);
+      response = write(xdr, info);
       // Write end debug trace is in Nfs3Utils.writeChannel
     } else if (nfsproc3 == NFSPROC3.CREATE) {
-      response = create(xdr, securityHandler, client);
+      response = create(xdr, info);
     } else if (nfsproc3 == NFSPROC3.MKDIR) {      
-      response = mkdir(xdr, securityHandler, client);
+      response = mkdir(xdr, info);
     } else if (nfsproc3 == NFSPROC3.SYMLINK) {
-      response = symlink(xdr, securityHandler, client);
+      response = symlink(xdr, info);
     } else if (nfsproc3 == NFSPROC3.MKNOD) {
-      response = mknod(xdr, securityHandler, client);
+      response = mknod(xdr, info);
     } else if (nfsproc3 == NFSPROC3.REMOVE) {
-      response = remove(xdr, securityHandler, client);
+      response = remove(xdr, info);
     } else if (nfsproc3 == NFSPROC3.RMDIR) {
-      response = rmdir(xdr, securityHandler, client);
+      response = rmdir(xdr, info);
     } else if (nfsproc3 == NFSPROC3.RENAME) {
-      response = rename(xdr, securityHandler, client);
+      response = rename(xdr, info);
     } else if (nfsproc3 == NFSPROC3.LINK) {
-      response = link(xdr, securityHandler, client);
+      response = link(xdr, info);
     } else if (nfsproc3 == NFSPROC3.READDIR) {
-      response = readdir(xdr, securityHandler, client);
+      response = readdir(xdr, info);
     } else if (nfsproc3 == NFSPROC3.READDIRPLUS) {
-      response = readdirplus(xdr, securityHandler, client);
+      response = readdirplus(xdr, info);
     } else if (nfsproc3 == NFSPROC3.FSSTAT) {
-      response = fsstat(xdr, securityHandler, client);
+      response = fsstat(xdr, info);
     } else if (nfsproc3 == NFSPROC3.FSINFO) {
-      response = fsinfo(xdr, securityHandler, client);
+      response = fsinfo(xdr, info);
     } else if (nfsproc3 == NFSPROC3.PATHCONF) {
-      response = pathconf(xdr, securityHandler, client);
+      response = pathconf(xdr,info);
     } else if (nfsproc3 == NFSPROC3.COMMIT) {
-      response = commit(xdr, channel, xid, securityHandler, client);
+      response = commit(xdr, info);
     } else {
       // Invalid procedure
       RpcAcceptedReply.getInstance(xid,
@@ -2027,8 +2070,21 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
     return nfsproc3 == null || nfsproc3.isIdempotent();
   }
   
-  private boolean checkAccessPrivilege(final InetAddress client,
+  private boolean checkAccessPrivilege(RpcInfo info,
+      final AccessPrivilege expected) {
+    SocketAddress remoteAddress = info.remoteAddress();
+    return checkAccessPrivilege(remoteAddress, expected);
+  }
+
+  private boolean checkAccessPrivilege(SocketAddress remoteAddress,
       final AccessPrivilege expected) {
+    // Port monitoring
+    if (!doPortMonitoring(remoteAddress)) {
+      return false;
+    }
+    
+    // Check export table
+    InetAddress client = ((InetSocketAddress) remoteAddress).getAddress();
     AccessPrivilege access = exports.getAccessPrivilege(client);
     if (access == AccessPrivilege.NONE) {
       return false;

+ 16 - 8
hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/TestReaddir.java → hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestReaddir.java

@@ -16,12 +16,14 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hdfs.nfs;
+package org.apache.hadoop.hdfs.nfs.nfs3;
 
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
 import java.util.List;
 
 import org.apache.hadoop.fs.Path;
@@ -38,10 +40,15 @@ import org.apache.hadoop.nfs.nfs3.response.READDIR3Response;
 import org.apache.hadoop.nfs.nfs3.response.READDIR3Response.Entry3;
 import org.apache.hadoop.nfs.nfs3.response.READDIRPLUS3Response;
 import org.apache.hadoop.nfs.nfs3.response.READDIRPLUS3Response.EntryPlus3;
+import org.apache.hadoop.oncrpc.RpcInfo;
+import org.apache.hadoop.oncrpc.RpcMessage;
 import org.apache.hadoop.oncrpc.XDR;
 import org.apache.hadoop.oncrpc.security.SecurityHandler;
 import org.apache.hadoop.security.authorize.DefaultImpersonationProvider;
 import org.apache.hadoop.security.authorize.ProxyUsers;
+import org.jboss.netty.buffer.ChannelBuffer;
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelHandlerContext;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -122,7 +129,7 @@ public class TestReaddir {
     xdr_req.writeInt(100); // count
 
     READDIR3Response response = nfsd.readdir(xdr_req.asReadOnlyWrap(),
-        securityHandler, InetAddress.getLocalHost());
+        securityHandler, new InetSocketAddress("localhost", 1234));
     List<Entry3> dirents = response.getDirList().getEntries();
     assertTrue(dirents.size() == 5); // inculding dot, dotdot
 
@@ -139,7 +146,7 @@ public class TestReaddir {
     xdr_req.writeInt(100); // count
 
     response = nfsd.readdir(xdr_req.asReadOnlyWrap(), securityHandler,
-        InetAddress.getLocalHost());
+        new InetSocketAddress("localhost", 1234));
     dirents = response.getDirList().getEntries();
     assertTrue(dirents.size() == 1);
     Entry3 entry = dirents.get(0);
@@ -149,7 +156,7 @@ public class TestReaddir {
     hdfs.delete(new Path(testdir + "/f2"), false);
 
     response = nfsd.readdir(xdr_req.asReadOnlyWrap(), securityHandler,
-        InetAddress.getLocalHost());
+        new InetSocketAddress("localhost", 1234));
     dirents = response.getDirList().getEntries();
     assertTrue(dirents.size() == 2); // No dot, dotdot
   }
@@ -170,8 +177,9 @@ public class TestReaddir {
     xdr_req.writeInt(100); // dirCount
     xdr_req.writeInt(1000); // maxCount
 
-    READDIRPLUS3Response responsePlus = nfsd.readdirplus(
-        xdr_req.asReadOnlyWrap(), securityHandler, InetAddress.getLocalHost());
+    READDIRPLUS3Response responsePlus = nfsd.readdirplus(xdr_req
+        .asReadOnlyWrap(), securityHandler, new InetSocketAddress("localhost",
+        1234));
     List<EntryPlus3> direntPlus = responsePlus.getDirListPlus().getEntries();
     assertTrue(direntPlus.size() == 5); // including dot, dotdot
 
@@ -189,7 +197,7 @@ public class TestReaddir {
     xdr_req.writeInt(1000); // maxCount
 
     responsePlus = nfsd.readdirplus(xdr_req.asReadOnlyWrap(), securityHandler,
-        InetAddress.getLocalHost());
+        new InetSocketAddress("localhost", 1234));
     direntPlus = responsePlus.getDirListPlus().getEntries();
     assertTrue(direntPlus.size() == 1);
     EntryPlus3 entryPlus = direntPlus.get(0);
@@ -199,7 +207,7 @@ public class TestReaddir {
     hdfs.delete(new Path(testdir + "/f2"), false);
 
     responsePlus = nfsd.readdirplus(xdr_req.asReadOnlyWrap(), securityHandler,
-        InetAddress.getLocalHost());
+        new InetSocketAddress("localhost", 1234));
     direntPlus = responsePlus.getDirListPlus().getEntries();
     assertTrue(direntPlus.size() == 2); // No dot, dotdot
   }

+ 7 - 7
hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestWrites.java

@@ -22,7 +22,7 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
-import java.net.InetAddress;
+import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.concurrent.ConcurrentNavigableMap;
@@ -318,7 +318,7 @@ public class TestWrites {
       XDR createXdr = new XDR();
       createReq.serialize(createXdr);
       CREATE3Response createRsp = nfsd.create(createXdr.asReadOnlyWrap(),
-          securityHandler, InetAddress.getLocalHost());
+          securityHandler, new InetSocketAddress("localhost", 1234));
       FileHandle handle = createRsp.getObjHandle();
 
       // Test DATA_SYNC
@@ -331,7 +331,7 @@ public class TestWrites {
       XDR writeXdr = new XDR();
       writeReq.serialize(writeXdr);
       nfsd.write(writeXdr.asReadOnlyWrap(), null, 1, securityHandler,
-          InetAddress.getLocalHost());
+          new InetSocketAddress("localhost", 1234));
 
       waitWrite(nfsd, handle, 60000);
 
@@ -340,7 +340,7 @@ public class TestWrites {
       XDR readXdr = new XDR();
       readReq.serialize(readXdr);
       READ3Response readRsp = nfsd.read(readXdr.asReadOnlyWrap(),
-          securityHandler, InetAddress.getLocalHost());
+          securityHandler, new InetSocketAddress("localhost", 1234));
 
       assertTrue(Arrays.equals(buffer, readRsp.getData().array()));
 
@@ -352,7 +352,7 @@ public class TestWrites {
       XDR createXdr2 = new XDR();
       createReq2.serialize(createXdr2);
       CREATE3Response createRsp2 = nfsd.create(createXdr2.asReadOnlyWrap(),
-          securityHandler, InetAddress.getLocalHost());
+          securityHandler, new InetSocketAddress("localhost", 1234));
       FileHandle handle2 = createRsp2.getObjHandle();
 
       WRITE3Request writeReq2 = new WRITE3Request(handle2, 0, 10,
@@ -360,7 +360,7 @@ public class TestWrites {
       XDR writeXdr2 = new XDR();
       writeReq2.serialize(writeXdr2);
       nfsd.write(writeXdr2.asReadOnlyWrap(), null, 1, securityHandler,
-          InetAddress.getLocalHost());
+          new InetSocketAddress("localhost", 1234));
 
       waitWrite(nfsd, handle2, 60000);
 
@@ -369,7 +369,7 @@ public class TestWrites {
       XDR readXdr2 = new XDR();
       readReq2.serialize(readXdr2);
       READ3Response readRsp2 = nfsd.read(readXdr2.asReadOnlyWrap(),
-          securityHandler, InetAddress.getLocalHost());
+          securityHandler, new InetSocketAddress("localhost", 1234));
 
       assertTrue(Arrays.equals(buffer, readRsp2.getData().array()));
       // FILE_SYNC should sync the file size

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

@@ -443,6 +443,17 @@ Release 2.5.0 - UNRELEASED
     HDFS-6499. Use NativeIO#renameTo instead of File#renameTo in
     FileJournalManager. (Yongjun Zhang via atm)
 
+    HDFS-6518. TestCacheDirectives#testExceedsCapacity should
+    take FSN read lock when accessing pendingCached list.
+    (wang)
+
+    HDFS-6528. Add XAttrs to TestOfflineImageViewer. (Stephen Chu via wang)
+
+    HDFS-6545. Finalizing rolling upgrade can make NN unavailable for a long
+    duration. (kihwal)
+
+    HDFS-6530. Fix Balancer documentation.  (szetszwo)
+
   OPTIMIZATIONS
 
     HDFS-6214. Webhdfs has poor throughput for files >2GB (daryn)
@@ -636,6 +647,24 @@ Release 2.5.0 - UNRELEASED
     HDFS-6375. Listing extended attributes with the search permission.
     (Charles Lamb via wang)
 
+    HDFS-6539. test_native_mini_dfs is skipped in hadoop-hdfs/pom.xml
+    (decstery via cmccabe)
+
+    HDFS-6527. Edit log corruption due to defered INode removal. (kihwal and
+    jing9 via jing9)
+
+    HDFS-6552. add DN storage to a BlockInfo will not replace the different
+    storage from same DN. (Amir Langer via Arpit Agarwal)
+
+    HDFS-6551. Rename with OVERWRITE option may throw NPE when the target
+    file/directory is a reference INode. (jing9)
+
+    HDFS-6439. NFS should not reject NFS requests to the NULL procedure whether
+    port monitoring is enabled or not. (brandonli)
+
+    HDFS-6559. Fix wrong option "dfsadmin -rollingUpgrade start" in the
+    document. (Akira Ajisaka via Arpit Agarwal)
+
   BREAKDOWN OF HDFS-2006 SUBTASKS AND RELATED JIRAS
 
     HDFS-6299. Protobuf for XAttr and client-side implementation. (Yi Liu via umamahesh)

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/pom.xml

@@ -409,7 +409,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
                     </exec>
                     <exec executable="sh" failonerror="true" dir="${project.build.directory}/native/">
                         <arg value="-c"/>
-                        <arg value="[ x$SKIPTESTS = xtrue ] || ${project.build.directory}/native/test_libhdfs_threaded"/>
+                        <arg value="[ x$SKIPTESTS = xtrue ] || ${project.build.directory}/native/test_native_mini_dfs"/>
                       <env key="CLASSPATH" value="${test_classpath}:${compile_classpath}"/>
                       <env key="SKIPTESTS" value="${skipTests}"/>
                     </exec>

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java

@@ -203,7 +203,7 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
       } else {
         // The block is on the DN but belongs to a different storage.
         // Update our state.
-        removeStorage(storage);
+        removeStorage(getStorageInfo(idx));
         added = false;      // Just updating storage. Return false.
       }
     }

+ 11 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -45,7 +45,6 @@ import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
-import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -899,9 +898,10 @@ public class FSDirectory implements Closeable {
     
     boolean undoRemoveDst = false;
     INode removedDst = null;
+    long removedNum = 0;
     try {
       if (dstInode != null) { // dst exists remove it
-        if (removeLastINode(dstIIP) != -1) {
+        if ((removedNum = removeLastINode(dstIIP)) != -1) {
           removedDst = dstIIP.getLastINode();
           undoRemoveDst = true;
         }
@@ -941,13 +941,15 @@ public class FSDirectory implements Closeable {
         long filesDeleted = -1;
         if (removedDst != null) {
           undoRemoveDst = false;
-          BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
-          List<INode> removedINodes = new ChunkedArrayList<INode>();
-          filesDeleted = removedDst.cleanSubtree(Snapshot.CURRENT_STATE_ID,
-              dstIIP.getLatestSnapshotId(), collectedBlocks, removedINodes, true)
-              .get(Quota.NAMESPACE);
-          getFSNamesystem().removePathAndBlocks(src, collectedBlocks,
-              removedINodes);
+          if (removedNum > 0) {
+            BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
+            List<INode> removedINodes = new ChunkedArrayList<INode>();
+            filesDeleted = removedDst.cleanSubtree(Snapshot.CURRENT_STATE_ID,
+                dstIIP.getLatestSnapshotId(), collectedBlocks, removedINodes,
+                true).get(Quota.NAMESPACE);
+            getFSNamesystem().removePathAndBlocks(src, collectedBlocks,
+                removedINodes);
+          }
         }
 
         if (snapshottableDirs.size() > 0) {

+ 16 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -3073,6 +3073,13 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
           + (lease != null ? lease.toString()
               : "Holder " + holder + " does not have any open files."));
     }
+    // No further modification is allowed on a deleted file.
+    // A file is considered deleted, if it has no parent or is marked
+    // as deleted in the snapshot feature.
+    if (file.getParent() == null || (file.isWithSnapshot() &&
+        file.getFileWithSnapshotFeature().isCurrentFileDeleted())) {
+      throw new FileNotFoundException(src);
+    }
     String clientName = file.getFileUnderConstructionFeature().getClientName();
     if (holder != null && !clientName.equals(holder)) {
       throw new LeaseExpiredException("Lease mismatch on " + ident +
@@ -3507,6 +3514,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     getEditLog().logSync(); 
     removeBlocks(collectedBlocks); // Incremental deletion of blocks
     collectedBlocks.clear();
+
     dir.writeLock();
     try {
       dir.removeFromInodeMap(removedINodes);
@@ -7694,14 +7702,20 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
       returnInfo = finalizeRollingUpgradeInternal(now());
       getEditLog().logFinalizeRollingUpgrade(returnInfo.getFinalizeTime());
-      getFSImage().saveNamespace(this);
+      if (haEnabled) {
+        // roll the edit log to make sure the standby NameNode can tail
+        getFSImage().rollEditLog();
+      }
       getFSImage().renameCheckpoint(NameNodeFile.IMAGE_ROLLBACK,
           NameNodeFile.IMAGE);
     } finally {
       writeUnlock();
     }
 
-    // getEditLog().logSync() is not needed since it does saveNamespace 
+    if (!haEnabled) {
+      // Sync not needed for ha since the edit was rolled after logging.
+      getEditLog().logSync();
+    }
 
     if (auditLog.isInfoEnabled() && isExternalInvocation()) {
       logAuditEvent(true, "finalizeRollingUpgrade", null, null, null);

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/site/apt/Federation.apt.vm

@@ -287,13 +287,14 @@ HDFS Federation
 
   Policy could be:
 
-  * <<<node>>> - this is the <default> policy. This balances the storage at 
+  * <<<datanode>>> - this is the <default> policy. This balances the storage at 
     the datanode level. This is similar to balancing policy from prior releases.
 
   * <<<blockpool>>> - this balances the storage at the block pool level. 
     Balancing at block pool level balances storage at the datanode level also.
 
-  Note that Balander only balances the data and does not balance the namespace.
+  Note that Balancer only balances the data and does not balance the namespace.
+  For the complete command usage, see {{{../hadoop-common/CommandsManual.html#balancer}balancer}}.
 
 ** Decommissioning
 

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

@@ -322,6 +322,22 @@ HDFS NFS Gateway
   Then the users can access HDFS as part of the local file system except that, 
   hard link and random write are not supported yet.
 
+* {Allow mounts from unprivileged clients}
+
+  In environments where root access on client machines is not generally
+  available, some measure of security can be obtained by ensuring that only NFS
+  clients originating from privileged ports can connect to the NFS server. This
+  feature is referred to as "port monitoring." This feature is not enabled by default
+  in the HDFS NFS Gateway, but can be optionally enabled by setting the
+  following config in hdfs-site.xml on the NFS Gateway machine:
+
+-------------------------------------------------------------------
+<property>
+  <name>nfs.port.monitoring.disabled</name>
+  <value>false</value>
+</property>
+-------------------------------------------------------------------
+
 * {User authentication and mapping}
 
   NFS gateway in this release uses AUTH_UNIX style authentication. When the user on NFS client

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HdfsUserGuide.apt.vm

@@ -77,7 +77,7 @@ HDFS Users Guide
           * <<<fetchdt>>>: a utility to fetch DelegationToken and store it in a
             file on the local system.
 
-          * Rebalancer: tool to balance the cluster when the data is
+          * Balancer: tool to balance the cluster when the data is
             unevenly distributed among DataNodes.
 
           * Upgrade and rollback: after a software upgrade, it is possible
@@ -316,7 +316,7 @@ HDFS Users Guide
 
    For command usage, see {{{../hadoop-common/CommandsManual.html#namenode}namenode}}.
 
-* Rebalancer
+* Balancer
 
    HDFS data might not always be be placed uniformly across the DataNode.
    One common reason is addition of new DataNodes to an existing cluster.
@@ -338,7 +338,7 @@ HDFS Users Guide
    Due to multiple competing considerations, data might not be uniformly
    placed across the DataNodes. HDFS provides a tool for administrators
    that analyzes block placement and rebalanaces data across the DataNode.
-   A brief administrator's guide for rebalancer as a PDF is attached to
+   A brief administrator's guide for balancer is available at 
    {{{https://issues.apache.org/jira/browse/HADOOP-1652}HADOOP-1652}}.
 
    For command usage, see {{{../hadoop-common/CommandsManual.html#balancer}balancer}}.

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/site/xdoc/HdfsRollingUpgrade.xml

@@ -217,7 +217,7 @@
     
   <subsection name="DFSAdmin Commands" id="dfsadminCommands">
   <h4><code>dfsadmin -rollingUpgrade</code></h4>
-  <source>hdfs dfsadmin -rollingUpgrade &lt;query|start|finalize&gt;</source>
+  <source>hdfs dfsadmin -rollingUpgrade &lt;query|prepare|finalize&gt;</source>
   <p>
     Execute a rolling upgrade action.
     <ul><li>Options:<table>

+ 162 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java

@@ -34,6 +34,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.*;
+import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
@@ -54,10 +55,16 @@ import org.apache.hadoop.util.ToolRunner;
 import org.junit.Test;
 
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY;
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryType.*;
+import static org.apache.hadoop.fs.permission.FsAction.*;
+import static org.apache.hadoop.hdfs.server.namenode.AclTestHelpers.aclEntry;
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.CoreMatchers.not;
 import static org.junit.Assert.*;
 
+import com.google.common.collect.Lists;
+
 /**
  * This class tests commands from DFSShell.
  */
@@ -1621,11 +1628,13 @@ public class TestDFSShell {
     assertEquals("expected to fail -1", res , -1);
   }
   
-  // Preserve Copy Option is -ptopx (timestamps, ownership, permission, XATTR)
+  // Preserve Copy Option is -ptopxa (timestamps, ownership, permission, XATTR,
+  // ACLs)
   @Test (timeout = 120000)
   public void testCopyCommandsWithPreserveOption() throws Exception {
     Configuration conf = new Configuration();
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_XATTRS_ENABLED_KEY, true);
+    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY, true);
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1)
         .format(true).build();
     FsShell shell = null;
@@ -1638,6 +1647,14 @@ public class TestDFSShell {
       fs.mkdirs(hdfsTestDir);
       Path src = new Path(hdfsTestDir, "srcfile");
       fs.create(src).close();
+
+      fs.setAcl(src, Lists.newArrayList(
+          aclEntry(ACCESS, USER, ALL),
+          aclEntry(ACCESS, USER, "foo", ALL),
+          aclEntry(ACCESS, GROUP, READ_EXECUTE),
+          aclEntry(ACCESS, GROUP, "bar", READ_EXECUTE),
+          aclEntry(ACCESS, OTHER, EXECUTE)));
+
       FileStatus status = fs.getFileStatus(src);
       final long mtime = status.getModificationTime();
       final long atime = status.getAccessTime();
@@ -1661,41 +1678,93 @@ public class TestDFSShell {
       assertEquals(atime, targetStatus.getAccessTime());
       assertEquals(owner, targetStatus.getOwner());
       assertEquals(group, targetStatus.getGroup());
-      assertTrue(perm.equals(targetStatus.getPermission()));
+      FsPermission targetPerm = targetStatus.getPermission();
+      assertTrue(perm.equals(targetPerm));
       Map<String, byte[]> xattrs = fs.getXAttrs(target1);
       assertTrue(xattrs.isEmpty());
+      List<AclEntry> acls = fs.getAclStatus(target1).getEntries();
+      assertTrue(acls.isEmpty());
+      assertFalse(targetPerm.getAclBit());
 
       // -ptop
       Path target2 = new Path(hdfsTestDir, "targetfile2");
       argv = new String[] { "-cp", "-ptop", src.toUri().toString(), 
           target2.toUri().toString() };
       ret = ToolRunner.run(shell, argv);
-      assertEquals("cp -p is not working", SUCCESS, ret);
-      targetStatus = fs.getFileStatus(target1);
+      assertEquals("cp -ptop is not working", SUCCESS, ret);
+      targetStatus = fs.getFileStatus(target2);
       assertEquals(mtime, targetStatus.getModificationTime());
       assertEquals(atime, targetStatus.getAccessTime());
       assertEquals(owner, targetStatus.getOwner());
       assertEquals(group, targetStatus.getGroup());
-      assertTrue(perm.equals(targetStatus.getPermission()));
+      targetPerm = targetStatus.getPermission();
+      assertTrue(perm.equals(targetPerm));
       xattrs = fs.getXAttrs(target2);
       assertTrue(xattrs.isEmpty());
-      
+      acls = fs.getAclStatus(target2).getEntries();
+      assertTrue(acls.isEmpty());
+      assertFalse(targetPerm.getAclBit());
+
       // -ptopx
       Path target3 = new Path(hdfsTestDir, "targetfile3");
       argv = new String[] { "-cp", "-ptopx", src.toUri().toString(), 
           target3.toUri().toString() };
       ret = ToolRunner.run(shell, argv);
-      assertEquals("cp -p is not working", SUCCESS, ret);
-      targetStatus = fs.getFileStatus(target1);
+      assertEquals("cp -ptopx is not working", SUCCESS, ret);
+      targetStatus = fs.getFileStatus(target3);
       assertEquals(mtime, targetStatus.getModificationTime());
       assertEquals(atime, targetStatus.getAccessTime());
       assertEquals(owner, targetStatus.getOwner());
       assertEquals(group, targetStatus.getGroup());
-      assertTrue(perm.equals(targetStatus.getPermission()));
+      targetPerm = targetStatus.getPermission();
+      assertTrue(perm.equals(targetPerm));
       xattrs = fs.getXAttrs(target3);
       assertEquals(xattrs.size(), 2);
       assertArrayEquals(new byte[]{0x31, 0x32, 0x33}, xattrs.get("user.a1"));
       assertArrayEquals(new byte[]{0x31, 0x31, 0x31}, xattrs.get("trusted.a1"));
+      acls = fs.getAclStatus(target3).getEntries();
+      assertTrue(acls.isEmpty());
+      assertFalse(targetPerm.getAclBit());
+
+      // -ptopa
+      Path target4 = new Path(hdfsTestDir, "targetfile4");
+      argv = new String[] { "-cp", "-ptopa", src.toUri().toString(),
+          target4.toUri().toString() };
+      ret = ToolRunner.run(shell, argv);
+      assertEquals("cp -ptopa is not working", SUCCESS, ret);
+      targetStatus = fs.getFileStatus(target4);
+      assertEquals(mtime, targetStatus.getModificationTime());
+      assertEquals(atime, targetStatus.getAccessTime());
+      assertEquals(owner, targetStatus.getOwner());
+      assertEquals(group, targetStatus.getGroup());
+      targetPerm = targetStatus.getPermission();
+      assertTrue(perm.equals(targetPerm));
+      xattrs = fs.getXAttrs(target4);
+      assertTrue(xattrs.isEmpty());
+      acls = fs.getAclStatus(target4).getEntries();
+      assertFalse(acls.isEmpty());
+      assertTrue(targetPerm.getAclBit());
+      assertEquals(fs.getAclStatus(src), fs.getAclStatus(target4));
+
+      // -ptoa (verify -pa option will preserve permissions also)
+      Path target5 = new Path(hdfsTestDir, "targetfile5");
+      argv = new String[] { "-cp", "-ptoa", src.toUri().toString(),
+          target5.toUri().toString() };
+      ret = ToolRunner.run(shell, argv);
+      assertEquals("cp -ptoa is not working", SUCCESS, ret);
+      targetStatus = fs.getFileStatus(target5);
+      assertEquals(mtime, targetStatus.getModificationTime());
+      assertEquals(atime, targetStatus.getAccessTime());
+      assertEquals(owner, targetStatus.getOwner());
+      assertEquals(group, targetStatus.getGroup());
+      targetPerm = targetStatus.getPermission();
+      assertTrue(perm.equals(targetPerm));
+      xattrs = fs.getXAttrs(target5);
+      assertTrue(xattrs.isEmpty());
+      acls = fs.getAclStatus(target5).getEntries();
+      assertFalse(acls.isEmpty());
+      assertTrue(targetPerm.getAclBit());
+      assertEquals(fs.getAclStatus(src), fs.getAclStatus(target5));
     } finally {
       if (null != shell) {
         shell.close();
@@ -1709,6 +1778,90 @@ public class TestDFSShell {
     }
   }
 
+  // Verify cp -pa option will preserve both ACL and sticky bit.
+  @Test (timeout = 120000)
+  public void testCopyCommandsPreserveAclAndStickyBit() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY, true);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1)
+        .format(true).build();
+    FsShell shell = null;
+    FileSystem fs = null;
+    final String testdir =
+        "/tmp/TestDFSShell-testCopyCommandsPreserveAclAndStickyBit-"
+        + counter.getAndIncrement();
+    final Path hdfsTestDir = new Path(testdir);
+    try {
+      fs = cluster.getFileSystem();
+      fs.mkdirs(hdfsTestDir);
+      Path src = new Path(hdfsTestDir, "srcfile");
+      fs.create(src).close();
+
+      fs.setAcl(src, Lists.newArrayList(
+          aclEntry(ACCESS, USER, ALL),
+          aclEntry(ACCESS, USER, "foo", ALL),
+          aclEntry(ACCESS, GROUP, READ_EXECUTE),
+          aclEntry(ACCESS, GROUP, "bar", READ_EXECUTE),
+          aclEntry(ACCESS, OTHER, EXECUTE)));
+      // set sticky bit
+      fs.setPermission(src,
+          new FsPermission(ALL, READ_EXECUTE, EXECUTE, true));
+
+      FileStatus status = fs.getFileStatus(src);
+      final long mtime = status.getModificationTime();
+      final long atime = status.getAccessTime();
+      final String owner = status.getOwner();
+      final String group = status.getGroup();
+      final FsPermission perm = status.getPermission();
+
+      shell = new FsShell(conf);
+
+      // -p preserves sticky bit and doesn't preserve ACL
+      Path target1 = new Path(hdfsTestDir, "targetfile1");
+      String[] argv = new String[] { "-cp", "-p", src.toUri().toString(),
+          target1.toUri().toString() };
+      int ret = ToolRunner.run(shell, argv);
+      assertEquals("cp is not working", SUCCESS, ret);
+      FileStatus targetStatus = fs.getFileStatus(target1);
+      assertEquals(mtime, targetStatus.getModificationTime());
+      assertEquals(atime, targetStatus.getAccessTime());
+      assertEquals(owner, targetStatus.getOwner());
+      assertEquals(group, targetStatus.getGroup());
+      FsPermission targetPerm = targetStatus.getPermission();
+      assertTrue(perm.equals(targetPerm));
+      List<AclEntry> acls = fs.getAclStatus(target1).getEntries();
+      assertTrue(acls.isEmpty());
+      assertFalse(targetPerm.getAclBit());
+
+      // -ptopa preserves both sticky bit and ACL
+      Path target2 = new Path(hdfsTestDir, "targetfile2");
+      argv = new String[] { "-cp", "-ptopa", src.toUri().toString(),
+          target2.toUri().toString() };
+      ret = ToolRunner.run(shell, argv);
+      assertEquals("cp -ptopa is not working", SUCCESS, ret);
+      targetStatus = fs.getFileStatus(target2);
+      assertEquals(mtime, targetStatus.getModificationTime());
+      assertEquals(atime, targetStatus.getAccessTime());
+      assertEquals(owner, targetStatus.getOwner());
+      assertEquals(group, targetStatus.getGroup());
+      targetPerm = targetStatus.getPermission();
+      assertTrue(perm.equals(targetPerm));
+      acls = fs.getAclStatus(target2).getEntries();
+      assertFalse(acls.isEmpty());
+      assertTrue(targetPerm.getAclBit());
+      assertEquals(fs.getAclStatus(src), fs.getAclStatus(target2));
+    } finally {
+      if (null != shell) {
+        shell.close();
+      }
+      if (null != fs) {
+        fs.delete(hdfsTestDir, true);
+        fs.close();
+      }
+      cluster.shutdown();
+    }
+  }
+
   // force Copy Option is -f
   @Test (timeout = 30000)
   public void testCopyCommandsWithForceOption() throws Exception {

+ 30 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java

@@ -29,6 +29,8 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+import org.junit.Assert;
 import org.junit.Test;
 
 /**
@@ -42,6 +44,34 @@ public class TestBlockInfo {
   private static final Log LOG = LogFactory
       .getLog("org.apache.hadoop.hdfs.TestBlockInfo");
 
+
+  @Test
+  public void testAddStorage() throws Exception {
+    BlockInfo blockInfo = new BlockInfo(3);
+
+    final DatanodeStorageInfo storage = DFSTestUtil.createDatanodeStorageInfo("storageID", "127.0.0.1");
+
+    boolean added = blockInfo.addStorage(storage);
+
+    Assert.assertTrue(added);
+    Assert.assertEquals(storage, blockInfo.getStorageInfo(0));
+  }
+
+
+  @Test
+  public void testReplaceStorageIfDifferetnOneAlreadyExistedFromSameDataNode() throws Exception {
+    BlockInfo blockInfo = new BlockInfo(3);
+
+    final DatanodeStorageInfo storage1 = DFSTestUtil.createDatanodeStorageInfo("storageID1", "127.0.0.1");
+    final DatanodeStorageInfo storage2 = new DatanodeStorageInfo(storage1.getDatanodeDescriptor(), new DatanodeStorage("storageID2"));
+
+    blockInfo.addStorage(storage1);
+    boolean added = blockInfo.addStorage(storage2);
+
+    Assert.assertFalse(added);
+    Assert.assertEquals(storage2, blockInfo.getStorageInfo(0));
+  }
+
   @Test
   public void testBlockListMoveToHead() throws Exception {
     LOG.info("BlockInfo moveToHead tests...");

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

@@ -1408,12 +1408,17 @@ public class TestCacheDirectives {
    */
   private void checkPendingCachedEmpty(MiniDFSCluster cluster)
       throws Exception {
-    final DatanodeManager datanodeManager =
-        cluster.getNamesystem().getBlockManager().getDatanodeManager();
-    for (DataNode dn : cluster.getDataNodes()) {
-      DatanodeDescriptor descriptor =
-          datanodeManager.getDatanode(dn.getDatanodeId());
-      Assert.assertTrue(descriptor.getPendingCached().isEmpty());
+    cluster.getNamesystem().readLock();
+    try {
+      final DatanodeManager datanodeManager =
+          cluster.getNamesystem().getBlockManager().getDatanodeManager();
+      for (DataNode dn : cluster.getDataNodes()) {
+        DatanodeDescriptor descriptor =
+            datanodeManager.getDatanode(dn.getDatanodeId());
+        Assert.assertTrue(descriptor.getPendingCached().isEmpty());
+      }
+    } finally {
+      cluster.getNamesystem().readUnlock();
     }
   }
 

+ 149 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeleteRace.java

@@ -0,0 +1,149 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.io.FileNotFoundException;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.StorageType;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
+import org.apache.hadoop.net.Node;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.internal.util.reflection.Whitebox;
+
+
+/**
+ * Test race between delete and other operations.  For now only addBlock()
+ * is tested since all others are acquiring FSNamesystem lock for the 
+ * whole duration.
+ */
+public class TestDeleteRace {
+  private static final Log LOG = LogFactory.getLog(TestDeleteRace.class);
+  private static final Configuration conf = new HdfsConfiguration();
+  private MiniDFSCluster cluster;
+
+  @Test  
+  public void testDeleteAddBlockRace() throws Exception {
+    testDeleteAddBlockRace(false);
+  }
+
+  @Test  
+  public void testDeleteAddBlockRaceWithSnapshot() throws Exception {
+    testDeleteAddBlockRace(true);
+  }
+
+  private void testDeleteAddBlockRace(boolean hasSnapshot) throws Exception {
+    try {
+      conf.setClass(DFSConfigKeys.DFS_BLOCK_REPLICATOR_CLASSNAME_KEY,
+          SlowBlockPlacementPolicy.class, BlockPlacementPolicy.class);
+      cluster = new MiniDFSCluster.Builder(conf).build();
+      FileSystem fs = cluster.getFileSystem();
+      final String fileName = "/testDeleteAddBlockRace";
+      Path filePath = new Path(fileName);
+
+      FSDataOutputStream out = null;
+      out = fs.create(filePath);
+      if (hasSnapshot) {
+        SnapshotTestHelper.createSnapshot((DistributedFileSystem) fs, new Path(
+            "/"), "s1");
+      }
+
+      Thread deleteThread = new DeleteThread(fs, filePath);
+      deleteThread.start();
+
+      try {
+        // write data and syn to make sure a block is allocated.
+        out.write(new byte[32], 0, 32);
+        out.hsync();
+        Assert.fail("Should have failed.");
+      } catch (FileNotFoundException e) {
+        GenericTestUtils.assertExceptionContains(filePath.getName(), e);
+      }
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  private static class SlowBlockPlacementPolicy extends
+      BlockPlacementPolicyDefault {
+    @Override
+    public DatanodeStorageInfo[] chooseTarget(String srcPath,
+                                      int numOfReplicas,
+                                      Node writer,
+                                      List<DatanodeStorageInfo> chosenNodes,
+                                      boolean returnChosenNodes,
+                                      Set<Node> excludedNodes,
+                                      long blocksize,
+                                      StorageType storageType) {
+      DatanodeStorageInfo[] results = super.chooseTarget(srcPath,
+          numOfReplicas, writer, chosenNodes, returnChosenNodes, excludedNodes,
+          blocksize, storageType);
+      try {
+        Thread.sleep(3000);
+      } catch (InterruptedException e) {}
+      return results;
+    }
+  }
+
+  private class DeleteThread extends Thread {
+    private FileSystem fs;
+    private Path path;
+
+    DeleteThread(FileSystem fs, Path path) {
+      this.fs = fs;
+      this.path = path;
+    }
+
+    @Override
+    public void run() {
+      try {
+        Thread.sleep(1000);
+        LOG.info("Deleting" + path);
+        final FSDirectory fsdir = cluster.getNamesystem().dir;
+        INode fileINode = fsdir.getINode4Write(path.toString());
+        INodeMap inodeMap = (INodeMap) Whitebox.getInternalState(fsdir,
+            "inodeMap");
+
+        fs.delete(path, false);
+        // after deletion, add the inode back to the inodeMap
+        inodeMap.put(fileINode);
+        LOG.info("Deleted" + path);
+      } catch (Exception e) {
+        LOG.info(e);
+      }
+    }
+  }
+}

+ 42 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java

@@ -171,8 +171,6 @@ public class TestRenameWithSnapshots {
   private static boolean existsInDiffReport(List<DiffReportEntry> entries,
       DiffType type, String relativePath) {
     for (DiffReportEntry entry : entries) {
-      System.out.println("DiffEntry is:" + entry.getType() + "\""
-          + new String(entry.getRelativePath()) + "\"");
       if ((entry.getType() == type)
           && ((new String(entry.getRelativePath())).compareTo(relativePath) == 0)) {
         return true;
@@ -2374,4 +2372,46 @@ public class TestRenameWithSnapshots {
     // save namespace and restart
     restartClusterAndCheckImage(true);
   }
+
+  @Test
+  public void testRenameWithOverWrite() throws Exception {
+    final Path root = new Path("/");
+    final Path foo = new Path(root, "foo");
+    final Path file1InFoo = new Path(foo, "file1");
+    final Path file2InFoo = new Path(foo, "file2");
+    final Path file3InFoo = new Path(foo, "file3");
+    DFSTestUtil.createFile(hdfs, file1InFoo, 1L, REPL, SEED);
+    DFSTestUtil.createFile(hdfs, file2InFoo, 1L, REPL, SEED);
+    DFSTestUtil.createFile(hdfs, file3InFoo, 1L, REPL, SEED);
+    final Path bar = new Path(root, "bar");
+    hdfs.mkdirs(bar);
+
+    SnapshotTestHelper.createSnapshot(hdfs, root, "s0");
+    // move file1 from foo to bar
+    final Path fileInBar = new Path(bar, "file1");
+    hdfs.rename(file1InFoo, fileInBar);
+    // rename bar to newDir
+    final Path newDir = new Path(root, "newDir");
+    hdfs.rename(bar, newDir);
+    // move file2 from foo to newDir
+    final Path file2InNewDir = new Path(newDir, "file2");
+    hdfs.rename(file2InFoo, file2InNewDir);
+    // move file3 from foo to newDir and rename it to file1, this will overwrite
+    // the original file1
+    final Path file1InNewDir = new Path(newDir, "file1");
+    hdfs.rename(file3InFoo, file1InNewDir, Rename.OVERWRITE);
+    SnapshotTestHelper.createSnapshot(hdfs, root, "s1");
+
+    SnapshotDiffReport report = hdfs.getSnapshotDiffReport(root, "s0", "s1");
+    LOG.info("DiffList is \n\"" + report.toString() + "\"");
+    List<DiffReportEntry> entries = report.getDiffList();
+    assertEquals(7, entries.size());
+    assertTrue(existsInDiffReport(entries, DiffType.MODIFY, ""));
+    assertTrue(existsInDiffReport(entries, DiffType.MODIFY, foo.getName()));
+    assertTrue(existsInDiffReport(entries, DiffType.DELETE, bar.getName()));
+    assertTrue(existsInDiffReport(entries, DiffType.CREATE, newDir.getName()));
+    assertTrue(existsInDiffReport(entries, DiffType.DELETE, "foo/file1"));
+    assertTrue(existsInDiffReport(entries, DiffType.DELETE, "foo/file2"));
+    assertTrue(existsInDiffReport(entries, DiffType.DELETE, "foo/file3"));
+  }
 }

+ 10 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java

@@ -138,6 +138,13 @@ public class TestOfflineImageViewer {
       hdfs.mkdirs(new Path("/snapshot/1"));
       hdfs.delete(snapshot, true);
 
+      // Set XAttrs so the fsimage contains XAttr ops
+      final Path xattr = new Path("/xattr");
+      hdfs.mkdirs(xattr);
+      hdfs.setXAttr(xattr, "user.a1", new byte[]{ 0x31, 0x32, 0x33 });
+      hdfs.setXAttr(xattr, "user.a2", new byte[]{ 0x37, 0x38, 0x39 });
+      writtenFiles.put(xattr.toString(), hdfs.getFileStatus(xattr));
+
       // Write results to the fsimage file
       hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER, false);
       hdfs.saveNamespace();
@@ -210,8 +217,8 @@ public class TestOfflineImageViewer {
     matcher = p.matcher(output.getBuffer());
     assertTrue(matcher.find() && matcher.groupCount() == 1);
     int totalDirs = Integer.parseInt(matcher.group(1));
-    // totalDirs includes root directory and empty directory
-    assertEquals(NUM_DIRS + 2, totalDirs);
+    // totalDirs includes root directory, empty directory, and xattr directory
+    assertEquals(NUM_DIRS + 3, totalDirs);
 
     FileStatus maxFile = Collections.max(writtenFiles.values(),
         new Comparator<FileStatus>() {
@@ -264,7 +271,7 @@ public class TestOfflineImageViewer {
 
       // verify the number of directories
       FileStatus[] statuses = webhdfs.listStatus(new Path("/"));
-      assertEquals(NUM_DIRS + 1, statuses.length); // contains empty directory
+      assertEquals(NUM_DIRS + 2, statuses.length); // contains empty and xattr directory
 
       // verify the number of files in the directory
       statuses = webhdfs.listStatus(new Path("/dir0"));

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

@@ -264,6 +264,9 @@ Release 2.5.0 - UNRELEASED
 
     MAPREDUCE-5920. Add Xattr option in DistCp docs. (Yi Liu via cnauroth)
 
+    MAPREDUCE-5924. Changed TaskAttemptImpl to ignore TA_COMMIT_PENDING event
+    at COMMIT_PENDING state. (Zhijie Shen via jianhe)
+
 Release 2.4.1 - 2014-06-23 
 
   INCOMPATIBLE CHANGES

+ 9 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java

@@ -335,6 +335,15 @@ public abstract class TaskAttemptImpl implements
      .addTransition(TaskAttemptStateInternal.COMMIT_PENDING,
          TaskAttemptStateInternal.FAIL_CONTAINER_CLEANUP,
          TaskAttemptEventType.TA_TIMED_OUT, CLEANUP_CONTAINER_TRANSITION)
+     // AM is likely to receive duplicate TA_COMMIT_PENDINGs as the task attempt
+     // will re-send the commit message until it doesn't encounter any
+     // IOException and succeeds in delivering the commit message.
+     // Ignoring the duplicate commit message is a short-term fix. In long term,
+     // we need to make use of retry cache to help this and other MR protocol
+     // APIs that can be considered as @AtMostOnce.
+     .addTransition(TaskAttemptStateInternal.COMMIT_PENDING,
+         TaskAttemptStateInternal.COMMIT_PENDING,
+         TaskAttemptEventType.TA_COMMIT_PENDING)
 
      // Transitions from SUCCESS_CONTAINER_CLEANUP state
      // kill and cleanup the container

+ 9 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRApp.java

@@ -112,6 +112,15 @@ public class TestMRApp {
     //wait for first attempt to commit pending
     app.waitForState(attempt, TaskAttemptState.COMMIT_PENDING);
 
+    //re-send the commit pending signal to the task
+    app.getContext().getEventHandler().handle(
+        new TaskAttemptEvent(
+            attempt.getID(),
+            TaskAttemptEventType.TA_COMMIT_PENDING));
+
+    //the task attempt should be still at COMMIT_PENDING
+    app.waitForState(attempt, TaskAttemptState.COMMIT_PENDING);
+
     //send the done signal to the task
     app.getContext().getEventHandler().handle(
         new TaskAttemptEvent(

+ 0 - 2
hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/protoc/ProtocMojo.java

@@ -1,6 +1,4 @@
 /*
- * Copyright 2012 The Apache Software Foundation.
- *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
  * You may obtain a copy of the License at

+ 0 - 2
hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/util/Exec.java

@@ -1,6 +1,4 @@
 /*
- * Copyright 2012 The Apache Software Foundation.
- *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
  * You may obtain a copy of the License at

+ 0 - 2
hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/util/FileSetUtils.java

@@ -1,6 +1,4 @@
 /*
- * Copyright 2012 The Apache Software Foundation.
- *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
  * You may obtain a copy of the License at

+ 0 - 2
hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/versioninfo/VersionInfoMojo.java

@@ -1,6 +1,4 @@
 /*
- * Copyright 2012 The Apache Software Foundation.
- *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
  * You may obtain a copy of the License at

+ 6 - 0
hadoop-project/pom.xml

@@ -774,6 +774,12 @@
         <version>1.8</version>
       </dependency>
 
+      <dependency>
+        <groupId>org.apache.directory.server</groupId>
+        <artifactId>apacheds-kerberos-codec</artifactId>
+        <version>2.0.0-M15</version>
+      </dependency>
+
       <dependency>
         <groupId>com.microsoft.windowsazure.storage</groupId>
         <artifactId>microsoft-windowsazure-storage-sdk</artifactId>

+ 0 - 2
hadoop-tools/hadoop-distcp/src/main/resources/distcp-default.xml

@@ -1,8 +1,6 @@
 <?xml version="1.0"?>
 <?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
 <!--
-  Copyright 2002-2004 The Apache Software Foundation
-
   Licensed under the Apache License, Version 2.0 (the "License");
   you may not use this file except in compliance with the License.
   You may obtain a copy of the License at

+ 0 - 2
hadoop-tools/hadoop-distcp/src/test/resources/sslConfig.xml

@@ -1,8 +1,6 @@
 <?xml version="1.0"?>
 <?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
 <!--
-  Copyright 2002-2004 The Apache Software Foundation
-
   Licensed under the Apache License, Version 2.0 (the "License");
   you may not use this file except in compliance with the License.
   You may obtain a copy of the License at

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

@@ -39,6 +39,9 @@ Release 2.5.0 - UNRELEASED
     YARN-1702. Added kill app functionality to RM web services. (Varun Vasudev
     via vinodkv)
 
+    YARN-1339. Recover DeletionService state upon nodemanager restart. (Jason Lowe
+    via junping_du)
+
   IMPROVEMENTS
 
     YARN-1479. Invalid NaN values in Hadoop REST API JSON response (Chen He via
@@ -164,6 +167,9 @@ Release 2.5.0 - UNRELEASED
     YARN-2125. Changed ProportionalCapacityPreemptionPolicy to log CSV in debug
     level. (Wangda Tan via jianhe)
 
+    YARN-2159. Better logging in SchedulerNode#allocateContainer.
+    (Ray Chiang via kasha)
+
   OPTIMIZATIONS
 
   BUG FIXES 
@@ -245,6 +251,14 @@ Release 2.5.0 - UNRELEASED
     YARN-2155. FairScheduler: Incorrect threshold check for preemption.
     (Wei Yan via kasha)
 
+    YARN-1885. Fixed a bug that RM may not send application-clean-up signal
+    to NMs where the completed applications previously ran in case of RM restart.
+    (Wangda Tan via jianhe)
+
+    YARN-2167. LeveldbIterator should get closed in
+    NMLeveldbStateStoreService#loadLocalizationState() within finally block
+    (Junping Du via jlowe)
+
 Release 2.4.1 - 2014-06-23 
 
   INCOMPATIBLE CHANGES

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestResourceTrackerOnHA.java

@@ -60,7 +60,7 @@ public class TestResourceTrackerOnHA extends ProtocolHATestBase{
     // make sure registerNodeManager works when failover happens
     RegisterNodeManagerRequest request =
         RegisterNodeManagerRequest.newInstance(nodeId, 0, resource,
-            YarnVersionInfo.getVersion(), null);
+            YarnVersionInfo.getVersion(), null, null);
     resourceTracker.registerNodeManager(request);
     Assert.assertTrue(waitForNodeManagerToConnect(10000, nodeId));
 

+ 25 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RegisterNodeManagerRequest.java

@@ -20,15 +20,17 @@ package org.apache.hadoop.yarn.server.api.protocolrecords;
 
 import java.util.List;
 
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.util.Records;
 
 public abstract class RegisterNodeManagerRequest {
-  
+
   public static RegisterNodeManagerRequest newInstance(NodeId nodeId,
       int httpPort, Resource resource, String nodeManagerVersionId,
-      List<NMContainerStatus> containerStatuses) {
+      List<NMContainerStatus> containerStatuses,
+      List<ApplicationId> runningApplications) {
     RegisterNodeManagerRequest request =
         Records.newRecord(RegisterNodeManagerRequest.class);
     request.setHttpPort(httpPort);
@@ -36,6 +38,7 @@ public abstract class RegisterNodeManagerRequest {
     request.setNodeId(nodeId);
     request.setNMVersion(nodeManagerVersionId);
     request.setContainerStatuses(containerStatuses);
+    request.setRunningApplications(runningApplications);
     return request;
   }
   
@@ -45,10 +48,30 @@ public abstract class RegisterNodeManagerRequest {
   public abstract String getNMVersion();
   public abstract List<NMContainerStatus> getNMContainerStatuses();
   
+  /**
+   * We introduce this here because currently YARN RM doesn't persist nodes info
+   * for application running. When RM restart happened, we cannot determinate if
+   * a node should do application cleanup (like log-aggregation, status update,
+   * etc.) or not. <p/>
+   * When we have this running application list in node manager register
+   * request, we can recover nodes info for running applications. And then we
+   * can take actions accordingly
+   * 
+   * @return running application list in this node
+   */
+  public abstract List<ApplicationId> getRunningApplications();
+  
   public abstract void setNodeId(NodeId nodeId);
   public abstract void setHttpPort(int port);
   public abstract void setResource(Resource resource);
   public abstract void setNMVersion(String version);
   public abstract void setContainerStatuses(
       List<NMContainerStatus> containerStatuses);
+  
+  /**
+   * Setter for {@link RegisterNodeManagerRequest#getRunningApplications()}
+   * @param runningApplications running application in this node
+   */
+  public abstract void setRunningApplications(
+      List<ApplicationId> runningApplications);
 }

+ 83 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RegisterNodeManagerRequestPBImpl.java

@@ -20,12 +20,23 @@ package org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb;
 
 
 import java.util.ArrayList;
+import java.util.Iterator;
 import java.util.List;
 
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.impl.pb.NodeIdPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerStatusPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.NodeIdPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NMContainerStatusProto;
@@ -44,6 +55,7 @@ public class RegisterNodeManagerRequestPBImpl extends RegisterNodeManagerRequest
   private Resource resource = null;
   private NodeId nodeId = null;
   private List<NMContainerStatus> containerStatuses = null;
+  private List<ApplicationId> runningApplications = null;
   
   public RegisterNodeManagerRequestPBImpl() {
     builder = RegisterNodeManagerRequestProto.newBuilder();
@@ -65,6 +77,9 @@ public class RegisterNodeManagerRequestPBImpl extends RegisterNodeManagerRequest
     if (this.containerStatuses != null) {
       addNMContainerStatusesToProto();
     }
+    if (this.runningApplications != null) {
+      addRunningApplicationsToProto();
+    }
     if (this.resource != null) {
       builder.setResource(convertToProtoFormat(this.resource));
     }
@@ -158,6 +173,66 @@ public class RegisterNodeManagerRequestPBImpl extends RegisterNodeManagerRequest
     maybeInitBuilder();
     builder.setHttpPort(httpPort);
   }
+  
+  @Override
+  public List<ApplicationId> getRunningApplications() {
+    initRunningApplications();
+    return runningApplications;
+  }
+  
+  private void initRunningApplications() {
+    if (this.runningApplications != null) {
+      return;
+    }
+    RegisterNodeManagerRequestProtoOrBuilder p = viaProto ? proto : builder;
+    List<ApplicationIdProto> list = p.getRunningApplicationsList();
+    this.runningApplications = new ArrayList<ApplicationId>();
+    for (ApplicationIdProto c : list) {
+      this.runningApplications.add(convertFromProtoFormat(c));
+    }
+  }
+
+  @Override
+  public void setRunningApplications(List<ApplicationId> apps) {
+    if (apps == null) {
+      return;
+    }
+    initRunningApplications();
+    this.runningApplications.addAll(apps);
+  }
+  
+  private void addRunningApplicationsToProto() {
+    maybeInitBuilder();
+    builder.clearRunningApplications();
+    if (runningApplications == null) {
+      return;
+    }
+    Iterable<ApplicationIdProto> it = new Iterable<ApplicationIdProto>() {
+      
+      @Override
+      public Iterator<ApplicationIdProto> iterator() {
+        return new Iterator<ApplicationIdProto>() {
+          Iterator<ApplicationId> iter = runningApplications.iterator();
+          
+          @Override
+          public boolean hasNext() {
+            return iter.hasNext();
+          }
+          
+          @Override
+          public ApplicationIdProto next() {
+            return convertToProtoFormat(iter.next());  
+          }
+          
+          @Override
+          public void remove() {
+            throw new UnsupportedOperationException();
+          }
+        };
+      }
+    };
+    builder.addAllRunningApplications(it);
+  }
 
   @Override
   public List<NMContainerStatus> getNMContainerStatuses() {
@@ -216,6 +291,14 @@ public class RegisterNodeManagerRequestPBImpl extends RegisterNodeManagerRequest
     maybeInitBuilder();
     builder.setNmVersion(version);
   }
+  
+  private ApplicationIdPBImpl convertFromProtoFormat(ApplicationIdProto p) {
+    return new ApplicationIdPBImpl(p);
+  }
+
+  private ApplicationIdProto convertToProtoFormat(ApplicationId t) {
+    return ((ApplicationIdPBImpl)t).getProto();
+  }
 
   private NodeIdPBImpl convertFromProtoFormat(NodeIdProto p) {
     return new NodeIdPBImpl(p);

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto

@@ -31,6 +31,7 @@ message RegisterNodeManagerRequestProto {
   optional ResourceProto resource = 4;
   optional string nm_version = 5;
   repeated NMContainerStatusProto container_statuses = 6;
+  repeated ApplicationIdProto runningApplications = 7;
 }
 
 message RegisterNodeManagerResponseProto {
@@ -66,4 +67,4 @@ message NMContainerStatusProto {
   optional PriorityProto priority = 4;
   optional string diagnostics = 5 [default = "N/A"];
   optional int32 container_exit_status = 6;
-}
+}

+ 4 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/api/protocolrecords/TestProtocolRecords.java

@@ -83,7 +83,8 @@ public class TestProtocolRecords {
     RegisterNodeManagerRequest request =
         RegisterNodeManagerRequest.newInstance(
           NodeId.newInstance("1.1.1.1", 1000), 8080,
-          Resource.newInstance(1024, 1), "NM-version-id", reports);
+            Resource.newInstance(1024, 1), "NM-version-id", reports,
+            Arrays.asList(appId));
     RegisterNodeManagerRequest requestProto =
         new RegisterNodeManagerRequestPBImpl(
           ((RegisterNodeManagerRequestPBImpl) request).getProto());
@@ -95,5 +96,7 @@ public class TestProtocolRecords {
       requestProto.getNodeId());
     Assert.assertEquals(Resource.newInstance(1024, 1),
       requestProto.getResource());
+    Assert.assertEquals(1, requestProto.getRunningApplications().size());
+    Assert.assertEquals(appId, requestProto.getRunningApplications().get(0)); 
   }
 }

+ 81 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/api/protocolrecords/TestRegisterNodeManagerRequest.java

@@ -0,0 +1,81 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.hadoop.yarn.server.api.protocolrecords;
+
+import java.util.Arrays;
+
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RegisterNodeManagerRequestPBImpl;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestRegisterNodeManagerRequest {
+  @Test
+  public void testRegisterNodeManagerRequest() {
+    RegisterNodeManagerRequest request =
+        RegisterNodeManagerRequest.newInstance(
+            NodeId.newInstance("host", 1234), 1234, Resource.newInstance(0, 0),
+            "version", Arrays.asList(NMContainerStatus.newInstance(
+                ContainerId.newInstance(
+                    ApplicationAttemptId.newInstance(
+                        ApplicationId.newInstance(1234L, 1), 1), 1),
+                ContainerState.RUNNING, Resource.newInstance(1024, 1), "good",
+                -1)), Arrays.asList(ApplicationId.newInstance(1234L, 1),
+                ApplicationId.newInstance(1234L, 2)));
+
+    // serialze to proto, and get request from proto
+    RegisterNodeManagerRequest request1 =
+        new RegisterNodeManagerRequestPBImpl(
+            ((RegisterNodeManagerRequestPBImpl) request).getProto());
+
+    // check values
+    Assert.assertEquals(request1.getNMContainerStatuses().size(), request
+        .getNMContainerStatuses().size());
+    Assert.assertEquals(request1.getNMContainerStatuses().get(0).getContainerId(),
+        request.getNMContainerStatuses().get(0).getContainerId());
+    Assert.assertEquals(request1.getRunningApplications().size(), request
+        .getRunningApplications().size());
+    Assert.assertEquals(request1.getRunningApplications().get(0), request
+        .getRunningApplications().get(0));
+    Assert.assertEquals(request1.getRunningApplications().get(1), request
+        .getRunningApplications().get(1));
+  }
+  
+  @Test
+  public void testRegisterNodeManagerRequestWithNullArrays() {
+    RegisterNodeManagerRequest request =
+        RegisterNodeManagerRequest.newInstance(NodeId.newInstance("host", 1234),
+            1234, Resource.newInstance(0, 0), "version", null, null);
+
+    // serialze to proto, and get request from proto
+    RegisterNodeManagerRequest request1 =
+        new RegisterNodeManagerRequestPBImpl(
+            ((RegisterNodeManagerRequestPBImpl) request).getProto());
+
+    // check values
+    Assert.assertEquals(0, request1.getNMContainerStatuses().size());
+    Assert.assertEquals(0, request1.getRunningApplications().size());
+  }
+}

+ 171 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DeletionService.java

@@ -21,10 +21,13 @@ package org.apache.hadoop.yarn.server.nodemanager;
 import static java.util.concurrent.TimeUnit.SECONDS;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.ThreadFactory;
@@ -40,6 +43,10 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.DeletionServiceDeleteTaskProto;
+import org.apache.hadoop.yarn.server.nodemanager.recovery.NMNullStateStoreService;
+import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
+import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService.RecoveredDeletionServiceState;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
@@ -50,6 +57,8 @@ public class DeletionService extends AbstractService {
   private final ContainerExecutor exec;
   private ScheduledThreadPoolExecutor sched;
   private static final FileContext lfs = getLfs();
+  private final NMStateStoreService stateStore;
+  private AtomicInteger nextTaskId = new AtomicInteger(0);
 
   static final FileContext getLfs() {
     try {
@@ -60,13 +69,17 @@ public class DeletionService extends AbstractService {
   }
 
   public DeletionService(ContainerExecutor exec) {
+    this(exec, new NMNullStateStoreService());
+  }
+
+  public DeletionService(ContainerExecutor exec,
+      NMStateStoreService stateStore) {
     super(DeletionService.class.getName());
     this.exec = exec;
     this.debugDelay = 0;
+    this.stateStore = stateStore;
   }
   
-  /**
-   * 
   /**
    * Delete the path(s) as this user.
    * @param user The user to delete as, or the JVM user if null
@@ -76,19 +89,20 @@ public class DeletionService extends AbstractService {
   public void delete(String user, Path subDir, Path... baseDirs) {
     // TODO if parent owned by NM, rename within parent inline
     if (debugDelay != -1) {
-      if (baseDirs == null || baseDirs.length == 0) {
-        sched.schedule(new FileDeletionTask(this, user, subDir, null),
-          debugDelay, TimeUnit.SECONDS);
-      } else {
-        sched.schedule(
-          new FileDeletionTask(this, user, subDir, Arrays.asList(baseDirs)),
-          debugDelay, TimeUnit.SECONDS);
+      List<Path> baseDirList = null;
+      if (baseDirs != null && baseDirs.length != 0) {
+        baseDirList = Arrays.asList(baseDirs);
       }
+      FileDeletionTask task =
+          new FileDeletionTask(this, user, subDir, baseDirList);
+      recordDeletionTaskInStateStore(task);
+      sched.schedule(task, debugDelay, TimeUnit.SECONDS);
     }
   }
   
   public void scheduleFileDeletionTask(FileDeletionTask fileDeletionTask) {
     if (debugDelay != -1) {
+      recordDeletionTaskInStateStore(fileDeletionTask);
       sched.schedule(fileDeletionTask, debugDelay, TimeUnit.SECONDS);
     }
   }
@@ -109,6 +123,9 @@ public class DeletionService extends AbstractService {
     }
     sched.setExecuteExistingDelayedTasksAfterShutdownPolicy(false);
     sched.setKeepAliveTime(60L, SECONDS);
+    if (stateStore.canRecover()) {
+      recover(stateStore.loadDeletionServiceState());
+    }
     super.serviceInit(conf);
   }
 
@@ -139,6 +156,8 @@ public class DeletionService extends AbstractService {
   }
 
   public static class FileDeletionTask implements Runnable {
+    public static final int INVALID_TASK_ID = -1;
+    private int taskId;
     private final String user;
     private final Path subDir;
     private final List<Path> baseDirs;
@@ -152,6 +171,12 @@ public class DeletionService extends AbstractService {
     
     private FileDeletionTask(DeletionService delService, String user,
         Path subDir, List<Path> baseDirs) {
+      this(INVALID_TASK_ID, delService, user, subDir, baseDirs);
+    }
+
+    private FileDeletionTask(int taskId, DeletionService delService,
+        String user, Path subDir, List<Path> baseDirs) {
+      this.taskId = taskId;
       this.delService = delService;
       this.user = user;
       this.subDir = subDir;
@@ -198,6 +223,12 @@ public class DeletionService extends AbstractService {
       return this.success;
     }
     
+    public synchronized FileDeletionTask[] getSuccessorTasks() {
+      FileDeletionTask[] successors =
+          new FileDeletionTask[successorTaskSet.size()];
+      return successorTaskSet.toArray(successors);
+    }
+
     @Override
     public void run() {
       if (LOG.isDebugEnabled()) {
@@ -286,6 +317,12 @@ public class DeletionService extends AbstractService {
      * dependent tasks of it has failed marking its success = false.  
      */
     private synchronized void fileDeletionTaskFinished() {
+      try {
+        delService.stateStore.removeDeletionTask(taskId);
+      } catch (IOException e) {
+        LOG.error("Unable to remove deletion task " + taskId
+            + " from state store", e);
+      }
       Iterator<FileDeletionTask> successorTaskI =
           this.successorTaskSet.iterator();
       while (successorTaskI.hasNext()) {
@@ -318,4 +355,129 @@ public class DeletionService extends AbstractService {
       Path[] baseDirs) {
     return new FileDeletionTask(this, user, subDir, Arrays.asList(baseDirs));
   }
+
+  private void recover(RecoveredDeletionServiceState state)
+      throws IOException {
+    List<DeletionServiceDeleteTaskProto> taskProtos = state.getTasks();
+    Map<Integer, DeletionTaskRecoveryInfo> idToInfoMap =
+        new HashMap<Integer, DeletionTaskRecoveryInfo>(taskProtos.size());
+    Set<Integer> successorTasks = new HashSet<Integer>();
+    for (DeletionServiceDeleteTaskProto proto : taskProtos) {
+      DeletionTaskRecoveryInfo info = parseTaskProto(proto);
+      idToInfoMap.put(info.task.taskId, info);
+      nextTaskId.set(Math.max(nextTaskId.get(), info.task.taskId));
+      successorTasks.addAll(info.successorTaskIds);
+    }
+
+    // restore the task dependencies and schedule the deletion tasks that
+    // have no predecessors
+    final long now = System.currentTimeMillis();
+    for (DeletionTaskRecoveryInfo info : idToInfoMap.values()) {
+      for (Integer successorId : info.successorTaskIds){
+        DeletionTaskRecoveryInfo successor = idToInfoMap.get(successorId);
+        if (successor != null) {
+          info.task.addFileDeletionTaskDependency(successor.task);
+        } else {
+          LOG.error("Unable to locate dependency task for deletion task "
+              + info.task.taskId + " at " + info.task.getSubDir());
+        }
+      }
+      if (!successorTasks.contains(info.task.taskId)) {
+        long msecTilDeletion = info.deletionTimestamp - now;
+        sched.schedule(info.task, msecTilDeletion, TimeUnit.MILLISECONDS);
+      }
+    }
+  }
+
+  private DeletionTaskRecoveryInfo parseTaskProto(
+      DeletionServiceDeleteTaskProto proto) throws IOException {
+    int taskId = proto.getId();
+    String user = proto.hasUser() ? proto.getUser() : null;
+    Path subdir = null;
+    List<Path> basePaths = null;
+    if (proto.hasSubdir()) {
+      subdir = new Path(proto.getSubdir());
+    }
+    List<String> basedirs = proto.getBasedirsList();
+    if (basedirs != null && basedirs.size() > 0) {
+      basePaths = new ArrayList<Path>(basedirs.size());
+      for (String basedir : basedirs) {
+        basePaths.add(new Path(basedir));
+      }
+    }
+
+    FileDeletionTask task = new FileDeletionTask(taskId, this, user,
+        subdir, basePaths);
+    return new DeletionTaskRecoveryInfo(task,
+        proto.getSuccessorIdsList(),
+        proto.getDeletionTime());
+  }
+
+  private int generateTaskId() {
+    // get the next ID but avoid an invalid ID
+    int taskId = nextTaskId.incrementAndGet();
+    while (taskId == FileDeletionTask.INVALID_TASK_ID) {
+      taskId = nextTaskId.incrementAndGet();
+    }
+    return taskId;
+  }
+
+  private void recordDeletionTaskInStateStore(FileDeletionTask task) {
+    if (!stateStore.canRecover()) {
+      // optimize the case where we aren't really recording
+      return;
+    }
+    if (task.taskId != FileDeletionTask.INVALID_TASK_ID) {
+      return;  // task already recorded
+    }
+
+    task.taskId = generateTaskId();
+
+    FileDeletionTask[] successors = task.getSuccessorTasks();
+
+    // store successors first to ensure task IDs have been generated for them
+    for (FileDeletionTask successor : successors) {
+      recordDeletionTaskInStateStore(successor);
+    }
+
+    DeletionServiceDeleteTaskProto.Builder builder =
+        DeletionServiceDeleteTaskProto.newBuilder();
+    builder.setId(task.taskId);
+    if (task.getUser() != null) {
+      builder.setUser(task.getUser());
+    }
+    if (task.getSubDir() != null) {
+      builder.setSubdir(task.getSubDir().toString());
+    }
+    builder.setDeletionTime(System.currentTimeMillis() +
+        TimeUnit.MILLISECONDS.convert(debugDelay, TimeUnit.SECONDS));
+    if (task.getBaseDirs() != null) {
+      for (Path dir : task.getBaseDirs()) {
+        builder.addBasedirs(dir.toString());
+      }
+    }
+    for (FileDeletionTask successor : successors) {
+      builder.addSuccessorIds(successor.taskId);
+    }
+
+    try {
+      stateStore.storeDeletionTask(task.taskId, builder.build());
+    } catch (IOException e) {
+      LOG.error("Unable to store deletion task " + task.taskId + " for "
+          + task.getSubDir(), e);
+    }
+  }
+
+  private static class DeletionTaskRecoveryInfo {
+    FileDeletionTask task;
+    List<Integer> successorTaskIds;
+    long deletionTimestamp;
+
+    public DeletionTaskRecoveryInfo(FileDeletionTask task,
+        List<Integer> successorTaskIds, long deletionTimestamp) {
+      this.task = task;
+      this.successorTaskIds = successorTaskIds;
+      this.deletionTimestamp = deletionTimestamp;
+    }
+  }
 }

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java

@@ -114,7 +114,7 @@ public class NodeManager extends CompositeService
   }
 
   protected DeletionService createDeletionService(ContainerExecutor exec) {
-    return new DeletionService(exec);
+    return new DeletionService(exec, nmStore);
   }
 
   protected NMContext createNMContext(

+ 7 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java

@@ -250,7 +250,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
     List<NMContainerStatus> containerReports = getNMContainerStatuses();
     RegisterNodeManagerRequest request =
         RegisterNodeManagerRequest.newInstance(nodeId, httpPort, totalResource,
-          nodeManagerVersionId, containerReports);
+          nodeManagerVersionId, containerReports, getRunningApplications());
     if (containerReports != null) {
       LOG.info("Registering with RM using containers :" + containerReports);
     }
@@ -374,6 +374,12 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
     }
     return containerStatuses;
   }
+  
+  private List<ApplicationId> getRunningApplications() {
+    List<ApplicationId> runningApplications = new ArrayList<ApplicationId>();
+    runningApplications.addAll(this.context.getApplications().keySet());
+    return runningApplications;
+  }
 
   // These NMContainerStatus are sent on NM registration and used by YARN only.
   private List<NMContainerStatus> getNMContainerStatuses() {

+ 60 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java

@@ -38,6 +38,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceProto;
+import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.DeletionServiceDeleteTaskProto;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.LocalizedResourceProto;
 import org.apache.hadoop.yarn.server.utils.LeveldbIterator;
 import org.apache.hadoop.yarn.util.ConverterUtils;
@@ -58,6 +59,9 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
   private static final String DB_SCHEMA_VERSION_KEY = "schema-version";
   private static final String DB_SCHEMA_VERSION = "1.0";
 
+  private static final String DELETION_TASK_KEY_PREFIX =
+      "DeletionService/deltask_";
+
   private static final String LOCALIZATION_KEY_PREFIX = "Localization/";
   private static final String LOCALIZATION_PUBLIC_KEY_PREFIX =
       LOCALIZATION_KEY_PREFIX + "public/";
@@ -91,8 +95,9 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
       throws IOException {
     RecoveredLocalizationState state = new RecoveredLocalizationState();
 
+    LeveldbIterator iter = null;
     try {
-      LeveldbIterator iter = new LeveldbIterator(db);
+      iter = new LeveldbIterator(db);
       iter.seek(bytes(LOCALIZATION_PUBLIC_KEY_PREFIX));
       state.publicTrackerState = loadResourceTrackerState(iter,
           LOCALIZATION_PUBLIC_KEY_PREFIX);
@@ -118,6 +123,10 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
       }
     } catch (DBException e) {
       throw new IOException(e.getMessage(), e);
+    } finally {
+      if (iter != null) {
+        iter.close();
+      }
     }
 
     return state;
@@ -308,6 +317,56 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
   }
 
 
+  @Override
+  public RecoveredDeletionServiceState loadDeletionServiceState()
+      throws IOException {
+    RecoveredDeletionServiceState state = new RecoveredDeletionServiceState();
+    state.tasks = new ArrayList<DeletionServiceDeleteTaskProto>();
+    LeveldbIterator iter = null;
+    try {
+      iter = new LeveldbIterator(db);
+      iter.seek(bytes(DELETION_TASK_KEY_PREFIX));
+      while (iter.hasNext()) {
+        Entry<byte[], byte[]> entry = iter.next();
+        String key = asString(entry.getKey());
+        if (!key.startsWith(DELETION_TASK_KEY_PREFIX)) {
+          break;
+        }
+        state.tasks.add(
+            DeletionServiceDeleteTaskProto.parseFrom(entry.getValue()));
+      }
+    } catch (DBException e) {
+      throw new IOException(e.getMessage(), e);
+    } finally {
+      if (iter != null) {
+        iter.close();
+      }
+    }
+    return state;
+  }
+
+  @Override
+  public void storeDeletionTask(int taskId,
+      DeletionServiceDeleteTaskProto taskProto) throws IOException {
+    String key = DELETION_TASK_KEY_PREFIX + taskId;
+    try {
+      db.put(bytes(key), taskProto.toByteArray());
+    } catch (DBException e) {
+      throw new IOException(e.getMessage(), e);
+    }
+  }
+
+  @Override
+  public void removeDeletionTask(int taskId) throws IOException {
+    String key = DELETION_TASK_KEY_PREFIX + taskId;
+    try {
+      db.delete(bytes(key));
+    } catch (DBException e) {
+      throw new IOException(e.getMessage(), e);
+    }
+  }
+
+
   @Override
   protected void initStorage(Configuration conf)
       throws IOException {

+ 17 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.java

@@ -24,6 +24,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceProto;
+import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.DeletionServiceDeleteTaskProto;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.LocalizedResourceProto;
 
 // The state store to use when state isn't being stored
@@ -60,6 +61,22 @@ public class NMNullStateStoreService extends NMStateStoreService {
       Path localPath) throws IOException {
   }
 
+  @Override
+  public RecoveredDeletionServiceState loadDeletionServiceState()
+      throws IOException {
+    throw new UnsupportedOperationException(
+        "Recovery not supported by this state store");
+  }
+
+  @Override
+  public void storeDeletionTask(int taskId,
+      DeletionServiceDeleteTaskProto taskProto) throws IOException {
+  }
+
+  @Override
+  public void removeDeletionTask(int taskId) throws IOException {
+  }
+
   @Override
   protected void initStorage(Configuration conf) throws IOException {
   }

+ 18 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMStateStoreService.java

@@ -31,6 +31,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceProto;
+import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.DeletionServiceDeleteTaskProto;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.LocalizedResourceProto;
 
 @Private
@@ -91,6 +92,14 @@ public abstract class NMStateStoreService extends AbstractService {
     }
   }
 
+  public static class RecoveredDeletionServiceState {
+    List<DeletionServiceDeleteTaskProto> tasks;
+
+    public List<DeletionServiceDeleteTaskProto> getTasks() {
+      return tasks;
+    }
+  }
+
   /** Initialize the state storage */
   @Override
   public void serviceInit(Configuration conf) throws IOException {
@@ -155,6 +164,15 @@ public abstract class NMStateStoreService extends AbstractService {
       ApplicationId appId, Path localPath) throws IOException;
 
 
+  public abstract RecoveredDeletionServiceState loadDeletionServiceState()
+      throws IOException;
+
+  public abstract void storeDeletionTask(int taskId,
+      DeletionServiceDeleteTaskProto taskProto) throws IOException;
+
+  public abstract void removeDeletionTask(int taskId) throws IOException;
+
+
   protected abstract void initStorage(Configuration conf) throws IOException;
 
   protected abstract void startStorage() throws IOException;

+ 9 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/proto/yarn_server_nodemanager_recovery.proto

@@ -24,6 +24,15 @@ package hadoop.yarn;
 
 import "yarn_protos.proto";
 
+message DeletionServiceDeleteTaskProto {
+  optional int32 id = 1;
+  optional string user = 2;
+  optional string subdir = 3;
+  optional int64 deletionTime = 4;
+  repeated string basedirs = 5;
+  repeated int32 successorIds = 6;
+}
+
 message LocalizedResourceProto {
   optional LocalResourceProto resource = 1;
   optional string localPath = 2;

+ 55 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestDeletionService.java

@@ -34,6 +34,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.nodemanager.DeletionService.FileDeletionTask;
+import org.apache.hadoop.yarn.server.nodemanager.recovery.NMMemoryStateStoreService;
 import org.junit.AfterClass;
 import org.junit.Test;
 import org.mockito.Mockito;
@@ -285,4 +286,58 @@ public class TestDeletionService {
       del.stop();
     }
   }
+
+  @Test
+  public void testRecovery() throws Exception {
+    Random r = new Random();
+    long seed = r.nextLong();
+    r.setSeed(seed);
+    System.out.println("SEED: " + seed);
+    List<Path> baseDirs = buildDirs(r, base, 4);
+    createDirs(new Path("."), baseDirs);
+    List<Path> content = buildDirs(r, new Path("."), 10);
+    for (Path b : baseDirs) {
+      createDirs(b, content);
+    }
+    Configuration conf = new YarnConfiguration();
+    conf.setBoolean(YarnConfiguration.NM_RECOVERY_ENABLED, true);
+    conf.setInt(YarnConfiguration.DEBUG_NM_DELETE_DELAY_SEC, 1);
+    NMMemoryStateStoreService stateStore = new NMMemoryStateStoreService();
+    stateStore.init(conf);
+    stateStore.start();
+    DeletionService del =
+      new DeletionService(new FakeDefaultContainerExecutor(), stateStore);
+    try {
+      del.init(conf);
+      del.start();
+      for (Path p : content) {
+        assertTrue(lfs.util().exists(new Path(baseDirs.get(0), p)));
+        del.delete((Long.parseLong(p.getName()) % 2) == 0 ? null : "dingo",
+            p, baseDirs.toArray(new Path[4]));
+      }
+
+      // restart the deletion service
+      del.stop();
+      del = new DeletionService(new FakeDefaultContainerExecutor(),
+          stateStore);
+      del.init(conf);
+      del.start();
+
+      // verify paths are still eventually deleted
+      int msecToWait = 10 * 1000;
+      for (Path p : baseDirs) {
+        for (Path q : content) {
+          Path fp = new Path(p, q);
+          while (msecToWait > 0 && lfs.util().exists(fp)) {
+            Thread.sleep(100);
+            msecToWait -= 100;
+          }
+          assertFalse(lfs.util().exists(fp));
+        }
+      }
+    } finally {
+      del.close();
+      stateStore.close();
+    }
+  }
 }

+ 27 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMMemoryStateStoreService.java

@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.yarn.server.nodemanager.recovery;
 
+import java.io.IOException;
+import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -25,10 +27,12 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceProto;
+import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.DeletionServiceDeleteTaskProto;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.LocalizedResourceProto;
 
 public class NMMemoryStateStoreService extends NMStateStoreService {
   private Map<TrackerKey, TrackerState> trackerStates;
+  private Map<Integer, DeletionServiceDeleteTaskProto> deleteTasks;
 
   public NMMemoryStateStoreService() {
     super(NMMemoryStateStoreService.class.getName());
@@ -110,6 +114,7 @@ public class NMMemoryStateStoreService extends NMStateStoreService {
   @Override
   protected void initStorage(Configuration conf) {
     trackerStates = new HashMap<TrackerKey, TrackerState>();
+    deleteTasks = new HashMap<Integer, DeletionServiceDeleteTaskProto>();
   }
 
   @Override
@@ -121,6 +126,28 @@ public class NMMemoryStateStoreService extends NMStateStoreService {
   }
 
 
+  @Override
+  public RecoveredDeletionServiceState loadDeletionServiceState()
+      throws IOException {
+    RecoveredDeletionServiceState result =
+        new RecoveredDeletionServiceState();
+    result.tasks = new ArrayList<DeletionServiceDeleteTaskProto>(
+        deleteTasks.values());
+    return result;
+  }
+
+  @Override
+  public synchronized void storeDeletionTask(int taskId,
+      DeletionServiceDeleteTaskProto taskProto) throws IOException {
+    deleteTasks.put(taskId, taskProto);
+  }
+
+  @Override
+  public synchronized void removeDeletionTask(int taskId) throws IOException {
+    deleteTasks.remove(taskId);
+  }
+
+
   private static class TrackerState {
     Map<Path, LocalResourceProto> inProgressMap =
         new HashMap<Path, LocalResourceProto>();

+ 56 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java

@@ -35,8 +35,10 @@ import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
 import org.apache.hadoop.yarn.api.records.impl.pb.LocalResourcePBImpl;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceProto;
+import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.DeletionServiceDeleteTaskProto;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.LocalizedResourceProto;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService.LocalResourceTrackerState;
+import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService.RecoveredDeletionServiceState;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService.RecoveredLocalizationState;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService.RecoveredUserResources;
 import org.apache.hadoop.yarn.util.ConverterUtils;
@@ -404,4 +406,58 @@ public class TestNMLeveldbStateStoreService {
         state.getUserResources();
     assertTrue(userResources.isEmpty());
   }
+
+  @Test
+  public void testDeletionTaskStorage() throws IOException {
+    // test empty when no state
+    RecoveredDeletionServiceState state =
+        stateStore.loadDeletionServiceState();
+    assertTrue(state.getTasks().isEmpty());
+
+    // store a deletion task and verify recovered
+    DeletionServiceDeleteTaskProto proto =
+        DeletionServiceDeleteTaskProto.newBuilder()
+        .setId(7)
+        .setUser("someuser")
+        .setSubdir("some/subdir")
+        .addBasedirs("some/dir/path")
+        .addBasedirs("some/other/dir/path")
+        .setDeletionTime(123456L)
+        .addSuccessorIds(8)
+        .addSuccessorIds(9)
+        .build();
+    stateStore.storeDeletionTask(proto.getId(), proto);
+    restartStateStore();
+    state = stateStore.loadDeletionServiceState();
+    assertEquals(1, state.getTasks().size());
+    assertEquals(proto, state.getTasks().get(0));
+
+    // store another deletion task
+    DeletionServiceDeleteTaskProto proto2 =
+        DeletionServiceDeleteTaskProto.newBuilder()
+        .setId(8)
+        .setUser("user2")
+        .setSubdir("subdir2")
+        .setDeletionTime(789L)
+        .build();
+    stateStore.storeDeletionTask(proto2.getId(), proto2);
+    restartStateStore();
+    state = stateStore.loadDeletionServiceState();
+    assertEquals(2, state.getTasks().size());
+    assertTrue(state.getTasks().contains(proto));
+    assertTrue(state.getTasks().contains(proto2));
+
+    // delete a task and verify gone after recovery
+    stateStore.removeDeletionTask(proto2.getId());
+    restartStateStore();
+    state = stateStore.loadDeletionServiceState();
+    assertEquals(1, state.getTasks().size());
+    assertEquals(proto, state.getTasks().get(0));
+
+    // delete the last task and verify none left
+    stateStore.removeDeletionTask(proto.getId());
+    restartStateStore();
+    state = stateStore.loadDeletionServiceState();
+    assertTrue(state.getTasks().isEmpty());
+  }
 }

+ 14 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java

@@ -244,15 +244,6 @@ public class ResourceTrackerService extends AbstractService implements
     Resource capability = request.getResource();
     String nodeManagerVersion = request.getNMVersion();
 
-    if (!rmContext.isWorkPreservingRecoveryEnabled()) {
-      if (!request.getNMContainerStatuses().isEmpty()) {
-        LOG.info("received container statuses on node manager register :"
-            + request.getNMContainerStatuses());
-        for (NMContainerStatus status : request.getNMContainerStatuses()) {
-          handleNMContainerStatus(status);
-        }
-      }
-    }
     RegisterNodeManagerResponse response = recordFactory
         .newRecordInstance(RegisterNodeManagerResponse.class);
 
@@ -311,7 +302,8 @@ public class ResourceTrackerService extends AbstractService implements
     RMNode oldNode = this.rmContext.getRMNodes().putIfAbsent(nodeId, rmNode);
     if (oldNode == null) {
       this.rmContext.getDispatcher().getEventHandler().handle(
-          new RMNodeStartedEvent(nodeId, request.getNMContainerStatuses()));
+              new RMNodeStartedEvent(nodeId, request.getNMContainerStatuses(),
+                  request.getRunningApplications()));
     } else {
       LOG.info("Reconnect from the node at: " + host);
       this.nmLivelinessMonitor.unregister(nodeId);
@@ -322,6 +314,18 @@ public class ResourceTrackerService extends AbstractService implements
     // present for any running application.
     this.nmTokenSecretManager.removeNodeKey(nodeId);
     this.nmLivelinessMonitor.register(nodeId);
+    
+    // Handle received container status, this should be processed after new
+    // RMNode inserted
+    if (!rmContext.isWorkPreservingRecoveryEnabled()) {
+      if (!request.getNMContainerStatuses().isEmpty()) {
+        LOG.info("received container statuses on node manager register :"
+            + request.getNMContainerStatuses());
+        for (NMContainerStatus status : request.getNMContainerStatuses()) {
+          handleNMContainerStatus(status);
+        }
+      }
+    }
 
     String message =
         "NodeManager from node " + host + "(cmPort: " + cmPort + " httpPort: "

+ 10 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java

@@ -19,16 +19,16 @@
 package org.apache.hadoop.yarn.server.resourcemanager.rmapp;
 
 import java.util.Collection;
-
 import java.util.Map;
 import java.util.Set;
 
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
@@ -208,6 +208,14 @@ public interface RMApp extends EventHandler<RMAppEvent> {
    * @return the flag indicating whether the applications's state is stored.
    */
   boolean isAppFinalStateStored();
+  
+  
+  /**
+   * Nodes on which the containers for this {@link RMApp} ran.
+   * @return the set of nodes that ran any containers from this {@link RMApp}
+   * Add more node on which containers for this {@link RMApp} ran
+   */
+  Set<NodeId> getRanNodes();
 
   /**
    * Create the external user-facing state of ApplicationMaster from the

+ 3 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppEventType.java

@@ -38,6 +38,9 @@ public enum RMAppEventType {
   ATTEMPT_FAILED,
   ATTEMPT_KILLED,
   NODE_UPDATE,
+  
+  // Source: Container and ResourceTracker
+  APP_RUNNING_ON_NODE,
 
   // Source: RMStateStore
   APP_NEW_SAVED,

+ 49 - 12
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java

@@ -25,6 +25,7 @@ import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.ConcurrentSkipListSet;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
@@ -71,7 +72,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanAppEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
 import org.apache.hadoop.yarn.state.MultipleArcTransition;
@@ -116,6 +116,7 @@ public class RMAppImpl implements RMApp, Recoverable {
   private EventHandler handler;
   private static final AppFinishedTransition FINISHED_TRANSITION =
       new AppFinishedTransition();
+  private Set<NodeId> ranNodes = new ConcurrentSkipListSet<NodeId>();
 
   // These states stored are only valid when app is at killing or final_saving.
   private RMAppState stateBeforeKilling;
@@ -180,7 +181,6 @@ public class RMAppImpl implements RMApp, Recoverable {
         new FinalSavingTransition(
           new AppKilledTransition(), RMAppState.KILLED))
 
-
      // Transitions from ACCEPTED state
     .addTransition(RMAppState.ACCEPTED, RMAppState.ACCEPTED,
         RMAppEventType.NODE_UPDATE, new RMAppNodeUpdateTransition())
@@ -200,6 +200,9 @@ public class RMAppImpl implements RMApp, Recoverable {
         new FinalSavingTransition(FINISHED_TRANSITION, RMAppState.FINISHED))
     .addTransition(RMAppState.ACCEPTED, RMAppState.KILLING,
         RMAppEventType.KILL, new KillAttemptTransition())
+    .addTransition(RMAppState.ACCEPTED, RMAppState.ACCEPTED, 
+        RMAppEventType.APP_RUNNING_ON_NODE,
+        new AppRunningOnNodeTransition())
     // ACCECPTED state can once again receive APP_ACCEPTED event, because on
     // recovery the app returns ACCEPTED state and the app once again go
     // through the scheduler and triggers one more APP_ACCEPTED event at
@@ -220,6 +223,9 @@ public class RMAppImpl implements RMApp, Recoverable {
     .addTransition(RMAppState.RUNNING, RMAppState.FINISHED,
       // UnManagedAM directly jumps to finished
         RMAppEventType.ATTEMPT_FINISHED, FINISHED_TRANSITION)
+    .addTransition(RMAppState.RUNNING, RMAppState.RUNNING, 
+        RMAppEventType.APP_RUNNING_ON_NODE,
+        new AppRunningOnNodeTransition())
     .addTransition(RMAppState.RUNNING,
         EnumSet.of(RMAppState.ACCEPTED, RMAppState.FINAL_SAVING),
         RMAppEventType.ATTEMPT_FAILED,
@@ -235,6 +241,9 @@ public class RMAppImpl implements RMApp, Recoverable {
     .addTransition(RMAppState.FINAL_SAVING, RMAppState.FINAL_SAVING,
         RMAppEventType.ATTEMPT_FINISHED,
         new AttemptFinishedAtFinalSavingTransition())
+    .addTransition(RMAppState.FINAL_SAVING, RMAppState.FINAL_SAVING, 
+        RMAppEventType.APP_RUNNING_ON_NODE,
+        new AppRunningOnNodeTransition())
     // ignorable transitions
     .addTransition(RMAppState.FINAL_SAVING, RMAppState.FINAL_SAVING,
         EnumSet.of(RMAppEventType.NODE_UPDATE, RMAppEventType.KILL,
@@ -243,6 +252,9 @@ public class RMAppImpl implements RMApp, Recoverable {
      // Transitions from FINISHING state
     .addTransition(RMAppState.FINISHING, RMAppState.FINISHED,
         RMAppEventType.ATTEMPT_FINISHED, FINISHED_TRANSITION)
+    .addTransition(RMAppState.FINISHING, RMAppState.FINISHING, 
+        RMAppEventType.APP_RUNNING_ON_NODE,
+        new AppRunningOnNodeTransition())
     // ignorable transitions
     .addTransition(RMAppState.FINISHING, RMAppState.FINISHING,
       EnumSet.of(RMAppEventType.NODE_UPDATE,
@@ -251,6 +263,9 @@ public class RMAppImpl implements RMApp, Recoverable {
         RMAppEventType.KILL))
 
      // Transitions from KILLING state
+    .addTransition(RMAppState.KILLING, RMAppState.KILLING, 
+        RMAppEventType.APP_RUNNING_ON_NODE,
+        new AppRunningOnNodeTransition())
     .addTransition(RMAppState.KILLING, RMAppState.FINAL_SAVING,
         RMAppEventType.ATTEMPT_KILLED,
         new FinalSavingTransition(
@@ -267,6 +282,9 @@ public class RMAppImpl implements RMApp, Recoverable {
 
      // Transitions from FINISHED state
      // ignorable transitions
+    .addTransition(RMAppState.FINISHED, RMAppState.FINISHED, 
+        RMAppEventType.APP_RUNNING_ON_NODE,
+        new AppRunningOnNodeTransition())
     .addTransition(RMAppState.FINISHED, RMAppState.FINISHED,
         EnumSet.of(
             RMAppEventType.NODE_UPDATE,
@@ -276,11 +294,17 @@ public class RMAppImpl implements RMApp, Recoverable {
 
      // Transitions from FAILED state
      // ignorable transitions
+    .addTransition(RMAppState.FAILED, RMAppState.FAILED, 
+        RMAppEventType.APP_RUNNING_ON_NODE,
+        new AppRunningOnNodeTransition())
     .addTransition(RMAppState.FAILED, RMAppState.FAILED,
         EnumSet.of(RMAppEventType.KILL, RMAppEventType.NODE_UPDATE))
 
      // Transitions from KILLED state
      // ignorable transitions
+    .addTransition(RMAppState.KILLED, RMAppState.KILLED, 
+        RMAppEventType.APP_RUNNING_ON_NODE,
+        new AppRunningOnNodeTransition())
     .addTransition(
         RMAppState.KILLED,
         RMAppState.KILLED,
@@ -695,6 +719,23 @@ public class RMAppImpl implements RMApp, Recoverable {
           nodeUpdateEvent.getNode());
     };
   }
+  
+  private static final class AppRunningOnNodeTransition extends RMAppTransition {
+    public void transition(RMAppImpl app, RMAppEvent event) {
+      RMAppRunningOnNodeEvent nodeAddedEvent = (RMAppRunningOnNodeEvent) event;
+      
+      // if final state already stored, notify RMNode
+      if (isAppInFinalState(app)) {
+        app.handler.handle(
+            new RMNodeCleanAppEvent(nodeAddedEvent.getNodeId(), nodeAddedEvent
+                .getApplicationId()));
+        return;
+      }
+      
+      // otherwise, add it to ranNodes for further process
+      app.ranNodes.add(nodeAddedEvent.getNodeId());
+    };
+  }
 
   /**
    * Move an app to a new queue.
@@ -1037,17 +1078,8 @@ public class RMAppImpl implements RMApp, Recoverable {
       this.finalState = finalState;
     }
 
-    private Set<NodeId> getNodesOnWhichAttemptRan(RMAppImpl app) {
-      Set<NodeId> nodes = new HashSet<NodeId>();
-      for (RMAppAttempt attempt : app.attempts.values()) {
-        nodes.addAll(attempt.getRanNodes());
-      }
-      return nodes;
-    }
-
     public void transition(RMAppImpl app, RMAppEvent event) {
-      Set<NodeId> nodes = getNodesOnWhichAttemptRan(app);
-      for (NodeId nodeId : nodes) {
+      for (NodeId nodeId : app.getRanNodes()) {
         app.handler.handle(
             new RMNodeCleanAppEvent(nodeId, app.applicationId));
       }
@@ -1148,4 +1180,9 @@ public class RMAppImpl implements RMApp, Recoverable {
   private RMAppState getRecoveredFinalState() {
     return this.recoveredFinalState;
   }
+
+  @Override
+  public Set<NodeId> getRanNodes() {
+    return ranNodes;
+  }
 }

+ 11 - 16
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/event/RMAppAttemptContainerAcquiredEvent.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppRunningOnNodeEvent.java

@@ -16,25 +16,20 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event;
+package org.apache.hadoop.yarn.server.resourcemanager.rmapp;
 
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.NodeId;
 
-public class RMAppAttemptContainerAcquiredEvent extends RMAppAttemptEvent {
+public class RMAppRunningOnNodeEvent extends RMAppEvent {
+  private final NodeId node;
 
-  private final Container container;
-
-  public RMAppAttemptContainerAcquiredEvent(ApplicationAttemptId appAttemptId, 
-      Container container) {
-    super(appAttemptId, RMAppAttemptEventType.CONTAINER_ACQUIRED);
-    this.container = container;
+  public RMAppRunningOnNodeEvent(ApplicationId appId, NodeId node) {
+    super(appId, RMAppEventType.APP_RUNNING_ON_NODE);
+    this.node = node;
   }
-
-  public Container getContainer() {
-    return this.container;
+  
+  public NodeId getNodeId() {
+    return node;
   }
-
 }

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

@@ -19,7 +19,6 @@
 package org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt;
 
 import java.util.List;
-import java.util.Set;
 
 import javax.crypto.SecretKey;
 
@@ -32,7 +31,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
-import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.EventHandler;
@@ -114,12 +112,6 @@ public interface RMAppAttempt extends EventHandler<RMAppAttemptEvent> {
    */
   FinalApplicationStatus getFinalApplicationStatus();
 
-  /**
-   * Nodes on which the containers for this {@link RMAppAttempt} ran.
-   * @return the set of nodes that ran any containers from this {@link RMAppAttempt}
-   */
-  Set<NodeId> getRanNodes();
-
   /**
    * Return a list of the last set of finished containers, resetting the
    * finished containers to empty.

+ 0 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptEventType.java

@@ -36,7 +36,6 @@ public enum RMAppAttemptEventType {
   UNREGISTERED,
 
   // Source: Containers
-  CONTAINER_ACQUIRED,
   CONTAINER_ALLOCATED,
   CONTAINER_FINISHED,
   

+ 4 - 39
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java

@@ -26,16 +26,13 @@ import java.net.URISyntaxException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.EnumSet;
-import java.util.HashSet;
 import java.util.List;
-import java.util.Set;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
 
 import javax.crypto.SecretKey;
 
-import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -54,7 +51,6 @@ import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
-import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
@@ -80,7 +76,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppFailedAttemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppFinishedAttemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAcquiredEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAllocatedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerFinishedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptLaunchFailedEvent;
@@ -103,6 +98,8 @@ import org.apache.hadoop.yarn.state.StateMachine;
 import org.apache.hadoop.yarn.state.StateMachineFactory;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 
+import com.google.common.annotations.VisibleForTesting;
+
 @SuppressWarnings({"unchecked", "rawtypes"})
 public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
 
@@ -133,10 +130,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
   private final ApplicationSubmissionContext submissionContext;
   private Token<AMRMTokenIdentifier> amrmToken = null;
   private SecretKey clientTokenMasterKey = null;
-
-  //nodes on while this attempt's containers ran
-  private Set<NodeId> ranNodes =
-    new HashSet<NodeId>();
+  
   private List<ContainerStatus> justFinishedContainers =
     new ArrayList<ContainerStatus>();
   private Container masterContainer;
@@ -219,10 +213,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
       .addTransition(RMAppAttemptState.ALLOCATED_SAVING, 
           RMAppAttemptState.ALLOCATED,
           RMAppAttemptEventType.ATTEMPT_NEW_SAVED, new AttemptStoredTransition())
-      .addTransition(RMAppAttemptState.ALLOCATED_SAVING, 
-          RMAppAttemptState.ALLOCATED_SAVING,
-          RMAppAttemptEventType.CONTAINER_ACQUIRED, 
-          new ContainerAcquiredTransition())
+          
        // App could be killed by the client. So need to handle this. 
       .addTransition(RMAppAttemptState.ALLOCATED_SAVING, 
           RMAppAttemptState.FINAL_SAVING,
@@ -249,10 +240,6 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
             RMAppAttemptState.KILLED), RMAppAttemptState.KILLED))
 
        // Transitions from ALLOCATED State
-      .addTransition(RMAppAttemptState.ALLOCATED,
-          RMAppAttemptState.ALLOCATED,
-          RMAppAttemptEventType.CONTAINER_ACQUIRED,
-          new ContainerAcquiredTransition())
       .addTransition(RMAppAttemptState.ALLOCATED, RMAppAttemptState.LAUNCHED,
           RMAppAttemptEventType.LAUNCHED, new AMLaunchedTransition())
       .addTransition(RMAppAttemptState.ALLOCATED, RMAppAttemptState.FINAL_SAVING,
@@ -296,10 +283,6 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
           RMAppAttemptEventType.STATUS_UPDATE, new StatusUpdateTransition())
       .addTransition(RMAppAttemptState.RUNNING, RMAppAttemptState.RUNNING,
           RMAppAttemptEventType.CONTAINER_ALLOCATED)
-      .addTransition(
-                RMAppAttemptState.RUNNING, RMAppAttemptState.RUNNING,
-                RMAppAttemptEventType.CONTAINER_ACQUIRED,
-                new ContainerAcquiredTransition())
       .addTransition(
           RMAppAttemptState.RUNNING,
           EnumSet.of(RMAppAttemptState.RUNNING, RMAppAttemptState.FINAL_SAVING),
@@ -337,7 +320,6 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
             // should be fixed to reject container allocate request at Final
             // Saving in scheduler
               RMAppAttemptEventType.CONTAINER_ALLOCATED,
-              RMAppAttemptEventType.CONTAINER_ACQUIRED,
               RMAppAttemptEventType.ATTEMPT_NEW_SAVED,
               RMAppAttemptEventType.KILL))
 
@@ -619,11 +601,6 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     }
   }
 
-  @Override
-  public Set<NodeId> getRanNodes() {
-    return ranNodes;
-  }
-
   @Override
   public Container getMasterContainer() {
     this.readLock.lock();
@@ -705,7 +682,6 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
 
   public void transferStateFromPreviousAttempt(RMAppAttempt attempt) {
     this.justFinishedContainers = attempt.getJustFinishedContainers();
-    this.ranNodes = attempt.getRanNodes();
   }
 
   private void recoverAppAttemptCredentials(Credentials appAttemptTokens)
@@ -1402,17 +1378,6 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     finalStatus = unregisterEvent.getFinalApplicationStatus();
   }
 
-  private static final class ContainerAcquiredTransition extends
-      BaseTransition {
-    @Override
-    public void transition(RMAppAttemptImpl appAttempt,
-        RMAppAttemptEvent event) {
-      RMAppAttemptContainerAcquiredEvent acquiredEvent
-        = (RMAppAttemptContainerAcquiredEvent) event;
-      appAttempt.ranNodes.add(acquiredEvent.getContainer().getNodeId());
-    }
-  }
-
   private static final class ContainerFinishedTransition
       implements
       MultipleArcTransition<RMAppAttemptImpl, RMAppAttemptEvent, RMAppAttemptState> {

+ 7 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java

@@ -37,7 +37,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAcquiredEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRunningOnNodeEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAllocatedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerFinishedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
@@ -365,9 +365,9 @@ public class RMContainerImpl implements RMContainer {
             RMContainerEventType.FINISHED));
         return RMContainerState.COMPLETED;
       } else if (report.getContainerState().equals(ContainerState.RUNNING)) {
-        // Tell the appAttempt
-        container.eventHandler.handle(new RMAppAttemptContainerAcquiredEvent(
-            container.getApplicationAttemptId(), container.getContainer()));
+        // Tell the app
+        container.eventHandler.handle(new RMAppRunningOnNodeEvent(container
+            .getApplicationAttemptId().getApplicationId(), container.nodeId));
         return RMContainerState.RUNNING;
       } else {
         // This can never happen.
@@ -408,9 +408,9 @@ public class RMContainerImpl implements RMContainer {
       // Register with containerAllocationExpirer.
       container.containerAllocationExpirer.register(container.getContainerId());
 
-      // Tell the appAttempt
-      container.eventHandler.handle(new RMAppAttemptContainerAcquiredEvent(
-          container.getApplicationAttemptId(), container.getContainer()));
+      // Tell the app
+      container.eventHandler.handle(new RMAppRunningOnNodeEvent(container
+          .getApplicationAttemptId().getApplicationId(), container.nodeId));
     }
   }
 

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

@@ -55,6 +55,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.ClusterMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.NodesListManagerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.NodesListManagerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRunningOnNodeEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
@@ -473,7 +475,13 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
       } else {
         // Increment activeNodes explicitly because this is a new node.
         ClusterMetrics.getMetrics().incrNumActiveNodes();
-        containers = startEvent.getContainerRecoveryReports();
+        containers = startEvent.getNMContainerStatuses();
+      }
+      
+      if (null != startEvent.getRunningApplications()) {
+        for (ApplicationId appId : startEvent.getRunningApplications()) {
+          handleRunningAppOnNode(rmNode, rmNode.context, appId, rmNode.nodeId);
+        }
       }
 
       rmNode.context.getDispatcher().getEventHandler()
@@ -482,6 +490,24 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
         new NodesListManagerEvent(
             NodesListManagerEventType.NODE_USABLE, rmNode));
     }
+
+    void handleRunningAppOnNode(RMNodeImpl rmNode, RMContext context,
+        ApplicationId appId, NodeId nodeId) {
+      RMApp app = context.getRMApps().get(appId);
+      
+      // if we failed getting app by appId, maybe something wrong happened, just
+      // add the app to the finishedApplications list so that the app can be
+      // cleaned up on the NM
+      if (null == app) {
+        LOG.warn("Cannot get RMApp by appId=" + appId
+            + ", just added it to finishedApplications list for cleanup");
+        rmNode.finishedApplications.add(appId);
+        return;
+      }
+
+      context.getDispatcher().getEventHandler()
+          .handle(new RMAppRunningOnNodeEvent(appId, nodeId));
+    }
   }
 
   public static class ReconnectNodeTransition implements
@@ -517,7 +543,7 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
         }
         rmNode.context.getRMNodes().put(newNode.getNodeID(), newNode);
         rmNode.context.getDispatcher().getEventHandler().handle(
-            new RMNodeStartedEvent(newNode.getNodeID(), null));
+            new RMNodeStartedEvent(newNode.getNodeID(), null, null));
       }
       rmNode.context.getDispatcher().getEventHandler().handle(
           new NodesListManagerEvent(

+ 14 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeStartedEvent.java

@@ -20,19 +20,28 @@ package org.apache.hadoop.yarn.server.resourcemanager.rmnode;
 
 import java.util.List;
 
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
 
 public class RMNodeStartedEvent extends RMNodeEvent {
 
-  private List<NMContainerStatus> containerReports;
+  private List<NMContainerStatus> containerStatuses;
+  private List<ApplicationId> runningApplications;
 
-  public RMNodeStartedEvent(NodeId nodeId, List<NMContainerStatus> containerReports) {
+  public RMNodeStartedEvent(NodeId nodeId,
+      List<NMContainerStatus> containerReports,
+      List<ApplicationId> runningApplications) {
     super(nodeId, RMNodeEventType.STARTED);
-    this.containerReports = containerReports;
+    this.containerStatuses = containerReports;
+    this.runningApplications = runningApplications;
   }
 
-  public List<NMContainerStatus> getContainerRecoveryReports() {
-    return this.containerReports;
+  public List<NMContainerStatus> getNMContainerStatuses() {
+    return this.containerStatuses;
+  }
+  
+  public List<ApplicationId> getRunningApplications() {
+    return runningApplications;
   }
 }

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java

@@ -130,9 +130,9 @@ public abstract class SchedulerNode {
 
     LOG.info("Assigned container " + container.getId() + " of capacity "
         + container.getResource() + " on host " + rmNode.getNodeAddress()
-        + ", which currently has " + numContainers + " containers, "
+        + ", which has " + numContainers + " containers, "
         + getUsedResource() + " used and " + getAvailableResource()
-        + " available");
+        + " available after allocation");
   }
 
   /**

برخی فایل ها در این مقایسه diff نمایش داده نمی شوند زیرا تعداد فایل ها بسیار زیاد است