Selaa lähdekoodia

Merge r1609845 through r1616427 from trunk.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-6584@1616428 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 10 vuotta sitten
vanhempi
commit
b50cb5e056
100 muutettua tiedostoa jossa 4131 lisäystä ja 1466 poistoa
  1. 38 0
      hadoop-common-project/hadoop-auth/dev-support/findbugsExcludeFile.xml
  2. 7 0
      hadoop-common-project/hadoop-auth/pom.xml
  3. 72 8
      hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java
  4. 49 0
      hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/RandomSignerSecretProvider.java
  5. 139 0
      hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/RolloverSignerSecretProvider.java
  6. 32 14
      hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/Signer.java
  7. 62 0
      hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/SignerSecretProvider.java
  8. 49 0
      hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/StringSignerSecretProvider.java
  9. 110 8
      hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/TestAuthenticationFilter.java
  10. 63 0
      hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestRandomSignerSecretProvider.java
  11. 79 0
      hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestRolloverSignerSecretProvider.java
  12. 69 16
      hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestSigner.java
  13. 33 0
      hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestStringSignerSecretProvider.java
  14. 43 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  15. 9 5
      hadoop-common-project/hadoop-common/src/main/bin/hadoop
  16. 0 2
      hadoop-common-project/hadoop-common/src/main/bin/hadoop-config.sh
  17. 10 3
      hadoop-common-project/hadoop-common/src/main/bin/hadoop.cmd
  18. 11 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java
  19. 10 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java
  20. 45 44
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyShell.java
  21. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
  22. 5 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Delete.java
  23. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java
  24. 12 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java
  25. 7 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/jmx/JMXJsonServlet.java
  26. 22 4
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java
  27. 2 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java
  28. 36 29
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/CredentialShell.java
  29. 19 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/FileBasedKeyStoresFactory.java
  30. 125 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Classpath.java
  31. 14 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/DiskChecker.java
  32. 37 0
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/nativeio/NativeIO.c
  33. 17 2
      hadoop-common-project/hadoop-common/src/site/apt/CommandsManual.apt.vm
  34. 9 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderCryptoExtension.java
  35. 7 7
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderFactory.java
  36. 28 28
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyShell.java
  37. 3 3
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java
  38. 58 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java
  39. 34 15
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredShell.java
  40. 41 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/KeyStoreTestUtil.java
  41. 57 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestSSLFactory.java
  42. 176 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestClasspath.java
  43. 33 0
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSJMXServlet.java
  44. 8 1
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebApp.java
  45. 1 1
      hadoop-common-project/hadoop-kms/src/main/webapp/WEB-INF/web.xml
  46. 6 6
      hadoop-common-project/hadoop-kms/src/site/apt/index.apt.vm
  47. 2 1
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java
  48. 215 166
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
  49. 587 3
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestRpcProgramNfs3.java
  50. 50 3
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  51. 0 44
      hadoop-hdfs-project/hadoop-hdfs/src/main/conf/hadoop-metrics2.properties
  52. 7 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  53. 28 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
  54. 11 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/Block.java
  55. 61 160
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
  56. 173 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/KeyManager.java
  57. 51 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Matcher.java
  58. 124 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/MovedBlocks.java
  59. 66 143
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java
  60. 4 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
  61. 109 69
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  62. 9 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
  63. 3 16
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CorruptReplicasMap.java
  64. 4 37
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
  65. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
  66. 15 19
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingDataNodeMessages.java
  67. 188 115
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
  68. 10 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java
  69. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
  70. 21 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  71. 4 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNodeLayoutVersion.java
  72. 122 22
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
  73. 36 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeUtil.java
  74. 22 40
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java
  75. 68 28
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
  76. 1 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
  77. 5 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java
  78. 0 228
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LDir.java
  79. 0 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  80. 6 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  81. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  82. 8 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  83. 8 5
      hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HdfsNfsGateway.apt.vm
  84. 28 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
  85. 11 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientFailover.java
  86. 5 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSFinalize.java
  87. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java
  88. 9 7
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStorageStateRecovery.java
  89. 7 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgrade.java
  90. 11 10
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
  91. 73 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
  92. 5 10
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeBlockScanner.java
  93. 48 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeLayoutUpgrade.java
  94. 57 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java
  95. 13 27
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCorruption.java
  96. 28 16
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/UpgradeUtilities.java
  97. 95 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
  98. 10 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
  99. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
  100. 10 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestCorruptReplicaInfo.java

+ 38 - 0
hadoop-common-project/hadoop-auth/dev-support/findbugsExcludeFile.xml

@@ -0,0 +1,38 @@
+<!--
+   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.
+-->
+<FindBugsFilter>
+  <!--
+    Caller is not supposed to modify returned values even though there's nothing
+    stopping them; we do this for performance reasons.
+  -->
+  <Match>
+    <Class name="org.apache.hadoop.security.authentication.util.RolloverSignerSecretProvider" />
+    <Method name="getAllSecrets" />
+    <Bug pattern="EI_EXPOSE_REP" />
+  </Match>
+  <Match>
+    <Class name="org.apache.hadoop.security.authentication.util.StringSignerSecretProvider" />
+    <Method name="getAllSecrets" />
+    <Bug pattern="EI_EXPOSE_REP" />
+  </Match>
+  <Match>
+    <Class name="org.apache.hadoop.security.authentication.util.StringSignerSecretProvider" />
+    <Method name="getCurrentSecret" />
+    <Bug pattern="EI_EXPOSE_REP" />
+  </Match>
+
+</FindBugsFilter>

+ 7 - 0
hadoop-common-project/hadoop-auth/pom.xml

@@ -150,6 +150,13 @@
           </execution>
         </executions>
       </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>findbugs-maven-plugin</artifactId>
+        <configuration>
+          <excludeFilterFile>${basedir}/dev-support/findbugsExcludeFile.xml</excludeFilterFile>
+        </configuration>
+      </plugin>
     </plugins>
   </build>
 

+ 72 - 8
hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java

@@ -19,6 +19,9 @@ import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.authentication.util.Signer;
 import org.apache.hadoop.security.authentication.util.SignerException;
+import org.apache.hadoop.security.authentication.util.RandomSignerSecretProvider;
+import org.apache.hadoop.security.authentication.util.SignerSecretProvider;
+import org.apache.hadoop.security.authentication.util.StringSignerSecretProvider;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -107,11 +110,28 @@ public class AuthenticationFilter implements Filter {
    */
   public static final String COOKIE_PATH = "cookie.path";
 
-  private static final Random RAN = new Random();
+  /**
+   * Constant for the configuration property that indicates the name of the
+   * SignerSecretProvider class to use.  If not specified, SIGNATURE_SECRET
+   * will be used or a random secret.
+   */
+  public static final String SIGNER_SECRET_PROVIDER_CLASS =
+          "signer.secret.provider";
+
+  /**
+   * Constant for the attribute that can be used for providing a custom
+   * object that subclasses the SignerSecretProvider.  Note that this should be
+   * set in the ServletContext and the class should already be initialized.  
+   * If not specified, SIGNER_SECRET_PROVIDER_CLASS will be used.
+   */
+  public static final String SIGNATURE_PROVIDER_ATTRIBUTE =
+      "org.apache.hadoop.security.authentication.util.SignerSecretProvider";
 
   private Signer signer;
+  private SignerSecretProvider secretProvider;
   private AuthenticationHandler authHandler;
   private boolean randomSecret;
+  private boolean customSecretProvider;
   private long validity;
   private String cookieDomain;
   private String cookiePath;
@@ -159,14 +179,46 @@ public class AuthenticationFilter implements Filter {
     } catch (IllegalAccessException ex) {
       throw new ServletException(ex);
     }
-    String signatureSecret = config.getProperty(configPrefix + SIGNATURE_SECRET);
-    if (signatureSecret == null) {
-      signatureSecret = Long.toString(RAN.nextLong());
-      randomSecret = true;
-      LOG.warn("'signature.secret' configuration not set, using a random value as secret");
+
+    validity = Long.parseLong(config.getProperty(AUTH_TOKEN_VALIDITY, "36000"))
+        * 1000; //10 hours
+    secretProvider = (SignerSecretProvider) filterConfig.getServletContext().
+        getAttribute(SIGNATURE_PROVIDER_ATTRIBUTE);
+    if (secretProvider == null) {
+      String signerSecretProviderClassName =
+          config.getProperty(configPrefix + SIGNER_SECRET_PROVIDER_CLASS, null);
+      if (signerSecretProviderClassName == null) {
+        String signatureSecret =
+            config.getProperty(configPrefix + SIGNATURE_SECRET, null);
+        if (signatureSecret != null) {
+          secretProvider = new StringSignerSecretProvider(signatureSecret);
+        } else {
+          secretProvider = new RandomSignerSecretProvider();
+          randomSecret = true;
+        }
+      } else {
+        try {
+          Class<?> klass = Thread.currentThread().getContextClassLoader().
+              loadClass(signerSecretProviderClassName);
+          secretProvider = (SignerSecretProvider) klass.newInstance();
+          customSecretProvider = true;
+        } catch (ClassNotFoundException ex) {
+          throw new ServletException(ex);
+        } catch (InstantiationException ex) {
+          throw new ServletException(ex);
+        } catch (IllegalAccessException ex) {
+          throw new ServletException(ex);
+        }
+      }
+      try {
+        secretProvider.init(config, validity);
+      } catch (Exception ex) {
+        throw new ServletException(ex);
+      }
+    } else {
+      customSecretProvider = true;
     }
-    signer = new Signer(signatureSecret.getBytes());
-    validity = Long.parseLong(config.getProperty(AUTH_TOKEN_VALIDITY, "36000")) * 1000; //10 hours
+    signer = new Signer(secretProvider);
 
     cookieDomain = config.getProperty(COOKIE_DOMAIN, null);
     cookiePath = config.getProperty(COOKIE_PATH, null);
@@ -190,6 +242,15 @@ public class AuthenticationFilter implements Filter {
     return randomSecret;
   }
 
+  /**
+   * Returns if a custom implementation of a SignerSecretProvider is being used.
+   *
+   * @return if a custom implementation of a SignerSecretProvider is being used.
+   */
+  protected boolean isCustomSignerSecretProvider() {
+    return customSecretProvider;
+  }
+
   /**
    * Returns the validity time of the generated tokens.
    *
@@ -228,6 +289,9 @@ public class AuthenticationFilter implements Filter {
       authHandler.destroy();
       authHandler = null;
     }
+    if (secretProvider != null) {
+      secretProvider.destroy();
+    }
   }
 
   /**

+ 49 - 0
hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/RandomSignerSecretProvider.java

@@ -0,0 +1,49 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License. See accompanying LICENSE file.
+ */
+package org.apache.hadoop.security.authentication.util;
+
+import java.util.Random;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A SignerSecretProvider that uses a random number as it's secret.  It rolls
+ * the secret at a regular interval.
+ */
+@InterfaceStability.Unstable
+@InterfaceAudience.Private
+public class RandomSignerSecretProvider extends RolloverSignerSecretProvider {
+
+  private final Random rand;
+
+  public RandomSignerSecretProvider() {
+    super();
+    rand = new Random();
+  }
+
+  /**
+   * This constructor lets you set the seed of the Random Number Generator and
+   * is meant for testing.
+   * @param seed the seed for the random number generator
+   */
+  public RandomSignerSecretProvider(long seed) {
+    super();
+    rand = new Random(seed);
+  }
+
+  @Override
+  protected byte[] generateNewSecret() {
+    return Long.toString(rand.nextLong()).getBytes();
+  }
+}

+ 139 - 0
hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/RolloverSignerSecretProvider.java

@@ -0,0 +1,139 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License. See accompanying LICENSE file.
+ */
+package org.apache.hadoop.security.authentication.util;
+
+import java.util.Properties;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An abstract SignerSecretProvider that can be use used as the base for a
+ * rolling secret.  The secret will roll over at the same interval as the token
+ * validity, so there are only ever a maximum of two valid secrets at any
+ * given time.  This class handles storing and returning the secrets, as well
+ * as the rolling over.  At a minimum, subclasses simply need to implement the
+ * generateNewSecret() method.  More advanced implementations can override
+ * other methods to provide more advanced behavior, but should be careful when
+ * doing so.
+ */
+@InterfaceStability.Unstable
+@InterfaceAudience.Private
+public abstract class RolloverSignerSecretProvider
+    extends SignerSecretProvider {
+
+  private static Logger LOG = LoggerFactory.getLogger(
+    RolloverSignerSecretProvider.class);
+  /**
+   * Stores the currently valid secrets.  The current secret is the 0th element
+   * in the array.
+   */
+  private volatile byte[][] secrets;
+  private ScheduledExecutorService scheduler;
+  private boolean schedulerRunning;
+  private boolean isDestroyed;
+
+  public RolloverSignerSecretProvider() {
+    schedulerRunning = false;
+    isDestroyed = false;
+  }
+
+  /**
+   * Initialize the SignerSecretProvider.  It initializes the current secret
+   * and starts the scheduler for the rollover to run at an interval of
+   * tokenValidity.
+   * @param config filter configuration
+   * @param tokenValidity The amount of time a token is valid for
+   * @throws Exception
+   */
+  @Override
+  public void init(Properties config, long tokenValidity) throws Exception {
+    initSecrets(generateNewSecret(), null);
+    startScheduler(tokenValidity, tokenValidity);
+  }
+
+  /**
+   * Initializes the secrets array.  This should typically be called only once,
+   * during init but some implementations may wish to call it other times.
+   * previousSecret can be null if there isn't a previous secret, but
+   * currentSecret should never be null.
+   * @param currentSecret The current secret
+   * @param previousSecret The previous secret
+   */
+  protected void initSecrets(byte[] currentSecret, byte[] previousSecret) {
+    secrets = new byte[][]{currentSecret, previousSecret};
+  }
+
+  /**
+   * Starts the scheduler for the rollover to run at an interval.
+   * @param initialDelay The initial delay in the rollover in milliseconds
+   * @param period The interval for the rollover in milliseconds
+   */
+  protected synchronized void startScheduler(long initialDelay, long period) {
+    if (!schedulerRunning) {
+      schedulerRunning = true;
+      scheduler = Executors.newSingleThreadScheduledExecutor();
+      scheduler.scheduleAtFixedRate(new Runnable() {
+        @Override
+        public void run() {
+          rollSecret();
+        }
+      }, initialDelay, period, TimeUnit.MILLISECONDS);
+    }
+  }
+
+  @Override
+  public synchronized void destroy() {
+    if (!isDestroyed) {
+      isDestroyed = true;
+      if (scheduler != null) {
+        scheduler.shutdown();
+      }
+      schedulerRunning = false;
+      super.destroy();
+    }
+  }
+
+  /**
+   * Rolls the secret.  It is called automatically at the rollover interval.
+   */
+  protected synchronized void rollSecret() {
+    if (!isDestroyed) {
+      LOG.debug("rolling secret");
+      byte[] newSecret = generateNewSecret();
+      secrets = new byte[][]{newSecret, secrets[0]};
+    }
+  }
+
+  /**
+   * Subclasses should implement this to return a new secret.  It will be called
+   * automatically at the secret rollover interval. It should never return null.
+   * @return a new secret
+   */
+  protected abstract byte[] generateNewSecret();
+
+  @Override
+  public byte[] getCurrentSecret() {
+    return secrets[0];
+  }
+
+  @Override
+  public byte[][] getAllSecrets() {
+    return secrets;
+  }
+}

+ 32 - 14
hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/Signer.java

@@ -24,18 +24,19 @@ import java.security.NoSuchAlgorithmException;
 public class Signer {
   private static final String SIGNATURE = "&s=";
 
-  private byte[] secret;
+  private SignerSecretProvider secretProvider;
 
   /**
-   * Creates a Signer instance using the specified secret.
+   * Creates a Signer instance using the specified SignerSecretProvider.  The
+   * SignerSecretProvider should already be initialized.
    *
-   * @param secret secret to use for creating the digest.
+   * @param secretProvider The SignerSecretProvider to use
    */
-  public Signer(byte[] secret) {
-    if (secret == null) {
-      throw new IllegalArgumentException("secret cannot be NULL");
+  public Signer(SignerSecretProvider secretProvider) {
+    if (secretProvider == null) {
+      throw new IllegalArgumentException("secretProvider cannot be NULL");
     }
-    this.secret = secret.clone();
+    this.secretProvider = secretProvider;
   }
 
   /**
@@ -47,11 +48,12 @@ public class Signer {
    *
    * @return the signed string.
    */
-  public String sign(String str) {
+  public synchronized String sign(String str) {
     if (str == null || str.length() == 0) {
       throw new IllegalArgumentException("NULL or empty string to sign");
     }
-    String signature = computeSignature(str);
+    byte[] secret = secretProvider.getCurrentSecret();
+    String signature = computeSignature(secret, str);
     return str + SIGNATURE + signature;
   }
 
@@ -71,21 +73,19 @@ public class Signer {
     }
     String originalSignature = signedStr.substring(index + SIGNATURE.length());
     String rawValue = signedStr.substring(0, index);
-    String currentSignature = computeSignature(rawValue);
-    if (!originalSignature.equals(currentSignature)) {
-      throw new SignerException("Invalid signature");
-    }
+    checkSignatures(rawValue, originalSignature);
     return rawValue;
   }
 
   /**
    * Returns then signature of a string.
    *
+   * @param secret The secret to use
    * @param str string to sign.
    *
    * @return the signature for the string.
    */
-  protected String computeSignature(String str) {
+  protected String computeSignature(byte[] secret, String str) {
     try {
       MessageDigest md = MessageDigest.getInstance("SHA");
       md.update(str.getBytes());
@@ -97,4 +97,22 @@ public class Signer {
     }
   }
 
+  protected void checkSignatures(String rawValue, String originalSignature)
+      throws SignerException {
+    boolean isValid = false;
+    byte[][] secrets = secretProvider.getAllSecrets();
+    for (int i = 0; i < secrets.length; i++) {
+      byte[] secret = secrets[i];
+      if (secret != null) {
+        String currentSignature = computeSignature(secret, rawValue);
+        if (originalSignature.equals(currentSignature)) {
+          isValid = true;
+          break;
+        }
+      }
+    }
+    if (!isValid) {
+      throw new SignerException("Invalid signature");
+    }
+  }
 }

+ 62 - 0
hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/SignerSecretProvider.java

@@ -0,0 +1,62 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License. See accompanying LICENSE file.
+ */
+package org.apache.hadoop.security.authentication.util;
+
+import java.util.Properties;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * The SignerSecretProvider is an abstract way to provide a secret to be used
+ * by the Signer so that we can have different implementations that potentially
+ * do more complicated things in the backend.
+ * See the RolloverSignerSecretProvider class for an implementation that
+ * supports rolling over the secret at a regular interval.
+ */
+@InterfaceStability.Unstable
+@InterfaceAudience.Private
+public abstract class SignerSecretProvider {
+
+  /**
+   * Initialize the SignerSecretProvider
+   * @param config filter configuration
+   * @param tokenValidity The amount of time a token is valid for
+   * @throws Exception
+   */
+  public abstract void init(Properties config, long tokenValidity)
+      throws Exception;
+
+  /**
+   * Will be called on shutdown; subclasses should perform any cleanup here.
+   */
+  public void destroy() {}
+
+  /**
+   * Returns the current secret to be used by the Signer for signing new
+   * cookies.  This should never return null.
+   * <p>
+   * Callers should be careful not to modify the returned value.
+   * @return the current secret
+   */
+  public abstract byte[] getCurrentSecret();
+
+  /**
+   * Returns all secrets that a cookie could have been signed with and are still
+   * valid; this should include the secret returned by getCurrentSecret().
+   * <p>
+   * Callers should be careful not to modify the returned value.
+   * @return the secrets
+   */
+  public abstract byte[][] getAllSecrets();
+}

+ 49 - 0
hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/StringSignerSecretProvider.java

@@ -0,0 +1,49 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License. See accompanying LICENSE file.
+ */
+package org.apache.hadoop.security.authentication.util;
+
+import java.util.Properties;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A SignerSecretProvider that simply creates a secret based on a given String.
+ */
+@InterfaceStability.Unstable
+@InterfaceAudience.Private
+public class StringSignerSecretProvider extends SignerSecretProvider {
+
+  private byte[] secret;
+  private byte[][] secrets;
+
+  public StringSignerSecretProvider(String secretStr) {
+    secret = secretStr.getBytes();
+    secrets = new byte[][]{secret};
+  }
+
+  @Override
+  public void init(Properties config, long tokenValidity) throws Exception {
+    // do nothing
+  }
+
+  @Override
+  public byte[] getCurrentSecret() {
+    return secret;
+  }
+
+  @Override
+  public byte[][] getAllSecrets() {
+    return secrets;
+  }
+}

+ 110 - 8
hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/TestAuthenticationFilter.java

@@ -23,6 +23,7 @@ import java.util.Vector;
 
 import javax.servlet.FilterChain;
 import javax.servlet.FilterConfig;
+import javax.servlet.ServletContext;
 import javax.servlet.ServletException;
 import javax.servlet.ServletRequest;
 import javax.servlet.ServletResponse;
@@ -33,6 +34,8 @@ import javax.servlet.http.HttpServletResponse;
 import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.authentication.util.Signer;
+import org.apache.hadoop.security.authentication.util.SignerSecretProvider;
+import org.apache.hadoop.security.authentication.util.StringSignerSecretProvider;
 import org.junit.Assert;
 import org.junit.Test;
 import org.mockito.Mockito;
@@ -157,9 +160,14 @@ public class TestAuthenticationFilter {
       Mockito.when(config.getInitParameterNames()).thenReturn(
         new Vector<String>(Arrays.asList(AuthenticationFilter.AUTH_TYPE,
                                  AuthenticationFilter.AUTH_TOKEN_VALIDITY)).elements());
+      ServletContext context = Mockito.mock(ServletContext.class);
+      Mockito.when(context.getAttribute(
+          AuthenticationFilter.SIGNATURE_PROVIDER_ATTRIBUTE)).thenReturn(null);
+      Mockito.when(config.getServletContext()).thenReturn(context);
       filter.init(config);
       Assert.assertEquals(PseudoAuthenticationHandler.class, filter.getAuthenticationHandler().getClass());
       Assert.assertTrue(filter.isRandomSecret());
+      Assert.assertFalse(filter.isCustomSignerSecretProvider());
       Assert.assertNull(filter.getCookieDomain());
       Assert.assertNull(filter.getCookiePath());
       Assert.assertEquals(TOKEN_VALIDITY_SEC, filter.getValidity());
@@ -167,6 +175,26 @@ public class TestAuthenticationFilter {
       filter.destroy();
     }
 
+    // string secret
+    filter = new AuthenticationFilter();
+    try {
+      FilterConfig config = Mockito.mock(FilterConfig.class);
+      Mockito.when(config.getInitParameter(AuthenticationFilter.AUTH_TYPE)).thenReturn("simple");
+      Mockito.when(config.getInitParameter(AuthenticationFilter.SIGNATURE_SECRET)).thenReturn("secret");
+      Mockito.when(config.getInitParameterNames()).thenReturn(
+        new Vector<String>(Arrays.asList(AuthenticationFilter.AUTH_TYPE,
+                                 AuthenticationFilter.SIGNATURE_SECRET)).elements());
+      ServletContext context = Mockito.mock(ServletContext.class);
+      Mockito.when(context.getAttribute(
+          AuthenticationFilter.SIGNATURE_PROVIDER_ATTRIBUTE)).thenReturn(null);
+      Mockito.when(config.getServletContext()).thenReturn(context);
+      filter.init(config);
+      Assert.assertFalse(filter.isRandomSecret());
+      Assert.assertFalse(filter.isCustomSignerSecretProvider());
+    } finally {
+      filter.destroy();
+    }
+
     // custom secret
     filter = new AuthenticationFilter();
     try {
@@ -176,8 +204,26 @@ public class TestAuthenticationFilter {
       Mockito.when(config.getInitParameterNames()).thenReturn(
         new Vector<String>(Arrays.asList(AuthenticationFilter.AUTH_TYPE,
                                  AuthenticationFilter.SIGNATURE_SECRET)).elements());
+      ServletContext context = Mockito.mock(ServletContext.class);
+      Mockito.when(context.getAttribute(
+          AuthenticationFilter.SIGNATURE_PROVIDER_ATTRIBUTE)).thenReturn(
+            new SignerSecretProvider() {
+              @Override
+              public void init(Properties config, long tokenValidity) {
+              }
+              @Override
+              public byte[] getCurrentSecret() {
+                return null;
+              }
+              @Override
+              public byte[][] getAllSecrets() {
+                return null;
+              }
+            });
+      Mockito.when(config.getServletContext()).thenReturn(context);
       filter.init(config);
       Assert.assertFalse(filter.isRandomSecret());
+      Assert.assertTrue(filter.isCustomSignerSecretProvider());
     } finally {
       filter.destroy();
     }
@@ -193,6 +239,10 @@ public class TestAuthenticationFilter {
         new Vector<String>(Arrays.asList(AuthenticationFilter.AUTH_TYPE,
                                  AuthenticationFilter.COOKIE_DOMAIN,
                                  AuthenticationFilter.COOKIE_PATH)).elements());
+      ServletContext context = Mockito.mock(ServletContext.class);
+      Mockito.when(context.getAttribute(
+          AuthenticationFilter.SIGNATURE_PROVIDER_ATTRIBUTE)).thenReturn(null);
+      Mockito.when(config.getServletContext()).thenReturn(context);
       filter.init(config);
       Assert.assertEquals(".foo.com", filter.getCookieDomain());
       Assert.assertEquals("/bar", filter.getCookiePath());
@@ -213,6 +263,10 @@ public class TestAuthenticationFilter {
         new Vector<String>(
           Arrays.asList(AuthenticationFilter.AUTH_TYPE,
                         "management.operation.return")).elements());
+      ServletContext context = Mockito.mock(ServletContext.class);
+      Mockito.when(context.getAttribute(
+          AuthenticationFilter.SIGNATURE_PROVIDER_ATTRIBUTE)).thenReturn(null);
+      Mockito.when(config.getServletContext()).thenReturn(context);
       filter.init(config);
       Assert.assertTrue(DummyAuthenticationHandler.init);
     } finally {
@@ -248,6 +302,10 @@ public class TestAuthenticationFilter {
       Mockito.when(config.getInitParameterNames()).thenReturn(
           new Vector<String>(Arrays.asList(AuthenticationFilter.AUTH_TYPE,
               AuthenticationFilter.AUTH_TOKEN_VALIDITY)).elements());
+      ServletContext context = Mockito.mock(ServletContext.class);
+      Mockito.when(context.getAttribute(
+          AuthenticationFilter.SIGNATURE_PROVIDER_ATTRIBUTE)).thenReturn(null);
+      Mockito.when(config.getServletContext()).thenReturn(context);
 
       filter.init(config);
       Assert.assertEquals(PseudoAuthenticationHandler.class, 
@@ -270,6 +328,10 @@ public class TestAuthenticationFilter {
         new Vector<String>(
           Arrays.asList(AuthenticationFilter.AUTH_TYPE,
                         "management.operation.return")).elements());
+      ServletContext context = Mockito.mock(ServletContext.class);
+      Mockito.when(context.getAttribute(
+          AuthenticationFilter.SIGNATURE_PROVIDER_ATTRIBUTE)).thenReturn(null);
+      Mockito.when(config.getServletContext()).thenReturn(context);
       filter.init(config);
 
       HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
@@ -297,11 +359,15 @@ public class TestAuthenticationFilter {
           Arrays.asList(AuthenticationFilter.AUTH_TYPE,
                         AuthenticationFilter.SIGNATURE_SECRET,
                         "management.operation.return")).elements());
+      ServletContext context = Mockito.mock(ServletContext.class);
+      Mockito.when(context.getAttribute(
+          AuthenticationFilter.SIGNATURE_PROVIDER_ATTRIBUTE)).thenReturn(null);
+      Mockito.when(config.getServletContext()).thenReturn(context);
       filter.init(config);
 
       AuthenticationToken token = new AuthenticationToken("u", "p", DummyAuthenticationHandler.TYPE);
       token.setExpires(System.currentTimeMillis() + TOKEN_VALIDITY_SEC);
-      Signer signer = new Signer("secret".getBytes());
+      Signer signer = new Signer(new StringSignerSecretProvider("secret"));
       String tokenSigned = signer.sign(token.toString());
 
       Cookie cookie = new Cookie(AuthenticatedURL.AUTH_COOKIE, tokenSigned);
@@ -330,12 +396,16 @@ public class TestAuthenticationFilter {
           Arrays.asList(AuthenticationFilter.AUTH_TYPE,
                         AuthenticationFilter.SIGNATURE_SECRET,
                         "management.operation.return")).elements());
+      ServletContext context = Mockito.mock(ServletContext.class);
+      Mockito.when(context.getAttribute(
+          AuthenticationFilter.SIGNATURE_PROVIDER_ATTRIBUTE)).thenReturn(null);
+      Mockito.when(config.getServletContext()).thenReturn(context);
       filter.init(config);
 
       AuthenticationToken token =
           new AuthenticationToken("u", "p", DummyAuthenticationHandler.TYPE);
       token.setExpires(System.currentTimeMillis() - TOKEN_VALIDITY_SEC);
-      Signer signer = new Signer("secret".getBytes());
+      Signer signer = new Signer(new StringSignerSecretProvider("secret"));
       String tokenSigned = signer.sign(token.toString());
 
       Cookie cookie = new Cookie(AuthenticatedURL.AUTH_COOKIE, tokenSigned);
@@ -371,11 +441,15 @@ public class TestAuthenticationFilter {
           Arrays.asList(AuthenticationFilter.AUTH_TYPE,
                         AuthenticationFilter.SIGNATURE_SECRET,
                         "management.operation.return")).elements());
+      ServletContext context = Mockito.mock(ServletContext.class);
+      Mockito.when(context.getAttribute(
+          AuthenticationFilter.SIGNATURE_PROVIDER_ATTRIBUTE)).thenReturn(null);
+      Mockito.when(config.getServletContext()).thenReturn(context);
       filter.init(config);
 
       AuthenticationToken token = new AuthenticationToken("u", "p", "invalidtype");
       token.setExpires(System.currentTimeMillis() + TOKEN_VALIDITY_SEC);
-      Signer signer = new Signer("secret".getBytes());
+      Signer signer = new Signer(new StringSignerSecretProvider("secret"));
       String tokenSigned = signer.sign(token.toString());
 
       Cookie cookie = new Cookie(AuthenticatedURL.AUTH_COOKIE, tokenSigned);
@@ -409,6 +483,10 @@ public class TestAuthenticationFilter {
         new Vector<String>(
           Arrays.asList(AuthenticationFilter.AUTH_TYPE,
                         "management.operation.return")).elements());
+      ServletContext context = Mockito.mock(ServletContext.class);
+      Mockito.when(context.getAttribute(
+          AuthenticationFilter.SIGNATURE_PROVIDER_ATTRIBUTE)).thenReturn(null);
+      Mockito.when(config.getServletContext()).thenReturn(context);
       filter.init(config);
 
       HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
@@ -458,6 +536,10 @@ public class TestAuthenticationFilter {
             AuthenticationFilter.AUTH_TOKEN_VALIDITY,
             AuthenticationFilter.SIGNATURE_SECRET, "management.operation" +
             ".return", "expired.token")).elements());
+      ServletContext context = Mockito.mock(ServletContext.class);
+      Mockito.when(context.getAttribute(
+          AuthenticationFilter.SIGNATURE_PROVIDER_ATTRIBUTE)).thenReturn(null);
+      Mockito.when(config.getServletContext()).thenReturn(context);
 
     if (withDomainPath) {
       Mockito.when(config.getInitParameter(AuthenticationFilter
@@ -511,7 +593,7 @@ public class TestAuthenticationFilter {
         Mockito.verify(chain).doFilter(Mockito.any(ServletRequest.class),
                 Mockito.any(ServletResponse.class));
 
-        Signer signer = new Signer("secret".getBytes());
+        Signer signer = new Signer(new StringSignerSecretProvider("secret"));
         String value = signer.verifyAndExtract(v);
         AuthenticationToken token = AuthenticationToken.parse(value);
         assertThat(token.getExpires(), not(0L));
@@ -578,6 +660,10 @@ public class TestAuthenticationFilter {
         new Vector<String>(
           Arrays.asList(AuthenticationFilter.AUTH_TYPE,
                         "management.operation.return")).elements());
+      ServletContext context = Mockito.mock(ServletContext.class);
+      Mockito.when(context.getAttribute(
+          AuthenticationFilter.SIGNATURE_PROVIDER_ATTRIBUTE)).thenReturn(null);
+      Mockito.when(config.getServletContext()).thenReturn(context);
       filter.init(config);
 
       HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
@@ -585,7 +671,7 @@ public class TestAuthenticationFilter {
 
       AuthenticationToken token = new AuthenticationToken("u", "p", "t");
       token.setExpires(System.currentTimeMillis() + TOKEN_VALIDITY_SEC);
-      Signer signer = new Signer("secret".getBytes());
+      Signer signer = new Signer(new StringSignerSecretProvider("secret"));
       String tokenSigned = signer.sign(token.toString());
 
       Cookie cookie = new Cookie(AuthenticatedURL.AUTH_COOKIE, tokenSigned);
@@ -628,6 +714,10 @@ public class TestAuthenticationFilter {
         new Vector<String>(
           Arrays.asList(AuthenticationFilter.AUTH_TYPE,
                         "management.operation.return")).elements());
+      ServletContext context = Mockito.mock(ServletContext.class);
+      Mockito.when(context.getAttribute(
+          AuthenticationFilter.SIGNATURE_PROVIDER_ATTRIBUTE)).thenReturn(null);
+      Mockito.when(config.getServletContext()).thenReturn(context);
       filter.init(config);
 
       HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
@@ -691,6 +781,10 @@ public class TestAuthenticationFilter {
           Arrays.asList(AuthenticationFilter.AUTH_TYPE,
                         AuthenticationFilter.SIGNATURE_SECRET,
                         "management.operation.return")).elements());
+      ServletContext context = Mockito.mock(ServletContext.class);
+      Mockito.when(context.getAttribute(
+          AuthenticationFilter.SIGNATURE_PROVIDER_ATTRIBUTE)).thenReturn(null);
+      Mockito.when(config.getServletContext()).thenReturn(context);
       filter.init(config);
 
       HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
@@ -698,7 +792,7 @@ public class TestAuthenticationFilter {
 
       AuthenticationToken token = new AuthenticationToken("u", "p", DummyAuthenticationHandler.TYPE);
       token.setExpires(System.currentTimeMillis() - TOKEN_VALIDITY_SEC);
-      Signer signer = new Signer(secret.getBytes());
+      Signer signer = new Signer(new StringSignerSecretProvider(secret));
       String tokenSigned = signer.sign(token.toString());
 
       Cookie cookie = new Cookie(AuthenticatedURL.AUTH_COOKIE, tokenSigned);
@@ -758,6 +852,10 @@ public class TestAuthenticationFilter {
           Arrays.asList(AuthenticationFilter.AUTH_TYPE,
                         AuthenticationFilter.SIGNATURE_SECRET,
                         "management.operation.return")).elements());
+      ServletContext context = Mockito.mock(ServletContext.class);
+      Mockito.when(context.getAttribute(
+          AuthenticationFilter.SIGNATURE_PROVIDER_ATTRIBUTE)).thenReturn(null);
+      Mockito.when(config.getServletContext()).thenReturn(context);
       filter.init(config);
 
       HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
@@ -765,7 +863,7 @@ public class TestAuthenticationFilter {
 
       AuthenticationToken token = new AuthenticationToken("u", "p", "invalidtype");
       token.setExpires(System.currentTimeMillis() + TOKEN_VALIDITY_SEC);
-      Signer signer = new Signer(secret.getBytes());
+      Signer signer = new Signer(new StringSignerSecretProvider(secret));
       String tokenSigned = signer.sign(token.toString());
 
       Cookie cookie = new Cookie(AuthenticatedURL.AUTH_COOKIE, tokenSigned);
@@ -793,6 +891,10 @@ public class TestAuthenticationFilter {
         new Vector<String>(
           Arrays.asList(AuthenticationFilter.AUTH_TYPE,
                         "management.operation.return")).elements());
+      ServletContext context = Mockito.mock(ServletContext.class);
+      Mockito.when(context.getAttribute(
+          AuthenticationFilter.SIGNATURE_PROVIDER_ATTRIBUTE)).thenReturn(null);
+      Mockito.when(config.getServletContext()).thenReturn(context);
       filter.init(config);
 
       HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
@@ -812,7 +914,7 @@ public class TestAuthenticationFilter {
 
       AuthenticationToken token = new AuthenticationToken("u", "p", "t");
       token.setExpires(System.currentTimeMillis() + TOKEN_VALIDITY_SEC);
-      Signer signer = new Signer("secret".getBytes());
+      Signer signer = new Signer(new StringSignerSecretProvider("secret"));
       String tokenSigned = signer.sign(token.toString());
       Cookie cookie = new Cookie(AuthenticatedURL.AUTH_COOKIE, tokenSigned);
       Mockito.when(request.getCookies()).thenReturn(new Cookie[]{cookie});

+ 63 - 0
hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestRandomSignerSecretProvider.java

@@ -0,0 +1,63 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License. See accompanying LICENSE file.
+ */
+package org.apache.hadoop.security.authentication.util;
+
+import java.util.Random;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestRandomSignerSecretProvider {
+
+  @Test
+  public void testGetAndRollSecrets() throws Exception {
+    long rolloverFrequency = 15 * 1000; // rollover every 15 sec
+    // use the same seed so we can predict the RNG
+    long seed = System.currentTimeMillis();
+    Random rand = new Random(seed);
+    byte[] secret1 = Long.toString(rand.nextLong()).getBytes();
+    byte[] secret2 = Long.toString(rand.nextLong()).getBytes();
+    byte[] secret3 = Long.toString(rand.nextLong()).getBytes();
+    RandomSignerSecretProvider secretProvider =
+        new RandomSignerSecretProvider(seed);
+    try {
+      secretProvider.init(null, rolloverFrequency);
+
+      byte[] currentSecret = secretProvider.getCurrentSecret();
+      byte[][] allSecrets = secretProvider.getAllSecrets();
+      Assert.assertArrayEquals(secret1, currentSecret);
+      Assert.assertEquals(2, allSecrets.length);
+      Assert.assertArrayEquals(secret1, allSecrets[0]);
+      Assert.assertNull(allSecrets[1]);
+      Thread.sleep(rolloverFrequency + 2000);
+
+      currentSecret = secretProvider.getCurrentSecret();
+      allSecrets = secretProvider.getAllSecrets();
+      Assert.assertArrayEquals(secret2, currentSecret);
+      Assert.assertEquals(2, allSecrets.length);
+      Assert.assertArrayEquals(secret2, allSecrets[0]);
+      Assert.assertArrayEquals(secret1, allSecrets[1]);
+      Thread.sleep(rolloverFrequency + 2000);
+
+      currentSecret = secretProvider.getCurrentSecret();
+      allSecrets = secretProvider.getAllSecrets();
+      Assert.assertArrayEquals(secret3, currentSecret);
+      Assert.assertEquals(2, allSecrets.length);
+      Assert.assertArrayEquals(secret3, allSecrets[0]);
+      Assert.assertArrayEquals(secret2, allSecrets[1]);
+      Thread.sleep(rolloverFrequency + 2000);
+    } finally {
+      secretProvider.destroy();
+    }
+  }
+}

+ 79 - 0
hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestRolloverSignerSecretProvider.java

@@ -0,0 +1,79 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License. See accompanying LICENSE file.
+ */
+package org.apache.hadoop.security.authentication.util;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestRolloverSignerSecretProvider {
+
+  @Test
+  public void testGetAndRollSecrets() throws Exception {
+    long rolloverFrequency = 15 * 1000; // rollover every 15 sec
+    byte[] secret1 = "doctor".getBytes();
+    byte[] secret2 = "who".getBytes();
+    byte[] secret3 = "tardis".getBytes();
+    TRolloverSignerSecretProvider secretProvider =
+        new TRolloverSignerSecretProvider(
+            new byte[][]{secret1, secret2, secret3});
+    try {
+      secretProvider.init(null, rolloverFrequency);
+
+      byte[] currentSecret = secretProvider.getCurrentSecret();
+      byte[][] allSecrets = secretProvider.getAllSecrets();
+      Assert.assertArrayEquals(secret1, currentSecret);
+      Assert.assertEquals(2, allSecrets.length);
+      Assert.assertArrayEquals(secret1, allSecrets[0]);
+      Assert.assertNull(allSecrets[1]);
+      Thread.sleep(rolloverFrequency + 2000);
+
+      currentSecret = secretProvider.getCurrentSecret();
+      allSecrets = secretProvider.getAllSecrets();
+      Assert.assertArrayEquals(secret2, currentSecret);
+      Assert.assertEquals(2, allSecrets.length);
+      Assert.assertArrayEquals(secret2, allSecrets[0]);
+      Assert.assertArrayEquals(secret1, allSecrets[1]);
+      Thread.sleep(rolloverFrequency + 2000);
+
+      currentSecret = secretProvider.getCurrentSecret();
+      allSecrets = secretProvider.getAllSecrets();
+      Assert.assertArrayEquals(secret3, currentSecret);
+      Assert.assertEquals(2, allSecrets.length);
+      Assert.assertArrayEquals(secret3, allSecrets[0]);
+      Assert.assertArrayEquals(secret2, allSecrets[1]);
+      Thread.sleep(rolloverFrequency + 2000);
+    } finally {
+      secretProvider.destroy();
+    }
+  }
+
+  class TRolloverSignerSecretProvider extends RolloverSignerSecretProvider {
+
+    private byte[][] newSecretSequence;
+    private int newSecretSequenceIndex;
+
+    public TRolloverSignerSecretProvider(byte[][] newSecretSequence)
+        throws Exception {
+      super();
+      this.newSecretSequence = newSecretSequence;
+      this.newSecretSequenceIndex = 0;
+    }
+
+    @Override
+    protected byte[] generateNewSecret() {
+      return newSecretSequence[newSecretSequenceIndex++];
+    }
+
+  }
+}

+ 69 - 16
hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestSigner.java

@@ -13,24 +13,15 @@
  */
 package org.apache.hadoop.security.authentication.util;
 
+import java.util.Properties;
 import org.junit.Assert;
 import org.junit.Test;
 
 public class TestSigner {
 
-  @Test
-  public void testNoSecret() throws Exception {
-    try {
-      new Signer(null);
-      Assert.fail();
-    }
-    catch (IllegalArgumentException ex) {
-    }
-  }
-
   @Test
   public void testNullAndEmptyString() throws Exception {
-    Signer signer = new Signer("secret".getBytes());
+    Signer signer = new Signer(new StringSignerSecretProvider("secret"));
     try {
       signer.sign(null);
       Assert.fail();
@@ -51,17 +42,17 @@ public class TestSigner {
 
   @Test
   public void testSignature() throws Exception {
-    Signer signer = new Signer("secret".getBytes());
+    Signer signer = new Signer(new StringSignerSecretProvider("secret"));
     String s1 = signer.sign("ok");
     String s2 = signer.sign("ok");
     String s3 = signer.sign("wrong");
     Assert.assertEquals(s1, s2);
-    Assert.assertNotSame(s1, s3);
+    Assert.assertNotEquals(s1, s3);
   }
 
   @Test
   public void testVerify() throws Exception {
-    Signer signer = new Signer("secret".getBytes());
+    Signer signer = new Signer(new StringSignerSecretProvider("secret"));
     String t = "test";
     String s = signer.sign(t);
     String e = signer.verifyAndExtract(s);
@@ -70,7 +61,7 @@ public class TestSigner {
 
   @Test
   public void testInvalidSignedText() throws Exception {
-    Signer signer = new Signer("secret".getBytes());
+    Signer signer = new Signer(new StringSignerSecretProvider("secret"));
     try {
       signer.verifyAndExtract("test");
       Assert.fail();
@@ -83,7 +74,7 @@ public class TestSigner {
 
   @Test
   public void testTampering() throws Exception {
-    Signer signer = new Signer("secret".getBytes());
+    Signer signer = new Signer(new StringSignerSecretProvider("secret"));
     String t = "test";
     String s = signer.sign(t);
     s += "x";
@@ -96,4 +87,66 @@ public class TestSigner {
       Assert.fail();
     }
   }
+
+  @Test
+  public void testMultipleSecrets() throws Exception {
+    TestSignerSecretProvider secretProvider = new TestSignerSecretProvider();
+    Signer signer = new Signer(secretProvider);
+    secretProvider.setCurrentSecret("secretB");
+    String t1 = "test";
+    String s1 = signer.sign(t1);
+    String e1 = signer.verifyAndExtract(s1);
+    Assert.assertEquals(t1, e1);
+    secretProvider.setPreviousSecret("secretA");
+    String t2 = "test";
+    String s2 = signer.sign(t2);
+    String e2 = signer.verifyAndExtract(s2);
+    Assert.assertEquals(t2, e2);
+    Assert.assertEquals(s1, s2); //check is using current secret for signing
+    secretProvider.setCurrentSecret("secretC");
+    secretProvider.setPreviousSecret("secretB");
+    String t3 = "test";
+    String s3 = signer.sign(t3);
+    String e3 = signer.verifyAndExtract(s3);
+    Assert.assertEquals(t3, e3);
+    Assert.assertNotEquals(s1, s3); //check not using current secret for signing
+    String e1b = signer.verifyAndExtract(s1);
+    Assert.assertEquals(t1, e1b); // previous secret still valid
+    secretProvider.setCurrentSecret("secretD");
+    secretProvider.setPreviousSecret("secretC");
+    try {
+      signer.verifyAndExtract(s1);  // previous secret no longer valid
+      Assert.fail();
+    } catch (SignerException ex) {
+      // Expected
+    }
+  }
+
+  class TestSignerSecretProvider extends SignerSecretProvider {
+
+    private byte[] currentSecret;
+    private byte[] previousSecret;
+
+    @Override
+    public void init(Properties config, long tokenValidity) {
+    }
+
+    @Override
+    public byte[] getCurrentSecret() {
+      return currentSecret;
+    }
+
+    @Override
+    public byte[][] getAllSecrets() {
+      return new byte[][]{currentSecret, previousSecret};
+    }
+
+    public void setCurrentSecret(String secretStr) {
+      currentSecret = secretStr.getBytes();
+    }
+
+    public void setPreviousSecret(String previousSecretStr) {
+      previousSecret = previousSecretStr.getBytes();
+    }
+  }
 }

+ 33 - 0
hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestStringSignerSecretProvider.java

@@ -0,0 +1,33 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License. See accompanying LICENSE file.
+ */
+package org.apache.hadoop.security.authentication.util;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestStringSignerSecretProvider {
+
+  @Test
+  public void testGetSecrets() throws Exception {
+    String secretStr = "secret";
+    StringSignerSecretProvider secretProvider
+        = new StringSignerSecretProvider(secretStr);
+    secretProvider.init(null, -1);
+    byte[] secretBytes = secretStr.getBytes();
+    Assert.assertArrayEquals(secretBytes, secretProvider.getCurrentSecret());
+    byte[][] allSecrets = secretProvider.getAllSecrets();
+    Assert.assertEquals(1, allSecrets.length);
+    Assert.assertArrayEquals(secretBytes, allSecrets[0]);
+  }
+}

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

@@ -195,6 +195,10 @@ Trunk (Unreleased)
     HADOOP-10756. KMS audit log should consolidate successful similar requests. 
     (asuresh via tucu)
 
+    HADOOP-10793. KeyShell args should use single-dash style. (wang)
+
+    HADOOP-10936. Change default KeyProvider bitlength to 128. (wang)
+
   BUG FIXES
 
     HADOOP-9451. Fault single-layer config if node group topology is enabled.
@@ -408,6 +412,15 @@ Trunk (Unreleased)
     HADOOP-10881. Clarify usage of encryption and encrypted encryption
     key in KeyProviderCryptoExtension. (wang)
 
+    HADOOP-10920. site plugin couldn't parse hadoop-kms index.apt.vm.
+    (Akira Ajisaka via wang)
+
+    HADOOP-10925. Compilation fails in native link0 function on Windows.
+    (cnauroth)
+
+    HADOOP-10939. Fix TestKeyProviderFactory testcases to use default 128 bit
+    length keys. (Arun Suresh via wang)
+
   OPTIMIZATIONS
 
     HADOOP-7761. Improve the performance of raw comparisons. (todd)
@@ -466,6 +479,17 @@ Release 2.6.0 - UNRELEASED
     HADOOP-8069. Enable TCP_NODELAY by default for IPC. (Todd Lipcon via
     Arpit Agarwal)
 
+    HADOOP-10902. Deletion of directories with snapshots will not output
+    reason for trash move failure. (Stephen Chu via wang)
+
+    HADOOP-10900. CredentialShell args should use single-dash style. (wang)
+
+    HADOOP-10903. Enhance hadoop classpath command to expand wildcards or write
+    classpath into jar manifest. (cnauroth)
+
+    HADOOP-10791. AuthenticationFilter should support externalizing the 
+    secret for signing and provide rotation support. (rkanter via tucu)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -500,6 +524,25 @@ Release 2.6.0 - UNRELEASED
     HADOOP-10876. The constructor of Path should not take an empty URL as a
     parameter. (Zhihai Xu via wang)
 
+    HADOOP-10928. Incorrect usage on `hadoop credential list`.
+    (Josh Elser via wang)
+
+    HADOOP-10927. Fix CredentialShell help behavior and error codes.
+    (Josh Elser via wang)
+
+    HADOOP-10937. Need to set version name correctly before decrypting EEK.
+    (Arun Suresh via wang)
+
+    HADOOP-10918. JMXJsonServlet fails when used within Tomcat. (tucu)
+
+    HADOOP-10933. FileBasedKeyStoresFactory Should use Configuration.getPassword 
+    for SSL Passwords. (lmccay via tucu)
+
+    HADOOP-10759. Remove hardcoded JAVA_HEAP_MAX. (Sam Liu via Eric Yang)
+
+    HADOOP-10905. LdapGroupsMapping Should use configuration.getPassword for SSL
+    and LDAP Passwords. (lmccay via brandonli)
+
 Release 2.5.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 9 - 5
hadoop-common-project/hadoop-common/src/main/bin/hadoop

@@ -35,6 +35,7 @@ function print_usage(){
   echo "  distcp <srcurl> <desturl> copy file or directories recursively"
   echo "  archive -archiveName NAME -p <parent path> <src>* <dest> create a hadoop archive"
   echo "  classpath            prints the class path needed to get the"
+  echo "  credential           interact with credential providers"
   echo "                       Hadoop jar and the required libraries"
   echo "  daemonlog            get/set the log level for each daemon"
   echo " or"
@@ -90,11 +91,6 @@ case $COMMAND in
     fi
     ;;
 
-  classpath)
-    echo $CLASSPATH
-    exit
-    ;;
-
   #core commands  
   *)
     # the core commands
@@ -118,6 +114,14 @@ case $COMMAND in
       CLASSPATH=${CLASSPATH}:${TOOL_PATH}
     elif [ "$COMMAND" = "credential" ] ; then
       CLASS=org.apache.hadoop.security.alias.CredentialShell
+    elif [ "$COMMAND" = "classpath" ] ; then
+      if [ "$#" -eq 1 ]; then
+        # No need to bother starting up a JVM for this simple case.
+        echo $CLASSPATH
+        exit
+      else
+        CLASS=org.apache.hadoop.util.Classpath
+      fi
     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/bin/hadoop-config.sh

@@ -149,8 +149,6 @@ if [[ -z $JAVA_HOME ]]; then
 fi
 
 JAVA=$JAVA_HOME/bin/java
-# some Java parameters
-JAVA_HEAP_MAX=-Xmx1000m 
 
 # check envvars which might override default args
 if [ "$HADOOP_HEAPSIZE" != "" ]; then

+ 10 - 3
hadoop-common-project/hadoop-common/src/main/bin/hadoop.cmd

@@ -115,11 +115,14 @@ call :updatepath %HADOOP_BIN_PATH%
   )
 
   if %hadoop-command% == classpath (
-    @echo %CLASSPATH%
-    goto :eof
+    if not defined hadoop-command-arguments (
+      @rem No need to bother starting up a JVM for this simple case.
+      @echo %CLASSPATH%
+      exit /b
+    )
   )
   
-  set corecommands=fs version jar checknative distcp daemonlog archive
+  set corecommands=fs version jar checknative distcp daemonlog archive classpath
   for %%i in ( %corecommands% ) do (
     if %hadoop-command% == %%i set corecommand=true  
   )
@@ -175,6 +178,10 @@ call :updatepath %HADOOP_BIN_PATH%
   set CLASSPATH=%CLASSPATH%;%TOOL_PATH%
   goto :eof
 
+:classpath
+  set CLASS=org.apache.hadoop.util.Classpath
+  goto :eof
+
 :updatepath
   set path_to_add=%*
   set current_path_comparable=%path%

+ 11 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java

@@ -54,7 +54,7 @@ public abstract class KeyProvider {
   public static final String DEFAULT_CIPHER = "AES/CTR/NoPadding";
   public static final String DEFAULT_BITLENGTH_NAME =
       "hadoop.security.key.default.bitlength";
-  public static final int DEFAULT_BITLENGTH = 256;
+  public static final int DEFAULT_BITLENGTH = 128;
 
   /**
    * The combination of both the key version name and the key material.
@@ -341,6 +341,16 @@ public abstract class KeyProvider {
     public Map<String, String> getAttributes() {
       return (attributes == null) ? Collections.EMPTY_MAP : attributes;
     }
+
+    @Override
+    public String toString() {
+      return "Options{" +
+          "cipher='" + cipher + '\'' +
+          ", bitLength=" + bitLength +
+          ", description='" + description + '\'' +
+          ", attributes=" + attributes +
+          '}';
+    }
   }
 
   /**

+ 10 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java

@@ -21,11 +21,13 @@ package org.apache.hadoop.crypto.key;
 import java.io.IOException;
 import java.security.GeneralSecurityException;
 import java.security.SecureRandom;
+
 import javax.crypto.Cipher;
 import javax.crypto.spec.IvParameterSpec;
 import javax.crypto.spec.SecretKeySpec;
 
 import com.google.common.base.Preconditions;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 
 /**
@@ -97,7 +99,7 @@ public class KeyProviderCryptoExtension extends
     public static EncryptedKeyVersion createForDecryption(String
         encryptionKeyVersionName, byte[] encryptedKeyIv,
         byte[] encryptedKeyMaterial) {
-      KeyVersion encryptedKeyVersion = new KeyVersion(null, null,
+      KeyVersion encryptedKeyVersion = new KeyVersion(null, EEK,
           encryptedKeyMaterial);
       return new EncryptedKeyVersion(null, encryptionKeyVersionName,
           encryptedKeyIv, encryptedKeyVersion);
@@ -258,6 +260,13 @@ public class KeyProviderCryptoExtension extends
           keyProvider.getKeyVersion(encryptionKeyVersionName);
       Preconditions.checkNotNull(encryptionKey,
           "KeyVersion name '%s' does not exist", encryptionKeyVersionName);
+      Preconditions.checkArgument(
+              encryptedKeyVersion.getEncryptedKeyVersion().getVersionName()
+                    .equals(KeyProviderCryptoExtension.EEK),
+                "encryptedKey version name must be '%s', is '%s'",
+                KeyProviderCryptoExtension.EEK,
+                encryptedKeyVersion.getEncryptedKeyVersion().getVersionName()
+            );
       final byte[] encryptionKeyMaterial = encryptionKey.getMaterial();
       // Encryption key IV is determined from encrypted key's IV
       final byte[] encryptionIV =

+ 45 - 44
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyShell.java

@@ -38,9 +38,9 @@ import org.apache.hadoop.util.ToolRunner;
  */
 public class KeyShell extends Configured implements Tool {
   final static private String USAGE_PREFIX = "Usage: hadoop key " +
-  		"[generic options]\n";
+      "[generic options]\n";
   final static private String COMMANDS =
-      "   [--help]\n" +
+      "   [-help]\n" +
       "   [" + CreateCommand.USAGE + "]\n" +
       "   [" + RollCommand.USAGE + "]\n" +
       "   [" + DeleteCommand.USAGE + "]\n" +
@@ -90,11 +90,11 @@ public class KeyShell extends Configured implements Tool {
   /**
    * Parse the command line arguments and initialize the data
    * <pre>
-   * % hadoop key create keyName [--size size] [--cipher algorithm]
-   *    [--provider providerPath]
-   * % hadoop key roll keyName [--provider providerPath]
+   * % hadoop key create keyName [-size size] [-cipher algorithm]
+   *    [-provider providerPath]
+   * % hadoop key roll keyName [-provider providerPath]
    * % hadoop key list [-provider providerPath]
-   * % hadoop key delete keyName [--provider providerPath] [-i]
+   * % hadoop key delete keyName [-provider providerPath] [-i]
    * </pre>
    * @param args Command line arguments.
    * @return 0 on success, 1 on failure.
@@ -107,47 +107,47 @@ public class KeyShell extends Configured implements Tool {
     for (int i = 0; i < args.length; i++) { // parse command line
       boolean moreTokens = (i < args.length - 1);
       if (args[i].equals("create")) {
-        String keyName = "--help";
+        String keyName = "-help";
         if (moreTokens) {
           keyName = args[++i];
         }
 
         command = new CreateCommand(keyName, options);
-        if ("--help".equals(keyName)) {
+        if ("-help".equals(keyName)) {
           printKeyShellUsage();
           return 1;
         }
       } else if (args[i].equals("delete")) {
-        String keyName = "--help";
+        String keyName = "-help";
         if (moreTokens) {
           keyName = args[++i];
         }
 
         command = new DeleteCommand(keyName);
-        if ("--help".equals(keyName)) {
+        if ("-help".equals(keyName)) {
           printKeyShellUsage();
           return 1;
         }
       } else if (args[i].equals("roll")) {
-        String keyName = "--help";
+        String keyName = "-help";
         if (moreTokens) {
           keyName = args[++i];
         }
 
         command = new RollCommand(keyName);
-        if ("--help".equals(keyName)) {
+        if ("-help".equals(keyName)) {
           printKeyShellUsage();
           return 1;
         }
       } else if ("list".equals(args[i])) {
         command = new ListCommand();
-      } else if ("--size".equals(args[i]) && moreTokens) {
+      } else if ("-size".equals(args[i]) && moreTokens) {
         options.setBitLength(Integer.parseInt(args[++i]));
-      } else if ("--cipher".equals(args[i]) && moreTokens) {
+      } else if ("-cipher".equals(args[i]) && moreTokens) {
         options.setCipher(args[++i]);
-      } else if ("--description".equals(args[i]) && moreTokens) {
+      } else if ("-description".equals(args[i]) && moreTokens) {
         options.setDescription(args[++i]);
-      } else if ("--attr".equals(args[i]) && moreTokens) {
+      } else if ("-attr".equals(args[i]) && moreTokens) {
         final String attrval[] = args[++i].split("=", 2);
         final String attr = attrval[0].trim();
         final String val = attrval[1].trim();
@@ -164,14 +164,14 @@ public class KeyShell extends Configured implements Tool {
           return 1;
         }
         attributes.put(attr, val);
-      } else if ("--provider".equals(args[i]) && moreTokens) {
+      } else if ("-provider".equals(args[i]) && moreTokens) {
         userSuppliedProvider = true;
         getConf().set(KeyProviderFactory.KEY_PROVIDER_PATH, args[++i]);
-      } else if ("--metadata".equals(args[i])) {
+      } else if ("-metadata".equals(args[i])) {
         getConf().setBoolean(LIST_METADATA, true);
-      } else if ("-i".equals(args[i]) || ("--interactive".equals(args[i]))) {
+      } else if ("-i".equals(args[i]) || ("-interactive".equals(args[i]))) {
         interactive = true;
-      } else if ("--help".equals(args[i])) {
+      } else if ("-help".equals(args[i])) {
         printKeyShellUsage();
         return 1;
       } else {
@@ -258,11 +258,11 @@ public class KeyShell extends Configured implements Tool {
 
   private class ListCommand extends Command {
     public static final String USAGE =
-        "list [--provider <provider>] [--metadata] [--help]";
+        "list [-provider <provider>] [-metadata] [-help]";
     public static final String DESC =
         "The list subcommand displays the keynames contained within\n" +
         "a particular provider as configured in core-site.xml or\n" +
-        "specified with the --provider argument. --metadata displays\n" +
+        "specified with the -provider argument. -metadata displays\n" +
         "the metadata.";
 
     private boolean metadata = false;
@@ -272,9 +272,9 @@ public class KeyShell extends Configured implements Tool {
       provider = getKeyProvider();
       if (provider == null) {
         out.println("There are no non-transient KeyProviders configured.\n"
-          + "Use the --provider option to specify a provider. If you\n"
+          + "Use the -provider option to specify a provider. If you\n"
           + "want to list a transient provider then you must use the\n"
-          + "--provider argument.");
+          + "-provider argument.");
         rc = false;
       }
       metadata = getConf().getBoolean(LIST_METADATA, false);
@@ -310,10 +310,10 @@ public class KeyShell extends Configured implements Tool {
   }
 
   private class RollCommand extends Command {
-    public static final String USAGE = "roll <keyname> [--provider <provider>] [--help]";
+    public static final String USAGE = "roll <keyname> [-provider <provider>] [-help]";
     public static final String DESC =
       "The roll subcommand creates a new version for the specified key\n" +
-      "within the provider indicated using the --provider argument\n";
+      "within the provider indicated using the -provider argument\n";
 
     String keyName = null;
 
@@ -326,13 +326,13 @@ public class KeyShell extends Configured implements Tool {
       provider = getKeyProvider();
       if (provider == null) {
         out.println("There are no valid KeyProviders configured. The key\n" +
-          "has not been rolled. Use the --provider option to specify\n" +
+          "has not been rolled. Use the -provider option to specify\n" +
           "a provider.");
         rc = false;
       }
       if (keyName == null) {
         out.println("Please provide a <keyname>.\n" +
-          "See the usage description by using --help.");
+          "See the usage description by using -help.");
         rc = false;
       }
       return rc;
@@ -367,11 +367,11 @@ public class KeyShell extends Configured implements Tool {
   }
 
   private class DeleteCommand extends Command {
-    public static final String USAGE = "delete <keyname> [--provider <provider>] [--help]";
+    public static final String USAGE = "delete <keyname> [-provider <provider>] [-help]";
     public static final String DESC =
         "The delete subcommand deletes all versions of the key\n" +
         "specified by the <keyname> argument from within the\n" +
-        "provider specified --provider.";
+        "provider specified -provider.";
 
     String keyName = null;
     boolean cont = true;
@@ -385,12 +385,12 @@ public class KeyShell extends Configured implements Tool {
       provider = getKeyProvider();
       if (provider == null) {
         out.println("There are no valid KeyProviders configured. Nothing\n"
-          + "was deleted. Use the --provider option to specify a provider.");
+          + "was deleted. Use the -provider option to specify a provider.");
         return false;
       }
       if (keyName == null) {
         out.println("There is no keyName specified. Please specify a " +
-            "<keyname>. See the usage description with --help.");
+            "<keyname>. See the usage description with -help.");
         return false;
       }
       if (interactive) {
@@ -436,19 +436,19 @@ public class KeyShell extends Configured implements Tool {
 
   private class CreateCommand extends Command {
     public static final String USAGE =
-      "create <keyname> [--cipher <cipher>] [--size <size>]\n" +
-      "                     [--description <description>]\n" +
-      "                     [--attr <attribute=value>]\n" +
-      "                     [--provider <provider>] [--help]";
+      "create <keyname> [-cipher <cipher>] [-size <size>]\n" +
+      "                     [-description <description>]\n" +
+      "                     [-attr <attribute=value>]\n" +
+      "                     [-provider <provider>] [-help]";
     public static final String DESC =
       "The create subcommand creates a new key for the name specified\n" +
       "by the <keyname> argument within the provider specified by the\n" +
-      "--provider argument. You may specify a cipher with the --cipher\n" +
+      "-provider argument. You may specify a cipher with the -cipher\n" +
       "argument. The default cipher is currently \"AES/CTR/NoPadding\".\n" +
-      "The default keysize is 256. You may specify the requested key\n" +
-      "length using the --size argument. Arbitrary attribute=value\n" +
-      "style attributes may be specified using the --attr argument.\n" +
-      "--attr may be specified multiple times, once per attribute.\n";
+      "The default keysize is 128. You may specify the requested key\n" +
+      "length using the -size argument. Arbitrary attribute=value\n" +
+      "style attributes may be specified using the -attr argument.\n" +
+      "-attr may be specified multiple times, once per attribute.\n";
 
     final String keyName;
     final Options options;
@@ -463,13 +463,13 @@ public class KeyShell extends Configured implements Tool {
       provider = getKeyProvider();
       if (provider == null) {
         out.println("There are no valid KeyProviders configured. No key\n" +
-          " was created. You can use the --provider option to specify\n" +
+          " was created. You can use the -provider option to specify\n" +
           " a provider to use.");
         rc = false;
       }
       if (keyName == null) {
         out.println("Please provide a <keyname>. See the usage description" +
-          " with --help.");
+          " with -help.");
         rc = false;
       }
       return rc;
@@ -479,7 +479,8 @@ public class KeyShell extends Configured implements Tool {
       warnIfTransientProvider();
       try {
         provider.createKey(keyName, options);
-        out.println(keyName + " has been successfully created.");
+        out.println(keyName + " has been successfully created with options "
+            + options.toString() + ".");
         provider.flush();
         printProviderWritten();
       } catch (InvalidParameterException e) {

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

@@ -653,7 +653,7 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension {
         encryptedKeyVersion.getEncryptedKeyVersion().getVersionName()
             .equals(KeyProviderCryptoExtension.EEK),
         "encryptedKey version name must be '%s', is '%s'",
-        KeyProviderCryptoExtension.EK,
+        KeyProviderCryptoExtension.EEK,
         encryptedKeyVersion.getEncryptedKeyVersion().getVersionName()
     );
     checkNotNull(encryptedKeyVersion.getEncryptedKeyVersion(), "encryptedKey");

+ 5 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Delete.java

@@ -118,7 +118,11 @@ class Delete {
         } catch(FileNotFoundException fnfe) {
           throw fnfe;
         } catch (IOException ioe) {
-            throw new IOException(ioe.getMessage() + ". Consider using -skipTrash option", ioe);
+          String msg = ioe.getMessage();
+          if (ioe.getCause() != null) {
+            msg += ": " + ioe.getCause().getMessage();
+	  }
+          throw new IOException(msg + ". Consider using -skipTrash option", ioe);
         }
       }
       return success;

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java

@@ -1005,7 +1005,7 @@ public final class HttpServer2 implements FilterContainer {
 
     String remoteUser = request.getRemoteUser();
     if (remoteUser == null) {
-      response.sendError(HttpServletResponse.SC_UNAUTHORIZED,
+      response.sendError(HttpServletResponse.SC_FORBIDDEN,
                          "Unauthenticated users are not " +
                          "authorized to access this page.");
       return false;
@@ -1013,7 +1013,7 @@ public final class HttpServer2 implements FilterContainer {
 
     if (servletContext.getAttribute(ADMINS_ACL) != null &&
         !userHasAdministratorAccess(servletContext, remoteUser)) {
-      response.sendError(HttpServletResponse.SC_UNAUTHORIZED, "User "
+      response.sendError(HttpServletResponse.SC_FORBIDDEN, "User "
           + remoteUser + " is unauthorized to access this page.");
       return false;
     }

+ 12 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java

@@ -33,6 +33,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.HardLink;
 import org.apache.hadoop.io.SecureIOUtils.AlreadyExistsException;
 import org.apache.hadoop.util.NativeCodeLoader;
 import org.apache.hadoop.util.Shell;
@@ -823,6 +824,14 @@ public class NativeIO {
     }
   }
 
+  public static void link(File src, File dst) throws IOException {
+    if (!nativeLoaded) {
+      HardLink.createHardLink(src, dst);
+    } else {
+      link0(src.getAbsolutePath(), dst.getAbsolutePath());
+    }
+  }
+
   /**
    * A version of renameTo that throws a descriptive exception when it fails.
    *
@@ -833,4 +842,7 @@ public class NativeIO {
    */
   private static native void renameTo0(String src, String dst)
       throws NativeIOException;
+
+  private static native void link0(String src, String dst)
+      throws NativeIOException;
 }

+ 7 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/jmx/JMXJsonServlet.java

@@ -143,6 +143,12 @@ public class JMXJsonServlet extends HttpServlet {
     jsonFactory = new JsonFactory();
   }
 
+  protected boolean isInstrumentationAccessAllowed(HttpServletRequest request, 
+      HttpServletResponse response) throws IOException {
+    return HttpServer2.isInstrumentationAccessAllowed(getServletContext(),
+        request, response);
+  }
+  
   /**
    * Process a GET request for the specified resource.
    * 
@@ -154,8 +160,7 @@ public class JMXJsonServlet extends HttpServlet {
   @Override
   public void doGet(HttpServletRequest request, HttpServletResponse response) {
     try {
-      if (!HttpServer2.isInstrumentationAccessAllowed(getServletContext(),
-                                                     request, response)) {
+      if (!isInstrumentationAccessAllowed(request, response)) {
         return;
       }
       JsonGenerator jg = null;

+ 22 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java

@@ -312,15 +312,15 @@ public class LdapGroupsMapping
     useSsl = conf.getBoolean(LDAP_USE_SSL_KEY, LDAP_USE_SSL_DEFAULT);
     keystore = conf.get(LDAP_KEYSTORE_KEY, LDAP_KEYSTORE_DEFAULT);
     
-    keystorePass =
-        conf.get(LDAP_KEYSTORE_PASSWORD_KEY, LDAP_KEYSTORE_PASSWORD_DEFAULT);
+    keystorePass = getPassword(conf, LDAP_KEYSTORE_PASSWORD_KEY,
+        LDAP_KEYSTORE_PASSWORD_DEFAULT);
     if (keystorePass.isEmpty()) {
       keystorePass = extractPassword(conf.get(LDAP_KEYSTORE_PASSWORD_FILE_KEY,
           LDAP_KEYSTORE_PASSWORD_FILE_DEFAULT));
     }
     
     bindUser = conf.get(BIND_USER_KEY, BIND_USER_DEFAULT);
-    bindPassword = conf.get(BIND_PASSWORD_KEY, BIND_PASSWORD_DEFAULT);
+    bindPassword = getPassword(conf, BIND_PASSWORD_KEY, BIND_PASSWORD_DEFAULT);
     if (bindPassword.isEmpty()) {
       bindPassword = extractPassword(
           conf.get(BIND_PASSWORD_FILE_KEY, BIND_PASSWORD_FILE_DEFAULT));
@@ -341,7 +341,25 @@ public class LdapGroupsMapping
 
     this.conf = conf;
   }
-  
+
+  String getPassword(Configuration conf, String alias, String defaultPass) {
+    String password = null;
+    try {
+      char[] passchars = conf.getPassword(alias);
+      if (passchars != null) {
+        password = new String(passchars);
+      }
+      else {
+        password = defaultPass;
+      }
+    }
+    catch (IOException ioe) {
+      LOG.warn("Exception while trying to password for alias " + alias + ": "
+          + ioe.getMessage());
+    }
+    return password;
+  }
+
   String extractPassword(String pwFile) {
     if (pwFile.isEmpty()) {
       // If there is no password file defined, we'll assume that we should do

+ 2 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java

@@ -77,7 +77,8 @@ public class SecurityUtil {
    * For use only by tests and initialization
    */
   @InterfaceAudience.Private
-  static void setTokenServiceUseIp(boolean flag) {
+  @VisibleForTesting
+  public static void setTokenServiceUseIp(boolean flag) {
     useIpForTokenService = flag;
     hostResolver = !useIpForTokenService
         ? new QualifiedHostResolver()

+ 36 - 29
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/CredentialShell.java

@@ -67,11 +67,11 @@ public class CredentialShell extends Configured implements Tool {
       if (command.validate()) {
           command.execute();
       } else {
-        exitCode = -1;
+        exitCode = 1;
       }
     } catch (Exception e) {
       e.printStackTrace(err);
-      return -1;
+      return 1;
     }
     return exitCode;
   }
@@ -79,47 +79,54 @@ public class CredentialShell extends Configured implements Tool {
   /**
    * 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]
+   * % hadoop credential create alias [-provider providerPath]
+   * % hadoop credential list [-provider providerPath]
+   * % hadoop credential delete alias [-provider providerPath] [-i]
    * </pre>
    * @param args
-   * @return
+   * @return 0 if the argument(s) were recognized, 1 otherwise
    * @throws IOException
    */
-  private int init(String[] args) throws IOException {
+  protected int init(String[] args) throws IOException {
+    // no args should print the help message
+    if (0 == args.length) {
+      printCredShellUsage();
+      ToolRunner.printGenericCommandUsage(System.err);
+      return 1;
+    }
+
     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")) {
+        if (alias.equals("-help")) {
           printCredShellUsage();
-          return -1;
+          return 0;
         }
       } else if (args[i].equals("delete")) {
         String alias = args[++i];
         command = new DeleteCommand(alias);
-        if (alias.equals("--help")) {
+        if (alias.equals("-help")) {
           printCredShellUsage();
-          return -1;
+          return 0;
         }
       } else if (args[i].equals("list")) {
         command = new ListCommand();
-      } else if (args[i].equals("--provider")) {
+      } 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"))) {
+      } else if (args[i].equals("-i") || (args[i].equals("-interactive"))) {
         interactive = true;
-      } else if (args[i].equals("-v") || (args[i].equals("--value"))) {
+      } else if (args[i].equals("-v") || (args[i].equals("-value"))) {
         value = args[++i];
-      } else if (args[i].equals("--help")) {
+      } else if (args[i].equals("-help")) {
         printCredShellUsage();
-        return -1;
+        return 0;
       } else {
         printCredShellUsage();
         ToolRunner.printGenericCommandUsage(System.err);
-        return -1;
+        return 1;
       }
     }
     return 0;
@@ -188,20 +195,20 @@ public class CredentialShell extends Configured implements Tool {
   }
 
   private class ListCommand extends Command {
-    public static final String USAGE = "list <alias> [--provider] [--help]";
+    public static final String USAGE = "list [-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.";
+        "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"
+            + "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.");
+            + "you MUST use the -provider argument.");
         rc = false;
       }
       return rc;
@@ -229,11 +236,11 @@ public class CredentialShell extends Configured implements Tool {
   }
 
   private class DeleteCommand extends Command {
-    public static final String USAGE = "delete <alias> [--provider] [--help]";
+    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";
+        "indicated through the -provider argument";
 
     String alias = null;
     boolean cont = true;
@@ -248,13 +255,13 @@ public class CredentialShell extends Configured implements Tool {
       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"
+            + "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.");
+            "mandatory <alias>. See the usage description with -help.");
         return false;
       }
       if (interactive) {
@@ -299,11 +306,11 @@ public class CredentialShell extends Configured implements Tool {
   }
 
   private class CreateCommand extends Command {
-    public static final String USAGE = "create <alias> [--provider] [--help]";
+    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.";
+        "the -provider argument.";
 
     String alias = null;
 
@@ -317,13 +324,13 @@ public class CredentialShell extends Configured implements Tool {
       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" +
+            + "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.");
+            "mandatory <alias>. See the usage description with -help.");
         rc = false;
       }
       return rc;

+ 19 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/FileBasedKeyStoresFactory.java

@@ -150,7 +150,7 @@ public class FileBasedKeyStoresFactory implements KeyStoresFactory {
       }
       String passwordProperty =
         resolvePropertyName(mode, SSL_KEYSTORE_PASSWORD_TPL_KEY);
-      String keystorePassword = conf.get(passwordProperty, "");
+      String keystorePassword = getPassword(conf, passwordProperty, "");
       if (keystorePassword.isEmpty()) {
         throw new GeneralSecurityException("The property '" + passwordProperty +
           "' has not been set in the ssl configuration file.");
@@ -160,7 +160,8 @@ public class FileBasedKeyStoresFactory implements KeyStoresFactory {
       // Key password defaults to the same value as store password for
       // compatibility with legacy configurations that did not use a separate
       // configuration property for key password.
-      keystoreKeyPassword = conf.get(keyPasswordProperty, keystorePassword);
+      keystoreKeyPassword = getPassword(
+          conf, keyPasswordProperty, keystorePassword);
       LOG.debug(mode.toString() + " KeyStore: " + keystoreLocation);
 
       InputStream is = new FileInputStream(keystoreLocation);
@@ -191,7 +192,7 @@ public class FileBasedKeyStoresFactory implements KeyStoresFactory {
     if (!truststoreLocation.isEmpty()) {
       String passwordProperty = resolvePropertyName(mode,
           SSL_TRUSTSTORE_PASSWORD_TPL_KEY);
-      String truststorePassword = conf.get(passwordProperty, "");
+      String truststorePassword = getPassword(conf, passwordProperty, "");
       if (truststorePassword.isEmpty()) {
         throw new GeneralSecurityException("The property '" + passwordProperty +
             "' has not been set in the ssl configuration file.");
@@ -217,6 +218,21 @@ public class FileBasedKeyStoresFactory implements KeyStoresFactory {
     }
   }
 
+  String getPassword(Configuration conf, String alias, String defaultPass) {
+    String password = defaultPass;
+    try {
+      char[] passchars = conf.getPassword(alias);
+      if (passchars != null) {
+        password = new String(passchars);
+      }
+    }
+    catch (IOException ioe) {
+      LOG.warn("Exception while trying to get password for alias " + alias +
+          ": " + ioe.getMessage());
+    }
+    return password;
+  }
+
   /**
    * Releases any resources being used.
    */

+ 125 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Classpath.java

@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.util;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.shell.CommandFormat;
+import org.apache.hadoop.fs.shell.CommandFormat.UnknownOptionException;
+
+/**
+ * Command-line utility for getting the full classpath needed to launch a Hadoop
+ * client application.  If the hadoop script is called with "classpath" as the
+ * command, then it simply prints the classpath and exits immediately without
+ * launching a JVM.  The output likely will include wildcards in the classpath.
+ * If there are arguments passed to the classpath command, then this class gets
+ * called.  With the --glob argument, it prints the full classpath with wildcards
+ * expanded.  This is useful in situations where wildcard syntax isn't usable.
+ * With the --jar argument, it writes the classpath as a manifest in a jar file.
+ * This is useful in environments with short limitations on the maximum command
+ * line length, where it may not be possible to specify the full classpath in a
+ * command.  For example, the maximum command line length on Windows is 8191
+ * characters.
+ */
+@InterfaceAudience.Private
+public final class Classpath {
+  private static final String usage =
+    "classpath [--glob|--jar <path>|-h|--help] :\n"
+    + "  Prints the classpath needed to get the Hadoop jar and the required\n"
+    + "  libraries.\n"
+    + "  Options:\n"
+    + "\n"
+    + "  --glob       expand wildcards\n"
+    + "  --jar <path> write classpath as manifest in jar named <path>\n"
+    + "  -h, --help   print help\n";
+
+  /**
+   * Main entry point.
+   *
+   * @param args command-line arguments
+   */
+  public static void main(String[] args) {
+    if (args.length < 1 || args[0].equals("-h") || args[0].equals("--help")) {
+      System.out.println(usage);
+      return;
+    }
+
+    // Copy args, because CommandFormat mutates the list.
+    List<String> argsList = new ArrayList<String>(Arrays.asList(args));
+    CommandFormat cf = new CommandFormat(0, Integer.MAX_VALUE, "-glob", "-jar");
+    try {
+      cf.parse(argsList);
+    } catch (UnknownOptionException e) {
+      terminate(1, "unrecognized option");
+      return;
+    }
+
+    String classPath = System.getProperty("java.class.path");
+
+    if (cf.getOpt("-glob")) {
+      // The classpath returned from the property has been globbed already.
+      System.out.println(classPath);
+    } else if (cf.getOpt("-jar")) {
+      if (argsList.isEmpty() || argsList.get(0) == null ||
+          argsList.get(0).isEmpty()) {
+        terminate(1, "-jar option requires path of jar file to write");
+        return;
+      }
+
+      // Write the classpath into the manifest of a temporary jar file.
+      Path workingDir = new Path(System.getProperty("user.dir"));
+      final String tmpJarPath;
+      try {
+        tmpJarPath = FileUtil.createJarWithClassPath(classPath, workingDir,
+          System.getenv());
+      } catch (IOException e) {
+        terminate(1, "I/O error creating jar: " + e.getMessage());
+        return;
+      }
+
+      // Rename the temporary file to its final location.
+      String jarPath = argsList.get(0);
+      try {
+        FileUtil.replaceFile(new File(tmpJarPath), new File(jarPath));
+      } catch (IOException e) {
+        terminate(1, "I/O error renaming jar temporary file to path: " +
+          e.getMessage());
+        return;
+      }
+    }
+  }
+
+  /**
+   * Prints a message to stderr and exits with a status code.
+   *
+   * @param status exit code
+   * @param msg message
+   */
+  private static void terminate(int status, String msg) {
+    System.err.println(msg);
+    ExitUtil.terminate(status, msg);
+  }
+}

+ 14 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/DiskChecker.java

@@ -78,6 +78,20 @@ public class DiskChecker {
            (mkdirsWithExistsCheck(new File(parent)) &&
                                       (canonDir.mkdir() || canonDir.exists()));
   }
+
+  /**
+   * Recurse down a directory tree, checking all child directories.
+   * @param dir
+   * @throws DiskErrorException
+   */
+  public static void checkDirs(File dir) throws DiskErrorException {
+    checkDir(dir);
+    for (File child : dir.listFiles()) {
+      if (child.isDirectory()) {
+        checkDirs(child);
+      }
+    }
+  }
   
   /**
    * Create the directory if it doesn't exist and check that dir is readable,

+ 37 - 0
hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/nativeio/NativeIO.c

@@ -1054,6 +1054,43 @@ done:
 #endif
 }
 
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_io_nativeio_NativeIO_link0(JNIEnv *env,
+jclass clazz, jstring jsrc, jstring jdst)
+{
+#ifdef UNIX
+  const char *src = NULL, *dst = NULL;
+
+  src = (*env)->GetStringUTFChars(env, jsrc, NULL);
+  if (!src) goto done; // exception was thrown
+  dst = (*env)->GetStringUTFChars(env, jdst, NULL);
+  if (!dst) goto done; // exception was thrown
+  if (link(src, dst)) {
+    throw_ioe(env, errno);
+  }
+
+done:
+  if (src) (*env)->ReleaseStringUTFChars(env, jsrc, src);
+  if (dst) (*env)->ReleaseStringUTFChars(env, jdst, dst);
+#endif
+
+#ifdef WINDOWS
+  LPCTSTR src = NULL, dst = NULL;
+
+  src = (LPCTSTR) (*env)->GetStringChars(env, jsrc, NULL);
+  if (!src) goto done; // exception was thrown
+  dst = (LPCTSTR) (*env)->GetStringChars(env, jdst, NULL);
+  if (!dst) goto done; // exception was thrown
+  if (!CreateHardLink(dst, src, NULL)) {
+    throw_ioe(env, GetLastError());
+  }
+
+done:
+  if (src) (*env)->ReleaseStringChars(env, jsrc, src);
+  if (dst) (*env)->ReleaseStringChars(env, jdst, dst);
+#endif
+}
+
 JNIEXPORT jlong JNICALL
 Java_org_apache_hadoop_io_nativeio_NativeIO_getMemlockLimit0(
 JNIEnv *env, jclass clazz)

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

@@ -296,9 +296,24 @@ User Commands
 * <<<classpath>>>
 
    Prints the class path needed to get the Hadoop jar and the required
-   libraries.
+   libraries.  If called without arguments, then prints the classpath set up by
+   the command scripts, which is likely to contain wildcards in the classpath
+   entries.  Additional options print the classpath after wildcard expansion or
+   write the classpath into the manifest of a jar file.  The latter is useful in
+   environments where wildcards cannot be used and the expanded classpath exceeds
+   the maximum supported command line length.
 
-   Usage: <<<hadoop classpath>>>
+   Usage: <<<hadoop classpath [--glob|--jar <path>|-h|--help]>>>
+
+*-----------------+-----------------------------------------------------------+
+|| COMMAND_OPTION || Description
+*-----------------+-----------------------------------------------------------+
+| --glob          | expand wildcards
+*-----------------+-----------------------------------------------------------+
+| --jar <path>    | write classpath as manifest in jar named <path>
+*-----------------+-----------------------------------------------------------+
+| -h, --help      | print help
+*-----------------+-----------------------------------------------------------+
 
 Administration Commands
 

+ 9 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderCryptoExtension.java

@@ -26,10 +26,10 @@ import javax.crypto.spec.IvParameterSpec;
 import javax.crypto.spec.SecretKeySpec;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-
 import static org.apache.hadoop.crypto.key.KeyProvider.KeyVersion;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
@@ -118,8 +118,15 @@ public class TestKeyProviderCryptoExtension {
         new IvParameterSpec(KeyProviderCryptoExtension.EncryptedKeyVersion
             .deriveIV(encryptedKeyIv)));
     final byte[] manualMaterial = cipher.doFinal(encryptedKeyMaterial);
+
+    // Test the createForDecryption factory method
+    EncryptedKeyVersion eek2 =
+        EncryptedKeyVersion.createForDecryption(
+            eek.getEncryptionKeyVersionName(), eek.getEncryptedKeyIv(),
+            eek.getEncryptedKeyVersion().getMaterial());
+
     // Decrypt it with the API
-    KeyVersion decryptedKey = kpExt.decryptEncryptedKey(eek);
+    KeyVersion decryptedKey = kpExt.decryptEncryptedKey(eek2);
     final byte[] apiMaterial = decryptedKey.getMaterial();
 
     assertArrayEquals("Wrong key material from decryptEncryptedKey",

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

@@ -100,9 +100,9 @@ public class TestKeyProviderFactory {
   static void checkSpecificProvider(Configuration conf,
                                    String ourUrl) throws Exception {
     KeyProvider provider = KeyProviderFactory.getProviders(conf).get(0);
-    byte[] key1 = new byte[32];
-    byte[] key2 = new byte[32];
-    byte[] key3 = new byte[32];
+    byte[] key1 = new byte[16];
+    byte[] key2 = new byte[16];
+    byte[] key3 = new byte[16];
     for(int i =0; i < key1.length; ++i) {
       key1[i] = (byte) i;
       key2[i] = (byte) (i * 2);
@@ -146,7 +146,7 @@ public class TestKeyProviderFactory {
           KeyProvider.options(conf).setBitLength(8));
       assertTrue("should throw", false);
     } catch (IOException e) {
-      assertEquals("Wrong key length. Required 8, but got 256", e.getMessage());
+      assertEquals("Wrong key length. Required 8, but got 128", e.getMessage());
     }
     provider.createKey("key4", new byte[]{1},
         KeyProvider.options(conf).setBitLength(8));
@@ -162,7 +162,7 @@ public class TestKeyProviderFactory {
       provider.rollNewVersion("key4", key1);
       assertTrue("should throw", false);
     } catch (IOException e) {
-      assertEquals("Wrong key length. Required 8, but got 256", e.getMessage());
+      assertEquals("Wrong key length. Required 8, but got 128", e.getMessage());
     }
     try {
       provider.rollNewVersion("no-such-key", key1);
@@ -228,7 +228,7 @@ public class TestKeyProviderFactory {
   public void checkPermissionRetention(Configuration conf, String ourUrl, Path path) throws Exception {
     KeyProvider provider = KeyProviderFactory.getProviders(conf).get(0);
     // let's add a new key and flush and check that permissions are still set to 777
-    byte[] key = new byte[32];
+    byte[] key = new byte[16];
     for(int i =0; i < key.length; ++i) {
       key[i] = (byte) i;
     }
@@ -261,7 +261,7 @@ public class TestKeyProviderFactory {
       conf.set(JavaKeyStoreProvider.KEYSTORE_PASSWORD_FILE_KEY,
           "javakeystoreprovider.password");
       KeyProvider provider = KeyProviderFactory.getProviders(conf).get(0);
-      provider.createKey("key3", new byte[32], KeyProvider.options(conf));
+      provider.createKey("key3", new byte[16], KeyProvider.options(conf));
       provider.flush();
     } catch (Exception ex) {
       Assert.fail("could not create keystore with password file");

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

@@ -73,7 +73,7 @@ public class TestKeyShell {
   private void deleteKey(KeyShell ks, String keyName) throws Exception {
     int rc;
     outContent.reset();
-    final String[] delArgs = {"delete", keyName, "--provider", jceksProvider};
+    final String[] delArgs = {"delete", keyName, "-provider", jceksProvider};
     rc = ks.run(delArgs);
     assertEquals(0, rc);
     assertTrue(outContent.toString().contains(keyName + " has been " +
@@ -90,8 +90,8 @@ public class TestKeyShell {
   private String listKeys(KeyShell ks, boolean wantMetadata) throws Exception {
     int rc;
     outContent.reset();
-    final String[] listArgs = {"list", "--provider", jceksProvider };
-    final String[] listArgsM = {"list", "--metadata", "--provider", jceksProvider };
+    final String[] listArgs = {"list", "-provider", jceksProvider };
+    final String[] listArgsM = {"list", "-metadata", "-provider", jceksProvider };
     rc = ks.run(wantMetadata ? listArgsM : listArgs);
     assertEquals(0, rc);
     return outContent.toString();
@@ -106,11 +106,11 @@ public class TestKeyShell {
     ks.setConf(new Configuration());
 
     outContent.reset();
-    final String[] args1 = {"create", keyName, "--provider", jceksProvider};
+    final String[] args1 = {"create", keyName, "-provider", jceksProvider};
     rc = ks.run(args1);
     assertEquals(0, rc);
     assertTrue(outContent.toString().contains(keyName + " has been " +
-            "successfully created."));
+            "successfully created"));
 
     String listOut = listKeys(ks, false);
     assertTrue(listOut.contains(keyName));
@@ -121,7 +121,7 @@ public class TestKeyShell {
     assertTrue(listOut.contains("created"));
 
     outContent.reset();
-    final String[] args2 = {"roll", keyName, "--provider", jceksProvider};
+    final String[] args2 = {"roll", keyName, "-provider", jceksProvider};
     rc = ks.run(args2);
     assertEquals(0, rc);
     assertTrue(outContent.toString().contains("key1 has been successfully " +
@@ -137,15 +137,15 @@ public class TestKeyShell {
   @Test
   public void testKeySuccessfulCreationWithDescription() throws Exception {
     outContent.reset();
-    final String[] args1 = {"create", "key1", "--provider", jceksProvider,
-                      "--description", "someDescription"};
+    final String[] args1 = {"create", "key1", "-provider", jceksProvider,
+                      "-description", "someDescription"};
     int rc = 0;
     KeyShell ks = new KeyShell();
     ks.setConf(new Configuration());
     rc = ks.run(args1);
     assertEquals(0, rc);
     assertTrue(outContent.toString().contains("key1 has been successfully " +
-        "created."));
+        "created"));
 
     String listOut = listKeys(ks, true);
     assertTrue(listOut.contains("description"));
@@ -154,7 +154,7 @@ public class TestKeyShell {
 
   @Test
   public void testInvalidKeySize() throws Exception {
-    final String[] args1 = {"create", "key1", "--size", "56", "--provider",
+    final String[] args1 = {"create", "key1", "-size", "56", "-provider",
             jceksProvider};
 
     int rc = 0;
@@ -167,7 +167,7 @@ public class TestKeyShell {
 
   @Test
   public void testInvalidCipher() throws Exception {
-    final String[] args1 = {"create", "key1", "--cipher", "LJM", "--provider",
+    final String[] args1 = {"create", "key1", "-cipher", "LJM", "-provider",
             jceksProvider};
 
     int rc = 0;
@@ -180,7 +180,7 @@ public class TestKeyShell {
 
   @Test
   public void testInvalidProvider() throws Exception {
-    final String[] args1 = {"create", "key1", "--cipher", "AES", "--provider",
+    final String[] args1 = {"create", "key1", "-cipher", "AES", "-provider",
       "sdff://file/tmp/keystore.jceks"};
     
     int rc = 0;
@@ -194,7 +194,7 @@ public class TestKeyShell {
 
   @Test
   public void testTransientProviderWarning() throws Exception {
-    final String[] args1 = {"create", "key1", "--cipher", "AES", "--provider",
+    final String[] args1 = {"create", "key1", "-cipher", "AES", "-provider",
       "user:///"};
     
     int rc = 0;
@@ -224,8 +224,8 @@ public class TestKeyShell {
   @Test
   public void testFullCipher() throws Exception {
     final String keyName = "key1";
-    final String[] args1 = {"create", keyName, "--cipher", "AES/CBC/pkcs5Padding",
-        "--provider", jceksProvider};
+    final String[] args1 = {"create", keyName, "-cipher", "AES/CBC/pkcs5Padding",
+        "-provider", jceksProvider};
     
     int rc = 0;
     KeyShell ks = new KeyShell();
@@ -233,7 +233,7 @@ public class TestKeyShell {
     rc = ks.run(args1);
     assertEquals(0, rc);
     assertTrue(outContent.toString().contains(keyName + " has been " +
-            "successfully " +	"created."));
+            "successfully created"));
 
     deleteKey(ks, keyName);
   }
@@ -245,12 +245,12 @@ public class TestKeyShell {
     ks.setConf(new Configuration());
 
     /* Simple creation test */
-    final String[] args1 = {"create", "keyattr1", "--provider", jceksProvider,
-            "--attr", "foo=bar"};
+    final String[] args1 = {"create", "keyattr1", "-provider", jceksProvider,
+            "-attr", "foo=bar"};
     rc = ks.run(args1);
     assertEquals(0, rc);
     assertTrue(outContent.toString().contains("keyattr1 has been " +
-            "successfully " + "created."));
+            "successfully created"));
 
     /* ...and list to see that we have the attr */
     String listOut = listKeys(ks, true);
@@ -259,8 +259,8 @@ public class TestKeyShell {
 
     /* Negative tests: no attribute */
     outContent.reset();
-    final String[] args2 = {"create", "keyattr2", "--provider", jceksProvider,
-            "--attr", "=bar"};
+    final String[] args2 = {"create", "keyattr2", "-provider", jceksProvider,
+            "-attr", "=bar"};
     rc = ks.run(args2);
     assertEquals(1, rc);
 
@@ -288,10 +288,10 @@ public class TestKeyShell {
 
     /* Test several attrs together... */
     outContent.reset();
-    final String[] args3 = {"create", "keyattr3", "--provider", jceksProvider,
-            "--attr", "foo = bar",
-            "--attr", " glarch =baz  ",
-            "--attr", "abc=def"};
+    final String[] args3 = {"create", "keyattr3", "-provider", jceksProvider,
+            "-attr", "foo = bar",
+            "-attr", " glarch =baz  ",
+            "-attr", "abc=def"};
     rc = ks.run(args3);
     assertEquals(0, rc);
 
@@ -304,9 +304,9 @@ public class TestKeyShell {
 
     /* Negative test - repeated attributes should fail */
     outContent.reset();
-    final String[] args4 = {"create", "keyattr4", "--provider", jceksProvider,
-            "--attr", "foo=bar",
-            "--attr", "foo=glarch"};
+    final String[] args4 = {"create", "keyattr4", "-provider", jceksProvider,
+            "-attr", "foo=bar",
+            "-attr", "foo=glarch"};
     rc = ks.run(args4);
     assertEquals(1, rc);
 

+ 3 - 3
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java

@@ -414,7 +414,7 @@ public class TestHttpServer extends HttpServerFunctionalTest {
         assertEquals(HttpURLConnection.HTTP_OK, getHttpStatusCode(serverURL
             + servlet, user));
       }
-      assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED, getHttpStatusCode(
+      assertEquals(HttpURLConnection.HTTP_FORBIDDEN, getHttpStatusCode(
           serverURL + servlet, "userE"));
     }
     myServer.stop();
@@ -474,7 +474,7 @@ public class TestHttpServer extends HttpServerFunctionalTest {
     response = Mockito.mock(HttpServletResponse.class);
     conf.setBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, true);
     Assert.assertFalse(HttpServer2.hasAdministratorAccess(context, request, response));
-    Mockito.verify(response).sendError(Mockito.eq(HttpServletResponse.SC_UNAUTHORIZED), Mockito.anyString());
+    Mockito.verify(response).sendError(Mockito.eq(HttpServletResponse.SC_FORBIDDEN), Mockito.anyString());
 
     //authorization ON & user NOT NULL & ACLs NULL
     response = Mockito.mock(HttpServletResponse.class);
@@ -487,7 +487,7 @@ public class TestHttpServer extends HttpServerFunctionalTest {
     Mockito.when(acls.isUserAllowed(Mockito.<UserGroupInformation>any())).thenReturn(false);
     Mockito.when(context.getAttribute(HttpServer2.ADMINS_ACL)).thenReturn(acls);
     Assert.assertFalse(HttpServer2.hasAdministratorAccess(context, request, response));
-    Mockito.verify(response).sendError(Mockito.eq(HttpServletResponse.SC_UNAUTHORIZED), Mockito.anyString());
+    Mockito.verify(response).sendError(Mockito.eq(HttpServletResponse.SC_FORBIDDEN), Mockito.anyString());
 
     //authorization ON & user NOT NULL & ACLs NOT NULL & user in in ACLs
     response = Mockito.mock(HttpServletResponse.class);

+ 58 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java

@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.security;
 
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
 import static org.mockito.Mockito.*;
 
 import java.io.File;
@@ -38,6 +40,9 @@ import javax.naming.directory.SearchControls;
 import javax.naming.directory.SearchResult;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.alias.CredentialProvider;
+import org.apache.hadoop.security.alias.CredentialProviderFactory;
+import org.apache.hadoop.security.alias.JavaKeyStoreProvider;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -154,4 +159,57 @@ public class TestLdapGroupsMapping {
     Assert.assertEquals("hadoop",
         mapping.extractPassword(secretFile.getPath()));
   }
+
+  @Test
+  public void testConfGetPassword() throws Exception {
+    File testDir = new File(System.getProperty("test.build.data",
+                                               "target/test-dir"));
+    Configuration conf = new Configuration();
+    final String ourUrl =
+        JavaKeyStoreProvider.SCHEME_NAME + "://file/" + testDir + "/test.jks";
+
+    File file = new File(testDir, "test.jks");
+    file.delete();
+    conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, ourUrl);
+
+    CredentialProvider provider =
+        CredentialProviderFactory.getProviders(conf).get(0);
+    char[] bindpass = {'b', 'i', 'n', 'd', 'p', 'a', 's', 's'};
+    char[] storepass = {'s', 't', 'o', 'r', 'e', 'p', 'a', 's', 's'};
+
+    // ensure that we get nulls when the key isn't there
+    assertEquals(null, provider.getCredentialEntry(
+        LdapGroupsMapping.BIND_PASSWORD_KEY));
+    assertEquals(null, provider.getCredentialEntry
+        (LdapGroupsMapping.LDAP_KEYSTORE_PASSWORD_KEY));
+
+    // create new aliases
+    try {
+      provider.createCredentialEntry(
+          LdapGroupsMapping.BIND_PASSWORD_KEY, bindpass);
+
+      provider.createCredentialEntry(
+          LdapGroupsMapping.LDAP_KEYSTORE_PASSWORD_KEY, storepass);
+      provider.flush();
+    } catch (Exception e) {
+      e.printStackTrace();
+      throw e;
+    }
+    // make sure we get back the right key
+    assertArrayEquals(bindpass, provider.getCredentialEntry(
+        LdapGroupsMapping.BIND_PASSWORD_KEY).getCredential());
+    assertArrayEquals(storepass, provider.getCredentialEntry(
+        LdapGroupsMapping.LDAP_KEYSTORE_PASSWORD_KEY).getCredential());
+
+    LdapGroupsMapping mapping = new LdapGroupsMapping();
+    Assert.assertEquals("bindpass",
+        mapping.getPassword(conf, LdapGroupsMapping.BIND_PASSWORD_KEY, ""));
+    Assert.assertEquals("storepass",
+        mapping.getPassword(conf, LdapGroupsMapping.LDAP_KEYSTORE_PASSWORD_KEY,
+           ""));
+    // let's make sure that a password that doesn't exist returns an
+    // empty string as currently expected and used to trigger a call to
+    // extract password
+    Assert.assertEquals("", mapping.getPassword(conf,"invalid-alias", ""));
+  }
 }

+ 34 - 15
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/alias/TestCredShell.java

@@ -17,16 +17,18 @@
  */
 package org.apache.hadoop.security.alias;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.PrintStream;
 import java.util.ArrayList;
+import java.util.Arrays;
 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;
 
@@ -45,7 +47,7 @@ public class TestCredShell {
   @Test
   public void testCredentialSuccessfulLifecycle() throws Exception {
     outContent.reset();
-    String[] args1 = {"create", "credential1", "--value", "p@ssw0rd", "--provider", 
+    String[] args1 = {"create", "credential1", "-value", "p@ssw0rd", "-provider",
         "jceks://file" + tmpDir + "/credstore.jceks"};
     int rc = 0;
     CredentialShell cs = new CredentialShell();
@@ -56,14 +58,14 @@ public class TestCredShell {
     		"created."));
 
     outContent.reset();
-    String[] args2 = {"list", "--provider", 
+    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", 
+    String[] args4 = {"delete", "credential1", "-provider",
         "jceks://file" + tmpDir + "/credstore.jceks"};
     rc = cs.run(args4);
     assertEquals(0, rc);
@@ -71,7 +73,7 @@ public class TestCredShell {
     		"deleted."));
 
     outContent.reset();
-    String[] args5 = {"list", "--provider", 
+    String[] args5 = {"list", "-provider",
         "jceks://file" + tmpDir + "/credstore.jceks"};
     rc = cs.run(args5);
     assertEquals(0, rc);
@@ -80,21 +82,21 @@ public class TestCredShell {
 
   @Test
   public void testInvalidProvider() throws Exception {
-    String[] args1 = {"create", "credential1", "--value", "p@ssw0rd", "--provider", 
+    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);
+    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", 
+    String[] args1 = {"create", "credential1", "-value", "p@ssw0rd", "-provider",
       "user:///"};
     
     int rc = 0;
@@ -105,7 +107,7 @@ public class TestCredShell {
     assertTrue(outContent.toString().contains("WARNING: you are modifying a " +
     		"transient provider."));
 
-    String[] args2 = {"delete", "credential1", "--provider", "user:///"};
+    String[] args2 = {"delete", "credential1", "-provider", "user:///"};
     rc = cs.run(args2);
     assertEquals(outContent.toString(), 0, rc);
     assertTrue(outContent.toString().contains("credential1 has been successfully " +
@@ -122,14 +124,14 @@ public class TestCredShell {
     config.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, "user:///");
     cs.setConf(config);
     rc = cs.run(args1);
-    assertEquals(-1, rc);
+    assertEquals(1, rc);
     assertTrue(outContent.toString().contains("There are no valid " +
     		"CredentialProviders configured."));
   }
   
   @Test
   public void testPromptForCredentialWithEmptyPasswd() throws Exception {
-    String[] args1 = {"create", "credential1", "--provider", 
+    String[] args1 = {"create", "credential1", "-provider",
         "jceks://file" + tmpDir + "/credstore.jceks"};
     ArrayList<String> passwords = new ArrayList<String>();
     passwords.add(null);
@@ -139,13 +141,13 @@ public class TestCredShell {
     shell.setConf(new Configuration());
     shell.setPasswordReader(new MockPasswordReader(passwords));
     rc = shell.run(args1);
-    assertEquals(outContent.toString(), -1, rc);
+    assertEquals(outContent.toString(), 1, rc);
     assertTrue(outContent.toString().contains("Passwords don't match"));
   }
 
   @Test
   public void testPromptForCredential() throws Exception {
-    String[] args1 = {"create", "credential1", "--provider", 
+    String[] args1 = {"create", "credential1", "-provider",
         "jceks://file" + tmpDir + "/credstore.jceks"};
     ArrayList<String> passwords = new ArrayList<String>();
     passwords.add("p@ssw0rd");
@@ -159,7 +161,7 @@ public class TestCredShell {
     assertTrue(outContent.toString().contains("credential1 has been successfully " +
         "created."));
     
-    String[] args2 = {"delete", "credential1", "--provider", 
+    String[] args2 = {"delete", "credential1", "-provider",
         "jceks://file" + tmpDir + "/credstore.jceks"};
     rc = shell.run(args2);
     assertEquals(0, rc);
@@ -186,4 +188,21 @@ public class TestCredShell {
       System.out.println(message);
     }
   }
+
+  @Test
+  public void testEmptyArgList() throws Exception {
+    CredentialShell shell = new CredentialShell();
+    shell.setConf(new Configuration());
+    assertEquals(1, shell.init(new String[0]));
+  }
+
+  @Test
+  public void testCommandHelpExitsNormally() throws Exception {
+    for (String cmd : Arrays.asList("create", "list", "delete")) {
+      CredentialShell shell = new CredentialShell();
+      shell.setConf(new Configuration());
+      assertEquals("Expected help argument on " + cmd + " to return 0",
+              0, shell.init(new String[] {cmd, "-help"}));
+    }
+  }
 }

+ 41 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/KeyStoreTestUtil.java

@@ -19,6 +19,10 @@
 package org.apache.hadoop.security.ssl;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.alias.CredentialProvider;
+import org.apache.hadoop.security.alias.CredentialProviderFactory;
+import org.apache.hadoop.security.alias.JavaKeyStoreProvider;
+
 import sun.security.x509.AlgorithmId;
 import sun.security.x509.CertificateAlgorithmId;
 import sun.security.x509.CertificateIssuerName;
@@ -382,4 +386,41 @@ public class KeyStoreTestUtil {
       writer.close();
     }
   }
+
+  public static void provisionPasswordsToCredentialProvider() throws Exception {
+    File testDir = new File(System.getProperty("test.build.data",
+        "target/test-dir"));
+
+    Configuration conf = new Configuration();
+    final String ourUrl =
+    JavaKeyStoreProvider.SCHEME_NAME + "://file/" + testDir + "/test.jks";
+
+    File file = new File(testDir, "test.jks");
+    file.delete();
+    conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, ourUrl);
+
+    CredentialProvider provider =
+        CredentialProviderFactory.getProviders(conf).get(0);
+    char[] keypass = {'k', 'e', 'y', 'p', 'a', 's', 's'};
+    char[] storepass = {'s', 't', 'o', 'r', 'e', 'p', 'a', 's', 's'};
+
+    // create new aliases
+    try {
+      provider.createCredentialEntry(
+          FileBasedKeyStoresFactory.resolvePropertyName(SSLFactory.Mode.SERVER,
+              FileBasedKeyStoresFactory.SSL_KEYSTORE_PASSWORD_TPL_KEY),
+              storepass);
+
+      provider.createCredentialEntry(
+          FileBasedKeyStoresFactory.resolvePropertyName(SSLFactory.Mode.SERVER,
+              FileBasedKeyStoresFactory.SSL_KEYSTORE_KEYPASSWORD_TPL_KEY),
+              keypass);
+
+      // write out so that it can be found in checks
+      provider.flush();
+    } catch (Exception e) {
+      e.printStackTrace();
+      throw e;
+    }
+  }
 }

+ 57 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestSSLFactory.java

@@ -17,8 +17,14 @@
  */
 package org.apache.hadoop.security.ssl;
 
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.security.alias.CredentialProvider;
+import org.apache.hadoop.security.alias.CredentialProviderFactory;
+import org.apache.hadoop.security.alias.JavaKeyStoreProvider;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -211,6 +217,13 @@ public class TestSSLFactory {
       "password", "password", null);
   }
 
+  @Test
+  public void testServerCredProviderPasswords() throws Exception {
+    KeyStoreTestUtil.provisionPasswordsToCredentialProvider();
+    checkSSLFactoryInitWithPasswords(SSLFactory.Mode.SERVER,
+        "storepass", "keypass", null, null, true);
+  }
+
   /**
    * Checks that SSLFactory initialization is successful with the given
    * arguments.  This is a helper method for writing test cases that cover
@@ -218,7 +231,7 @@ public class TestSSLFactory {
    * It takes care of bootstrapping a keystore, a truststore, and SSL client or
    * server configuration.  Then, it initializes an SSLFactory.  If no exception
    * is thrown, then initialization was successful.
-   * 
+   *
    * @param mode SSLFactory.Mode mode to test
    * @param password String store password to set on keystore
    * @param keyPassword String key password to set on keystore
@@ -231,6 +244,34 @@ public class TestSSLFactory {
   private void checkSSLFactoryInitWithPasswords(SSLFactory.Mode mode,
       String password, String keyPassword, String confPassword,
       String confKeyPassword) throws Exception {
+    checkSSLFactoryInitWithPasswords(mode, password, keyPassword,
+        confPassword, confKeyPassword, false);
+  }
+
+ /**
+   * Checks that SSLFactory initialization is successful with the given
+   * arguments.  This is a helper method for writing test cases that cover
+   * different combinations of settings for the store password and key password.
+   * It takes care of bootstrapping a keystore, a truststore, and SSL client or
+   * server configuration.  Then, it initializes an SSLFactory.  If no exception
+   * is thrown, then initialization was successful.
+   *
+   * @param mode SSLFactory.Mode mode to test
+   * @param password String store password to set on keystore
+   * @param keyPassword String key password to set on keystore
+   * @param confPassword String store password to set in SSL config file, or null
+   *   to avoid setting in SSL config file
+   * @param confKeyPassword String key password to set in SSL config file, or
+   *   null to avoid setting in SSL config file
+   * @param useCredProvider boolean to indicate whether passwords should be set
+   * into the config or not. When set to true nulls are set and aliases are
+   * expected to be resolved through credential provider API through the
+   * Configuration.getPassword method
+   * @throws Exception for any error
+   */
+  private void checkSSLFactoryInitWithPasswords(SSLFactory.Mode mode,
+      String password, String keyPassword, String confPassword,
+      String confKeyPassword, boolean useCredProvider) throws Exception {
     String keystore = new File(KEYSTORES_DIR, "keystore.jks").getAbsolutePath();
     String truststore = new File(KEYSTORES_DIR, "truststore.jks")
       .getAbsolutePath();
@@ -249,10 +290,25 @@ public class TestSSLFactory {
     // Create SSL configuration file, for either server or client.
     final String sslConfFileName;
     final Configuration sslConf;
+
+    // if the passwords are provisioned in a cred provider then don't set them
+    // in the configuration properly - expect them to be resolved through the
+    // provider
+    if (useCredProvider) {
+      confPassword = null;
+      confKeyPassword = null;
+    }
     if (mode == SSLFactory.Mode.SERVER) {
       sslConfFileName = "ssl-server.xml";
       sslConf = KeyStoreTestUtil.createServerSSLConfig(keystore, confPassword,
         confKeyPassword, truststore);
+      if (useCredProvider) {
+        File testDir = new File(System.getProperty("test.build.data",
+            "target/test-dir"));
+        final String ourUrl =
+            JavaKeyStoreProvider.SCHEME_NAME + "://file/" + testDir + "/test.jks";
+        sslConf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, ourUrl);
+      }
     } else {
       sslConfFileName = "ssl-client.xml";
       sslConf = KeyStoreTestUtil.createClientSSLConfig(keystore, confPassword,

+ 176 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestClasspath.java

@@ -0,0 +1,176 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.util;
+
+import static org.junit.Assert.*;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.charset.Charset;
+import java.util.jar.Attributes;
+import java.util.jar.JarFile;
+import java.util.jar.Manifest;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.io.IOUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Tests covering the classpath command-line utility.
+ */
+public class TestClasspath {
+
+  private static final Log LOG = LogFactory.getLog(TestClasspath.class);
+  private static final File TEST_DIR = new File(
+    System.getProperty("test.build.data", "/tmp"), "TestClasspath");
+  private static final Charset UTF8 = Charset.forName("UTF-8");
+
+  static {
+    ExitUtil.disableSystemExit();
+  }
+
+  private PrintStream oldStdout, oldStderr;
+  private ByteArrayOutputStream stdout, stderr;
+  private PrintStream printStdout, printStderr;
+
+  @Before
+  public void setUp() {
+    assertTrue(FileUtil.fullyDelete(TEST_DIR));
+    assertTrue(TEST_DIR.mkdirs());
+    oldStdout = System.out;
+    oldStderr = System.err;
+
+    stdout = new ByteArrayOutputStream();
+    printStdout = new PrintStream(stdout);
+    System.setOut(printStdout);
+
+    stderr = new ByteArrayOutputStream();
+    printStderr = new PrintStream(stderr);
+    System.setErr(printStderr);
+  }
+
+  @After
+  public void tearDown() {
+    System.setOut(oldStdout);
+    System.setErr(oldStderr);
+    IOUtils.cleanup(LOG, printStdout, printStderr);
+    assertTrue(FileUtil.fullyDelete(TEST_DIR));
+  }
+
+  @Test
+  public void testGlob() {
+    Classpath.main(new String[] { "--glob" });
+    String strOut = new String(stdout.toByteArray(), UTF8);
+    assertEquals(System.getProperty("java.class.path"), strOut.trim());
+    assertTrue(stderr.toByteArray().length == 0);
+  }
+
+  @Test
+  public void testJar() throws IOException {
+    File file = new File(TEST_DIR, "classpath.jar");
+    Classpath.main(new String[] { "--jar", file.getAbsolutePath() });
+    assertTrue(stdout.toByteArray().length == 0);
+    assertTrue(stderr.toByteArray().length == 0);
+    assertTrue(file.exists());
+    assertJar(file);
+  }
+
+  @Test
+  public void testJarReplace() throws IOException {
+    // Run the command twice with the same output jar file, and expect success.
+    testJar();
+    testJar();
+  }
+
+  @Test
+  public void testJarFileMissing() throws IOException {
+    try {
+      Classpath.main(new String[] { "--jar" });
+      fail("expected exit");
+    } catch (ExitUtil.ExitException e) {
+      assertTrue(stdout.toByteArray().length == 0);
+      String strErr = new String(stderr.toByteArray(), UTF8);
+      assertTrue(strErr.contains("requires path of jar"));
+    }
+  }
+
+  @Test
+  public void testHelp() {
+    Classpath.main(new String[] { "--help" });
+    String strOut = new String(stdout.toByteArray(), UTF8);
+    assertTrue(strOut.contains("Prints the classpath"));
+    assertTrue(stderr.toByteArray().length == 0);
+  }
+
+  @Test
+  public void testHelpShort() {
+    Classpath.main(new String[] { "-h" });
+    String strOut = new String(stdout.toByteArray(), UTF8);
+    assertTrue(strOut.contains("Prints the classpath"));
+    assertTrue(stderr.toByteArray().length == 0);
+  }
+
+  @Test
+  public void testUnrecognized() {
+    try {
+      Classpath.main(new String[] { "--notarealoption" });
+      fail("expected exit");
+    } catch (ExitUtil.ExitException e) {
+      assertTrue(stdout.toByteArray().length == 0);
+      String strErr = new String(stderr.toByteArray(), UTF8);
+      assertTrue(strErr.contains("unrecognized option"));
+    }
+  }
+
+  /**
+   * Asserts that the specified file is a jar file with a manifest containing a
+   * non-empty classpath attribute.
+   *
+   * @param file File to check
+   * @throws IOException if there is an I/O error
+   */
+  private static void assertJar(File file) throws IOException {
+    JarFile jarFile = null;
+    try {
+      jarFile = new JarFile(file);
+      Manifest manifest = jarFile.getManifest();
+      assertNotNull(manifest);
+      Attributes mainAttributes = manifest.getMainAttributes();
+      assertNotNull(mainAttributes);
+      assertTrue(mainAttributes.containsKey(Attributes.Name.CLASS_PATH));
+      String classPathAttr = mainAttributes.getValue(Attributes.Name.CLASS_PATH);
+      assertNotNull(classPathAttr);
+      assertFalse(classPathAttr.isEmpty());
+    } finally {
+      // It's too bad JarFile doesn't implement Closeable.
+      if (jarFile != null) {
+        try {
+          jarFile.close();
+        } catch (IOException e) {
+          LOG.warn("exception closing jarFile: " + jarFile, e);
+        }
+      }
+    }
+  }
+}

+ 33 - 0
hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSJMXServlet.java

@@ -0,0 +1,33 @@
+/**
+ * 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.crypto.key.kms.server;
+
+import org.apache.hadoop.jmx.JMXJsonServlet;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+
+public class KMSJMXServlet extends JMXJsonServlet {
+
+  @Override
+  protected boolean isInstrumentationAccessAllowed(HttpServletRequest request,
+      HttpServletResponse response) throws IOException {
+    return true;
+  }
+}

+ 8 - 1
hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebApp.java

@@ -181,12 +181,19 @@ public class KMSWebApp implements ServletContextListener {
         keyProvider = new CachingKeyProvider(keyProvider, keyTimeOutMillis,
             currKeyTimeOutMillis);
       }
+      LOG.info("Initialized KeyProvider " + keyProvider);
+
       keyProviderCryptoExtension = KeyProviderCryptoExtension.
           createKeyProviderCryptoExtension(keyProvider);
       keyProviderCryptoExtension = 
           new EagerKeyGeneratorKeyProviderCryptoExtension(kmsConf, 
               keyProviderCryptoExtension);
-
+      LOG.info("Initialized KeyProviderCryptoExtension "
+          + keyProviderCryptoExtension);
+      final int defaultBitlength = kmsConf
+          .getInt(KeyProvider.DEFAULT_BITLENGTH_NAME,
+              KeyProvider.DEFAULT_BITLENGTH);
+      LOG.info("Default key bitlength is {}", defaultBitlength);
       LOG.info("KMS Started");
     } catch (Throwable ex) {
       System.out.println();

+ 1 - 1
hadoop-common-project/hadoop-kms/src/main/webapp/WEB-INF/web.xml

@@ -42,7 +42,7 @@
 
   <servlet>
     <servlet-name>jmx-servlet</servlet-name>
-    <servlet-class>org.apache.hadoop.jmx.JMXJsonServlet</servlet-class>
+    <servlet-class>org.apache.hadoop.crypto.key.kms.server.KMSJMXServlet</servlet-class>
   </servlet>
 
   <servlet-mapping>

+ 6 - 6
hadoop-common-project/hadoop-kms/src/site/apt/index.apt.vm

@@ -106,14 +106,14 @@ Hadoop Key Management Server (KMS) - Documentation Sets ${project.version}
 
 ** KMS Aggregated Audit logs
 
-Audit logs are aggregated for API accesses to the GET_KEY_VERSION, 
-GET_CURRENT_KEY, DECRYPT_EEK, GENERATE_EEK operations.
+  Audit logs are aggregated for API accesses to the GET_KEY_VERSION,
+  GET_CURRENT_KEY, DECRYPT_EEK, GENERATE_EEK operations.
 
-Entries are grouped by the (user,key,operation) combined key for a configurable
-aggregation interval after which the number of accesses to the specified
-end-point by the user for a given key is flushed to the audit log.
+  Entries are grouped by the (user,key,operation) combined key for a
+  configurable aggregation interval after which the number of accesses to the
+  specified end-point by the user for a given key is flushed to the audit log.
 
-The Aggregation interval is configured via the property :
+  The Aggregation interval is configured via the property :
 
 +---+
   <property>

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java

@@ -65,6 +65,7 @@ import org.mortbay.jetty.Server;
 import org.mortbay.jetty.webapp.WebAppContext;
 
 import com.google.common.collect.Maps;
+import org.apache.hadoop.security.authentication.util.StringSignerSecretProvider;
 
 public class TestHttpFSServer extends HFSTestCase {
 
@@ -683,7 +684,7 @@ public class TestHttpFSServer extends HFSTestCase {
       new AuthenticationToken("u", "p",
         HttpFSKerberosAuthenticationHandlerForTesting.TYPE);
     token.setExpires(System.currentTimeMillis() + 100000000);
-    Signer signer = new Signer("secret".getBytes());
+    Signer signer = new Signer(new StringSignerSecretProvider("secret"));
     String tokenSigned = signer.sign(token.toString());
 
     url = new URL(TestJettyHelper.getJettyURL(),

Tiedoston diff-näkymää rajattu, sillä se on liian suuri
+ 215 - 166
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java


+ 587 - 3
hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestRpcProgramNfs3.java

@@ -18,19 +18,603 @@
 package org.apache.hadoop.hdfs.nfs.nfs3;
 
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import org.jboss.netty.channel.Channel;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.Mockito;
 
 import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
+import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.nfs.nfs3.FileHandle;
 import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
-import org.junit.Assert;
-import org.junit.Test;
+import org.apache.hadoop.nfs.nfs3.Nfs3Constant.WriteStableHow;
+import org.apache.hadoop.nfs.nfs3.Nfs3Status;
+import org.apache.hadoop.nfs.nfs3.request.LOOKUP3Request;
+import org.apache.hadoop.nfs.nfs3.request.READ3Request;
+import org.apache.hadoop.nfs.nfs3.request.WRITE3Request;
+import org.apache.hadoop.nfs.nfs3.response.ACCESS3Response;
+import org.apache.hadoop.nfs.nfs3.response.COMMIT3Response;
+import org.apache.hadoop.nfs.nfs3.response.CREATE3Response;
+import org.apache.hadoop.nfs.nfs3.response.FSSTAT3Response;
+import org.apache.hadoop.nfs.nfs3.response.FSINFO3Response;
+import org.apache.hadoop.nfs.nfs3.response.GETATTR3Response;
+import org.apache.hadoop.nfs.nfs3.response.LOOKUP3Response;
+import org.apache.hadoop.nfs.nfs3.response.PATHCONF3Response;
+import org.apache.hadoop.nfs.nfs3.response.READ3Response;
+import org.apache.hadoop.nfs.nfs3.response.REMOVE3Response;
+import org.apache.hadoop.nfs.nfs3.response.RMDIR3Response;
+import org.apache.hadoop.nfs.nfs3.response.RENAME3Response;
+import org.apache.hadoop.nfs.nfs3.response.READDIR3Response;
+import org.apache.hadoop.nfs.nfs3.response.READDIRPLUS3Response;
+import org.apache.hadoop.nfs.nfs3.response.READLINK3Response;
+import org.apache.hadoop.nfs.nfs3.response.SETATTR3Response;
+import org.apache.hadoop.nfs.nfs3.response.SYMLINK3Response;
+import org.apache.hadoop.nfs.nfs3.response.WRITE3Response;
+import org.apache.hadoop.nfs.nfs3.request.SetAttr3;
+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;
 
 
 /**
  * Tests for {@link RpcProgramNfs3}
  */
 public class TestRpcProgramNfs3 {
+  static DistributedFileSystem hdfs;
+  static MiniDFSCluster cluster = null;
+  static NfsConfiguration config = new NfsConfiguration();
+  static NameNode nn;
+  static Nfs3 nfs;
+  static RpcProgramNfs3 nfsd;
+  static SecurityHandler securityHandler;
+  static SecurityHandler securityHandlerUnpriviledged;
+  static String testdir = "/tmp";
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    String currentUser = System.getProperty("user.name");
+
+    config.set("fs.permissions.umask-mode", "u=rwx,g=,o=");
+    config.set(DefaultImpersonationProvider.getTestProvider()
+        .getProxySuperuserGroupConfKey(currentUser), "*");
+    config.set(DefaultImpersonationProvider.getTestProvider()
+        .getProxySuperuserIpConfKey(currentUser), "*");
+    ProxyUsers.refreshSuperUserGroupsConfiguration(config);
+
+    cluster = new MiniDFSCluster.Builder(config).numDataNodes(1).build();
+    cluster.waitActive();
+    hdfs = cluster.getFileSystem();
+    nn = cluster.getNameNode();
+
+    // Use ephemeral ports in case tests are running in parallel
+    config.setInt("nfs3.mountd.port", 0);
+    config.setInt("nfs3.server.port", 0);
+
+    // Start NFS with allowed.hosts set to "* rw"
+    config.set("dfs.nfs.exports.allowed.hosts", "* rw");
+    nfs = new Nfs3(config);
+    nfs.startServiceInternal(false);
+    nfsd = (RpcProgramNfs3) nfs.getRpcProgram();
+
+
+    // Mock SecurityHandler which returns system user.name
+    securityHandler = Mockito.mock(SecurityHandler.class);
+    Mockito.when(securityHandler.getUser()).thenReturn(currentUser);
+
+    // Mock SecurityHandler which returns a dummy username "harry"
+    securityHandlerUnpriviledged = Mockito.mock(SecurityHandler.class);
+    Mockito.when(securityHandlerUnpriviledged.getUser()).thenReturn("harry");
+  }
+
+  @AfterClass
+  public static void shutdown() throws Exception {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Before
+  public void createFiles() throws IllegalArgumentException, IOException {
+    hdfs.delete(new Path(testdir), true);
+    hdfs.mkdirs(new Path(testdir));
+    hdfs.mkdirs(new Path(testdir + "/foo"));
+    DFSTestUtil.createFile(hdfs, new Path(testdir + "/bar"), 0, (short) 1, 0);
+  }
+
+  @Test(timeout = 60000)
+  public void testGetattr() throws Exception {
+    HdfsFileStatus status = nn.getRpcServer().getFileInfo("/tmp/bar");
+    long dirId = status.getFileId();
+    FileHandle handle = new FileHandle(dirId);
+    XDR xdr_req = new XDR();
+    handle.serialize(xdr_req);
+
+    // Attempt by an unpriviledged user should fail.
+    GETATTR3Response response1 = nfsd.getattr(xdr_req.asReadOnlyWrap(),
+        securityHandlerUnpriviledged,
+        new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect return code", Nfs3Status.NFS3ERR_ACCES,
+        response1.getStatus());
+
+    // Attempt by a priviledged user should pass.
+    GETATTR3Response response2 = nfsd.getattr(xdr_req.asReadOnlyWrap(),
+        securityHandler, new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect return code", Nfs3Status.NFS3_OK,
+        response2.getStatus());
+  }
+
+  @Test(timeout = 60000)
+  public void testSetattr() throws Exception {
+    HdfsFileStatus status = nn.getRpcServer().getFileInfo(testdir);
+    long dirId = status.getFileId();
+    XDR xdr_req = new XDR();
+    FileHandle handle = new FileHandle(dirId);
+    handle.serialize(xdr_req);
+    xdr_req.writeString("bar");
+    SetAttr3 symAttr = new SetAttr3();
+    symAttr.serialize(xdr_req);
+    xdr_req.writeBoolean(false);
+
+    // Attempt by an unpriviledged user should fail.
+    SETATTR3Response response1 = nfsd.setattr(xdr_req.asReadOnlyWrap(),
+        securityHandlerUnpriviledged,
+        new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect return code", Nfs3Status.NFS3ERR_ACCES,
+        response1.getStatus());
+
+    // Attempt by a priviledged user should pass.
+    SETATTR3Response response2 = nfsd.setattr(xdr_req.asReadOnlyWrap(),
+        securityHandler, new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect return code", Nfs3Status.NFS3_OK,
+        response2.getStatus());
+  }
+
+  @Test(timeout = 60000)
+  public void testLookup() throws Exception {
+    HdfsFileStatus status = nn.getRpcServer().getFileInfo(testdir);
+    long dirId = status.getFileId();
+    FileHandle handle = new FileHandle(dirId);
+    LOOKUP3Request lookupReq = new LOOKUP3Request(handle, "bar");
+    XDR xdr_req = new XDR();
+    lookupReq.serialize(xdr_req);
+
+    // Attempt by an unpriviledged user should fail.
+    LOOKUP3Response response1 = nfsd.lookup(xdr_req.asReadOnlyWrap(),
+        securityHandlerUnpriviledged,
+        new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect return code", Nfs3Status.NFS3ERR_ACCES,
+        response1.getStatus());
+
+    // Attempt by a priviledged user should pass.
+    LOOKUP3Response response2 = nfsd.lookup(xdr_req.asReadOnlyWrap(),
+        securityHandler, new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect return code", Nfs3Status.NFS3_OK,
+        response2.getStatus());
+  }
+
+  @Test(timeout = 60000)
+  public void testAccess() throws Exception {
+    HdfsFileStatus status = nn.getRpcServer().getFileInfo("/tmp/bar");
+    long dirId = status.getFileId();
+    FileHandle handle = new FileHandle(dirId);
+    XDR xdr_req = new XDR();
+    handle.serialize(xdr_req);
+
+    // Attempt by an unpriviledged user should fail.
+    ACCESS3Response response1 = nfsd.access(xdr_req.asReadOnlyWrap(),
+        securityHandlerUnpriviledged,
+        new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect return code", Nfs3Status.NFS3ERR_ACCES,
+        response1.getStatus());
+
+    // Attempt by a priviledged user should pass.
+    ACCESS3Response response2 = nfsd.access(xdr_req.asReadOnlyWrap(),
+        securityHandler, new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect return code", Nfs3Status.NFS3_OK,
+        response2.getStatus());
+  }
+
+  @Test(timeout = 60000)
+  public void testReadlink() throws Exception {
+    // Create a symlink first.
+    HdfsFileStatus status = nn.getRpcServer().getFileInfo(testdir);
+    long dirId = status.getFileId();
+    XDR xdr_req = new XDR();
+    FileHandle handle = new FileHandle(dirId);
+    handle.serialize(xdr_req);
+    xdr_req.writeString("fubar");
+    SetAttr3 symAttr = new SetAttr3();
+    symAttr.serialize(xdr_req);
+    xdr_req.writeString("bar");
+
+    SYMLINK3Response response = nfsd.symlink(xdr_req.asReadOnlyWrap(),
+        securityHandler, new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect return code:", Nfs3Status.NFS3_OK,
+        response.getStatus());
+
+    // Now perform readlink operations.
+    FileHandle handle2 = response.getObjFileHandle();
+    XDR xdr_req2 = new XDR();
+    handle2.serialize(xdr_req2);
+
+    // Attempt by an unpriviledged user should fail.
+    READLINK3Response response1 = nfsd.readlink(xdr_req2.asReadOnlyWrap(),
+        securityHandlerUnpriviledged,
+        new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect return code:", Nfs3Status.NFS3ERR_ACCES,
+        response1.getStatus());
+
+    // Attempt by a priviledged user should pass.
+    READLINK3Response response2 = nfsd.readlink(xdr_req2.asReadOnlyWrap(),
+        securityHandler, new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect return code:", Nfs3Status.NFS3_OK,
+        response2.getStatus());
+  }
+
+  @Test(timeout = 60000)
+  public void testRead() throws Exception {
+    HdfsFileStatus status = nn.getRpcServer().getFileInfo("/tmp/bar");
+    long dirId = status.getFileId();
+    FileHandle handle = new FileHandle(dirId);
+
+    READ3Request readReq = new READ3Request(handle, 0, 5);
+    XDR xdr_req = new XDR();
+    readReq.serialize(xdr_req);
+
+    // Attempt by an unpriviledged user should fail.
+    /* Hits HDFS-6582. It needs to be fixed first.
+    READ3Response response1 = nfsd.read(xdr_req.asReadOnlyWrap(),
+        securityHandlerUnpriviledged,
+        new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect return code:", Nfs3Status.NFS3ERR_ACCES,
+        response1.getStatus());
+    */
+
+    // Attempt by a priviledged user should pass.
+    READ3Response response2 = nfsd.read(xdr_req.asReadOnlyWrap(),
+        securityHandler, new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect return code:", Nfs3Status.NFS3_OK,
+        response2.getStatus());
+  }
+
+  @Test(timeout = 60000)
+  public void testWrite() throws Exception {
+    HdfsFileStatus status = nn.getRpcServer().getFileInfo("/tmp/bar");
+    long dirId = status.getFileId();
+    FileHandle handle = new FileHandle(dirId);
+
+    byte[] buffer = new byte[10];
+    for (int i = 0; i < 10; i++) {
+      buffer[i] = (byte) i;
+    }
+
+    WRITE3Request writeReq = new WRITE3Request(handle, 0, 10,
+        WriteStableHow.DATA_SYNC, ByteBuffer.wrap(buffer));
+    XDR xdr_req = new XDR();
+    writeReq.serialize(xdr_req);
+
+    // Attempt by an unpriviledged user should fail.
+    WRITE3Response response1 = nfsd.write(xdr_req.asReadOnlyWrap(),
+        null, 1, securityHandlerUnpriviledged,
+        new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect return code:", Nfs3Status.NFS3ERR_ACCES,
+        response1.getStatus());
+
+    // Attempt by a priviledged user should pass.
+    WRITE3Response response2 = nfsd.write(xdr_req.asReadOnlyWrap(),
+        null, 1, securityHandler,
+        new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect response:", null, response2);
+  }
+
+  @Test(timeout = 60000)
+  public void testCreate() throws Exception {
+    HdfsFileStatus status = nn.getRpcServer().getFileInfo(testdir);
+    long dirId = status.getFileId();
+    XDR xdr_req = new XDR();
+    FileHandle handle = new FileHandle(dirId);
+    handle.serialize(xdr_req);
+    xdr_req.writeString("fubar");
+    xdr_req.writeInt(Nfs3Constant.CREATE_UNCHECKED);
+    SetAttr3 symAttr = new SetAttr3();
+    symAttr.serialize(xdr_req);
+
+    // Attempt by an unpriviledged user should fail.
+    CREATE3Response response1 = nfsd.create(xdr_req.asReadOnlyWrap(),
+        securityHandlerUnpriviledged,
+        new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect return code:", Nfs3Status.NFS3ERR_ACCES,
+        response1.getStatus());
+
+    // Attempt by a priviledged user should pass.
+    CREATE3Response response2 = nfsd.create(xdr_req.asReadOnlyWrap(),
+        securityHandler, new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect return code:", Nfs3Status.NFS3_OK,
+        response2.getStatus());
+  }
+
+  @Test(timeout = 60000)
+  public void testMkdir() throws Exception {
+    HdfsFileStatus status = nn.getRpcServer().getFileInfo(testdir);
+    long dirId = status.getFileId();
+    XDR xdr_req = new XDR();
+    FileHandle handle = new FileHandle(dirId);
+    handle.serialize(xdr_req);
+    xdr_req.writeString("fubar");
+    SetAttr3 symAttr = new SetAttr3();
+    symAttr.serialize(xdr_req);
+    xdr_req.writeString("bar");
+
+    // Attempt to remove by an unpriviledged user should fail.
+    SYMLINK3Response response1 = nfsd.symlink(xdr_req.asReadOnlyWrap(),
+        securityHandlerUnpriviledged,
+        new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect return code:", Nfs3Status.NFS3ERR_ACCES,
+        response1.getStatus());
+
+    // Attempt to remove by a priviledged user should pass.
+    SYMLINK3Response response2 = nfsd.symlink(xdr_req.asReadOnlyWrap(),
+        securityHandler, new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect return code:", Nfs3Status.NFS3_OK,
+        response2.getStatus());
+  }
+
+  @Test(timeout = 60000)
+  public void testSymlink() throws Exception {
+    HdfsFileStatus status = nn.getRpcServer().getFileInfo(testdir);
+    long dirId = status.getFileId();
+    XDR xdr_req = new XDR();
+    FileHandle handle = new FileHandle(dirId);
+    handle.serialize(xdr_req);
+    xdr_req.writeString("fubar");
+    SetAttr3 symAttr = new SetAttr3();
+    symAttr.serialize(xdr_req);
+    xdr_req.writeString("bar");
+
+    // Attempt by an unpriviledged user should fail.
+    SYMLINK3Response response1 = nfsd.symlink(xdr_req.asReadOnlyWrap(),
+        securityHandlerUnpriviledged,
+        new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect return code:", Nfs3Status.NFS3ERR_ACCES,
+        response1.getStatus());
+
+    // Attempt by a priviledged user should pass.
+    SYMLINK3Response response2 = nfsd.symlink(xdr_req.asReadOnlyWrap(),
+        securityHandler, new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect return code:", Nfs3Status.NFS3_OK,
+        response2.getStatus());
+  }
+
+  @Test(timeout = 60000)
+  public void testRemove() throws Exception {
+    HdfsFileStatus status = nn.getRpcServer().getFileInfo(testdir);
+    long dirId = status.getFileId();
+    XDR xdr_req = new XDR();
+    FileHandle handle = new FileHandle(dirId);
+    handle.serialize(xdr_req);
+    xdr_req.writeString("bar");
+
+    // Attempt by an unpriviledged user should fail.
+    REMOVE3Response response1 = nfsd.remove(xdr_req.asReadOnlyWrap(),
+        securityHandlerUnpriviledged,
+        new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect return code:", Nfs3Status.NFS3ERR_ACCES,
+        response1.getStatus());
+
+    // Attempt by a priviledged user should pass.
+    REMOVE3Response response2 = nfsd.remove(xdr_req.asReadOnlyWrap(),
+        securityHandler, new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect return code:", Nfs3Status.NFS3_OK,
+        response2.getStatus());
+  }
+
+  @Test(timeout = 60000)
+  public void testRmdir() throws Exception {
+    HdfsFileStatus status = nn.getRpcServer().getFileInfo(testdir);
+    long dirId = status.getFileId();
+    XDR xdr_req = new XDR();
+    FileHandle handle = new FileHandle(dirId);
+    handle.serialize(xdr_req);
+    xdr_req.writeString("foo");
+
+    // Attempt by an unpriviledged user should fail.
+    RMDIR3Response response1 = nfsd.rmdir(xdr_req.asReadOnlyWrap(),
+        securityHandlerUnpriviledged,
+        new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect return code:", Nfs3Status.NFS3ERR_ACCES,
+        response1.getStatus());
+
+    // Attempt by a priviledged user should pass.
+    RMDIR3Response response2 = nfsd.rmdir(xdr_req.asReadOnlyWrap(),
+        securityHandler, new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect return code:", Nfs3Status.NFS3_OK,
+        response2.getStatus());
+  }
+
+  @Test(timeout = 60000)
+  public void testRename() throws Exception {
+    HdfsFileStatus status = nn.getRpcServer().getFileInfo(testdir);
+    long dirId = status.getFileId();
+    XDR xdr_req = new XDR();
+    FileHandle handle = new FileHandle(dirId);
+    handle.serialize(xdr_req);
+    xdr_req.writeString("bar");
+    handle.serialize(xdr_req);
+    xdr_req.writeString("fubar");
+
+    // Attempt by an unpriviledged user should fail.
+    RENAME3Response response1 = nfsd.rename(xdr_req.asReadOnlyWrap(),
+        securityHandlerUnpriviledged,
+        new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect return code:", Nfs3Status.NFS3ERR_ACCES,
+        response1.getStatus());
+
+    // Attempt by a priviledged user should pass.
+    RENAME3Response response2 = nfsd.rename(xdr_req.asReadOnlyWrap(),
+        securityHandler, new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect return code:", Nfs3Status.NFS3_OK,
+        response2.getStatus());
+  }
+
+  @Test(timeout = 60000)
+  public void testReaddir() throws Exception {
+    HdfsFileStatus status = nn.getRpcServer().getFileInfo(testdir);
+    long dirId = status.getFileId();
+    FileHandle handle = new FileHandle(dirId);
+    XDR xdr_req = new XDR();
+    handle.serialize(xdr_req);
+    xdr_req.writeLongAsHyper(0);
+    xdr_req.writeLongAsHyper(0);
+    xdr_req.writeInt(100);
+
+    // Attempt by an unpriviledged user should fail.
+    READDIR3Response response1 = nfsd.readdir(xdr_req.asReadOnlyWrap(),
+        securityHandlerUnpriviledged,
+        new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect return code:", Nfs3Status.NFS3ERR_ACCES,
+        response1.getStatus());
+
+    // Attempt by a priviledged user should pass.
+    READDIR3Response response2 = nfsd.readdir(xdr_req.asReadOnlyWrap(),
+        securityHandler, new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect return code:", Nfs3Status.NFS3_OK,
+        response2.getStatus());
+  }
+
+  @Test(timeout = 60000)
+  public void testReaddirplus() throws Exception {
+    HdfsFileStatus status = nn.getRpcServer().getFileInfo(testdir);
+    long dirId = status.getFileId();
+    FileHandle handle = new FileHandle(dirId);
+    XDR xdr_req = new XDR();
+    handle.serialize(xdr_req);
+    xdr_req.writeLongAsHyper(0);
+    xdr_req.writeLongAsHyper(0);
+    xdr_req.writeInt(3);
+    xdr_req.writeInt(2);
+
+    // Attempt by an unpriviledged user should fail.
+    READDIRPLUS3Response response1 = nfsd.readdirplus(xdr_req.asReadOnlyWrap(),
+        securityHandlerUnpriviledged,
+        new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect return code:", Nfs3Status.NFS3ERR_ACCES,
+        response1.getStatus());
+
+    // Attempt by a priviledged user should pass.
+    READDIRPLUS3Response response2 = nfsd.readdirplus(xdr_req.asReadOnlyWrap(),
+        securityHandler, new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect return code:", Nfs3Status.NFS3_OK,
+        response2.getStatus());
+  }
+
+  @Test(timeout = 60000)
+  public void testFsstat() throws Exception {
+    HdfsFileStatus status = nn.getRpcServer().getFileInfo("/tmp/bar");
+    long dirId = status.getFileId();
+    FileHandle handle = new FileHandle(dirId);
+    XDR xdr_req = new XDR();
+    handle.serialize(xdr_req);
+
+    // Attempt by an unpriviledged user should fail.
+    FSSTAT3Response response1 = nfsd.fsstat(xdr_req.asReadOnlyWrap(),
+        securityHandlerUnpriviledged,
+        new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect return code:", Nfs3Status.NFS3ERR_ACCES,
+        response1.getStatus());
+
+    // Attempt by a priviledged user should pass.
+    FSSTAT3Response response2 = nfsd.fsstat(xdr_req.asReadOnlyWrap(),
+        securityHandler, new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect return code:", Nfs3Status.NFS3_OK,
+        response2.getStatus());
+  }
+
+  @Test(timeout = 60000)
+  public void testFsinfo() throws Exception {
+    HdfsFileStatus status = nn.getRpcServer().getFileInfo("/tmp/bar");
+    long dirId = status.getFileId();
+    FileHandle handle = new FileHandle(dirId);
+    XDR xdr_req = new XDR();
+    handle.serialize(xdr_req);
+
+    // Attempt by an unpriviledged user should fail.
+    FSINFO3Response response1 = nfsd.fsinfo(xdr_req.asReadOnlyWrap(),
+        securityHandlerUnpriviledged,
+        new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect return code:", Nfs3Status.NFS3ERR_ACCES,
+        response1.getStatus());
+
+    // Attempt by a priviledged user should pass.
+    FSINFO3Response response2 = nfsd.fsinfo(xdr_req.asReadOnlyWrap(),
+        securityHandler, new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect return code:", Nfs3Status.NFS3_OK,
+        response2.getStatus());
+  }
+
+  @Test(timeout = 60000)
+  public void testPathconf() throws Exception {
+    HdfsFileStatus status = nn.getRpcServer().getFileInfo("/tmp/bar");
+    long dirId = status.getFileId();
+    FileHandle handle = new FileHandle(dirId);
+    XDR xdr_req = new XDR();
+    handle.serialize(xdr_req);
+
+    // Attempt by an unpriviledged user should fail.
+    PATHCONF3Response response1 = nfsd.pathconf(xdr_req.asReadOnlyWrap(),
+        securityHandlerUnpriviledged,
+        new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect return code:", Nfs3Status.NFS3ERR_ACCES,
+        response1.getStatus());
+
+    // Attempt by a priviledged user should pass.
+    PATHCONF3Response response2 = nfsd.pathconf(xdr_req.asReadOnlyWrap(),
+        securityHandler, new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect return code:", Nfs3Status.NFS3_OK,
+        response2.getStatus());
+  }
+
+  @Test(timeout = 60000)
+  public void testCommit() throws Exception {
+    HdfsFileStatus status = nn.getRpcServer().getFileInfo("/tmp/bar");
+    long dirId = status.getFileId();
+    FileHandle handle = new FileHandle(dirId);
+    XDR xdr_req = new XDR();
+    handle.serialize(xdr_req);
+    xdr_req.writeLongAsHyper(0);
+    xdr_req.writeInt(5);
+
+    Channel ch = Mockito.mock(Channel.class);
+
+    // Attempt by an unpriviledged user should fail.
+    COMMIT3Response response1 = nfsd.commit(xdr_req.asReadOnlyWrap(),
+        ch, 1, securityHandlerUnpriviledged,
+        new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect return code:", Nfs3Status.NFS3ERR_ACCES,
+        response1.getStatus());
+
+    // Attempt by a priviledged user should pass.
+    COMMIT3Response response2 = nfsd.commit(xdr_req.asReadOnlyWrap(),
+        ch, 1, securityHandler,
+        new InetSocketAddress("localhost", 1234));
+    assertEquals("Incorrect COMMIT3Response:", null, response2);
+  }
+
   @Test(timeout=1000)
   public void testIdempotent() {
     Object[][] procedures = {

+ 50 - 3
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -150,9 +150,15 @@ Trunk (Unreleased)
     HDFS-6609. Use DirectorySnapshottableFeature to represent a snapshottable
     directory. (Jing Zhao via wheat9)
 
+    HDFS-6482. Use block ID-based block layout on datanodes (James Thomas via
+    Colin Patrick McCabe)
+
   OPTIMIZATIONS
 
   BUG FIXES
+ 
+    HDFS-6517. Remove hadoop-metrics2.properties from hdfs project (Akira 
+               AJISAKA via aw)
 
     HADOOP-9635 Fix potential Stack Overflow in DomainSocket.c (V. Karthik Kumar
                 via cmccabe)
@@ -357,6 +363,30 @@ Release 2.6.0 - UNRELEASED
 
     HDFS-6685. Balancer should preserve storage type of replicas.  (szetszwo)
 
+    HDFS-6798. Add test case for incorrect data node condition during
+    balancing. (Benoy Antony via Arpit Agarwal)
+
+    HDFS-6796. Improve the argument check during balancer command line parsing.
+    (Benoy Antony via szetszwo)
+
+    HDFS-6794. Update BlockManager methods to use DatanodeStorageInfo
+    where possible (Arpit Agarwal)
+
+    HDFS-6802. Some tests in TestDFSClientFailover are missing @Test
+    annotation. (Akira Ajisaka via wang)
+
+    HDFS-6788. Improve synchronization in BPOfferService with read write lock.
+    (Yongjun Zhang via wang)
+
+    HDFS-6787. Remove duplicate code in FSDirectory#unprotectedConcat. (Yi Liu via umamahesh)
+
+    HDFS-6809. Move Balancer's inner classes MovedBlocks and Matcher as to
+    standalone classes and separates KeyManager from NameNodeConnector.
+    (szetszwo)
+
+    HDFS-6812. Remove addBlock and replaceBlock from DatanodeDescriptor.
+    (szetszwo)
+
   OPTIMIZATIONS
 
     HDFS-6690. Deduplicate xattr names in memory. (wang)
@@ -429,6 +459,26 @@ Release 2.6.0 - UNRELEASED
     HDFS-6797. DataNode logs wrong layoutversion during upgrade. (Benoy Antony
     via Arpit Agarwal)
 
+    HDFS-6810. StorageReport array is initialized with wrong size in
+    DatanodeDescriptor#getStorageReports. (szetszwo via Arpit Agarwal)
+
+    HDFS-5723. Append failed FINALIZED replica should not be accepted as valid
+    when that block is underconstruction (vinayakumarb)
+
+    HDFS-5185. DN fails to startup if one of the data dir is full. (vinayakumarb)
+
+    HDFS-6451. NFS should not return NFS3ERR_IO for AccessControlException 
+    (Abhiraj Butala via brandonli)
+
+    HDFS-6717. JIRA HDFS-5804 breaks default nfs-gateway behavior for unsecured config
+    (brandonli)
+
+    HDFS-6790. DFSUtil Should Use configuration.getPassword for SSL passwords
+    (Larry McCay via brandonli)
+
+    HDFS-6791. A block could remain under replicated if all of its replicas are on
+    decommissioned nodes. (Ming Ma via jing9)
+
 Release 2.5.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -981,9 +1031,6 @@ Release 2.5.0 - UNRELEASED
     HDFS-6723. New NN webUI no longer displays decommissioned state for dead node.
     (Ming Ma via wheat9)
 
-    HDFS-6717. JIRA HDFS-5804 breaks default nfs-gateway behavior for unsecured config
-    (brandonli)
-
     HDFS-6768. Fix a few unit tests that use hard-coded port numbers. (Arpit
     Agarwal)
 

+ 0 - 44
hadoop-hdfs-project/hadoop-hdfs/src/main/conf/hadoop-metrics2.properties

@@ -1,44 +0,0 @@
-#
-#   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.
-#
-
-# syntax: [prefix].[source|sink].[instance].[options]
-# See javadoc of package-info.java for org.apache.hadoop.metrics2 for details
-
-*.sink.file.class=org.apache.hadoop.metrics2.sink.FileSink
-# default sampling period, in seconds
-*.period=10
-
-# The namenode-metrics.out will contain metrics from all context
-#namenode.sink.file.filename=namenode-metrics.out
-# Specifying a special sampling period for namenode:
-#namenode.sink.*.period=8
-
-#datanode.sink.file.filename=datanode-metrics.out
-
-# the following example split metrics of different
-# context to different sinks (in this case files)
-#jobtracker.sink.file_jvm.context=jvm
-#jobtracker.sink.file_jvm.filename=jobtracker-jvm-metrics.out
-#jobtracker.sink.file_mapred.context=mapred
-#jobtracker.sink.file_mapred.filename=jobtracker-mapred-metrics.out
-
-#tasktracker.sink.file.filename=tasktracker-metrics.out
-
-#maptask.sink.file.filename=maptask-metrics.out
-
-#reducetask.sink.file.filename=reducetask-metrics.out
-

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

@@ -201,6 +201,9 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_ADMIN = "dfs.cluster.administrators";
   public static final String  DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_KEY = "dfs.https.server.keystore.resource";
   public static final String  DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_DEFAULT = "ssl-server.xml";
+  public static final String  DFS_SERVER_HTTPS_KEYPASSWORD_KEY = "ssl.server.keystore.keypassword";
+  public static final String  DFS_SERVER_HTTPS_KEYSTORE_PASSWORD_KEY = "ssl.server.keystore.password";
+  public static final String  DFS_SERVER_HTTPS_TRUSTSTORE_PASSWORD_KEY = "ssl.server.truststore.password";
   public static final String  DFS_NAMENODE_NAME_DIR_RESTORE_KEY = "dfs.namenode.name.dir.restore";
   public static final boolean DFS_NAMENODE_NAME_DIR_RESTORE_DEFAULT = false;
   public static final String  DFS_NAMENODE_SUPPORT_ALLOW_FORMAT_KEY = "dfs.namenode.support.allow.format";
@@ -381,8 +384,6 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_DATANODE_HTTP_ADDRESS_DEFAULT = "0.0.0.0:" + DFS_DATANODE_HTTP_DEFAULT_PORT;
   public static final String  DFS_DATANODE_MAX_RECEIVER_THREADS_KEY = "dfs.datanode.max.transfer.threads";
   public static final int     DFS_DATANODE_MAX_RECEIVER_THREADS_DEFAULT = 4096;
-  public static final String  DFS_DATANODE_NUMBLOCKS_KEY = "dfs.datanode.numblocks";
-  public static final int     DFS_DATANODE_NUMBLOCKS_DEFAULT = 64;
   public static final String  DFS_DATANODE_SCAN_PERIOD_HOURS_KEY = "dfs.datanode.scan.period.hours";
   public static final int     DFS_DATANODE_SCAN_PERIOD_HOURS_DEFAULT = 0;
   public static final String  DFS_DATANODE_TRANSFERTO_ALLOWED_KEY = "dfs.datanode.transferTo.allowed";
@@ -675,4 +676,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
    public static final String DFS_DATANODE_SLOW_IO_WARNING_THRESHOLD_KEY =
      "dfs.datanode.slow.io.warning.threshold.ms";
    public static final long DFS_DATANODE_SLOW_IO_WARNING_THRESHOLD_DEFAULT = 300;
+
+   public static final String DFS_DATANODE_BLOCK_ID_LAYOUT_UPGRADE_THREADS_KEY =
+       "dfs.datanode.block.id.layout.upgrade.threads";
+   public static final int DFS_DATANODE_BLOCK_ID_LAYOUT_UPGRADE_THREADS = 12;
 }

+ 28 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java

@@ -33,6 +33,9 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_A
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMESERVICES;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMESERVICE_ID;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_KEYPASSWORD_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_KEYSTORE_PASSWORD_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_TRUSTSTORE_PASSWORD_KEY;
 
 import java.io.IOException;
 import java.io.PrintStream;
@@ -1531,15 +1534,37 @@ public class DFSUtil {
         .needsClientAuth(
             sslConf.getBoolean(DFS_CLIENT_HTTPS_NEED_AUTH_KEY,
                 DFS_CLIENT_HTTPS_NEED_AUTH_DEFAULT))
-        .keyPassword(sslConf.get("ssl.server.keystore.keypassword"))
+        .keyPassword(getPassword(sslConf, DFS_SERVER_HTTPS_KEYPASSWORD_KEY))
         .keyStore(sslConf.get("ssl.server.keystore.location"),
-            sslConf.get("ssl.server.keystore.password"),
+            getPassword(sslConf, DFS_SERVER_HTTPS_KEYSTORE_PASSWORD_KEY),
             sslConf.get("ssl.server.keystore.type", "jks"))
         .trustStore(sslConf.get("ssl.server.truststore.location"),
-            sslConf.get("ssl.server.truststore.password"),
+            getPassword(sslConf, DFS_SERVER_HTTPS_TRUSTSTORE_PASSWORD_KEY),
             sslConf.get("ssl.server.truststore.type", "jks"));
   }
 
+  /**
+   * Leverages the Configuration.getPassword method to attempt to get
+   * passwords from the CredentialProvider API before falling back to
+   * clear text in config - if falling back is allowed.
+   * @param conf Configuration instance
+   * @param alias name of the credential to retreive
+   * @return String credential value or null
+   */
+  static String getPassword(Configuration conf, String alias) {
+    String password = null;
+    try {
+      char[] passchars = conf.getPassword(alias);
+      if (passchars != null) {
+        password = new String(passchars);
+      }
+    }
+    catch (IOException ioe) {
+      password = null;
+    }
+    return password;
+  }
+
   /**
    * Converts a Date into an ISO-8601 formatted datetime string.
    */

+ 11 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/Block.java

@@ -50,6 +50,9 @@ public class Block implements Writable, Comparable<Block> {
   public static final Pattern metaFilePattern = Pattern
       .compile(BLOCK_FILE_PREFIX + "(-??\\d++)_(\\d++)\\" + METADATA_EXTENSION
           + "$");
+  public static final Pattern metaOrBlockFilePattern = Pattern
+      .compile(BLOCK_FILE_PREFIX + "(-??\\d++)(_(\\d++)\\" + METADATA_EXTENSION
+          + ")?$");
 
   public static boolean isBlockFilename(File f) {
     String name = f.getName();
@@ -65,6 +68,11 @@ public class Block implements Writable, Comparable<Block> {
     return metaFilePattern.matcher(name).matches();
   }
 
+  public static File metaToBlockFile(File metaFile) {
+    return new File(metaFile.getParent(), metaFile.getName().substring(
+        0, metaFile.getName().lastIndexOf('_')));
+  }
+
   /**
    * Get generation stamp from the name of the metafile name
    */
@@ -75,10 +83,10 @@ public class Block implements Writable, Comparable<Block> {
   }
 
   /**
-   * Get the blockId from the name of the metafile name
+   * Get the blockId from the name of the meta or block file
    */
-  public static long getBlockId(String metaFile) {
-    Matcher m = metaFilePattern.matcher(metaFile);
+  public static long getBlockId(String metaOrBlockFile) {
+    Matcher m = metaOrBlockFilePattern.matcher(metaOrBlockFile);
     return m.matches() ? Long.parseLong(m.group(1)) : 0;
   }
 

+ 61 - 160
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java

@@ -58,6 +58,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
@@ -85,7 +86,6 @@ import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetworkTopology;
-import org.apache.hadoop.net.Node;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.HostsFileReader;
 import org.apache.hadoop.util.StringUtils;
@@ -195,10 +195,12 @@ import com.google.common.base.Preconditions;
 @InterfaceAudience.Private
 public class Balancer {
   static final Log LOG = LogFactory.getLog(Balancer.class);
-  final private static long GB = 1L << 30; //1GB
-  final private static long MAX_SIZE_TO_MOVE = 10*GB;
-  final private static long MAX_BLOCKS_SIZE_TO_FETCH = 2*GB;
-  private static long WIN_WIDTH = 5400*1000L; // 1.5 hour
+
+  private static final Path BALANCER_ID_PATH = new Path("/system/balancer.id");
+
+  private static final long GB = 1L << 30; //1GB
+  private static final long MAX_SIZE_TO_MOVE = 10*GB;
+  private static final long MAX_BLOCKS_SIZE_TO_FETCH = 2*GB;
 
   /** The maximum number of concurrent blocks moves for 
    * balancing purpose at a datanode
@@ -219,6 +221,8 @@ public class Balancer {
       + "\tIncludes only the specified datanodes.";
   
   private final NameNodeConnector nnc;
+  private final KeyManager keyManager;
+
   private final BalancingPolicy policy;
   private final SaslDataTransferClient saslClient;
   private final double threshold;
@@ -241,7 +245,8 @@ public class Balancer {
   
   private final Map<Block, BalancerBlock> globalBlockList
                  = new HashMap<Block, BalancerBlock>();
-  private final MovedBlocks movedBlocks = new MovedBlocks();
+  private final MovedBlocks<BalancerDatanode.StorageGroup> movedBlocks;
+
   /** Map (datanodeUuid,storageType -> StorageGroup) */
   private final StorageGroupMap storageGroupMap = new StorageGroupMap();
   
@@ -326,7 +331,7 @@ public class Balancer {
           if (isGoodBlockCandidate(source, target, block)) {
             this.block = block;
             if ( chooseProxySource() ) {
-              movedBlocks.add(block);
+              movedBlocks.put(block);
               if (LOG.isDebugEnabled()) {
                 LOG.debug("Decided to move " + this);
               }
@@ -399,10 +404,10 @@ public class Balancer {
         
         OutputStream unbufOut = sock.getOutputStream();
         InputStream unbufIn = sock.getInputStream();
-        ExtendedBlock eb = new ExtendedBlock(nnc.blockpoolID, block.getBlock());
-        Token<BlockTokenIdentifier> accessToken = nnc.getAccessToken(eb);
+        ExtendedBlock eb = new ExtendedBlock(nnc.getBlockpoolID(), block.getBlock());
+        Token<BlockTokenIdentifier> accessToken = keyManager.getAccessToken(eb);
         IOStreamPair saslStreams = saslClient.socketSend(sock, unbufOut,
-          unbufIn, nnc, accessToken, target.getDatanode());
+          unbufIn, keyManager, accessToken, target.getDatanode());
         unbufOut = saslStreams.out;
         unbufIn = saslStreams.in;
         out = new DataOutputStream(new BufferedOutputStream(unbufOut,
@@ -483,47 +488,9 @@ public class Balancer {
   }
   
   /* A class for keeping track of blocks in the Balancer */
-  static private class BalancerBlock {
-    private final Block block; // the block
-    /** The locations of the replicas of the block. */
-    private final List<BalancerDatanode.StorageGroup> locations
-        = new ArrayList<BalancerDatanode.StorageGroup>(3);
-    
-    /* Constructor */
-    private BalancerBlock(Block block) {
-      this.block = block;
-    }
-    
-    /* clean block locations */
-    private synchronized void clearLocations() {
-      locations.clear();
-    }
-    
-    /* add a location */
-    private synchronized void addLocation(BalancerDatanode.StorageGroup g) {
-      if (!locations.contains(g)) {
-        locations.add(g);
-      }
-    }
-    
-    /** @return if the block is located on the given storage group. */
-    private synchronized boolean isLocatedOn(BalancerDatanode.StorageGroup g) {
-      return locations.contains(g);
-    }
-    
-    /* Return its locations */
-    private synchronized List<BalancerDatanode.StorageGroup> getLocations() {
-      return locations;
-    }
-    
-    /* Return the block */
-    private Block getBlock() {
-      return block;
-    }
-    
-    /* Return the length of the block */
-    private long getNumBytes() {
-      return block.getNumBytes();
+  static class BalancerBlock extends MovedBlocks.Locations<BalancerDatanode.StorageGroup> {
+    BalancerBlock(Block block) {
+      super(block);
     }
   }
   
@@ -735,7 +702,7 @@ public class Balancer {
      */
     private long getBlockList() throws IOException {
       final long size = Math.min(MAX_BLOCKS_SIZE_TO_FETCH, blocksToReceive);
-      final BlockWithLocations[] newBlocks = nnc.namenode.getBlocks(
+      final BlockWithLocations[] newBlocks = nnc.getNamenode().getBlocks(
           getDatanode(), size).getBlocks();
 
       long bytesReceived = 0;
@@ -819,7 +786,7 @@ public class Balancer {
     private void filterMovedBlocks() {
       for (Iterator<BalancerBlock> blocks=getBlockIterator();
             blocks.hasNext();) {
-        if (movedBlocks.contains(blocks.next())) {
+        if (movedBlocks.contains(blocks.next().getBlock())) {
           blocks.remove();
         }
       }
@@ -925,6 +892,13 @@ public class Balancer {
     this.nodesToBeExcluded = p.nodesToBeExcluded;
     this.nodesToBeIncluded = p.nodesToBeIncluded;
     this.nnc = theblockpool;
+    this.keyManager = nnc.getKeyManager();
+    
+    final long movedWinWidth = conf.getLong(
+        DFSConfigKeys.DFS_BALANCER_MOVEDWINWIDTH_KEY, 
+        DFSConfigKeys.DFS_BALANCER_MOVEDWINWIDTH_DEFAULT);
+    movedBlocks = new MovedBlocks<BalancerDatanode.StorageGroup>(movedWinWidth);
+
     cluster = NetworkTopology.getInstance(conf);
 
     this.moverExecutor = Executors.newFixedThreadPool(
@@ -1094,36 +1068,6 @@ public class Balancer {
     LOG.info(items.size() + " " + name + ": " + items);
   }
 
-  /** A matcher interface for matching nodes. */
-  private interface Matcher {
-    /** Given the cluster topology, does the left node match the right node? */
-    boolean match(NetworkTopology cluster, Node left,  Node right);
-  }
-
-  /** Match datanodes in the same node group. */
-  static final Matcher SAME_NODE_GROUP = new Matcher() {
-    @Override
-    public boolean match(NetworkTopology cluster, Node left, Node right) {
-      return cluster.isOnSameNodeGroup(left, right);
-    }
-  };
-
-  /** Match datanodes in the same rack. */
-  static final Matcher SAME_RACK = new Matcher() {
-    @Override
-    public boolean match(NetworkTopology cluster, Node left, Node right) {
-      return cluster.isOnSameRack(left, right);
-    }
-  };
-
-  /** Match any datanode with any other datanode. */
-  static final Matcher ANY_OTHER = new Matcher() {
-    @Override
-    public boolean match(NetworkTopology cluster, Node left, Node right) {
-      return left != right;
-    }
-  };
-
   /**
    * Decide all <source, target> pairs and
    * the number of bytes to move from a source to a target
@@ -1134,13 +1078,13 @@ public class Balancer {
   private long chooseStorageGroups() {
     // First, match nodes on the same node group if cluster is node group aware
     if (cluster.isNodeGroupAware()) {
-      chooseStorageGroups(SAME_NODE_GROUP);
+      chooseStorageGroups(Matcher.SAME_NODE_GROUP);
     }
     
     // Then, match nodes on the same rack
-    chooseStorageGroups(SAME_RACK);
+    chooseStorageGroups(Matcher.SAME_RACK);
     // At last, match all remaining nodes
-    chooseStorageGroups(ANY_OTHER);
+    chooseStorageGroups(Matcher.ANY_OTHER);
     
     Preconditions.checkState(storageGroupMap.size() >= sources.size() + targets.size(),
         "Mismatched number of datanodes (" + storageGroupMap.size() + " < "
@@ -1307,56 +1251,6 @@ public class Balancer {
     } while (shouldWait);
   }
 
-  /** This window makes sure to keep blocks that have been moved within 1.5 hour.
-   * Old window has blocks that are older;
-   * Current window has blocks that are more recent;
-   * Cleanup method triggers the check if blocks in the old window are
-   * more than 1.5 hour old. If yes, purge the old window and then
-   * move blocks in current window to old window.
-   */ 
-  private static class MovedBlocks {
-    private long lastCleanupTime = Time.now();
-    final private static int CUR_WIN = 0;
-    final private static int OLD_WIN = 1;
-    final private static int NUM_WINS = 2;
-    final private List<HashMap<Block, BalancerBlock>> movedBlocks = 
-      new ArrayList<HashMap<Block, BalancerBlock>>(NUM_WINS);
-    
-    /* initialize the moved blocks collection */
-    private MovedBlocks() {
-      movedBlocks.add(new HashMap<Block,BalancerBlock>());
-      movedBlocks.add(new HashMap<Block,BalancerBlock>());
-    }
-
-    /* add a block thus marking a block to be moved */
-    synchronized private void add(BalancerBlock block) {
-      movedBlocks.get(CUR_WIN).put(block.getBlock(), block);
-    }
-
-    /* check if a block is marked as moved */
-    synchronized private boolean contains(BalancerBlock block) {
-      return contains(block.getBlock());
-    }
-
-    /* check if a block is marked as moved */
-    synchronized private boolean contains(Block block) {
-      return movedBlocks.get(CUR_WIN).containsKey(block) ||
-        movedBlocks.get(OLD_WIN).containsKey(block);
-    }
-
-    /* remove old blocks */
-    synchronized private void cleanup() {
-      long curTime = Time.now();
-      // check if old win is older than winWidth
-      if (lastCleanupTime + WIN_WIDTH <= curTime) {
-        // purge the old window
-        movedBlocks.set(OLD_WIN, movedBlocks.get(CUR_WIN));
-        movedBlocks.set(CUR_WIN, new HashMap<Block, BalancerBlock>());
-        lastCleanupTime = curTime;
-      }
-    }
-  }
-
   /* Decide if it is OK to move the given block from source to target
    * A block is a good candidate if
    * 1. the block is not in the process of being moved/has not been moved;
@@ -1369,7 +1263,7 @@ public class Balancer {
       return false;
     }
     // check if the block is moved or not
-    if (movedBlocks.contains(block)) {
+    if (movedBlocks.contains(block.getBlock())) {
       return false;
     }
     if (block.isLocatedOn(target)) {
@@ -1387,7 +1281,7 @@ public class Balancer {
     } else {
       boolean notOnSameRack = true;
       synchronized (block) {
-        for (BalancerDatanode.StorageGroup loc : block.locations) {
+        for (BalancerDatanode.StorageGroup loc : block.getLocations()) {
           if (cluster.isOnSameRack(loc.getDatanode(), target.getDatanode())) {
             notOnSameRack = false;
             break;
@@ -1399,7 +1293,7 @@ public class Balancer {
         goodBlock = true;
       } else {
         // good if source is on the same rack as on of the replicas
-        for (BalancerDatanode.StorageGroup loc : block.locations) {
+        for (BalancerDatanode.StorageGroup loc : block.getLocations()) {
           if (loc != source && 
               cluster.isOnSameRack(loc.getDatanode(), source.getDatanode())) {
             goodBlock = true;
@@ -1425,7 +1319,7 @@ public class Balancer {
   private boolean isOnSameNodeGroupWithReplicas(BalancerDatanode.StorageGroup target,
       BalancerBlock block, Source source) {
     final DatanodeInfo targetDn = target.getDatanode();
-    for (BalancerDatanode.StorageGroup loc : block.locations) {
+    for (BalancerDatanode.StorageGroup loc : block.getLocations()) {
       if (loc != source && 
           cluster.isOnSameNodeGroup(loc.getDatanode(), targetDn)) {
         return true;
@@ -1489,7 +1383,7 @@ public class Balancer {
        * decide the number of bytes need to be moved
        */
       final long bytesLeftToMove = init(
-          nnc.client.getDatanodeStorageReport(DatanodeReportType.LIVE));
+          nnc.getClient().getDatanodeStorageReport(DatanodeReportType.LIVE));
       if (bytesLeftToMove == 0) {
         System.out.println("The cluster is balanced. Exiting...");
         return ReturnStatus.SUCCESS;
@@ -1558,8 +1452,8 @@ public class Balancer {
     final long sleeptime = 2000*conf.getLong(
         DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
         DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT);
-    LOG.info("namenodes = " + namenodes);
-    LOG.info("p         = " + p);
+    LOG.info("namenodes  = " + namenodes);
+    LOG.info("parameters = " + p);
     
     final Formatter formatter = new Formatter(System.out);
     System.out.println("Time Stamp               Iteration#  Bytes Already Moved  Bytes Left To Move  Bytes Being Moved");
@@ -1568,7 +1462,10 @@ public class Balancer {
         = new ArrayList<NameNodeConnector>(namenodes.size());
     try {
       for (URI uri : namenodes) {
-        connectors.add(new NameNodeConnector(uri, conf));
+        final NameNodeConnector nnc = new NameNodeConnector(
+            Balancer.class.getSimpleName(), uri, BALANCER_ID_PATH, conf);
+        nnc.getKeyManager().startBlockKeyUpdater();
+        connectors.add(nnc);
       }
     
       boolean done = false;
@@ -1730,9 +1627,6 @@ public class Balancer {
     public int run(String[] args) {
       final long startTime = Time.now();
       final Configuration conf = getConf();
-      WIN_WIDTH = conf.getLong(
-          DFSConfigKeys.DFS_BALANCER_MOVEDWINWIDTH_KEY, 
-          DFSConfigKeys.DFS_BALANCER_MOVEDWINWIDTH_DEFAULT);
 
       try {
         checkReplicationPolicyCompatibility(conf);
@@ -1761,9 +1655,9 @@ public class Balancer {
       if (args != null) {
         try {
           for(int i = 0; i < args.length; i++) {
-            checkArgument(args.length >= 2, "args = " + Arrays.toString(args));
             if ("-threshold".equalsIgnoreCase(args[i])) {
-              i++;
+              checkArgument(++i < args.length,
+                "Threshold value is missing: args = " + Arrays.toString(args));
               try {
                 threshold = Double.parseDouble(args[i]);
                 if (threshold < 1 || threshold > 100) {
@@ -1778,7 +1672,8 @@ public class Balancer {
                 throw e;
               }
             } else if ("-policy".equalsIgnoreCase(args[i])) {
-              i++;
+              checkArgument(++i < args.length,
+                "Policy value is missing: args = " + Arrays.toString(args));
               try {
                 policy = BalancingPolicy.parse(args[i]);
               } catch(IllegalArgumentException e) {
@@ -1786,16 +1681,26 @@ public class Balancer {
                 throw e;
               }
             } else if ("-exclude".equalsIgnoreCase(args[i])) {
-              i++;
+              checkArgument(++i < args.length,
+                  "List of nodes to exclude | -f <filename> is missing: args = "
+                  + Arrays.toString(args));
               if ("-f".equalsIgnoreCase(args[i])) {
-                nodesTobeExcluded = Util.getHostListFromFile(args[++i]);
+                checkArgument(++i < args.length,
+                    "File containing nodes to exclude is not specified: args = "
+                    + Arrays.toString(args));
+                nodesTobeExcluded = Util.getHostListFromFile(args[i]);
               } else {
                 nodesTobeExcluded = Util.parseHostList(args[i]);
               }
             } else if ("-include".equalsIgnoreCase(args[i])) {
-              i++;
+              checkArgument(++i < args.length,
+                "List of nodes to include | -f <filename> is missing: args = "
+                + Arrays.toString(args));
               if ("-f".equalsIgnoreCase(args[i])) {
-                nodesTobeIncluded = Util.getHostListFromFile(args[++i]);
+                checkArgument(++i < args.length,
+                    "File containing nodes to include is not specified: args = "
+                    + Arrays.toString(args));
+                nodesTobeIncluded = Util.getHostListFromFile(args[i]);
                } else {
                 nodesTobeIncluded = Util.parseHostList(args[i]);
               }
@@ -1804,12 +1709,8 @@ public class Balancer {
                   + Arrays.toString(args));
             }
           }
-          if (!nodesTobeExcluded.isEmpty() && !nodesTobeIncluded.isEmpty()) {
-            System.err.println(
-                "-exclude and -include options cannot be specified together.");
-            throw new IllegalArgumentException(
-                "-exclude and -include options cannot be specified together.");
-          }
+          checkArgument(nodesTobeExcluded.isEmpty() || nodesTobeIncluded.isEmpty(),
+              "-exclude and -include options cannot be specified together.");
         } catch(RuntimeException e) {
           printUsage(System.err);
           throw e;

+ 173 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/KeyManager.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.hdfs.server.balancer;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.EnumSet;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactory;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
+import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
+import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.Daemon;
+import org.apache.hadoop.util.StringUtils;
+
+/**
+ * The class provides utilities for key and token management.
+ */
+@InterfaceAudience.Private
+public class KeyManager implements Closeable, DataEncryptionKeyFactory {
+  private static final Log LOG = LogFactory.getLog(KeyManager.class);
+
+  private final NamenodeProtocol namenode;
+
+  private final boolean isBlockTokenEnabled;
+  private final boolean encryptDataTransfer;
+  private boolean shouldRun;
+
+  private final BlockTokenSecretManager blockTokenSecretManager;
+  private final BlockKeyUpdater blockKeyUpdater;
+  private DataEncryptionKey encryptionKey;
+
+  public KeyManager(String blockpoolID, NamenodeProtocol namenode,
+      boolean encryptDataTransfer, Configuration conf) throws IOException {
+    this.namenode = namenode;
+    this.encryptDataTransfer = encryptDataTransfer;
+
+    final ExportedBlockKeys keys = namenode.getBlockKeys();
+    this.isBlockTokenEnabled = keys.isBlockTokenEnabled();
+    if (isBlockTokenEnabled) {
+      long updateInterval = keys.getKeyUpdateInterval();
+      long tokenLifetime = keys.getTokenLifetime();
+      LOG.info("Block token params received from NN: update interval="
+          + StringUtils.formatTime(updateInterval)
+          + ", token lifetime=" + StringUtils.formatTime(tokenLifetime));
+      String encryptionAlgorithm = conf.get(
+          DFSConfigKeys.DFS_DATA_ENCRYPTION_ALGORITHM_KEY);
+      this.blockTokenSecretManager = new BlockTokenSecretManager(
+          updateInterval, tokenLifetime, blockpoolID, encryptionAlgorithm);
+      this.blockTokenSecretManager.addKeys(keys);
+
+      // sync block keys with NN more frequently than NN updates its block keys
+      this.blockKeyUpdater = new BlockKeyUpdater(updateInterval / 4);
+      this.shouldRun = true;
+    } else {
+      this.blockTokenSecretManager = null;
+      this.blockKeyUpdater = null;
+    }
+  }
+  
+  public void startBlockKeyUpdater() {
+    if (blockKeyUpdater != null) {
+      blockKeyUpdater.daemon.start();
+    }
+  }
+
+  /** Get an access token for a block. */
+  public Token<BlockTokenIdentifier> getAccessToken(ExtendedBlock eb
+      ) throws IOException {
+    if (!isBlockTokenEnabled) {
+      return BlockTokenSecretManager.DUMMY_TOKEN;
+    } else {
+      if (!shouldRun) {
+        throw new IOException(
+            "Cannot get access token since BlockKeyUpdater is not running");
+      }
+      return blockTokenSecretManager.generateToken(null, eb,
+          EnumSet.of(AccessMode.REPLACE, AccessMode.COPY));
+    }
+  }
+
+  @Override
+  public DataEncryptionKey newDataEncryptionKey() {
+    if (encryptDataTransfer) {
+      synchronized (this) {
+        if (encryptionKey == null) {
+          encryptionKey = blockTokenSecretManager.generateDataEncryptionKey();
+        }
+        return encryptionKey;
+      }
+    } else {
+      return null;
+    }
+  }
+
+  @Override
+  public void close() {
+    shouldRun = false;
+    try {
+      if (blockKeyUpdater != null) {
+        blockKeyUpdater.daemon.interrupt();
+      }
+    } catch(Exception e) {
+      LOG.warn("Exception shutting down access key updater thread", e);
+    }
+  }
+
+  /**
+   * Periodically updates access keys.
+   */
+  class BlockKeyUpdater implements Runnable, Closeable {
+    private final Daemon daemon = new Daemon(this);
+    private final long sleepInterval;
+
+    BlockKeyUpdater(final long sleepInterval) {
+      this.sleepInterval = sleepInterval;
+      LOG.info("Update block keys every " + StringUtils.formatTime(sleepInterval));
+    }
+
+    @Override
+    public void run() {
+      try {
+        while (shouldRun) {
+          try {
+            blockTokenSecretManager.addKeys(namenode.getBlockKeys());
+          } catch (IOException e) {
+            LOG.error("Failed to set keys", e);
+          }
+          Thread.sleep(sleepInterval);
+        }
+      } catch (InterruptedException e) {
+        LOG.debug("InterruptedException in block key updater thread", e);
+      } catch (Throwable e) {
+        LOG.error("Exception in block key updater thread", e);
+        shouldRun = false;
+      }
+    }
+
+    @Override
+    public void close() throws IOException {
+      try {
+        daemon.interrupt();
+      } catch(Exception e) {
+        LOG.warn("Exception shutting down key updater thread", e);
+      }
+    }
+  }
+}

+ 51 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Matcher.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.hdfs.server.balancer;
+
+import org.apache.hadoop.net.NetworkTopology;
+import org.apache.hadoop.net.Node;
+
+/** A matcher interface for matching nodes. */
+public interface Matcher {
+  /** Given the cluster topology, does the left node match the right node? */
+  public boolean match(NetworkTopology cluster, Node left,  Node right);
+
+  /** Match datanodes in the same node group. */
+  public static final Matcher SAME_NODE_GROUP = new Matcher() {
+    @Override
+    public boolean match(NetworkTopology cluster, Node left, Node right) {
+      return cluster.isOnSameNodeGroup(left, right);
+    }
+  };
+
+  /** Match datanodes in the same rack. */
+  public static final Matcher SAME_RACK = new Matcher() {
+    @Override
+    public boolean match(NetworkTopology cluster, Node left, Node right) {
+      return cluster.isOnSameRack(left, right);
+    }
+  };
+
+  /** Match any datanode with any other datanode. */
+  public static final Matcher ANY_OTHER = new Matcher() {
+    @Override
+    public boolean match(NetworkTopology cluster, Node left, Node right) {
+      return left != right;
+    }
+  };
+}

+ 124 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/MovedBlocks.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.hdfs.server.balancer;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.util.Time;
+
+/**
+ * This window makes sure to keep blocks that have been moved within a fixed
+ * time interval (default is 1.5 hour). Old window has blocks that are older;
+ * Current window has blocks that are more recent; Cleanup method triggers the
+ * check if blocks in the old window are more than the fixed time interval. If
+ * yes, purge the old window and then move blocks in current window to old
+ * window.
+ * 
+ * @param <L> Location type
+ */
+public class MovedBlocks<L> {
+  /** A class for keeping track of a block and its locations */
+  public static class Locations<L> {
+    private final Block block; // the block
+    /** The locations of the replicas of the block. */
+    private final List<L> locations = new ArrayList<L>(3);
+    
+    public Locations(Block block) {
+      this.block = block;
+    }
+    
+    /** clean block locations */
+    public synchronized void clearLocations() {
+      locations.clear();
+    }
+    
+    /** add a location */
+    public synchronized void addLocation(L loc) {
+      if (!locations.contains(loc)) {
+        locations.add(loc);
+      }
+    }
+    
+    /** @return if the block is located on the given location. */
+    public synchronized boolean isLocatedOn(L loc) {
+      return locations.contains(loc);
+    }
+    
+    /** @return its locations */
+    public synchronized List<L> getLocations() {
+      return locations;
+    }
+    
+    /* @return the block */
+    public Block getBlock() {
+      return block;
+    }
+    
+    /* Return the length of the block */
+    public long getNumBytes() {
+      return block.getNumBytes();
+    }
+  }
+
+  private static final int CUR_WIN = 0;
+  private static final int OLD_WIN = 1;
+  private static final int NUM_WINS = 2;
+
+  private final long winTimeInterval;
+  private long lastCleanupTime = Time.monotonicNow();
+  private final List<Map<Block, Locations<L>>> movedBlocks
+      = new ArrayList<Map<Block, Locations<L>>>(NUM_WINS);
+  
+  /** initialize the moved blocks collection */
+  public MovedBlocks(long winTimeInterval) {
+    this.winTimeInterval = winTimeInterval;
+    movedBlocks.add(newMap());
+    movedBlocks.add(newMap());
+  }
+
+  private Map<Block, Locations<L>> newMap() {
+    return new HashMap<Block, Locations<L>>();
+  }
+
+  /** add a block thus marking a block to be moved */
+  public synchronized void put(Locations<L> block) {
+    movedBlocks.get(CUR_WIN).put(block.getBlock(), block);
+  }
+
+  /** @return if a block is marked as moved */
+  public synchronized boolean contains(Block block) {
+    return movedBlocks.get(CUR_WIN).containsKey(block) ||
+      movedBlocks.get(OLD_WIN).containsKey(block);
+  }
+
+  /** remove old blocks */
+  public synchronized void cleanup() {
+    long curTime = Time.monotonicNow();
+    // check if old win is older than winWidth
+    if (lastCleanupTime + winTimeInterval <= curTime) {
+      // purge the old window
+      movedBlocks.set(OLD_WIN, movedBlocks.get(CUR_WIN));
+      movedBlocks.set(CUR_WIN, newMap());
+      lastCleanupTime = curTime;
+    }
+  }
+}

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

@@ -17,113 +17,96 @@
  */
 package org.apache.hadoop.hdfs.server.balancer;
 
+import java.io.Closeable;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.net.InetAddress;
 import java.net.URI;
-import java.util.EnumSet;
 
 import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.NameNodeProxies;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactory;
-import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
-import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.util.Daemon;
 
 /**
- * The class provides utilities for {@link Balancer} to access a NameNode
+ * The class provides utilities for accessing a NameNode.
  */
 @InterfaceAudience.Private
-class NameNodeConnector implements DataEncryptionKeyFactory {
-  private static final Log LOG = Balancer.LOG;
-  private static final Path BALANCER_ID_PATH = new Path("/system/balancer.id");
+public class NameNodeConnector implements Closeable {
+  private static final Log LOG = LogFactory.getLog(NameNodeConnector.class);
+
   private static final int MAX_NOT_CHANGED_ITERATIONS = 5;
 
-  final URI nameNodeUri;
-  final String blockpoolID;
+  private final URI nameNodeUri;
+  private final String blockpoolID;
+
+  private final NamenodeProtocol namenode;
+  private final ClientProtocol client;
+  private final KeyManager keyManager;
 
-  final NamenodeProtocol namenode;
-  final ClientProtocol client;
-  final FileSystem fs;
-  final OutputStream out;
+  private final FileSystem fs;
+  private final Path idPath;
+  private final OutputStream out;
 
-  private final boolean isBlockTokenEnabled;
-  private final boolean encryptDataTransfer;
-  private boolean shouldRun;
-  private long keyUpdaterInterval;
-  // used for balancer
   private int notChangedIterations = 0;
-  private BlockTokenSecretManager blockTokenSecretManager;
-  private Daemon keyupdaterthread; // AccessKeyUpdater thread
-  private DataEncryptionKey encryptionKey;
 
-  NameNodeConnector(URI nameNodeUri,
+  public NameNodeConnector(String name, URI nameNodeUri, Path idPath,
       Configuration conf) throws IOException {
     this.nameNodeUri = nameNodeUri;
+    this.idPath = idPath;
     
-    this.namenode =
-      NameNodeProxies.createProxy(conf, nameNodeUri, NamenodeProtocol.class)
-        .getProxy();
-    this.client =
-      NameNodeProxies.createProxy(conf, nameNodeUri, ClientProtocol.class)
-        .getProxy();
+    this.namenode = NameNodeProxies.createProxy(conf, nameNodeUri,
+        NamenodeProtocol.class).getProxy();
+    this.client = NameNodeProxies.createProxy(conf, nameNodeUri,
+        ClientProtocol.class).getProxy();
     this.fs = FileSystem.get(nameNodeUri, conf);
 
     final NamespaceInfo namespaceinfo = namenode.versionRequest();
     this.blockpoolID = namespaceinfo.getBlockPoolID();
 
-    final ExportedBlockKeys keys = namenode.getBlockKeys();
-    this.isBlockTokenEnabled = keys.isBlockTokenEnabled();
-    if (isBlockTokenEnabled) {
-      long blockKeyUpdateInterval = keys.getKeyUpdateInterval();
-      long blockTokenLifetime = keys.getTokenLifetime();
-      LOG.info("Block token params received from NN: keyUpdateInterval="
-          + blockKeyUpdateInterval / (60 * 1000) + " min(s), tokenLifetime="
-          + blockTokenLifetime / (60 * 1000) + " min(s)");
-      String encryptionAlgorithm = conf.get(
-          DFSConfigKeys.DFS_DATA_ENCRYPTION_ALGORITHM_KEY);
-      this.blockTokenSecretManager = new BlockTokenSecretManager(
-          blockKeyUpdateInterval, blockTokenLifetime, blockpoolID,
-          encryptionAlgorithm);
-      this.blockTokenSecretManager.addKeys(keys);
-      /*
-       * Balancer should sync its block keys with NN more frequently than NN
-       * updates its block keys
-       */
-      this.keyUpdaterInterval = blockKeyUpdateInterval / 4;
-      LOG.info("Balancer will update its block keys every "
-          + keyUpdaterInterval / (60 * 1000) + " minute(s)");
-      this.keyupdaterthread = new Daemon(new BlockKeyUpdater());
-      this.shouldRun = true;
-      this.keyupdaterthread.start();
-    }
-    this.encryptDataTransfer = fs.getServerDefaults(new Path("/"))
-        .getEncryptDataTransfer();
-    // Check if there is another balancer running.
+    final FsServerDefaults defaults = fs.getServerDefaults(new Path("/"));
+    this.keyManager = new KeyManager(blockpoolID, namenode,
+        defaults.getEncryptDataTransfer(), conf);
     // Exit if there is another one running.
-    out = checkAndMarkRunningBalancer(); 
+    out = checkAndMarkRunning(); 
     if (out == null) {
-      throw new IOException("Another balancer is running");
+      throw new IOException("Another " + name + " is running.");
     }
   }
 
-  boolean shouldContinue(long dispatchBlockMoveBytes) {
+  /** @return the block pool ID */
+  public String getBlockpoolID() {
+    return blockpoolID;
+  }
+
+  /** @return the namenode proxy. */
+  public NamenodeProtocol getNamenode() {
+    return namenode;
+  }
+
+  /** @return the client proxy. */
+  public ClientProtocol getClient() {
+    return client;
+  }
+
+  /** @return the key manager */
+  public KeyManager getKeyManager() {
+    return keyManager;
+  }
+
+  /** Should the instance continue running? */
+  public boolean shouldContinue(long dispatchBlockMoveBytes) {
     if (dispatchBlockMoveBytes > 0) {
       notChangedIterations = 0;
     } else {
@@ -137,53 +120,25 @@ class NameNodeConnector implements DataEncryptionKeyFactory {
     return true;
   }
   
-  /** Get an access token for a block. */
-  Token<BlockTokenIdentifier> getAccessToken(ExtendedBlock eb
-      ) throws IOException {
-    if (!isBlockTokenEnabled) {
-      return BlockTokenSecretManager.DUMMY_TOKEN;
-    } else {
-      if (!shouldRun) {
-        throw new IOException(
-            "Can not get access token. BlockKeyUpdater is not running");
-      }
-      return blockTokenSecretManager.generateToken(null, eb,
-          EnumSet.of(BlockTokenSecretManager.AccessMode.REPLACE,
-          BlockTokenSecretManager.AccessMode.COPY));
-    }
-  }
-
-  @Override
-  public DataEncryptionKey newDataEncryptionKey() {
-    if (encryptDataTransfer) {
-      synchronized (this) {
-        if (encryptionKey == null) {
-          encryptionKey = blockTokenSecretManager.generateDataEncryptionKey();
-        }
-        return encryptionKey;
-      }
-    } else {
-      return null;
-    }
-  }
 
-  /* The idea for making sure that there is no more than one balancer
+  /**
+   * The idea for making sure that there is no more than one instance
    * running in an HDFS is to create a file in the HDFS, writes the hostname
-   * of the machine on which the balancer is running to the file, but did not
-   * close the file until the balancer exits. 
-   * This prevents the second balancer from running because it can not
+   * of the machine on which the instance is running to the file, but did not
+   * close the file until it exits. 
+   * 
+   * This prevents the second instance from running because it can not
    * creates the file while the first one is running.
    * 
-   * This method checks if there is any running balancer and 
-   * if no, mark yes if no.
+   * This method checks if there is any running instance. If no, mark yes.
    * Note that this is an atomic operation.
    * 
-   * Return null if there is a running balancer; otherwise the output stream
-   * to the newly created file.
+   * @return null if there is a running instance;
+   *         otherwise, the output stream to the newly created file.
    */
-  private OutputStream checkAndMarkRunningBalancer() throws IOException {
+  private OutputStream checkAndMarkRunning() throws IOException {
     try {
-      final DataOutputStream out = fs.create(BALANCER_ID_PATH);
+      final DataOutputStream out = fs.create(idPath);
       out.writeBytes(InetAddress.getLocalHost().getHostName());
       out.flush();
       return out;
@@ -196,24 +151,17 @@ class NameNodeConnector implements DataEncryptionKeyFactory {
     }
   }
 
-  /** Close the connection. */
-  void close() {
-    shouldRun = false;
-    try {
-      if (keyupdaterthread != null) {
-        keyupdaterthread.interrupt();
-      }
-    } catch(Exception e) {
-      LOG.warn("Exception shutting down access key updater thread", e);
-    }
+  @Override
+  public void close() {
+    keyManager.close();
 
     // close the output file
     IOUtils.closeStream(out); 
     if (fs != null) {
       try {
-        fs.delete(BALANCER_ID_PATH, true);
+        fs.delete(idPath, true);
       } catch(IOException ioe) {
-        LOG.warn("Failed to delete " + BALANCER_ID_PATH, ioe);
+        LOG.warn("Failed to delete " + idPath, ioe);
       }
     }
   }
@@ -221,31 +169,6 @@ class NameNodeConnector implements DataEncryptionKeyFactory {
   @Override
   public String toString() {
     return getClass().getSimpleName() + "[namenodeUri=" + nameNodeUri
-        + ", id=" + blockpoolID
-        + "]";
-  }
-
-  /**
-   * Periodically updates access keys.
-   */
-  class BlockKeyUpdater implements Runnable {
-    @Override
-    public void run() {
-      try {
-        while (shouldRun) {
-          try {
-            blockTokenSecretManager.addKeys(namenode.getBlockKeys());
-          } catch (IOException e) {
-            LOG.error("Failed to set keys", e);
-          }
-          Thread.sleep(keyUpdaterInterval);
-        }
-      } catch (InterruptedException e) {
-        LOG.debug("InterruptedException in block key updater thread", e);
-      } catch (Throwable e) {
-        LOG.error("Exception in block key updater thread", e);
-        shouldRun = false;
-      }
-    }
+        + ", bpid=" + blockpoolID + "]";
   }
 }

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

@@ -21,7 +21,6 @@ import java.util.LinkedList;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.util.LightWeightGSet;
 
@@ -254,18 +253,18 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
   }
   /**
    * Find specified DatanodeStorageInfo.
-   * @return index or -1 if not found.
+   * @return DatanodeStorageInfo or null if not found.
    */
-  int findStorageInfo(DatanodeInfo dn) {
+  DatanodeStorageInfo findStorageInfo(DatanodeDescriptor dn) {
     int len = getCapacity();
     for(int idx = 0; idx < len; idx++) {
       DatanodeStorageInfo cur = getStorageInfo(idx);
       if(cur == null)
         break;
       if(cur.getDatanodeDescriptor() == dn)
-        return idx;
+        return cur;
     }
-    return -1;
+    return null;
   }
   
   /**

+ 109 - 69
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -1082,6 +1082,7 @@ public class BlockManager {
    * Mark the block belonging to datanode as corrupt
    * @param blk Block to be marked as corrupt
    * @param dn Datanode which holds the corrupt replica
+   * @param storageID if known, null otherwise.
    * @param reason a textual reason why the block should be marked corrupt,
    * for logging purposes
    */
@@ -1098,19 +1099,29 @@ public class BlockManager {
           + blk + " not found");
       return;
     }
-    markBlockAsCorrupt(new BlockToMarkCorrupt(storedBlock,
-        blk.getGenerationStamp(), reason, Reason.CORRUPTION_REPORTED),
-        dn, storageID);
-  }
 
-  private void markBlockAsCorrupt(BlockToMarkCorrupt b,
-      DatanodeInfo dn, String storageID) throws IOException {
     DatanodeDescriptor node = getDatanodeManager().getDatanode(dn);
     if (node == null) {
-      throw new IOException("Cannot mark " + b
+      throw new IOException("Cannot mark " + blk
           + " as corrupt because datanode " + dn + " (" + dn.getDatanodeUuid()
           + ") does not exist");
     }
+    
+    markBlockAsCorrupt(new BlockToMarkCorrupt(storedBlock,
+            blk.getGenerationStamp(), reason, Reason.CORRUPTION_REPORTED),
+        storageID == null ? null : node.getStorageInfo(storageID),
+        node);
+  }
+
+  /**
+   * 
+   * @param b
+   * @param storageInfo storage that contains the block, if known. null otherwise.
+   * @throws IOException
+   */
+  private void markBlockAsCorrupt(BlockToMarkCorrupt b,
+      DatanodeStorageInfo storageInfo,
+      DatanodeDescriptor node) throws IOException {
 
     BlockCollection bc = b.corrupted.getBlockCollection();
     if (bc == null) {
@@ -1121,7 +1132,9 @@ public class BlockManager {
     } 
 
     // Add replica to the data-node if it is not already there
-    node.addBlock(storageID, b.stored);
+    if (storageInfo != null) {
+      storageInfo.addBlock(b.stored);
+    }
 
     // Add this replica to corruptReplicas Map
     corruptReplicas.addToCorruptReplicasMap(b.corrupted, node, b.reason,
@@ -1481,6 +1494,8 @@ public class BlockManager {
    * 
    * @throws IOException
    *           if the number of targets < minimum replication.
+   * @see BlockPlacementPolicy#chooseTarget(String, int, Node,
+   *      List, boolean, Set, long, StorageType)
    */
   public DatanodeStorageInfo[] chooseTarget4NewBlock(final String src,
       final int numOfReplicas, final DatanodeDescriptor client,
@@ -1719,7 +1734,7 @@ public class BlockManager {
    * @throws IOException
    */
   public boolean processReport(final DatanodeID nodeID,
-      final DatanodeStorage storage, final String poolId,
+      final DatanodeStorage storage,
       final BlockListAsLongs newReport) throws IOException {
     namesystem.writeLock();
     final long startTime = Time.now(); //after acquiring write lock
@@ -1751,9 +1766,9 @@ public class BlockManager {
       if (storageInfo.numBlocks() == 0) {
         // The first block report can be processed a lot more efficiently than
         // ordinary block reports.  This shortens restart times.
-        processFirstBlockReport(node, storage.getStorageID(), newReport);
+        processFirstBlockReport(storageInfo, newReport);
       } else {
-        processReport(node, storage, newReport);
+        processReport(storageInfo, newReport);
       }
       
       // Now that we have an up-to-date block report, we know that any
@@ -1815,9 +1830,8 @@ public class BlockManager {
     }
   }
   
-  private void processReport(final DatanodeDescriptor node,
-      final DatanodeStorage storage,
-      final BlockListAsLongs report) throws IOException {
+  private void processReport(final DatanodeStorageInfo storageInfo,
+                             final BlockListAsLongs report) throws IOException {
     // Normal case:
     // Modify the (block-->datanode) map, according to the difference
     // between the old and new block report.
@@ -1827,19 +1841,20 @@ public class BlockManager {
     Collection<Block> toInvalidate = new LinkedList<Block>();
     Collection<BlockToMarkCorrupt> toCorrupt = new LinkedList<BlockToMarkCorrupt>();
     Collection<StatefulBlockInfo> toUC = new LinkedList<StatefulBlockInfo>();
-    reportDiff(node, storage, report,
+    reportDiff(storageInfo, report,
         toAdd, toRemove, toInvalidate, toCorrupt, toUC);
-
+   
+    DatanodeDescriptor node = storageInfo.getDatanodeDescriptor();
     // Process the blocks on each queue
     for (StatefulBlockInfo b : toUC) { 
-      addStoredBlockUnderConstruction(b, node, storage.getStorageID());
+      addStoredBlockUnderConstruction(b, storageInfo);
     }
     for (Block b : toRemove) {
       removeStoredBlock(b, node);
     }
     int numBlocksLogged = 0;
     for (BlockInfo b : toAdd) {
-      addStoredBlock(b, node, storage.getStorageID(), null, numBlocksLogged < maxNumBlocksToLog);
+      addStoredBlock(b, storageInfo, null, numBlocksLogged < maxNumBlocksToLog);
       numBlocksLogged++;
     }
     if (numBlocksLogged > maxNumBlocksToLog) {
@@ -1853,7 +1868,7 @@ public class BlockManager {
       addToInvalidates(b, node);
     }
     for (BlockToMarkCorrupt b : toCorrupt) {
-      markBlockAsCorrupt(b, node, storage.getStorageID());
+      markBlockAsCorrupt(b, storageInfo, node);
     }
   }
 
@@ -1864,16 +1879,16 @@ public class BlockManager {
    * a toRemove list (since there won't be any).  It also silently discards 
    * any invalid blocks, thereby deferring their processing until 
    * the next block report.
-   * @param node - DatanodeDescriptor of the node that sent the report
+   * @param storageInfo - DatanodeStorageInfo that sent the report
    * @param report - the initial block report, to be processed
    * @throws IOException 
    */
-  private void processFirstBlockReport(final DatanodeDescriptor node,
-      final String storageID,
+  private void processFirstBlockReport(
+      final DatanodeStorageInfo storageInfo,
       final BlockListAsLongs report) throws IOException {
     if (report == null) return;
     assert (namesystem.hasWriteLock());
-    assert (node.getStorageInfo(storageID).numBlocks() == 0);
+    assert (storageInfo.numBlocks() == 0);
     BlockReportIterator itBR = report.getBlockReportIterator();
 
     while(itBR.hasNext()) {
@@ -1882,7 +1897,7 @@ public class BlockManager {
       
       if (shouldPostponeBlocksFromFuture &&
           namesystem.isGenStampInFuture(iblk)) {
-        queueReportedBlock(node, storageID, iblk, reportedState,
+        queueReportedBlock(storageInfo, iblk, reportedState,
             QUEUE_REASON_FUTURE_GENSTAMP);
         continue;
       }
@@ -1894,15 +1909,16 @@ public class BlockManager {
       // If block is corrupt, mark it and continue to next block.
       BlockUCState ucState = storedBlock.getBlockUCState();
       BlockToMarkCorrupt c = checkReplicaCorrupt(
-          iblk, reportedState, storedBlock, ucState, node);
+          iblk, reportedState, storedBlock, ucState,
+          storageInfo.getDatanodeDescriptor());
       if (c != null) {
         if (shouldPostponeBlocksFromFuture) {
           // In the Standby, we may receive a block report for a file that we
           // just have an out-of-date gen-stamp or state for, for example.
-          queueReportedBlock(node, storageID, iblk, reportedState,
+          queueReportedBlock(storageInfo, iblk, reportedState,
               QUEUE_REASON_CORRUPT_STATE);
         } else {
-          markBlockAsCorrupt(c, node, storageID);
+          markBlockAsCorrupt(c, storageInfo, storageInfo.getDatanodeDescriptor());
         }
         continue;
       }
@@ -1910,7 +1926,7 @@ public class BlockManager {
       // If block is under construction, add this replica to its list
       if (isBlockUnderConstruction(storedBlock, ucState, reportedState)) {
         ((BlockInfoUnderConstruction)storedBlock).addReplicaIfNotPresent(
-            node.getStorageInfo(storageID), iblk, reportedState);
+            storageInfo, iblk, reportedState);
         // OpenFileBlocks only inside snapshots also will be added to safemode
         // threshold. So we need to update such blocks to safemode
         // refer HDFS-5283
@@ -1923,12 +1939,12 @@ public class BlockManager {
       }      
       //add replica if appropriate
       if (reportedState == ReplicaState.FINALIZED) {
-        addStoredBlockImmediate(storedBlock, node, storageID);
+        addStoredBlockImmediate(storedBlock, storageInfo);
       }
     }
   }
 
-  private void reportDiff(DatanodeDescriptor dn, DatanodeStorage storage, 
+  private void reportDiff(DatanodeStorageInfo storageInfo, 
       BlockListAsLongs newReport, 
       Collection<BlockInfo> toAdd,              // add to DatanodeDescriptor
       Collection<Block> toRemove,           // remove from DatanodeDescriptor
@@ -1936,8 +1952,6 @@ public class BlockManager {
       Collection<BlockToMarkCorrupt> toCorrupt, // add to corrupt replicas list
       Collection<StatefulBlockInfo> toUC) { // add to under-construction list
 
-    final DatanodeStorageInfo storageInfo = dn.getStorageInfo(storage.getStorageID());
-
     // place a delimiter in the list which separates blocks 
     // that have been reported from those that have not
     BlockInfo delimiter = new BlockInfo(new Block(), 1);
@@ -1954,7 +1968,7 @@ public class BlockManager {
     while(itBR.hasNext()) {
       Block iblk = itBR.next();
       ReplicaState iState = itBR.getCurrentReplicaState();
-      BlockInfo storedBlock = processReportedBlock(dn, storage.getStorageID(),
+      BlockInfo storedBlock = processReportedBlock(storageInfo,
           iblk, iState, toAdd, toInvalidate, toCorrupt, toUC);
 
       // move block to the head of the list
@@ -1991,7 +2005,7 @@ public class BlockManager {
    * BlockInfoUnderConstruction's list of replicas.</li>
    * </ol>
    * 
-   * @param dn descriptor for the datanode that made the report
+   * @param storageInfo DatanodeStorageInfo that sent the report.
    * @param block reported block replica
    * @param reportedState reported replica state
    * @param toAdd add to DatanodeDescriptor
@@ -2003,14 +2017,16 @@ public class BlockManager {
    * @return the up-to-date stored block, if it should be kept.
    *         Otherwise, null.
    */
-  private BlockInfo processReportedBlock(final DatanodeDescriptor dn,
-      final String storageID,
+  private BlockInfo processReportedBlock(
+      final DatanodeStorageInfo storageInfo,
       final Block block, final ReplicaState reportedState, 
       final Collection<BlockInfo> toAdd, 
       final Collection<Block> toInvalidate, 
       final Collection<BlockToMarkCorrupt> toCorrupt,
       final Collection<StatefulBlockInfo> toUC) {
     
+    DatanodeDescriptor dn = storageInfo.getDatanodeDescriptor();
+
     if(LOG.isDebugEnabled()) {
       LOG.debug("Reported block " + block
           + " on " + dn + " size " + block.getNumBytes()
@@ -2019,7 +2035,7 @@ public class BlockManager {
   
     if (shouldPostponeBlocksFromFuture &&
         namesystem.isGenStampInFuture(block)) {
-      queueReportedBlock(dn, storageID, block, reportedState,
+      queueReportedBlock(storageInfo, block, reportedState,
           QUEUE_REASON_FUTURE_GENSTAMP);
       return null;
     }
@@ -2059,7 +2075,7 @@ public class BlockManager {
         // TODO: Pretty confident this should be s/storedBlock/block below,
         // since we should be postponing the info of the reported block, not
         // the stored block. See HDFS-6289 for more context.
-        queueReportedBlock(dn, storageID, storedBlock, reportedState,
+        queueReportedBlock(storageInfo, storedBlock, reportedState,
             QUEUE_REASON_CORRUPT_STATE);
       } else {
         toCorrupt.add(c);
@@ -2088,17 +2104,17 @@ public class BlockManager {
    * standby node. @see PendingDataNodeMessages.
    * @param reason a textual reason to report in the debug logs
    */
-  private void queueReportedBlock(DatanodeDescriptor dn, String storageID, Block block,
+  private void queueReportedBlock(DatanodeStorageInfo storageInfo, Block block,
       ReplicaState reportedState, String reason) {
     assert shouldPostponeBlocksFromFuture;
     
     if (LOG.isDebugEnabled()) {
       LOG.debug("Queueing reported block " + block +
           " in state " + reportedState + 
-          " from datanode " + dn + " for later processing " +
-          "because " + reason + ".");
+          " from datanode " + storageInfo.getDatanodeDescriptor() +
+          " for later processing because " + reason + ".");
     }
-    pendingDNMessages.enqueueReportedBlock(dn, storageID, block, reportedState);
+    pendingDNMessages.enqueueReportedBlock(storageInfo, block, reportedState);
   }
 
   /**
@@ -2121,7 +2137,7 @@ public class BlockManager {
       if (LOG.isDebugEnabled()) {
         LOG.debug("Processing previouly queued message " + rbi);
       }
-      processAndHandleReportedBlock(rbi.getNode(), rbi.getStorageID(), 
+      processAndHandleReportedBlock(rbi.getStorageInfo(), 
           rbi.getBlock(), rbi.getReportedState(), null);
     }
   }
@@ -2178,6 +2194,16 @@ public class BlockManager {
         } else {
           return null; // not corrupt
         }
+      case UNDER_CONSTRUCTION:
+        if (storedBlock.getGenerationStamp() > reported.getGenerationStamp()) {
+          final long reportedGS = reported.getGenerationStamp();
+          return new BlockToMarkCorrupt(storedBlock, reportedGS, "block is "
+              + ucState + " and reported state " + reportedState
+              + ", But reported genstamp " + reportedGS
+              + " does not match genstamp in block map "
+              + storedBlock.getGenerationStamp(), Reason.GENSTAMP_MISMATCH);
+        }
+        return null;
       default:
         return null;
       }
@@ -2241,19 +2267,20 @@ public class BlockManager {
   }
 
   void addStoredBlockUnderConstruction(StatefulBlockInfo ucBlock,
-      DatanodeDescriptor node, String storageID) throws IOException {
+      DatanodeStorageInfo storageInfo) throws IOException {
     BlockInfoUnderConstruction block = ucBlock.storedBlock;
-    block.addReplicaIfNotPresent(node.getStorageInfo(storageID),
-        ucBlock.reportedBlock, ucBlock.reportedState);
+    block.addReplicaIfNotPresent(
+        storageInfo, ucBlock.reportedBlock, ucBlock.reportedState);
 
-    if (ucBlock.reportedState == ReplicaState.FINALIZED && block.findDatanode(node) < 0) {
-      addStoredBlock(block, node, storageID, null, true);
+    if (ucBlock.reportedState == ReplicaState.FINALIZED &&
+        block.findDatanode(storageInfo.getDatanodeDescriptor()) < 0) {
+      addStoredBlock(block, storageInfo, null, true);
     }
   } 
 
   /**
    * Faster version of
-   * {@link #addStoredBlock(BlockInfo, DatanodeDescriptor, String, DatanodeDescriptor, boolean)}
+   * {@link #addStoredBlock(BlockInfo, DatanodeStorageInfo, DatanodeDescriptor, boolean)}
    * , intended for use with initial block report at startup. If not in startup
    * safe mode, will call standard addStoredBlock(). Assumes this method is
    * called "immediately" so there is no need to refresh the storedBlock from
@@ -2264,17 +2291,17 @@ public class BlockManager {
    * @throws IOException
    */
   private void addStoredBlockImmediate(BlockInfo storedBlock,
-      DatanodeDescriptor node, String storageID)
+      DatanodeStorageInfo storageInfo)
   throws IOException {
     assert (storedBlock != null && namesystem.hasWriteLock());
     if (!namesystem.isInStartupSafeMode() 
         || namesystem.isPopulatingReplQueues()) {
-      addStoredBlock(storedBlock, node, storageID, null, false);
+      addStoredBlock(storedBlock, storageInfo, null, false);
       return;
     }
 
     // just add it
-    node.addBlock(storageID, storedBlock);
+    storageInfo.addBlock(storedBlock);
 
     // Now check for completion of blocks and safe block count
     int numCurrentReplica = countLiveNodes(storedBlock);
@@ -2296,13 +2323,13 @@ public class BlockManager {
    * @return the block that is stored in blockMap.
    */
   private Block addStoredBlock(final BlockInfo block,
-                               DatanodeDescriptor node,
-                               String storageID,
+                               DatanodeStorageInfo storageInfo,
                                DatanodeDescriptor delNodeHint,
                                boolean logEveryBlock)
   throws IOException {
     assert block != null && namesystem.hasWriteLock();
     BlockInfo storedBlock;
+    DatanodeDescriptor node = storageInfo.getDatanodeDescriptor();
     if (block instanceof BlockInfoUnderConstruction) {
       //refresh our copy in case the block got completed in another thread
       storedBlock = blocksMap.getStoredBlock(block);
@@ -2322,7 +2349,7 @@ public class BlockManager {
     assert bc != null : "Block must belong to a file";
 
     // add block to the datanode
-    boolean added = node.addBlock(storageID, storedBlock);
+    boolean added = storageInfo.addBlock(storedBlock);
 
     int curReplicaDelta;
     if (added) {
@@ -2872,8 +2899,9 @@ public class BlockManager {
    * The given node is reporting that it received a certain block.
    */
   @VisibleForTesting
-  void addBlock(DatanodeDescriptor node, String storageID, Block block, String delHint)
+  void addBlock(DatanodeStorageInfo storageInfo, Block block, String delHint)
       throws IOException {
+    DatanodeDescriptor node = storageInfo.getDatanodeDescriptor();
     // Decrement number of blocks scheduled to this datanode.
     // for a retry request (of DatanodeProtocol#blockReceivedAndDeleted with 
     // RECEIVED_BLOCK), we currently also decrease the approximate number. 
@@ -2893,12 +2921,12 @@ public class BlockManager {
     // Modify the blocks->datanode map and node's map.
     //
     pendingReplications.decrement(block, node);
-    processAndHandleReportedBlock(node, storageID, block, ReplicaState.FINALIZED,
+    processAndHandleReportedBlock(storageInfo, block, ReplicaState.FINALIZED,
         delHintNode);
   }
   
-  private void processAndHandleReportedBlock(DatanodeDescriptor node,
-      String storageID, Block block,
+  private void processAndHandleReportedBlock(
+      DatanodeStorageInfo storageInfo, Block block,
       ReplicaState reportedState, DatanodeDescriptor delHintNode)
       throws IOException {
     // blockReceived reports a finalized block
@@ -2906,7 +2934,9 @@ public class BlockManager {
     Collection<Block> toInvalidate = new LinkedList<Block>();
     Collection<BlockToMarkCorrupt> toCorrupt = new LinkedList<BlockToMarkCorrupt>();
     Collection<StatefulBlockInfo> toUC = new LinkedList<StatefulBlockInfo>();
-    processReportedBlock(node, storageID, block, reportedState,
+    final DatanodeDescriptor node = storageInfo.getDatanodeDescriptor();
+
+    processReportedBlock(storageInfo, block, reportedState,
                               toAdd, toInvalidate, toCorrupt, toUC);
     // the block is only in one of the to-do lists
     // if it is in none then data-node already has it
@@ -2914,11 +2944,11 @@ public class BlockManager {
       : "The block should be only in one of the lists.";
 
     for (StatefulBlockInfo b : toUC) { 
-      addStoredBlockUnderConstruction(b, node, storageID);
+      addStoredBlockUnderConstruction(b, storageInfo);
     }
     long numBlocksLogged = 0;
     for (BlockInfo b : toAdd) {
-      addStoredBlock(b, node, storageID, delHintNode, numBlocksLogged < maxNumBlocksToLog);
+      addStoredBlock(b, storageInfo, delHintNode, numBlocksLogged < maxNumBlocksToLog);
       numBlocksLogged++;
     }
     if (numBlocksLogged > maxNumBlocksToLog) {
@@ -2932,7 +2962,7 @@ public class BlockManager {
       addToInvalidates(b, node);
     }
     for (BlockToMarkCorrupt b : toCorrupt) {
-      markBlockAsCorrupt(b, node, storageID);
+      markBlockAsCorrupt(b, storageInfo, node);
     }
   }
 
@@ -2959,13 +2989,15 @@ public class BlockManager {
           "Got incremental block report from unregistered or dead node");
     }
 
-    if (node.getStorageInfo(srdb.getStorage().getStorageID()) == null) {
+    DatanodeStorageInfo storageInfo =
+        node.getStorageInfo(srdb.getStorage().getStorageID());
+    if (storageInfo == null) {
       // The DataNode is reporting an unknown storage. Usually the NN learns
       // about new storages from heartbeats but during NN restart we may
       // receive a block report or incremental report before the heartbeat.
       // We must handle this for protocol compatibility. This issue was
       // uncovered by HDFS-6094.
-      node.updateStorage(srdb.getStorage());
+      storageInfo = node.updateStorage(srdb.getStorage());
     }
 
     for (ReceivedDeletedBlockInfo rdbi : srdb.getBlocks()) {
@@ -2975,14 +3007,13 @@ public class BlockManager {
         deleted++;
         break;
       case RECEIVED_BLOCK:
-        addBlock(node, srdb.getStorage().getStorageID(),
-            rdbi.getBlock(), rdbi.getDelHints());
+        addBlock(storageInfo, rdbi.getBlock(), rdbi.getDelHints());
         received++;
         break;
       case RECEIVING_BLOCK:
         receiving++;
-        processAndHandleReportedBlock(node, srdb.getStorage().getStorageID(),
-            rdbi.getBlock(), ReplicaState.RBW, null);
+        processAndHandleReportedBlock(storageInfo, rdbi.getBlock(),
+                                      ReplicaState.RBW, null);
         break;
       default:
         String msg = 
@@ -3174,6 +3205,15 @@ public class BlockManager {
         }
       }
     }
+
+    if (!status && !srcNode.isAlive) {
+      LOG.warn("srcNode " + srcNode + " is dead " +
+          "when decommission is in progress. Continue to mark " +
+          "it as decommission in progress. In that way, when it rejoins the " +
+          "cluster it can continue the decommission process.");
+      status = true;
+    }
+
     srcNode.decommissioningStatus.set(underReplicatedBlocks,
         decommissionOnlyReplicas, 
         underReplicatedInOpenFiles);

+ 9 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java

@@ -23,8 +23,8 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.util.GSet;
 import org.apache.hadoop.util.LightWeightGSet;
-import org.apache.hadoop.util.LightWeightGSet.SetIterator;
 
+import com.google.common.base.Preconditions;
 import com.google.common.base.Predicate;
 import com.google.common.collect.Iterables;
 
@@ -217,9 +217,14 @@ class BlocksMap {
     BlockInfo currentBlock = blocks.get(newBlock);
     assert currentBlock != null : "the block if not in blocksMap";
     // replace block in data-node lists
-    for(int idx = currentBlock.numNodes()-1; idx >= 0; idx--) {
-      DatanodeDescriptor dn = currentBlock.getDatanode(idx);
-      dn.replaceBlock(currentBlock, newBlock);
+    for (int i = currentBlock.numNodes() - 1; i >= 0; i--) {
+      final DatanodeDescriptor dn = currentBlock.getDatanode(i);
+      final DatanodeStorageInfo storage = currentBlock.findStorageInfo(dn);
+      final boolean removed = storage.removeBlock(currentBlock);
+      Preconditions.checkState(removed, "currentBlock not found.");
+
+      final boolean added = storage.addBlock(newBlock);
+      Preconditions.checkState(added, "newBlock already exists.");
     }
     // replace block in the map itself
     blocks.put(newBlock);

+ 3 - 16
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CorruptReplicasMap.java

@@ -48,18 +48,6 @@ public class CorruptReplicasMap{
 
   private final SortedMap<Block, Map<DatanodeDescriptor, Reason>> corruptReplicasMap =
     new TreeMap<Block, Map<DatanodeDescriptor, Reason>>();
-  
-  /**
-   * Mark the block belonging to datanode as corrupt.
-   *
-   * @param blk Block to be added to CorruptReplicasMap
-   * @param dn DatanodeDescriptor which holds the corrupt replica
-   * @param reason a textual reason (for logging purposes)
-   */
-  public void addToCorruptReplicasMap(Block blk, DatanodeDescriptor dn,
-      String reason) {
-    addToCorruptReplicasMap(blk, dn, reason, Reason.NONE);
-  }
 
   /**
    * Mark the block belonging to datanode as corrupt.
@@ -69,7 +57,7 @@ public class CorruptReplicasMap{
    * @param reason a textual reason (for logging purposes)
    * @param reasonCode the enum representation of the reason
    */
-  public void addToCorruptReplicasMap(Block blk, DatanodeDescriptor dn,
+  void addToCorruptReplicasMap(Block blk, DatanodeDescriptor dn,
       String reason, Reason reasonCode) {
     Map <DatanodeDescriptor, Reason> nodes = corruptReplicasMap.get(blk);
     if (nodes == null) {
@@ -127,7 +115,6 @@ public class CorruptReplicasMap{
   boolean removeFromCorruptReplicasMap(Block blk, DatanodeDescriptor datanode,
       Reason reason) {
     Map <DatanodeDescriptor, Reason> datanodes = corruptReplicasMap.get(blk);
-    boolean removed = false;
     if (datanodes==null)
       return false;
 
@@ -174,12 +161,12 @@ public class CorruptReplicasMap{
     return ((nodes != null) && (nodes.contains(node)));
   }
 
-  public int numCorruptReplicas(Block blk) {
+  int numCorruptReplicas(Block blk) {
     Collection<DatanodeDescriptor> nodes = getNodes(blk);
     return (nodes == null) ? 0 : nodes.size();
   }
   
-  public int size() {
+  int size() {
     return corruptReplicasMap.size();
   }
 

+ 4 - 37
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java

@@ -234,18 +234,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
     updateHeartbeat(StorageReport.EMPTY_ARRAY, 0L, 0L, 0, 0);
   }
 
-  /**
-   * Add data-node to the block. Add block to the head of the list of blocks
-   * belonging to the data-node.
-   */
-  public boolean addBlock(String storageID, BlockInfo b) {
-    DatanodeStorageInfo s = getStorageInfo(storageID);
-    if (s != null) {
-      return s.addBlock(b);
-    }
-    return false;
-  }
-
   @VisibleForTesting
   public DatanodeStorageInfo getStorageInfo(String storageID) {
     synchronized (storageMap) {
@@ -260,8 +248,8 @@ public class DatanodeDescriptor extends DatanodeInfo {
   }
 
   public StorageReport[] getStorageReports() {
-    final StorageReport[] reports = new StorageReport[storageMap.size()];
     final DatanodeStorageInfo[] infos = getStorageInfos();
+    final StorageReport[] reports = new StorageReport[infos.length];
     for(int i = 0; i < infos.length; i++) {
       reports[i] = infos[i].toStorageReport();
     }
@@ -284,13 +272,10 @@ public class DatanodeDescriptor extends DatanodeInfo {
    * data-node from the block.
    */
   boolean removeBlock(BlockInfo b) {
-    int index = b.findStorageInfo(this);
+    final DatanodeStorageInfo s = b.findStorageInfo(this);
     // if block exists on this datanode
-    if (index >= 0) {
-      DatanodeStorageInfo s = b.getStorageInfo(index);
-      if (s != null) {
-        return s.removeBlock(b);
-      }
+    if (s != null) {
+      return s.removeBlock(b);
     }
     return false;
   }
@@ -307,24 +292,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
     return false;
   }
 
-  /**
-   * Replace specified old block with a new one in the DataNodeDescriptor.
-   *
-   * @param oldBlock - block to be replaced
-   * @param newBlock - a replacement block
-   * @return the new block
-   */
-  public BlockInfo replaceBlock(BlockInfo oldBlock, BlockInfo newBlock) {
-    int index = oldBlock.findStorageInfo(this);
-    DatanodeStorageInfo s = oldBlock.getStorageInfo(index);
-    boolean done = s.removeBlock(oldBlock);
-    assert done : "Old block should belong to the data-node when replacing";
-
-    done = s.addBlock(newBlock);
-    assert done : "New block should not belong to the data-node when replacing";
-    return newBlock;
-  }
-
   public void resetBlocks() {
     setCapacity(0);
     setRemaining(0);

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

@@ -207,7 +207,7 @@ public class DatanodeStorageInfo {
     return blockPoolUsed;
   }
 
-  boolean addBlock(BlockInfo b) {
+  public boolean addBlock(BlockInfo b) {
     if(!b.addStorage(this))
       return false;
     // add to the head of the data-node list

+ 15 - 19
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingDataNodeMessages.java

@@ -23,6 +23,7 @@ import java.util.Queue;
 
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
+
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
@@ -41,14 +42,12 @@ class PendingDataNodeMessages {
     
   static class ReportedBlockInfo {
     private final Block block;
-    private final DatanodeDescriptor dn;
-    private final String storageID;
+    private final DatanodeStorageInfo storageInfo;
     private final ReplicaState reportedState;
 
-    ReportedBlockInfo(DatanodeDescriptor dn, String storageID, Block block,
+    ReportedBlockInfo(DatanodeStorageInfo storageInfo, Block block,
         ReplicaState reportedState) {
-      this.dn = dn;
-      this.storageID = storageID;
+      this.storageInfo = storageInfo;
       this.block = block;
       this.reportedState = reportedState;
     }
@@ -57,21 +56,18 @@ class PendingDataNodeMessages {
       return block;
     }
 
-    DatanodeDescriptor getNode() {
-      return dn;
-    }
-    
-    String getStorageID() {
-      return storageID;
-    }
-
     ReplicaState getReportedState() {
       return reportedState;
     }
+    
+    DatanodeStorageInfo getStorageInfo() {
+      return storageInfo;
+    }
 
     @Override
     public String toString() {
-      return "ReportedBlockInfo [block=" + block + ", dn=" + dn
+      return "ReportedBlockInfo [block=" + block + ", dn="
+          + storageInfo.getDatanodeDescriptor()
           + ", reportedState=" + reportedState + "]";
     }
   }
@@ -87,7 +83,7 @@ class PendingDataNodeMessages {
       Queue<ReportedBlockInfo> oldQueue = entry.getValue();
       while (!oldQueue.isEmpty()) {
         ReportedBlockInfo rbi = oldQueue.remove();
-        if (!rbi.getNode().equals(dn)) {
+        if (!rbi.getStorageInfo().getDatanodeDescriptor().equals(dn)) {
           newQueue.add(rbi);
         } else {
           count--;
@@ -97,11 +93,11 @@ class PendingDataNodeMessages {
     }
   }
   
-  void enqueueReportedBlock(DatanodeDescriptor dn, String storageID, Block block,
+  void enqueueReportedBlock(DatanodeStorageInfo storageInfo, Block block,
       ReplicaState reportedState) {
     block = new Block(block);
     getBlockQueue(block).add(
-        new ReportedBlockInfo(dn, storageID, block, reportedState));
+        new ReportedBlockInfo(storageInfo, block, reportedState));
     count++;
   }
   
@@ -127,7 +123,7 @@ class PendingDataNodeMessages {
     return queue;
   }
   
-  public int count() {
+  int count() {
     return count ;
   }
 
@@ -144,7 +140,7 @@ class PendingDataNodeMessages {
     return sb.toString();
   }
 
-  public Iterable<ReportedBlockInfo> takeAll() {
+  Iterable<ReportedBlockInfo> takeAll() {
     List<ReportedBlockInfo> rbis = Lists.newArrayListWithCapacity(
         count);
     for (Queue<ReportedBlockInfo> q : queueByBlockId.values()) {

+ 188 - 115
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java

@@ -21,6 +21,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
+
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
@@ -38,6 +39,8 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 /**
  * One instance per block-pool/namespace on the DN, which handles the
@@ -91,6 +94,28 @@ class BPOfferService {
    */
   private long lastActiveClaimTxId = -1;
 
+  private final ReentrantReadWriteLock mReadWriteLock =
+      new ReentrantReadWriteLock();
+  private final Lock mReadLock  = mReadWriteLock.readLock();
+  private final Lock mWriteLock = mReadWriteLock.writeLock();
+
+  // utility methods to acquire and release read lock and write lock
+  void readLock() {
+    mReadLock.lock();
+  }
+
+  void readUnlock() {
+    mReadLock.unlock();
+  }
+
+  void writeLock() {
+    mWriteLock.lock();
+  }
+
+  void writeUnlock() {
+    mWriteLock.unlock();
+  }
+
   BPOfferService(List<InetSocketAddress> nnAddrs, DataNode dn) {
     Preconditions.checkArgument(!nnAddrs.isEmpty(),
         "Must pass at least one NN.");
@@ -135,14 +160,19 @@ class BPOfferService {
     }
     return false;
   }
-  
-  synchronized String getBlockPoolId() {
-    if (bpNSInfo != null) {
-      return bpNSInfo.getBlockPoolID();
-    } else {
-      LOG.warn("Block pool ID needed, but service not yet registered with NN",
-          new Exception("trace"));
-      return null;
+
+  String getBlockPoolId() {
+    readLock();
+    try {
+      if (bpNSInfo != null) {
+        return bpNSInfo.getBlockPoolID();
+      } else {
+        LOG.warn("Block pool ID needed, but service not yet registered with NN",
+            new Exception("trace"));
+        return null;
+      }
+    } finally {
+      readUnlock();
     }
   }
 
@@ -150,27 +180,37 @@ class BPOfferService {
     return getNamespaceInfo() != null;
   }
 
-  synchronized NamespaceInfo getNamespaceInfo() {
-    return bpNSInfo;
+  NamespaceInfo getNamespaceInfo() {
+    readLock();
+    try {
+      return bpNSInfo;
+    } finally {
+      readUnlock();
+    }
   }
 
   @Override
-  public synchronized String toString() {
-    if (bpNSInfo == null) {
-      // If we haven't yet connected to our NN, we don't yet know our
-      // own block pool ID.
-      // If _none_ of the block pools have connected yet, we don't even
-      // know the DatanodeID ID of this DN.
-      String datanodeUuid = dn.getDatanodeUuid();
-
-      if (datanodeUuid == null || datanodeUuid.isEmpty()) {
-        datanodeUuid = "unassigned";
+  public String toString() {
+    readLock();
+    try {
+      if (bpNSInfo == null) {
+        // If we haven't yet connected to our NN, we don't yet know our
+        // own block pool ID.
+        // If _none_ of the block pools have connected yet, we don't even
+        // know the DatanodeID ID of this DN.
+        String datanodeUuid = dn.getDatanodeUuid();
+
+        if (datanodeUuid == null || datanodeUuid.isEmpty()) {
+          datanodeUuid = "unassigned";
+        }
+        return "Block pool <registering> (Datanode Uuid " + datanodeUuid + ")";
+      } else {
+        return "Block pool " + getBlockPoolId() +
+            " (Datanode Uuid " + dn.getDatanodeUuid() +
+            ")";
       }
-      return "Block pool <registering> (Datanode Uuid " + datanodeUuid + ")";
-    } else {
-      return "Block pool " + getBlockPoolId() +
-          " (Datanode Uuid " + dn.getDatanodeUuid() +
-          ")";
+    } finally {
+      readUnlock();
     }
   }
   
@@ -266,32 +306,37 @@ class BPOfferService {
    * verifies that this namespace matches (eg to prevent a misconfiguration
    * where a StandbyNode from a different cluster is specified)
    */
-  synchronized void verifyAndSetNamespaceInfo(NamespaceInfo nsInfo) throws IOException {
-    if (this.bpNSInfo == null) {
-      this.bpNSInfo = nsInfo;
-      boolean success = false;
-
-      // Now that we know the namespace ID, etc, we can pass this to the DN.
-      // The DN can now initialize its local storage if we are the
-      // first BP to handshake, etc.
-      try {
-        dn.initBlockPool(this);
-        success = true;
-      } finally {
-        if (!success) {
-          // The datanode failed to initialize the BP. We need to reset
-          // the namespace info so that other BPService actors still have
-          // a chance to set it, and re-initialize the datanode.
-          this.bpNSInfo = null;
+  void verifyAndSetNamespaceInfo(NamespaceInfo nsInfo) throws IOException {
+    writeLock();
+    try {
+      if (this.bpNSInfo == null) {
+        this.bpNSInfo = nsInfo;
+        boolean success = false;
+
+        // Now that we know the namespace ID, etc, we can pass this to the DN.
+        // The DN can now initialize its local storage if we are the
+        // first BP to handshake, etc.
+        try {
+          dn.initBlockPool(this);
+          success = true;
+        } finally {
+          if (!success) {
+            // The datanode failed to initialize the BP. We need to reset
+            // the namespace info so that other BPService actors still have
+            // a chance to set it, and re-initialize the datanode.
+            this.bpNSInfo = null;
+          }
         }
+      } else {
+        checkNSEquality(bpNSInfo.getBlockPoolID(), nsInfo.getBlockPoolID(),
+            "Blockpool ID");
+        checkNSEquality(bpNSInfo.getNamespaceID(), nsInfo.getNamespaceID(),
+            "Namespace ID");
+        checkNSEquality(bpNSInfo.getClusterID(), nsInfo.getClusterID(),
+            "Cluster ID");
       }
-    } else {
-      checkNSEquality(bpNSInfo.getBlockPoolID(), nsInfo.getBlockPoolID(),
-          "Blockpool ID");
-      checkNSEquality(bpNSInfo.getNamespaceID(), nsInfo.getNamespaceID(),
-          "Namespace ID");
-      checkNSEquality(bpNSInfo.getClusterID(), nsInfo.getClusterID(),
-          "Cluster ID");
+    } finally {
+      writeUnlock();
     }
   }
 
@@ -300,22 +345,27 @@ class BPOfferService {
    * NN, it calls this function to verify that the NN it connected to
    * is consistent with other NNs serving the block-pool.
    */
-  synchronized void registrationSucceeded(BPServiceActor bpServiceActor,
+  void registrationSucceeded(BPServiceActor bpServiceActor,
       DatanodeRegistration reg) throws IOException {
-    if (bpRegistration != null) {
-      checkNSEquality(bpRegistration.getStorageInfo().getNamespaceID(),
-          reg.getStorageInfo().getNamespaceID(), "namespace ID");
-      checkNSEquality(bpRegistration.getStorageInfo().getClusterID(),
-          reg.getStorageInfo().getClusterID(), "cluster ID");
-    } else {
-      bpRegistration = reg;
-    }
-    
-    dn.bpRegistrationSucceeded(bpRegistration, getBlockPoolId());
-    // Add the initial block token secret keys to the DN's secret manager.
-    if (dn.isBlockTokenEnabled) {
-      dn.blockPoolTokenSecretManager.addKeys(getBlockPoolId(),
-          reg.getExportedKeys());
+    writeLock();
+    try {
+      if (bpRegistration != null) {
+        checkNSEquality(bpRegistration.getStorageInfo().getNamespaceID(),
+            reg.getStorageInfo().getNamespaceID(), "namespace ID");
+        checkNSEquality(bpRegistration.getStorageInfo().getClusterID(),
+            reg.getStorageInfo().getClusterID(), "cluster ID");
+      } else {
+        bpRegistration = reg;
+      }
+
+      dn.bpRegistrationSucceeded(bpRegistration, getBlockPoolId());
+      // Add the initial block token secret keys to the DN's secret manager.
+      if (dn.isBlockTokenEnabled) {
+        dn.blockPoolTokenSecretManager.addKeys(getBlockPoolId(),
+            reg.getExportedKeys());
+      }
+    } finally {
+      writeUnlock();
     }
   }
 
@@ -333,25 +383,35 @@ class BPOfferService {
     }
   }
 
-  synchronized DatanodeRegistration createRegistration() {
-    Preconditions.checkState(bpNSInfo != null,
-        "getRegistration() can only be called after initial handshake");
-    return dn.createBPRegistration(bpNSInfo);
+  DatanodeRegistration createRegistration() {
+    writeLock();
+    try {
+      Preconditions.checkState(bpNSInfo != null,
+          "getRegistration() can only be called after initial handshake");
+      return dn.createBPRegistration(bpNSInfo);
+    } finally {
+      writeUnlock();
+    }
   }
 
   /**
    * Called when an actor shuts down. If this is the last actor
    * to shut down, shuts down the whole blockpool in the DN.
    */
-  synchronized void shutdownActor(BPServiceActor actor) {
-    if (bpServiceToActive == actor) {
-      bpServiceToActive = null;
-    }
+  void shutdownActor(BPServiceActor actor) {
+    writeLock();
+    try {
+      if (bpServiceToActive == actor) {
+        bpServiceToActive = null;
+      }
 
-    bpServices.remove(actor);
+      bpServices.remove(actor);
 
-    if (bpServices.isEmpty()) {
-      dn.shutdownBlockPool(this);
+      if (bpServices.isEmpty()) {
+        dn.shutdownBlockPool(this);
+      }
+    } finally {
+      writeUnlock();
     }
   }
 
@@ -392,11 +452,16 @@ class BPOfferService {
    * @return a proxy to the active NN, or null if the BPOS has not
    * acknowledged any NN as active yet.
    */
-  synchronized DatanodeProtocolClientSideTranslatorPB getActiveNN() {
-    if (bpServiceToActive != null) {
-      return bpServiceToActive.bpNamenode;
-    } else {
-      return null;
+  DatanodeProtocolClientSideTranslatorPB getActiveNN() {
+    readLock();
+    try {
+      if (bpServiceToActive != null) {
+        return bpServiceToActive.bpNamenode;
+      } else {
+        return null;
+      }
+    } finally {
+      readUnlock();
     }
   }
 
@@ -424,45 +489,50 @@ class BPOfferService {
    * @param actor the actor which received the heartbeat
    * @param nnHaState the HA-related heartbeat contents
    */
-  synchronized void updateActorStatesFromHeartbeat(
+  void updateActorStatesFromHeartbeat(
       BPServiceActor actor,
       NNHAStatusHeartbeat nnHaState) {
-    final long txid = nnHaState.getTxId();
-    
-    final boolean nnClaimsActive =
-      nnHaState.getState() == HAServiceState.ACTIVE;
-    final boolean bposThinksActive = bpServiceToActive == actor;
-    final boolean isMoreRecentClaim = txid > lastActiveClaimTxId; 
-    
-    if (nnClaimsActive && !bposThinksActive) {
-      LOG.info("Namenode " + actor + " trying to claim ACTIVE state with " +
-          "txid=" + txid);
-      if (!isMoreRecentClaim) {
-        // Split-brain scenario - an NN is trying to claim active
-        // state when a different NN has already claimed it with a higher
-        // txid.
-        LOG.warn("NN " + actor + " tried to claim ACTIVE state at txid=" +
-            txid + " but there was already a more recent claim at txid=" +
-            lastActiveClaimTxId);
-        return;
-      } else {
-        if (bpServiceToActive == null) {
-          LOG.info("Acknowledging ACTIVE Namenode " + actor);
+    writeLock();
+    try {
+      final long txid = nnHaState.getTxId();
+
+      final boolean nnClaimsActive =
+          nnHaState.getState() == HAServiceState.ACTIVE;
+      final boolean bposThinksActive = bpServiceToActive == actor;
+      final boolean isMoreRecentClaim = txid > lastActiveClaimTxId;
+
+      if (nnClaimsActive && !bposThinksActive) {
+        LOG.info("Namenode " + actor + " trying to claim ACTIVE state with " +
+            "txid=" + txid);
+        if (!isMoreRecentClaim) {
+          // Split-brain scenario - an NN is trying to claim active
+          // state when a different NN has already claimed it with a higher
+          // txid.
+          LOG.warn("NN " + actor + " tried to claim ACTIVE state at txid=" +
+              txid + " but there was already a more recent claim at txid=" +
+              lastActiveClaimTxId);
+          return;
         } else {
-          LOG.info("Namenode " + actor + " taking over ACTIVE state from " +
-              bpServiceToActive + " at higher txid=" + txid);
+          if (bpServiceToActive == null) {
+            LOG.info("Acknowledging ACTIVE Namenode " + actor);
+          } else {
+            LOG.info("Namenode " + actor + " taking over ACTIVE state from " +
+                bpServiceToActive + " at higher txid=" + txid);
+          }
+          bpServiceToActive = actor;
         }
-        bpServiceToActive = actor;
+      } else if (!nnClaimsActive && bposThinksActive) {
+        LOG.info("Namenode " + actor + " relinquishing ACTIVE state with " +
+            "txid=" + nnHaState.getTxId());
+        bpServiceToActive = null;
       }
-    } else if (!nnClaimsActive && bposThinksActive) {
-      LOG.info("Namenode " + actor + " relinquishing ACTIVE state with " +
-          "txid=" + nnHaState.getTxId());
-      bpServiceToActive = null;
-    }
-    
-    if (bpServiceToActive == actor) {
-      assert txid >= lastActiveClaimTxId;
-      lastActiveClaimTxId = txid;
+
+      if (bpServiceToActive == actor) {
+        assert txid >= lastActiveClaimTxId;
+        lastActiveClaimTxId = txid;
+      }
+    } finally {
+      writeUnlock();
     }
   }
 
@@ -533,12 +603,15 @@ class BPOfferService {
       actor.reRegister();
       return true;
     }
-    synchronized (this) {
+    writeLock();
+    try {
       if (actor == bpServiceToActive) {
         return processCommandFromActive(cmd, actor);
       } else {
         return processCommandFromStandby(cmd, actor);
       }
+    } finally {
+      writeUnlock();
     }
   }
 

+ 10 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java

@@ -152,7 +152,7 @@ public class BlockPoolSliceStorage extends Storage {
     // During startup some of them can upgrade or roll back
     // while others could be up-to-date for the regular startup.
     for (int idx = 0; idx < getNumStorageDirs(); idx++) {
-      doTransition(getStorageDir(idx), nsInfo, startOpt);
+      doTransition(datanode, getStorageDir(idx), nsInfo, startOpt);
       assert getCTime() == nsInfo.getCTime() 
           : "Data-node and name-node CTimes must be the same.";
     }
@@ -242,7 +242,7 @@ public class BlockPoolSliceStorage extends Storage {
    * @param startOpt startup option
    * @throws IOException
    */
-  private void doTransition(StorageDirectory sd,
+  private void doTransition(DataNode datanode, StorageDirectory sd,
       NamespaceInfo nsInfo, StartupOption startOpt) throws IOException {
     if (startOpt == StartupOption.ROLLBACK) {
       doRollback(sd, nsInfo); // rollback if applicable
@@ -275,7 +275,7 @@ public class BlockPoolSliceStorage extends Storage {
     }
     if (this.layoutVersion > HdfsConstants.DATANODE_LAYOUT_VERSION
         || this.cTime < nsInfo.getCTime()) {
-      doUpgrade(sd, nsInfo); // upgrade
+      doUpgrade(datanode, sd, nsInfo); // upgrade
       return;
     }
     // layoutVersion == LAYOUT_VERSION && this.cTime > nsInfo.cTime
@@ -304,7 +304,8 @@ public class BlockPoolSliceStorage extends Storage {
    * @param nsInfo Namespace Info from the namenode
    * @throws IOException on error
    */
-  void doUpgrade(StorageDirectory bpSd, NamespaceInfo nsInfo) throws IOException {
+  void doUpgrade(DataNode datanode, StorageDirectory bpSd, NamespaceInfo nsInfo)
+      throws IOException {
     // Upgrading is applicable only to release with federation or after
     if (!DataNodeLayoutVersion.supports(
         LayoutVersion.Feature.FEDERATION, layoutVersion)) {
@@ -340,7 +341,7 @@ public class BlockPoolSliceStorage extends Storage {
     rename(bpCurDir, bpTmpDir);
     
     // 3. Create new <SD>/current with block files hardlinks and VERSION
-    linkAllBlocks(bpTmpDir, bpCurDir);
+    linkAllBlocks(datanode, bpTmpDir, bpCurDir);
     this.layoutVersion = HdfsConstants.DATANODE_LAYOUT_VERSION;
     assert this.namespaceID == nsInfo.getNamespaceID() 
         : "Data-node and name-node layout versions must be the same.";
@@ -517,14 +518,15 @@ public class BlockPoolSliceStorage extends Storage {
    * @param toDir the current data directory
    * @throws IOException if error occurs during hardlink
    */
-  private void linkAllBlocks(File fromDir, File toDir) throws IOException {
+  private void linkAllBlocks(DataNode datanode, File fromDir, File toDir)
+      throws IOException {
     // do the link
     int diskLayoutVersion = this.getLayoutVersion();
     // hardlink finalized blocks in tmpDir
     HardLink hardLink = new HardLink();
-    DataStorage.linkBlocks(new File(fromDir, DataStorage.STORAGE_DIR_FINALIZED), 
+    DataStorage.linkBlocks(datanode, new File(fromDir, DataStorage.STORAGE_DIR_FINALIZED),
       new File(toDir,DataStorage.STORAGE_DIR_FINALIZED), diskLayoutVersion, hardLink);
-    DataStorage.linkBlocks(new File(fromDir, DataStorage.STORAGE_DIR_RBW), 
+    DataStorage.linkBlocks(datanode, new File(fromDir, DataStorage.STORAGE_DIR_RBW),
         new File(toDir, DataStorage.STORAGE_DIR_RBW), diskLayoutVersion, hardLink);
     LOG.info( hardLink.linkStats.report() );
   }

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java

@@ -253,7 +253,7 @@ class BlockReceiver implements Closeable {
       
       if (cause != null) { // possible disk error
         ioe = cause;
-        datanode.checkDiskError();
+        datanode.checkDiskErrorAsync();
       }
       
       throw ioe;
@@ -329,7 +329,7 @@ class BlockReceiver implements Closeable {
     }
     // disk check
     if(ioe != null) {
-      datanode.checkDiskError();
+      datanode.checkDiskErrorAsync();
       throw ioe;
     }
   }
@@ -639,7 +639,7 @@ class BlockReceiver implements Closeable {
           manageWriterOsCache(offsetInBlock);
         }
       } catch (IOException iex) {
-        datanode.checkDiskError();
+        datanode.checkDiskErrorAsync();
         throw iex;
       }
     }
@@ -1208,7 +1208,7 @@ class BlockReceiver implements Closeable {
         } catch (IOException e) {
           LOG.warn("IOException in BlockReceiver.run(): ", e);
           if (running) {
-            datanode.checkDiskError();
+            datanode.checkDiskErrorAsync();
             LOG.info(myString, e);
             running = false;
             if (!Thread.interrupted()) { // failure not caused by interruption

+ 21 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -1075,6 +1075,11 @@ public class DataNode extends Configured
     // In the case that this is the first block pool to connect, initialize
     // the dataset, block scanners, etc.
     initStorage(nsInfo);
+
+    // Exclude failed disks before initializing the block pools to avoid startup
+    // failures.
+    checkDiskError();
+
     initPeriodicScanners(conf);
     
     data.addBlockPool(nsInfo.getBlockPoolID(), conf);
@@ -1510,9 +1515,9 @@ public class DataNode extends Configured
   
   
   /**
-   *  Check if there is a disk failure and if so, handle the error
+   * Check if there is a disk failure asynchronously and if so, handle the error
    */
-  public void checkDiskError() {
+  public void checkDiskErrorAsync() {
     synchronized(checkDiskErrorMutex) {
       checkDiskErrorFlag = true;
       if(checkDiskErrorThread == null) {
@@ -1821,7 +1826,7 @@ public class DataNode extends Configured
         LOG.warn(bpReg + ":Failed to transfer " + b + " to " +
             targets[0] + " got ", ie);
         // check if there are any disk problem
-        checkDiskError();
+        checkDiskErrorAsync();
       } finally {
         xmitsInProgress.getAndDecrement();
         IOUtils.closeStream(blockSender);
@@ -2759,7 +2764,18 @@ public class DataNode extends Configured
   public ShortCircuitRegistry getShortCircuitRegistry() {
     return shortCircuitRegistry;
   }
-  
+
+  /**
+   * Check the disk error
+   */
+  private void checkDiskError() {
+    try {
+      data.checkDataDir();
+    } catch (DiskErrorException de) {
+      handleDiskError(de.getMessage());
+    }
+  }
+
   /**
    * Starts a new thread which will check for disk error check request 
    * every 5 sec
@@ -2776,9 +2792,7 @@ public class DataNode extends Configured
               }
               if(tempFlag) {
                 try {
-                  data.checkDataDir();
-                } catch (DiskErrorException de) {
-                  handleDiskError(de.getMessage());
+                  checkDiskError();
                 } catch (Exception e) {
                   LOG.warn("Unexpected exception occurred while checking disk error  " + e);
                   checkDiskErrorThread = null;

+ 4 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNodeLayoutVersion.java

@@ -62,7 +62,10 @@ public class DataNodeLayoutVersion {
    * </ul>
    */
   public static enum Feature implements LayoutFeature {
-    FIRST_LAYOUT(-55, -53, "First datanode layout", false);
+    FIRST_LAYOUT(-55, -53, "First datanode layout", false),
+    BLOCKID_BASED_LAYOUT(-56,
+        "The block ID of a finalized block uniquely determines its position " +
+            "in the directory structure");
    
     private final FeatureInfo info;
 

+ 122 - 22
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java

@@ -18,13 +18,19 @@
 
 package org.apache.hadoop.hdfs.server.datanode;
 
+import com.google.common.collect.Lists;
+import com.google.common.util.concurrent.Futures;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.*;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.HardLink;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
@@ -35,13 +41,30 @@ import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.DiskChecker;
 
-import java.io.*;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.RandomAccessFile;
 import java.nio.channels.FileLock;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
 
 /** 
  * Data storage information file.
@@ -261,6 +284,7 @@ public class DataStorage extends Storage {
           STORAGE_DIR_CURRENT));
       bpDataDirs.add(bpRoot);
     }
+
     // mkdir for the list of BlockPoolStorage
     makeBlockPoolDataDir(bpDataDirs, null);
     BlockPoolSliceStorage bpStorage = new BlockPoolSliceStorage(
@@ -488,7 +512,7 @@ public class DataStorage extends Storage {
     
     // do upgrade
     if (this.layoutVersion > HdfsConstants.DATANODE_LAYOUT_VERSION) {
-      doUpgrade(sd, nsInfo);  // upgrade
+      doUpgrade(datanode, sd, nsInfo);  // upgrade
       return;
     }
     
@@ -523,7 +547,8 @@ public class DataStorage extends Storage {
    * @param sd  storage directory
    * @throws IOException on error
    */
-  void doUpgrade(StorageDirectory sd, NamespaceInfo nsInfo) throws IOException {
+  void doUpgrade(DataNode datanode, StorageDirectory sd, NamespaceInfo nsInfo)
+      throws IOException {
     // If the existing on-disk layout version supportes federation, simply
     // update its layout version.
     if (DataNodeLayoutVersion.supports(
@@ -568,7 +593,8 @@ public class DataStorage extends Storage {
     BlockPoolSliceStorage bpStorage = new BlockPoolSliceStorage(nsInfo.getNamespaceID(), 
         nsInfo.getBlockPoolID(), nsInfo.getCTime(), nsInfo.getClusterID());
     bpStorage.format(curDir, nsInfo);
-    linkAllBlocks(tmpDir, bbwDir, new File(curBpDir, STORAGE_DIR_CURRENT));
+    linkAllBlocks(datanode, tmpDir, bbwDir, new File(curBpDir,
+        STORAGE_DIR_CURRENT));
     
     // 4. Write version file under <SD>/current
     layoutVersion = HdfsConstants.DATANODE_LAYOUT_VERSION;
@@ -746,22 +772,22 @@ public class DataStorage extends Storage {
    *
    * @throws IOException If error occurs during hardlink
    */
-  private void linkAllBlocks(File fromDir, File fromBbwDir, File toDir)
-      throws IOException {
+  private void linkAllBlocks(DataNode datanode, File fromDir, File fromBbwDir,
+      File toDir) throws IOException {
     HardLink hardLink = new HardLink();
     // do the link
     int diskLayoutVersion = this.getLayoutVersion();
     if (DataNodeLayoutVersion.supports(
         LayoutVersion.Feature.APPEND_RBW_DIR, diskLayoutVersion)) {
       // hardlink finalized blocks in tmpDir/finalized
-      linkBlocks(new File(fromDir, STORAGE_DIR_FINALIZED), 
+      linkBlocks(datanode, new File(fromDir, STORAGE_DIR_FINALIZED),
           new File(toDir, STORAGE_DIR_FINALIZED), diskLayoutVersion, hardLink);
       // hardlink rbw blocks in tmpDir/rbw
-      linkBlocks(new File(fromDir, STORAGE_DIR_RBW), 
+      linkBlocks(datanode, new File(fromDir, STORAGE_DIR_RBW),
           new File(toDir, STORAGE_DIR_RBW), diskLayoutVersion, hardLink);
     } else { // pre-RBW version
       // hardlink finalized blocks in tmpDir
-      linkBlocks(fromDir, new File(toDir, STORAGE_DIR_FINALIZED), 
+      linkBlocks(datanode, fromDir, new File(toDir, STORAGE_DIR_FINALIZED),
           diskLayoutVersion, hardLink);      
       if (fromBbwDir.exists()) {
         /*
@@ -770,15 +796,67 @@ public class DataStorage extends Storage {
          * NOT underneath the 'current' directory in those releases.  See
          * HDFS-3731 for details.
          */
-        linkBlocks(fromBbwDir,
+        linkBlocks(datanode, fromBbwDir,
             new File(toDir, STORAGE_DIR_RBW), diskLayoutVersion, hardLink);
       }
     } 
     LOG.info( hardLink.linkStats.report() );
   }
+
+  private static class LinkArgs {
+    public File src;
+    public File dst;
+
+    public LinkArgs(File src, File dst) {
+      this.src = src;
+      this.dst = dst;
+    }
+  }
+
+  static void linkBlocks(DataNode datanode, File from, File to, int oldLV,
+      HardLink hl) throws IOException {
+    boolean upgradeToIdBasedLayout = false;
+    // If we are upgrading from a version older than the one where we introduced
+    // block ID-based layout AND we're working with the finalized directory,
+    // we'll need to upgrade from the old flat layout to the block ID-based one
+    if (oldLV > DataNodeLayoutVersion.Feature.BLOCKID_BASED_LAYOUT.getInfo().
+        getLayoutVersion() && to.getName().equals(STORAGE_DIR_FINALIZED)) {
+      upgradeToIdBasedLayout = true;
+    }
+
+    final List<LinkArgs> idBasedLayoutSingleLinks = Lists.newArrayList();
+    linkBlocksHelper(from, to, oldLV, hl, upgradeToIdBasedLayout, to,
+        idBasedLayoutSingleLinks);
+    int numLinkWorkers = datanode.getConf().getInt(
+        DFSConfigKeys.DFS_DATANODE_BLOCK_ID_LAYOUT_UPGRADE_THREADS_KEY,
+        DFSConfigKeys.DFS_DATANODE_BLOCK_ID_LAYOUT_UPGRADE_THREADS);
+    ExecutorService linkWorkers = Executors.newFixedThreadPool(numLinkWorkers);
+    final int step = idBasedLayoutSingleLinks.size() / numLinkWorkers + 1;
+    List<Future<Void>> futures = Lists.newArrayList();
+    for (int i = 0; i < idBasedLayoutSingleLinks.size(); i += step) {
+      final int iCopy = i;
+      futures.add(linkWorkers.submit(new Callable<Void>() {
+        @Override
+        public Void call() throws IOException {
+          int upperBound = Math.min(iCopy + step,
+              idBasedLayoutSingleLinks.size());
+          for (int j = iCopy; j < upperBound; j++) {
+            LinkArgs cur = idBasedLayoutSingleLinks.get(j);
+            NativeIO.link(cur.src, cur.dst);
+          }
+          return null;
+        }
+      }));
+    }
+    linkWorkers.shutdown();
+    for (Future<Void> f : futures) {
+      Futures.get(f, IOException.class);
+    }
+  }
   
-  static void linkBlocks(File from, File to, int oldLV, HardLink hl) 
-  throws IOException {
+  static void linkBlocksHelper(File from, File to, int oldLV, HardLink hl,
+  boolean upgradeToIdBasedLayout, File blockRoot,
+      List<LinkArgs> idBasedLayoutSingleLinks) throws IOException {
     if (!from.exists()) {
       return;
     }
@@ -805,9 +883,6 @@ public class DataStorage extends Storage {
     // from is a directory
     hl.linkStats.countDirs++;
     
-    if (!to.mkdirs())
-      throw new IOException("Cannot create directory " + to);
-    
     String[] blockNames = from.list(new java.io.FilenameFilter() {
       @Override
       public boolean accept(File dir, String name) {
@@ -815,12 +890,36 @@ public class DataStorage extends Storage {
       }
     });
 
+    // If we are upgrading to block ID-based layout, we don't want to recreate
+    // any subdirs from the source that contain blocks, since we have a new
+    // directory structure
+    if (!upgradeToIdBasedLayout || !to.getName().startsWith(
+        BLOCK_SUBDIR_PREFIX)) {
+      if (!to.mkdirs())
+        throw new IOException("Cannot create directory " + to);
+    }
+
     // Block files just need hard links with the same file names
     // but a different directory
     if (blockNames.length > 0) {
-      HardLink.createHardLinkMult(from, blockNames, to);
-      hl.linkStats.countMultLinks++;
-      hl.linkStats.countFilesMultLinks += blockNames.length;
+      if (upgradeToIdBasedLayout) {
+        for (String blockName : blockNames) {
+          long blockId = Block.getBlockId(blockName);
+          File blockLocation = DatanodeUtil.idToBlockDir(blockRoot, blockId);
+          if (!blockLocation.exists()) {
+            if (!blockLocation.mkdirs()) {
+              throw new IOException("Failed to mkdirs " + blockLocation);
+            }
+          }
+          idBasedLayoutSingleLinks.add(new LinkArgs(new File(from, blockName),
+              new File(blockLocation, blockName)));
+          hl.linkStats.countSingleLinks++;
+        }
+      } else {
+        HardLink.createHardLinkMult(from, blockNames, to);
+        hl.linkStats.countMultLinks++;
+        hl.linkStats.countFilesMultLinks += blockNames.length;
+      }
     } else {
       hl.linkStats.countEmptyDirs++;
     }
@@ -834,8 +933,9 @@ public class DataStorage extends Storage {
         }
       });
     for(int i = 0; i < otherNames.length; i++)
-      linkBlocks(new File(from, otherNames[i]), 
-          new File(to, otherNames[i]), oldLV, hl);
+      linkBlocksHelper(new File(from, otherNames[i]),
+          new File(to, otherNames[i]), oldLV, hl, upgradeToIdBasedLayout,
+          blockRoot, idBasedLayoutSingleLinks);
   }
 
   /**

+ 36 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeUtil.java

@@ -30,6 +30,8 @@ public class DatanodeUtil {
 
   public static final String DISK_ERROR = "Possible disk error: ";
 
+  private static final String SEP = System.getProperty("file.separator");
+
   /** Get the cause of an I/O exception if caused by a possible disk error
    * @param ioe an I/O exception
    * @return cause if the I/O exception is caused by a possible disk error;
@@ -78,4 +80,38 @@ public class DatanodeUtil {
   public static File getUnlinkTmpFile(File f) {
     return new File(f.getParentFile(), f.getName()+UNLINK_BLOCK_SUFFIX);
   }
+
+  /**
+   * Checks whether there are any files anywhere in the directory tree rooted
+   * at dir (directories don't count as files). dir must exist
+   * @return true if there are no files
+   * @throws IOException if unable to list subdirectories
+   */
+  public static boolean dirNoFilesRecursive(File dir) throws IOException {
+    File[] contents = dir.listFiles();
+    if (contents == null) {
+      throw new IOException("Cannot list contents of " + dir);
+    }
+    for (File f : contents) {
+      if (!f.isDirectory() || (f.isDirectory() && !dirNoFilesRecursive(f))) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /**
+   * Get the directory where a finalized block with this ID should be stored.
+   * Do not attempt to create the directory.
+   * @param root the root directory where finalized blocks are stored
+   * @param blockId
+   * @return
+   */
+  public static File idToBlockDir(File root, long blockId) {
+    int d1 = (int)((blockId >> 16) & 0xff);
+    int d2 = (int)((blockId >> 8) & 0xff);
+    String path = DataStorage.BLOCK_SUBDIR_PREFIX + d1 + SEP +
+        DataStorage.BLOCK_SUBDIR_PREFIX + d2;
+    return new File(root, path);
+  }
 }

+ 22 - 40
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java

@@ -54,10 +54,10 @@ abstract public class ReplicaInfo extends Block implements Replica {
   private File baseDir;
   
   /**
-   * Ints representing the sub directory path from base dir to the directory
-   * containing this replica.
+   * Whether or not this replica's parent directory includes subdirs, in which
+   * case we can generate them based on the replica's block ID
    */
-  private int[] subDirs;
+  private boolean hasSubdirs;
   
   private static final Map<String, File> internedBaseDirs = new HashMap<String, File>();
 
@@ -151,18 +151,8 @@ abstract public class ReplicaInfo extends Block implements Replica {
    * @return the parent directory path where this replica is located
    */
   File getDir() {
-    if (subDirs == null) {
-      return null;
-    }
-
-    StringBuilder sb = new StringBuilder();
-    for (int i : subDirs) {
-      sb.append(DataStorage.BLOCK_SUBDIR_PREFIX);
-      sb.append(i);
-      sb.append("/");
-    }
-    File ret = new File(baseDir, sb.toString());
-    return ret;
+    return hasSubdirs ? DatanodeUtil.idToBlockDir(baseDir,
+        getBlockId()) : baseDir;
   }
 
   /**
@@ -175,54 +165,46 @@ abstract public class ReplicaInfo extends Block implements Replica {
 
   private void setDirInternal(File dir) {
     if (dir == null) {
-      subDirs = null;
       baseDir = null;
       return;
     }
 
-    ReplicaDirInfo replicaDirInfo = parseSubDirs(dir);
-    this.subDirs = replicaDirInfo.subDirs;
+    ReplicaDirInfo dirInfo = parseBaseDir(dir);
+    this.hasSubdirs = dirInfo.hasSubidrs;
     
     synchronized (internedBaseDirs) {
-      if (!internedBaseDirs.containsKey(replicaDirInfo.baseDirPath)) {
+      if (!internedBaseDirs.containsKey(dirInfo.baseDirPath)) {
         // Create a new String path of this file and make a brand new File object
         // to guarantee we drop the reference to the underlying char[] storage.
-        File baseDir = new File(replicaDirInfo.baseDirPath);
-        internedBaseDirs.put(replicaDirInfo.baseDirPath, baseDir);
+        File baseDir = new File(dirInfo.baseDirPath);
+        internedBaseDirs.put(dirInfo.baseDirPath, baseDir);
       }
-      this.baseDir = internedBaseDirs.get(replicaDirInfo.baseDirPath);
+      this.baseDir = internedBaseDirs.get(dirInfo.baseDirPath);
     }
   }
-  
+
   @VisibleForTesting
   public static class ReplicaDirInfo {
-    @VisibleForTesting
     public String baseDirPath;
-    
-    @VisibleForTesting
-    public int[] subDirs;
+    public boolean hasSubidrs;
+
+    public ReplicaDirInfo (String baseDirPath, boolean hasSubidrs) {
+      this.baseDirPath = baseDirPath;
+      this.hasSubidrs = hasSubidrs;
+    }
   }
   
   @VisibleForTesting
-  public static ReplicaDirInfo parseSubDirs(File dir) {
-    ReplicaDirInfo ret = new ReplicaDirInfo();
+  public static ReplicaDirInfo parseBaseDir(File dir) {
     
     File currentDir = dir;
-    List<Integer> subDirList = new ArrayList<Integer>();
+    boolean hasSubdirs = false;
     while (currentDir.getName().startsWith(DataStorage.BLOCK_SUBDIR_PREFIX)) {
-      // Prepend the integer into the list.
-      subDirList.add(0, Integer.parseInt(currentDir.getName().replaceFirst(
-          DataStorage.BLOCK_SUBDIR_PREFIX, "")));
+      hasSubdirs = true;
       currentDir = currentDir.getParentFile();
     }
-    ret.subDirs = new int[subDirList.size()];
-    for (int i = 0; i < subDirList.size(); i++) {
-      ret.subDirs[i] = subDirList.get(i);
-    }
-    
-    ret.baseDirPath = currentDir.getAbsolutePath();
     
-    return ret;
+    return new ReplicaDirInfo(currentDir.getAbsolutePath(), hasSubdirs);
   }
 
   /**

+ 68 - 28
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java

@@ -59,7 +59,8 @@ class BlockPoolSlice {
   private final String bpid;
   private final FsVolumeImpl volume; // volume to which this BlockPool belongs to
   private final File currentDir; // StorageDirectory/current/bpid/current
-  private final LDir finalizedDir; // directory store Finalized replica
+  // directory where finalized replicas are stored
+  private final File finalizedDir;
   private final File rbwDir; // directory store RBW replica
   private final File tmpDir; // directory store Temporary replica
   private static final String DU_CACHE_FILE = "dfsUsed";
@@ -82,8 +83,13 @@ class BlockPoolSlice {
     this.bpid = bpid;
     this.volume = volume;
     this.currentDir = new File(bpDir, DataStorage.STORAGE_DIR_CURRENT); 
-    final File finalizedDir = new File(
+    this.finalizedDir = new File(
         currentDir, DataStorage.STORAGE_DIR_FINALIZED);
+    if (!this.finalizedDir.exists()) {
+      if (!this.finalizedDir.mkdirs()) {
+        throw new IOException("Failed to mkdirs " + this.finalizedDir);
+      }
+    }
 
     // Files that were being written when the datanode was last shutdown
     // are now moved back to the data directory. It is possible that
@@ -95,10 +101,6 @@ class BlockPoolSlice {
       FileUtil.fullyDelete(tmpDir);
     }
     this.rbwDir = new File(currentDir, DataStorage.STORAGE_DIR_RBW);
-    final int maxBlocksPerDir = conf.getInt(
-        DFSConfigKeys.DFS_DATANODE_NUMBLOCKS_KEY,
-        DFSConfigKeys.DFS_DATANODE_NUMBLOCKS_DEFAULT);
-    this.finalizedDir = new LDir(finalizedDir, maxBlocksPerDir);
     if (!rbwDir.mkdirs()) {  // create rbw directory if not exist
       if (!rbwDir.isDirectory()) {
         throw new IOException("Mkdirs failed to create " + rbwDir.toString());
@@ -131,7 +133,7 @@ class BlockPoolSlice {
   }
 
   File getFinalizedDir() {
-    return finalizedDir.dir;
+    return finalizedDir;
   }
   
   File getRbwDir() {
@@ -239,25 +241,56 @@ class BlockPoolSlice {
   }
 
   File addBlock(Block b, File f) throws IOException {
-    File blockFile = finalizedDir.addBlock(b, f);
+    File blockDir = DatanodeUtil.idToBlockDir(finalizedDir, b.getBlockId());
+    if (!blockDir.exists()) {
+      if (!blockDir.mkdirs()) {
+        throw new IOException("Failed to mkdirs " + blockDir);
+      }
+    }
+    File blockFile = FsDatasetImpl.moveBlockFiles(b, f, blockDir);
     File metaFile = FsDatasetUtil.getMetaFile(blockFile, b.getGenerationStamp());
     dfsUsage.incDfsUsed(b.getNumBytes()+metaFile.length());
     return blockFile;
   }
     
   void checkDirs() throws DiskErrorException {
-    finalizedDir.checkDirTree();
+    DiskChecker.checkDirs(finalizedDir);
     DiskChecker.checkDir(tmpDir);
     DiskChecker.checkDir(rbwDir);
   }
     
   void getVolumeMap(ReplicaMap volumeMap) throws IOException {
     // add finalized replicas
-    finalizedDir.getVolumeMap(bpid, volumeMap, volume);
+    addToReplicasMap(volumeMap, finalizedDir, true);
     // add rbw replicas
     addToReplicasMap(volumeMap, rbwDir, false);
   }
 
+  /**
+   * Recover an unlinked tmp file on datanode restart. If the original block
+   * does not exist, then the tmp file is renamed to be the
+   * original file name and the original name is returned; otherwise the tmp
+   * file is deleted and null is returned.
+   */
+  File recoverTempUnlinkedBlock(File unlinkedTmp) throws IOException {
+    File blockFile = FsDatasetUtil.getOrigFile(unlinkedTmp);
+    if (blockFile.exists()) {
+      // If the original block file still exists, then no recovery is needed.
+      if (!unlinkedTmp.delete()) {
+        throw new IOException("Unable to cleanup unlinked tmp file " +
+            unlinkedTmp);
+      }
+      return null;
+    } else {
+      if (!unlinkedTmp.renameTo(blockFile)) {
+        throw new IOException("Unable to rename unlinked tmp file " +
+            unlinkedTmp);
+      }
+      return blockFile;
+    }
+  }
+
+
   /**
    * Add replicas under the given directory to the volume map
    * @param volumeMap the replicas map
@@ -267,23 +300,34 @@ class BlockPoolSlice {
    */
   void addToReplicasMap(ReplicaMap volumeMap, File dir, boolean isFinalized
       ) throws IOException {
-    File blockFiles[] = FileUtil.listFiles(dir);
-    for (File blockFile : blockFiles) {
-      if (!Block.isBlockFilename(blockFile))
+    File files[] = FileUtil.listFiles(dir);
+    for (File file : files) {
+      if (file.isDirectory()) {
+        addToReplicasMap(volumeMap, file, isFinalized);
+      }
+
+      if (isFinalized && FsDatasetUtil.isUnlinkTmpFile(file)) {
+        file = recoverTempUnlinkedBlock(file);
+        if (file == null) { // the original block still exists, so we cover it
+          // in another iteration and can continue here
+          continue;
+        }
+      }
+      if (!Block.isBlockFilename(file))
         continue;
       
       long genStamp = FsDatasetUtil.getGenerationStampFromFile(
-          blockFiles, blockFile);
-      long blockId = Block.filename2id(blockFile.getName());
+          files, file);
+      long blockId = Block.filename2id(file.getName());
       ReplicaInfo newReplica = null;
       if (isFinalized) {
         newReplica = new FinalizedReplica(blockId, 
-            blockFile.length(), genStamp, volume, blockFile.getParentFile());
+            file.length(), genStamp, volume, file.getParentFile());
       } else {
 
         boolean loadRwr = true;
-        File restartMeta = new File(blockFile.getParent()  +
-            File.pathSeparator + "." + blockFile.getName() + ".restart");
+        File restartMeta = new File(file.getParent()  +
+            File.pathSeparator + "." + file.getName() + ".restart");
         Scanner sc = null;
         try {
           sc = new Scanner(restartMeta);
@@ -291,8 +335,8 @@ class BlockPoolSlice {
           if (sc.hasNextLong() && (sc.nextLong() > Time.now())) {
             // It didn't expire. Load the replica as a RBW.
             newReplica = new ReplicaBeingWritten(blockId,
-                validateIntegrityAndSetLength(blockFile, genStamp), 
-                genStamp, volume, blockFile.getParentFile(), null);
+                validateIntegrityAndSetLength(file, genStamp),
+                genStamp, volume, file.getParentFile(), null);
             loadRwr = false;
           }
           sc.close();
@@ -301,7 +345,7 @@ class BlockPoolSlice {
               restartMeta.getPath());
           }
         } catch (FileNotFoundException fnfe) {
-          // nothing to do here
+          // nothing to do hereFile dir =
         } finally {
           if (sc != null) {
             sc.close();
@@ -310,15 +354,15 @@ class BlockPoolSlice {
         // Restart meta doesn't exist or expired.
         if (loadRwr) {
           newReplica = new ReplicaWaitingToBeRecovered(blockId,
-              validateIntegrityAndSetLength(blockFile, genStamp), 
-              genStamp, volume, blockFile.getParentFile());
+              validateIntegrityAndSetLength(file, genStamp),
+              genStamp, volume, file.getParentFile());
         }
       }
 
       ReplicaInfo oldReplica = volumeMap.add(bpid, newReplica);
       if (oldReplica != null) {
         FsDatasetImpl.LOG.warn("Two block files with the same block id exist " +
-            "on disk: " + oldReplica.getBlockFile() + " and " + blockFile );
+            "on disk: " + oldReplica.getBlockFile() + " and " + file );
       }
     }
   }
@@ -405,10 +449,6 @@ class BlockPoolSlice {
     }
   }
     
-  void clearPath(File f) {
-    finalizedDir.clearPath(f);
-  }
-    
   @Override
   public String toString() {
     return currentDir.getAbsolutePath();

+ 1 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java

@@ -1151,7 +1151,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
         return f;
    
       // if file is not null, but doesn't exist - possibly disk failed
-      datanode.checkDiskError();
+      datanode.checkDiskErrorAsync();
     }
     
     if (LOG.isDebugEnabled()) {
@@ -1224,13 +1224,6 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
               +  ". Parent not found for file " + f);
           continue;
         }
-        ReplicaState replicaState = info.getState();
-        if (replicaState == ReplicaState.FINALIZED || 
-            (replicaState == ReplicaState.RUR && 
-                ((ReplicaUnderRecovery)info).getOriginalReplica().getState() == 
-                  ReplicaState.FINALIZED)) {
-          v.clearPath(bpid, parent);
-        }
         volumeMap.remove(bpid, invalidBlks[i]);
       }
 

+ 5 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java

@@ -37,6 +37,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
+import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
@@ -235,10 +236,6 @@ class FsVolumeImpl implements FsVolumeSpi {
     // dfsUsage.incDfsUsed(b.getNumBytes()+metaFile.length());
     bp.addToReplicasMap(volumeMap, dir, isFinalized);
   }
-  
-  void clearPath(String bpid, File f) throws IOException {
-    getBlockPoolSlice(bpid).clearPath(f);
-  }
 
   @Override
   public String toString() {
@@ -274,7 +271,8 @@ class FsVolumeImpl implements FsVolumeSpi {
     File finalizedDir = new File(bpCurrentDir,
         DataStorage.STORAGE_DIR_FINALIZED);
     File rbwDir = new File(bpCurrentDir, DataStorage.STORAGE_DIR_RBW);
-    if (finalizedDir.exists() && FileUtil.list(finalizedDir).length != 0) {
+    if (finalizedDir.exists() && !DatanodeUtil.dirNoFilesRecursive(
+        finalizedDir)) {
       return false;
     }
     if (rbwDir.exists() && FileUtil.list(rbwDir).length != 0) {
@@ -301,7 +299,8 @@ class FsVolumeImpl implements FsVolumeSpi {
       if (!rbwDir.delete()) {
         throw new IOException("Failed to delete " + rbwDir);
       }
-      if (!finalizedDir.delete()) {
+      if (!DatanodeUtil.dirNoFilesRecursive(finalizedDir) ||
+          !FileUtil.fullyDelete(finalizedDir)) {
         throw new IOException("Failed to delete " + finalizedDir);
       }
       FileUtil.fullyDelete(tmpDir);

+ 0 - 228
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LDir.java

@@ -1,228 +0,0 @@
-/**
- * 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.datanode.fsdataset.impl;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.datanode.DataStorage;
-import org.apache.hadoop.util.DiskChecker;
-import org.apache.hadoop.util.DiskChecker.DiskErrorException;
-
-/**
- * A node type that can be built into a tree reflecting the
- * hierarchy of replicas on the local disk.
- */
-class LDir {
-  final File dir;
-  final int maxBlocksPerDir;
-
-  private int numBlocks = 0;
-  private LDir[] children = null;
-  private int lastChildIdx = 0;
-
-  LDir(File dir, int maxBlocksPerDir) throws IOException {
-    this.dir = dir;
-    this.maxBlocksPerDir = maxBlocksPerDir;
-
-    if (!dir.exists()) {
-      if (!dir.mkdirs()) {
-        throw new IOException("Failed to mkdirs " + dir);
-      }
-    } else {
-      File[] files = FileUtil.listFiles(dir); 
-      List<LDir> dirList = new ArrayList<LDir>();
-      for (int idx = 0; idx < files.length; idx++) {
-        if (files[idx].isDirectory()) {
-          dirList.add(new LDir(files[idx], maxBlocksPerDir));
-        } else if (Block.isBlockFilename(files[idx])) {
-          numBlocks++;
-        }
-      }
-      if (dirList.size() > 0) {
-        children = dirList.toArray(new LDir[dirList.size()]);
-      }
-    }
-  }
-      
-  File addBlock(Block b, File src) throws IOException {
-    //First try without creating subdirectories
-    File file = addBlock(b, src, false, false);          
-    return (file != null) ? file : addBlock(b, src, true, true);
-  }
-
-  private File addBlock(Block b, File src, boolean createOk, boolean resetIdx
-      ) throws IOException {
-    if (numBlocks < maxBlocksPerDir) {
-      final File dest = FsDatasetImpl.moveBlockFiles(b, src, dir);
-      numBlocks += 1;
-      return dest;
-    }
-          
-    if (lastChildIdx < 0 && resetIdx) {
-      //reset so that all children will be checked
-      lastChildIdx = DFSUtil.getRandom().nextInt(children.length);              
-    }
-          
-    if (lastChildIdx >= 0 && children != null) {
-      //Check if any child-tree has room for a block.
-      for (int i=0; i < children.length; i++) {
-        int idx = (lastChildIdx + i)%children.length;
-        File file = children[idx].addBlock(b, src, false, resetIdx);
-        if (file != null) {
-          lastChildIdx = idx;
-          return file; 
-        }
-      }
-      lastChildIdx = -1;
-    }
-          
-    if (!createOk) {
-      return null;
-    }
-          
-    if (children == null || children.length == 0) {
-      children = new LDir[maxBlocksPerDir];
-      for (int idx = 0; idx < maxBlocksPerDir; idx++) {
-        final File sub = new File(dir, DataStorage.BLOCK_SUBDIR_PREFIX+idx);
-        children[idx] = new LDir(sub, maxBlocksPerDir);
-      }
-    }
-          
-    //now pick a child randomly for creating a new set of subdirs.
-    lastChildIdx = DFSUtil.getRandom().nextInt(children.length);
-    return children[ lastChildIdx ].addBlock(b, src, true, false); 
-  }
-
-  void getVolumeMap(String bpid, ReplicaMap volumeMap, FsVolumeImpl volume
-      ) throws IOException {
-    if (children != null) {
-      for (int i = 0; i < children.length; i++) {
-        children[i].getVolumeMap(bpid, volumeMap, volume);
-      }
-    }
-
-    recoverTempUnlinkedBlock();
-    volume.addToReplicasMap(bpid, volumeMap, dir, true);
-  }
-      
-  /**
-   * Recover unlinked tmp files on datanode restart. If the original block
-   * does not exist, then the tmp file is renamed to be the
-   * original file name; otherwise the tmp file is deleted.
-   */
-  private void recoverTempUnlinkedBlock() throws IOException {
-    File files[] = FileUtil.listFiles(dir);
-    for (File file : files) {
-      if (!FsDatasetUtil.isUnlinkTmpFile(file)) {
-        continue;
-      }
-      File blockFile = FsDatasetUtil.getOrigFile(file);
-      if (blockFile.exists()) {
-        // If the original block file still exists, then no recovery  is needed.
-        if (!file.delete()) {
-          throw new IOException("Unable to cleanup unlinked tmp file " + file);
-        }
-      } else {
-        if (!file.renameTo(blockFile)) {
-          throw new IOException("Unable to cleanup detached file " + file);
-        }
-      }
-    }
-  }
-  
-  /**
-   * check if a data diretory is healthy
-   * @throws DiskErrorException
-   */
-  void checkDirTree() throws DiskErrorException {
-    DiskChecker.checkDir(dir);
-          
-    if (children != null) {
-      for (int i = 0; i < children.length; i++) {
-        children[i].checkDirTree();
-      }
-    }
-  }
-      
-  void clearPath(File f) {
-    String root = dir.getAbsolutePath();
-    String dir = f.getAbsolutePath();
-    if (dir.startsWith(root)) {
-      String[] dirNames = dir.substring(root.length()).
-        split(File.separator + DataStorage.BLOCK_SUBDIR_PREFIX);
-      if (clearPath(f, dirNames, 1))
-        return;
-    }
-    clearPath(f, null, -1);
-  }
-      
-  /**
-   * dirNames is an array of string integers derived from
-   * usual directory structure data/subdirN/subdirXY/subdirM ...
-   * If dirName array is non-null, we only check the child at 
-   * the children[dirNames[idx]]. This avoids iterating over
-   * children in common case. If directory structure changes 
-   * in later versions, we need to revisit this.
-   */
-  private boolean clearPath(File f, String[] dirNames, int idx) {
-    if ((dirNames == null || idx == dirNames.length) &&
-        dir.compareTo(f) == 0) {
-      numBlocks--;
-      return true;
-    }
-        
-    if (dirNames != null) {
-      //guess the child index from the directory name
-      if (idx > (dirNames.length - 1) || children == null) {
-        return false;
-      }
-      int childIdx; 
-      try {
-        childIdx = Integer.parseInt(dirNames[idx]);
-      } catch (NumberFormatException ignored) {
-        // layout changed? we could print a warning.
-        return false;
-      }
-      return (childIdx >= 0 && childIdx < children.length) ?
-        children[childIdx].clearPath(f, dirNames, idx+1) : false;
-    }
-
-    //guesses failed. back to blind iteration.
-    if (children != null) {
-      for(int i=0; i < children.length; i++) {
-        if (children[i].clearPath(f, null, -1)){
-          return true;
-        }
-      }
-    }
-    return false;
-  }
-
-  @Override
-  public String toString() {
-    return "FSDir{dir=" + dir + ", children="
-        + (children == null ? null : Arrays.asList(children)) + "}";
-  }
-}

+ 0 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -1085,9 +1085,6 @@ public class FSDirectory implements Closeable {
       count++;
     }
     
-    // update inodeMap
-    removeFromInodeMap(Arrays.asList(allSrcInodes));
-    
     trgInode.setModificationTime(timestamp, trgLatestSnapshot);
     trgParent.updateModificationTime(timestamp, trgLatestSnapshot);
     // update quota on the parent directory ('count' files removed, 0 space)

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

@@ -4358,8 +4358,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
           // Otherwise fsck will report these blocks as MISSING, especially if the
           // blocksReceived from Datanodes take a long time to arrive.
           for (int i = 0; i < trimmedTargets.size(); i++) {
-            trimmedTargets.get(i).addBlock(
-              trimmedStorages.get(i), storedBlock);
+            DatanodeStorageInfo storageInfo =
+                trimmedTargets.get(i).getStorageInfo(trimmedStorages.get(i));
+            if (storageInfo != null) {
+              storageInfo.addBlock(storedBlock);
+            }
           }
         }
 
@@ -5838,7 +5841,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   }
 
   public void processIncrementalBlockReport(final DatanodeID nodeID,
-      final String poolId, final StorageReceivedDeletedBlocks srdb)
+      final StorageReceivedDeletedBlocks srdb)
       throws IOException {
     writeLock();
     try {

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

@@ -1065,7 +1065,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
       // for the same node and storage, so the value returned by the last
       // call of this loop is the final updated value for noStaleStorage.
       //
-      noStaleStorages = bm.processReport(nodeReg, r.getStorage(), poolId, blocks);
+      noStaleStorages = bm.processReport(nodeReg, r.getStorage(), blocks);
       metrics.incrStorageBlockReportOps();
     }
 
@@ -1101,7 +1101,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
           +" blocks.");
     }
     for(StorageReceivedDeletedBlocks r : receivedAndDeletedBlocks) {
-      namesystem.processIncrementalBlockReport(nodeReg, poolId, r);
+      namesystem.processIncrementalBlockReport(nodeReg, r);
     }
   }
 

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

@@ -2053,4 +2053,12 @@
   </description>
 </property>
 
+<property>
+  <name>dfs.datanode.block.id.layout.upgrade.threads</name>
+  <value>12</value>
+  <description>The number of threads to use when creating hard links from
+    current to previous blocks during upgrade of a DataNode to block ID-based
+    block layout (see HDFS-6482 for details on the layout).</description>
+</property>
+
 </configuration>

+ 8 - 5
hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HdfsNfsGateway.apt.vm

@@ -47,18 +47,21 @@ HDFS NFS Gateway
    The NFS-gateway uses proxy user to proxy all the users accessing the NFS mounts. 
    In non-secure mode, the user running the gateway is the proxy user, while in secure mode the
    user in Kerberos keytab is the proxy user. Suppose the proxy user is 'nfsserver'
-   and users belonging to the groups 'nfs-users1'
-   and 'nfs-users2' use the NFS mounts, then in core-site.xml of the NameNode, the following
+   and users belonging to the groups 'users-group1'
+   and 'users-group2' use the NFS mounts, then in core-site.xml of the NameNode, the following
    two properities must be set and only NameNode needs restart after the configuration change
    (NOTE: replace the string 'nfsserver' with the proxy user name in your cluster):
 
 ----
 <property>
   <name>hadoop.proxyuser.nfsserver.groups</name>
-  <value>nfs-users1,nfs-users2</value>
+  <value>root,users-group1,users-group2</value>
   <description>
-         The 'nfsserver' user is allowed to proxy all members of the 'nfs-users1' and 
-         'nfs-users2' groups. Set this to '*' to allow nfsserver user to proxy any group.
+         The 'nfsserver' user is allowed to proxy all members of the 'users-group1' and 
+         'users-group2' groups. Note that in most cases you will need to include the
+         group "root" because the user "root" (which usually belonges to "root" group) will
+         generally be the user that initially executes the mount on the NFS client system. 
+         Set this to '*' to allow nfsserver user to proxy any group.
   </description>
 </property>
 ----

+ 28 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java

@@ -2353,8 +2353,8 @@ public class MiniDFSCluster {
    * @return data file corresponding to the block
    */
   public static File getBlockFile(File storageDir, ExtendedBlock blk) {
-    return new File(getFinalizedDir(storageDir, blk.getBlockPoolId()), 
-        blk.getBlockName());
+    return new File(DatanodeUtil.idToBlockDir(getFinalizedDir(storageDir,
+        blk.getBlockPoolId()), blk.getBlockId()), blk.getBlockName());
   }
 
   /**
@@ -2364,10 +2364,32 @@ public class MiniDFSCluster {
    * @return metadata file corresponding to the block
    */
   public static File getBlockMetadataFile(File storageDir, ExtendedBlock blk) {
-    return new File(getFinalizedDir(storageDir, blk.getBlockPoolId()), 
-        blk.getBlockName() + "_" + blk.getGenerationStamp() +
-        Block.METADATA_EXTENSION);
-    
+    return new File(DatanodeUtil.idToBlockDir(getFinalizedDir(storageDir,
+        blk.getBlockPoolId()), blk.getBlockId()), blk.getBlockName() + "_" +
+        blk.getGenerationStamp() + Block.METADATA_EXTENSION);
+  }
+
+  /**
+   * Return all block metadata files in given directory (recursive search)
+   */
+  public static List<File> getAllBlockMetadataFiles(File storageDir) {
+    List<File> results = new ArrayList<File>();
+    File[] files = storageDir.listFiles();
+    if (files == null) {
+      return null;
+    }
+    for (File f : files) {
+      if (f.getName().startsWith("blk_") && f.getName().endsWith(
+          Block.METADATA_EXTENSION)) {
+        results.add(f);
+      } else if (f.isDirectory()) {
+        List<File> subdirResults = getAllBlockMetadataFiles(f);
+        if (subdirResults != null) {
+          results.addAll(subdirResults);
+        }
+      }
+    }
+    return results;
   }
 
   /**

+ 11 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientFailover.java

@@ -52,6 +52,7 @@ import org.apache.hadoop.io.retry.DefaultFailoverProxyProvider;
 import org.apache.hadoop.io.retry.FailoverProxyProvider;
 import org.apache.hadoop.net.ConnectTimeoutException;
 import org.apache.hadoop.net.StandardSocketFactory;
+import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.StringUtils;
@@ -89,6 +90,11 @@ public class TestDFSClientFailover {
     cluster.shutdown();
   }
 
+  @After
+  public void clearConfig() {
+    SecurityUtil.setTokenServiceUseIp(true);
+  }
+
   /**
    * Make sure that client failover works when an active NN dies and the standby
    * takes over.
@@ -323,6 +329,7 @@ public class TestDFSClientFailover {
   /**
    * Test to verify legacy proxy providers are correctly wrapped.
    */
+  @Test
   public void testWrappedFailoverProxyProvider() throws Exception {
     // setup the config with the dummy provider class
     Configuration config = new HdfsConfiguration(conf);
@@ -332,6 +339,9 @@ public class TestDFSClientFailover {
         DummyLegacyFailoverProxyProvider.class.getName());
     Path p = new Path("hdfs://" + logicalName + "/");
 
+    // not to use IP address for token service
+    SecurityUtil.setTokenServiceUseIp(false);
+
     // Logical URI should be used.
     assertTrue("Legacy proxy providers should use logical URI.",
         HAUtil.useLogicalUri(config, p.toUri()));
@@ -340,6 +350,7 @@ public class TestDFSClientFailover {
   /**
    * Test to verify IPFailoverProxyProvider is not requiring logical URI.
    */
+  @Test
   public void testIPFailoverProxyProviderLogicalUri() throws Exception {
     // setup the config with the IP failover proxy provider class
     Configuration config = new HdfsConfiguration(conf);

+ 5 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSFinalize.java

@@ -79,8 +79,8 @@ public class TestDFSFinalize {
     File dnCurDirs[] = new File[dataNodeDirs.length];
     for (int i = 0; i < dataNodeDirs.length; i++) {
       dnCurDirs[i] = new File(dataNodeDirs[i],"current");
-      assertEquals(UpgradeUtilities.checksumContents(DATA_NODE, dnCurDirs[i]),
-                   UpgradeUtilities.checksumMasterDataNodeContents());
+      assertEquals(UpgradeUtilities.checksumContents(DATA_NODE, dnCurDirs[i],
+              false), UpgradeUtilities.checksumMasterDataNodeContents());
     }
     for (int i = 0; i < nameNodeDirs.length; i++) {
       assertFalse(new File(nameNodeDirs[i],"previous").isDirectory());
@@ -96,8 +96,9 @@ public class TestDFSFinalize {
         assertFalse(new File(bpRoot,"previous").isDirectory());
         
         File bpCurFinalizeDir = new File(bpRoot,"current/"+DataStorage.STORAGE_DIR_FINALIZED);
-        assertEquals(UpgradeUtilities.checksumContents(DATA_NODE, bpCurFinalizeDir),
-                     UpgradeUtilities.checksumMasterBlockPoolFinalizedContents());
+        assertEquals(UpgradeUtilities.checksumContents(DATA_NODE,
+                bpCurFinalizeDir, true),
+                UpgradeUtilities.checksumMasterBlockPoolFinalizedContents());
       }
     }
   }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java

@@ -81,7 +81,7 @@ public class TestDFSRollback {
         break;
       case DATA_NODE:
         assertEquals(
-            UpgradeUtilities.checksumContents(nodeType, curDir),
+            UpgradeUtilities.checksumContents(nodeType, curDir, false),
             UpgradeUtilities.checksumMasterDataNodeContents());
         break;
       }

+ 9 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStorageStateRecovery.java

@@ -239,7 +239,7 @@ public class TestDFSStorageStateRecovery {
         assertTrue(new File(baseDirs[i],"previous").isDirectory());
         assertEquals(
                      UpgradeUtilities.checksumContents(
-                                                       NAME_NODE, new File(baseDirs[i],"previous")),
+                     NAME_NODE, new File(baseDirs[i],"previous"), false),
                      UpgradeUtilities.checksumMasterNameNodeContents());
       }
     }
@@ -259,7 +259,8 @@ public class TestDFSStorageStateRecovery {
     if (currentShouldExist) {
       for (int i = 0; i < baseDirs.length; i++) {
         assertEquals(
-                     UpgradeUtilities.checksumContents(DATA_NODE, new File(baseDirs[i],"current")),
+                     UpgradeUtilities.checksumContents(DATA_NODE,
+                     new File(baseDirs[i],"current"), false),
                      UpgradeUtilities.checksumMasterDataNodeContents());
       }
     }
@@ -267,7 +268,8 @@ public class TestDFSStorageStateRecovery {
       for (int i = 0; i < baseDirs.length; i++) {
         assertTrue(new File(baseDirs[i],"previous").isDirectory());
         assertEquals(
-                     UpgradeUtilities.checksumContents(DATA_NODE, new File(baseDirs[i],"previous")),
+                     UpgradeUtilities.checksumContents(DATA_NODE,
+                     new File(baseDirs[i],"previous"), false),
                      UpgradeUtilities.checksumMasterDataNodeContents());
       }
     }
@@ -290,8 +292,8 @@ public class TestDFSStorageStateRecovery {
     if (currentShouldExist) {
       for (int i = 0; i < baseDirs.length; i++) {
         File bpCurDir = new File(baseDirs[i], Storage.STORAGE_DIR_CURRENT);
-        assertEquals(UpgradeUtilities.checksumContents(DATA_NODE, bpCurDir),
-                     UpgradeUtilities.checksumMasterBlockPoolContents());
+        assertEquals(UpgradeUtilities.checksumContents(DATA_NODE, bpCurDir,
+                false), UpgradeUtilities.checksumMasterBlockPoolContents());
       }
     }
     if (previousShouldExist) {
@@ -299,8 +301,8 @@ public class TestDFSStorageStateRecovery {
         File bpPrevDir = new File(baseDirs[i], Storage.STORAGE_DIR_PREVIOUS);
         assertTrue(bpPrevDir.isDirectory());
         assertEquals(
-                     UpgradeUtilities.checksumContents(DATA_NODE, bpPrevDir),
-                     UpgradeUtilities.checksumMasterBlockPoolContents());
+                     UpgradeUtilities.checksumContents(DATA_NODE, bpPrevDir,
+                     false), UpgradeUtilities.checksumMasterBlockPoolContents());
       }
     }
   }

+ 7 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgrade.java

@@ -100,7 +100,7 @@ public class TestDFSUpgrade {
       
       File previous = new File(baseDir, "previous");
       assertExists(previous);
-      assertEquals(UpgradeUtilities.checksumContents(NAME_NODE, previous),
+      assertEquals(UpgradeUtilities.checksumContents(NAME_NODE, previous, false),
           UpgradeUtilities.checksumMasterNameNodeContents());
     }
   }
@@ -114,23 +114,25 @@ public class TestDFSUpgrade {
   void checkDataNode(String[] baseDirs, String bpid) throws IOException {
     for (int i = 0; i < baseDirs.length; i++) {
       File current = new File(baseDirs[i], "current/" + bpid + "/current");
-      assertEquals(UpgradeUtilities.checksumContents(DATA_NODE, current),
+      assertEquals(UpgradeUtilities.checksumContents(DATA_NODE, current, false),
         UpgradeUtilities.checksumMasterDataNodeContents());
       
       // block files are placed under <sd>/current/<bpid>/current/finalized
       File currentFinalized = 
         MiniDFSCluster.getFinalizedDir(new File(baseDirs[i]), bpid);
-      assertEquals(UpgradeUtilities.checksumContents(DATA_NODE, currentFinalized),
+      assertEquals(UpgradeUtilities.checksumContents(DATA_NODE,
+          currentFinalized, true),
           UpgradeUtilities.checksumMasterBlockPoolFinalizedContents());
       
       File previous = new File(baseDirs[i], "current/" + bpid + "/previous");
       assertTrue(previous.isDirectory());
-      assertEquals(UpgradeUtilities.checksumContents(DATA_NODE, previous),
+      assertEquals(UpgradeUtilities.checksumContents(DATA_NODE, previous, false),
           UpgradeUtilities.checksumMasterDataNodeContents());
       
       File previousFinalized = 
         new File(baseDirs[i], "current/" + bpid + "/previous"+"/finalized");
-      assertEquals(UpgradeUtilities.checksumContents(DATA_NODE, previousFinalized),
+      assertEquals(UpgradeUtilities.checksumContents(DATA_NODE,
+          previousFinalized, true),
           UpgradeUtilities.checksumMasterBlockPoolFinalizedContents());
       
     }

+ 11 - 10
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java

@@ -24,6 +24,7 @@ import static org.junit.Assert.fail;
 
 import java.io.BufferedReader;
 import java.io.File;
+import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.FileReader;
 import java.io.IOException;
@@ -80,7 +81,7 @@ public class TestDFSUpgradeFromImage {
     long checksum;
   }
   
-  private static final Configuration upgradeConf;
+  static final Configuration upgradeConf;
   
   static {
     upgradeConf = new HdfsConfiguration();
@@ -95,7 +96,7 @@ public class TestDFSUpgradeFromImage {
   
   boolean printChecksum = false;
   
-  private void unpackStorage(String tarFileName)
+  void unpackStorage(String tarFileName, String referenceName)
       throws IOException {
     String tarFile = System.getProperty("test.cache.data", "build/test/cache")
         + "/" + tarFileName;
@@ -110,7 +111,7 @@ public class TestDFSUpgradeFromImage {
     
     BufferedReader reader = new BufferedReader(new FileReader(
         System.getProperty("test.cache.data", "build/test/cache")
-            + "/" + HADOOP_DFS_DIR_TXT));
+            + "/" + referenceName));
     String line;
     while ( (line = reader.readLine()) != null ) {
       
@@ -285,7 +286,7 @@ public class TestDFSUpgradeFromImage {
    */
   @Test
   public void testUpgradeFromRel22Image() throws IOException {
-    unpackStorage(HADOOP22_IMAGE);
+    unpackStorage(HADOOP22_IMAGE, HADOOP_DFS_DIR_TXT);
     upgradeAndVerify(new MiniDFSCluster.Builder(upgradeConf).
         numDataNodes(4));
   }
@@ -296,7 +297,7 @@ public class TestDFSUpgradeFromImage {
    */
   @Test
   public void testUpgradeFromCorruptRel22Image() throws IOException {
-    unpackStorage(HADOOP22_IMAGE);
+    unpackStorage(HADOOP22_IMAGE, HADOOP_DFS_DIR_TXT);
     
     // Overwrite the md5 stored in the VERSION files
     File baseDir = new File(MiniDFSCluster.getBaseDirectory());
@@ -333,7 +334,7 @@ public class TestDFSUpgradeFromImage {
    */
   @Test
   public void testUpgradeFromRel1ReservedImage() throws Exception {
-    unpackStorage(HADOOP1_RESERVED_IMAGE);
+    unpackStorage(HADOOP1_RESERVED_IMAGE, HADOOP_DFS_DIR_TXT);
     MiniDFSCluster cluster = null;
     // Try it once without setting the upgrade flag to ensure it fails
     final Configuration conf = new Configuration();
@@ -403,7 +404,7 @@ public class TestDFSUpgradeFromImage {
    */
   @Test
   public void testUpgradeFromRel023ReservedImage() throws Exception {
-    unpackStorage(HADOOP023_RESERVED_IMAGE);
+    unpackStorage(HADOOP023_RESERVED_IMAGE, HADOOP_DFS_DIR_TXT);
     MiniDFSCluster cluster = null;
     // Try it once without setting the upgrade flag to ensure it fails
     final Configuration conf = new Configuration();
@@ -468,7 +469,7 @@ public class TestDFSUpgradeFromImage {
    */
   @Test
   public void testUpgradeFromRel2ReservedImage() throws Exception {
-    unpackStorage(HADOOP2_RESERVED_IMAGE);
+    unpackStorage(HADOOP2_RESERVED_IMAGE, HADOOP_DFS_DIR_TXT);
     MiniDFSCluster cluster = null;
     // Try it once without setting the upgrade flag to ensure it fails
     final Configuration conf = new Configuration();
@@ -572,7 +573,7 @@ public class TestDFSUpgradeFromImage {
     } while (dirList.hasMore());
   }
   
-  private void upgradeAndVerify(MiniDFSCluster.Builder bld)
+  void upgradeAndVerify(MiniDFSCluster.Builder bld)
       throws IOException {
     MiniDFSCluster cluster = null;
     try {
@@ -601,7 +602,7 @@ public class TestDFSUpgradeFromImage {
    */
   @Test
   public void testUpgradeFromRel1BBWImage() throws IOException {
-    unpackStorage(HADOOP1_BBW_IMAGE);
+    unpackStorage(HADOOP1_BBW_IMAGE, HADOOP_DFS_DIR_TXT);
     Configuration conf = new Configuration(upgradeConf);
     conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, 
         System.getProperty("test.build.data") + File.separator + 

+ 73 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java

@@ -30,8 +30,12 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_A
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMESERVICES;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMESERVICE_ID;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_KEYPASSWORD_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_KEYSTORE_PASSWORD_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_TRUSTSTORE_PASSWORD_KEY;
 import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
 import static org.hamcrest.CoreMatchers.not;
+import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
@@ -39,6 +43,7 @@ import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import java.io.File;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.URI;
@@ -61,8 +66,12 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.alias.CredentialProvider;
+import org.apache.hadoop.security.alias.CredentialProviderFactory;
+import org.apache.hadoop.security.alias.JavaKeyStoreProvider;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Shell;
+import org.junit.Assert;
 import org.junit.Assume;
 import org.junit.Before;
 import org.junit.Test;
@@ -792,4 +801,68 @@ public class TestDFSUtil {
       }
     }
   }
+
+  @Test
+  public void testGetPassword() throws Exception {
+    File testDir = new File(System.getProperty("test.build.data",
+        "target/test-dir"));
+
+    Configuration conf = new Configuration();
+    final String ourUrl =
+    JavaKeyStoreProvider.SCHEME_NAME + "://file/" + testDir + "/test.jks";
+
+    File file = new File(testDir, "test.jks");
+    file.delete();
+    conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, ourUrl);
+
+    CredentialProvider provider =
+        CredentialProviderFactory.getProviders(conf).get(0);
+    char[] keypass = {'k', 'e', 'y', 'p', 'a', 's', 's'};
+    char[] storepass = {'s', 't', 'o', 'r', 'e', 'p', 'a', 's', 's'};
+    char[] trustpass = {'t', 'r', 'u', 's', 't', 'p', 'a', 's', 's'};
+
+    // ensure that we get nulls when the key isn't there
+    assertEquals(null, provider.getCredentialEntry(
+        DFS_SERVER_HTTPS_KEYPASSWORD_KEY));
+    assertEquals(null, provider.getCredentialEntry(
+        DFS_SERVER_HTTPS_KEYSTORE_PASSWORD_KEY));
+    assertEquals(null, provider.getCredentialEntry(
+        DFS_SERVER_HTTPS_TRUSTSTORE_PASSWORD_KEY));
+
+    // create new aliases
+    try {
+      provider.createCredentialEntry(
+          DFS_SERVER_HTTPS_KEYPASSWORD_KEY, keypass);
+
+      provider.createCredentialEntry(
+          DFS_SERVER_HTTPS_KEYSTORE_PASSWORD_KEY, storepass);
+
+      provider.createCredentialEntry(
+          DFS_SERVER_HTTPS_TRUSTSTORE_PASSWORD_KEY, trustpass);
+
+      // write out so that it can be found in checks
+      provider.flush();
+    } catch (Exception e) {
+      e.printStackTrace();
+      throw e;
+    }
+    // make sure we get back the right key directly from api
+    assertArrayEquals(keypass, provider.getCredentialEntry(
+        DFS_SERVER_HTTPS_KEYPASSWORD_KEY).getCredential());
+    assertArrayEquals(storepass, provider.getCredentialEntry(
+        DFS_SERVER_HTTPS_KEYSTORE_PASSWORD_KEY).getCredential());
+    assertArrayEquals(trustpass, provider.getCredentialEntry(
+        DFS_SERVER_HTTPS_TRUSTSTORE_PASSWORD_KEY).getCredential());
+
+    // use WebAppUtils as would be used by loadSslConfiguration
+    Assert.assertEquals("keypass",
+        DFSUtil.getPassword(conf, DFS_SERVER_HTTPS_KEYPASSWORD_KEY));
+    Assert.assertEquals("storepass",
+        DFSUtil.getPassword(conf, DFS_SERVER_HTTPS_KEYSTORE_PASSWORD_KEY));
+    Assert.assertEquals("trustpass",
+        DFSUtil.getPassword(conf, DFS_SERVER_HTTPS_TRUSTSTORE_PASSWORD_KEY));
+
+    // let's make sure that a password that doesn't exist returns null
+    Assert.assertEquals(null, DFSUtil.getPassword(conf,"invalid-alias"));
+  }
 }

+ 5 - 10
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeBlockScanner.java

@@ -445,19 +445,14 @@ public class TestDatanodeBlockScanner {
   
   @Test
   public void testReplicaInfoParsing() throws Exception {
-    testReplicaInfoParsingSingle(BASE_PATH, new int[0]);
-    testReplicaInfoParsingSingle(BASE_PATH + "/subdir1", new int[]{1});
-    testReplicaInfoParsingSingle(BASE_PATH + "/subdir43", new int[]{43});
-    testReplicaInfoParsingSingle(BASE_PATH + "/subdir1/subdir2/subdir3", new int[]{1, 2, 3});
-    testReplicaInfoParsingSingle(BASE_PATH + "/subdir1/subdir2/subdir43", new int[]{1, 2, 43});
-    testReplicaInfoParsingSingle(BASE_PATH + "/subdir1/subdir23/subdir3", new int[]{1, 23, 3});
-    testReplicaInfoParsingSingle(BASE_PATH + "/subdir13/subdir2/subdir3", new int[]{13, 2, 3});
+    testReplicaInfoParsingSingle(BASE_PATH);
+    testReplicaInfoParsingSingle(BASE_PATH + "/subdir1");
+    testReplicaInfoParsingSingle(BASE_PATH + "/subdir1/subdir2/subdir3");
   }
   
-  private static void testReplicaInfoParsingSingle(String subDirPath, int[] expectedSubDirs) {
+  private static void testReplicaInfoParsingSingle(String subDirPath) {
     File testFile = new File(subDirPath);
-    assertArrayEquals(expectedSubDirs, ReplicaInfo.parseSubDirs(testFile).subDirs);
-    assertEquals(BASE_PATH, ReplicaInfo.parseSubDirs(testFile).baseDirPath);
+    assertEquals(BASE_PATH, ReplicaInfo.parseBaseDir(testFile).baseDirPath);
   }
 
   @Test

+ 48 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeLayoutUpgrade.java

@@ -0,0 +1,48 @@
+/**
+ * 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;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+
+public class TestDatanodeLayoutUpgrade {
+  private static final String HADOOP_DATANODE_DIR_TXT =
+      "hadoop-datanode-dir.txt";
+  private static final String HADOOP24_DATANODE = "hadoop-24-datanode-dir.tgz";
+
+  @Test
+  // Upgrade from LDir-based layout to block ID-based layout -- change described
+  // in HDFS-6482
+  public void testUpgradeToIdBasedLayout() throws IOException {
+    TestDFSUpgradeFromImage upgrade = new TestDFSUpgradeFromImage();
+    upgrade.unpackStorage(HADOOP24_DATANODE, HADOOP_DATANODE_DIR_TXT);
+    Configuration conf = new Configuration(TestDFSUpgradeFromImage.upgradeConf);
+    conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY,
+        System.getProperty("test.build.data") + File.separator +
+            "dfs" + File.separator + "data");
+    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+        System.getProperty("test.build.data") + File.separator +
+            "dfs" + File.separator + "name");
+    upgrade.upgradeAndVerify(new MiniDFSCluster.Builder(conf).numDataNodes(1)
+    .manageDataDfsDirs(false).manageNameDfsDirs(false));
+  }
+}

+ 57 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.File;
 import java.io.FileNotFoundException;
@@ -32,6 +33,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.HardLink;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -39,6 +41,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.RemoteException;
 import org.junit.Assert;
 import org.junit.Test;
@@ -169,6 +172,7 @@ public class TestFileAppend{
       }
 
     } finally {
+      client.close();
       fs.close();
       cluster.shutdown();
     }
@@ -380,4 +384,57 @@ public class TestFileAppend{
     }
   }
 
+  /**
+   * Old replica of the block should not be accepted as valid for append/read
+   */
+  @Test
+  public void testFailedAppendBlockRejection() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    conf.set("dfs.client.block.write.replace-datanode-on-failure.enable",
+        "false");
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3)
+        .build();
+    DistributedFileSystem fs = null;
+    try {
+      fs = cluster.getFileSystem();
+      Path path = new Path("/test");
+      FSDataOutputStream out = fs.create(path);
+      out.writeBytes("hello\n");
+      out.close();
+
+      // stop one datanode
+      DataNodeProperties dnProp = cluster.stopDataNode(0);
+      String dnAddress = dnProp.datanode.getXferAddress().toString();
+      if (dnAddress.startsWith("/")) {
+        dnAddress = dnAddress.substring(1);
+      }
+
+      // append again to bump genstamps
+      for (int i = 0; i < 2; i++) {
+        out = fs.append(path);
+        out.writeBytes("helloagain\n");
+        out.close();
+      }
+
+      // re-open and make the block state as underconstruction
+      out = fs.append(path);
+      cluster.restartDataNode(dnProp, true);
+      // wait till the block report comes
+      Thread.sleep(2000);
+      // check the block locations, this should not contain restarted datanode
+      BlockLocation[] locations = fs.getFileBlockLocations(path, 0,
+          Long.MAX_VALUE);
+      String[] names = locations[0].getNames();
+      for (String node : names) {
+        if (node.equals(dnAddress)) {
+          fail("Failed append should not be present in latest block locations.");
+        }
+      }
+      out.close();
+    } finally {
+      IOUtils.closeStream(fs);
+      cluster.shutdown();
+    }
+  }
+
 }

+ 13 - 27
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCorruption.java

@@ -27,6 +27,7 @@ import java.io.DataOutputStream;
 import java.io.File;
 import java.io.FileOutputStream;
 import java.util.ArrayList;
+import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -35,6 +36,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
@@ -137,13 +139,15 @@ public class TestFileCorruption {
       final String bpid = cluster.getNamesystem().getBlockPoolId();
       File storageDir = cluster.getInstanceStorageDir(0, 0);
       File dataDir = MiniDFSCluster.getFinalizedDir(storageDir, bpid);
+      assertTrue("Data directory does not exist", dataDir.exists());
       ExtendedBlock blk = getBlock(bpid, dataDir);
       if (blk == null) {
         storageDir = cluster.getInstanceStorageDir(0, 1);
         dataDir = MiniDFSCluster.getFinalizedDir(storageDir, bpid);
         blk = getBlock(bpid, dataDir);
       }
-      assertFalse(blk==null);
+      assertFalse("Data directory does not contain any blocks or there was an "
+          + "IO error", blk==null);
 
       // start a third datanode
       cluster.startDataNodes(conf, 1, true, null, null);
@@ -174,33 +178,15 @@ public class TestFileCorruption {
     
   }
   
-  private ExtendedBlock getBlock(String bpid, File dataDir) {
-    assertTrue("data directory does not exist", dataDir.exists());
-    File[] blocks = dataDir.listFiles();
-    assertTrue("Blocks do not exist in dataDir", (blocks != null) && (blocks.length > 0));
-
-    int idx = 0;
-    String blockFileName = null;
-    for (; idx < blocks.length; idx++) {
-      blockFileName = blocks[idx].getName();
-      if (blockFileName.startsWith("blk_") && !blockFileName.endsWith(".meta")) {
-        break;
-      }
-    }
-    if (blockFileName == null) {
+  public static ExtendedBlock getBlock(String bpid, File dataDir) {
+    List<File> metadataFiles = MiniDFSCluster.getAllBlockMetadataFiles(dataDir);
+    if (metadataFiles == null || metadataFiles.isEmpty()) {
       return null;
     }
-    long blockId = Long.parseLong(blockFileName.substring("blk_".length()));
-    long blockTimeStamp = GenerationStamp.GRANDFATHER_GENERATION_STAMP;
-    for (idx=0; idx < blocks.length; idx++) {
-      String fileName = blocks[idx].getName();
-      if (fileName.startsWith(blockFileName) && fileName.endsWith(".meta")) {
-        int startIndex = blockFileName.length()+1;
-        int endIndex = fileName.length() - ".meta".length();
-        blockTimeStamp = Long.parseLong(fileName.substring(startIndex, endIndex));
-        break;
-      }
-    }
-    return new ExtendedBlock(bpid, blockId, blocks[idx].length(), blockTimeStamp);
+    File metadataFile = metadataFiles.get(0);
+    File blockFile = Block.metaToBlockFile(metadataFile);
+    return new ExtendedBlock(bpid, Block.getBlockId(blockFile.getName()),
+        blockFile.length(), Block.getGenerationStamp(metadataFile.getName()));
   }
+
 }

+ 28 - 16
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/UpgradeUtilities.java

@@ -158,21 +158,23 @@ public class UpgradeUtilities {
       FileUtil.fullyDelete(new File(datanodeStorage,"in_use.lock"));
     }
     namenodeStorageChecksum = checksumContents(NAME_NODE, 
-        new File(namenodeStorage, "current"));
+        new File(namenodeStorage, "current"), false);
     File dnCurDir = new File(datanodeStorage, "current");
-    datanodeStorageChecksum = checksumContents(DATA_NODE, dnCurDir);
+    datanodeStorageChecksum = checksumContents(DATA_NODE, dnCurDir, false);
     
     File bpCurDir = new File(BlockPoolSliceStorage.getBpRoot(bpid, dnCurDir),
         "current");
-    blockPoolStorageChecksum = checksumContents(DATA_NODE, bpCurDir);
+    blockPoolStorageChecksum = checksumContents(DATA_NODE, bpCurDir, false);
     
     File bpCurFinalizeDir = new File(BlockPoolSliceStorage.getBpRoot(bpid, dnCurDir),
         "current/"+DataStorage.STORAGE_DIR_FINALIZED);
-    blockPoolFinalizedStorageChecksum = checksumContents(DATA_NODE, bpCurFinalizeDir);
+    blockPoolFinalizedStorageChecksum = checksumContents(DATA_NODE,
+        bpCurFinalizeDir, true);
     
     File bpCurRbwDir = new File(BlockPoolSliceStorage.getBpRoot(bpid, dnCurDir),
         "current/"+DataStorage.STORAGE_DIR_RBW);
-    blockPoolRbwStorageChecksum = checksumContents(DATA_NODE, bpCurRbwDir);
+    blockPoolRbwStorageChecksum = checksumContents(DATA_NODE, bpCurRbwDir,
+        false);
   }
   
   // Private helper method that writes a file to the given file system.
@@ -266,36 +268,47 @@ public class UpgradeUtilities {
   
   /**
    * Compute the checksum of all the files in the specified directory.
-   * The contents of subdirectories are not included. This method provides
-   * an easy way to ensure equality between the contents of two directories.
+   * This method provides an easy way to ensure equality between the contents
+   * of two directories.
    *
    * @param nodeType if DATA_NODE then any file named "VERSION" is ignored.
    *    This is because this file file is changed every time
    *    the Datanode is started.
-   * @param dir must be a directory. Subdirectories are ignored.
+   * @param dir must be a directory
+   * @param recursive whether or not to consider subdirectories
    *
    * @throws IllegalArgumentException if specified directory is not a directory
    * @throws IOException if an IOException occurs while reading the files
    * @return the computed checksum value
    */
-  public static long checksumContents(NodeType nodeType, File dir) throws IOException {
+  public static long checksumContents(NodeType nodeType, File dir,
+      boolean recursive) throws IOException {
+    CRC32 checksum = new CRC32();
+    checksumContentsHelper(nodeType, dir, checksum, recursive);
+    return checksum.getValue();
+  }
+
+  public static void checksumContentsHelper(NodeType nodeType, File dir,
+      CRC32 checksum, boolean recursive) throws IOException {
     if (!dir.isDirectory()) {
       throw new IllegalArgumentException(
-                                         "Given argument is not a directory:" + dir);
+          "Given argument is not a directory:" + dir);
     }
     File[] list = dir.listFiles();
     Arrays.sort(list);
-    CRC32 checksum = new CRC32();
     for (int i = 0; i < list.length; i++) {
       if (!list[i].isFile()) {
+        if (recursive) {
+          checksumContentsHelper(nodeType, list[i], checksum, recursive);
+        }
         continue;
       }
 
       // skip VERSION and dfsUsed file for DataNodes
-      if (nodeType == DATA_NODE && 
-         (list[i].getName().equals("VERSION") || 
-         list[i].getName().equals("dfsUsed"))) {
-        continue; 
+      if (nodeType == DATA_NODE &&
+          (list[i].getName().equals("VERSION") ||
+              list[i].getName().equals("dfsUsed"))) {
+        continue;
       }
 
       FileInputStream fis = null;
@@ -312,7 +325,6 @@ public class UpgradeUtilities {
         }
       }
     }
-    return checksum.getValue();
   }
   
   /**

+ 95 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java

@@ -658,6 +658,71 @@ public class TestBalancer {
     oneNodeTest(conf, false);
   }
   
+  /* we first start a cluster and fill the cluster up to a certain size.
+   * then redistribute blocks according the required distribution.
+   * Then we start an empty datanode.
+   * Afterwards a balancer is run to balance the cluster.
+   * A partially filled datanode is excluded during balancing.
+   * This triggers a situation where one of the block's location is unknown.
+   */
+  @Test(timeout=100000)
+  public void testUnknownDatanode() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    initConf(conf);
+    long distribution[] = new long[] {50*CAPACITY/100, 70*CAPACITY/100, 0*CAPACITY/100};
+    long capacities[] = new long[]{CAPACITY, CAPACITY, CAPACITY};
+    String racks[] = new String[] {RACK0, RACK1, RACK1};
+
+    int numDatanodes = distribution.length;
+    if (capacities.length != numDatanodes || racks.length != numDatanodes) {
+      throw new IllegalArgumentException("Array length is not the same");
+    }
+
+    // calculate total space that need to be filled
+    final long totalUsedSpace = sum(distribution);
+
+    // fill the cluster
+    ExtendedBlock[] blocks = generateBlocks(conf, totalUsedSpace,
+        (short) numDatanodes);
+
+    // redistribute blocks
+    Block[][] blocksDN = distributeBlocks(
+        blocks, (short)(numDatanodes-1), distribution);
+
+    // restart the cluster: do NOT format the cluster
+    conf.set(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY, "0.0f");
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDatanodes)
+        .format(false)
+        .racks(racks)
+        .simulatedCapacities(capacities)
+        .build();
+    try {
+      cluster.waitActive();
+      client = NameNodeProxies.createProxy(conf, cluster.getFileSystem(0).getUri(),
+          ClientProtocol.class).getProxy();
+
+      for(int i = 0; i < 3; i++) {
+        cluster.injectBlocks(i, Arrays.asList(blocksDN[i]), null);
+      }
+
+      cluster.startDataNodes(conf, 1, true, null,
+          new String[]{RACK0}, null,new long[]{CAPACITY});
+      cluster.triggerHeartbeats();
+
+      Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(conf);
+      Set<String>  datanodes = new HashSet<String>();
+      datanodes.add(cluster.getDataNodes().get(0).getDatanodeId().getHostName());
+      Balancer.Parameters p = new Balancer.Parameters(
+          Balancer.Parameters.DEFAULT.policy,
+          Balancer.Parameters.DEFAULT.threshold,
+          datanodes, Balancer.Parameters.DEFAULT.nodesToBeIncluded);
+      final int r = Balancer.run(namenodes, p, conf);
+      assertEquals(Balancer.ReturnStatus.SUCCESS.code, r);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
   /**
    * Test parse method in Balancer#Cli class with threshold value out of
    * boundaries.
@@ -789,13 +854,42 @@ public class TestBalancer {
     } catch (IllegalArgumentException e) {
 
     }
-    parameters = new String[] { "-threshold 1 -policy" };
+    parameters = new String[] {"-threshold", "1", "-policy"};
+    try {
+      Balancer.Cli.parse(parameters);
+      fail(reason);
+    } catch (IllegalArgumentException e) {
+
+    }
+    parameters = new String[] {"-threshold", "1", "-include"};
     try {
       Balancer.Cli.parse(parameters);
       fail(reason);
     } catch (IllegalArgumentException e) {
 
     }
+    parameters = new String[] {"-threshold", "1", "-exclude"};
+    try {
+      Balancer.Cli.parse(parameters);
+      fail(reason);
+    } catch (IllegalArgumentException e) {
+
+    }
+    parameters = new String[] {"-include",  "-f"};
+    try {
+      Balancer.Cli.parse(parameters);
+      fail(reason);
+    } catch (IllegalArgumentException e) {
+
+    }
+    parameters = new String[] {"-exclude",  "-f"};
+    try {
+      Balancer.Cli.parse(parameters);
+      fail(reason);
+    } catch (IllegalArgumentException e) {
+
+    }
+
     parameters = new String[] {"-include",  "testnode1", "-exclude", "testnode2"};
     try {
       Balancer.Cli.parse(parameters);

+ 10 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java

@@ -268,4 +268,14 @@ public class BlockManagerTestUtil {
     }
     return reports.toArray(StorageReport.EMPTY_ARRAY);
   }
+
+  /**
+   * Have DatanodeManager check decommission state.
+   * @param dm the DatanodeManager to manipulate
+   */
+  public static void checkDecommissionState(DatanodeManager dm,
+      DatanodeDescriptor node) {
+    dm.checkDecommissionState(node);
+  }
+
 }

+ 5 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java

@@ -368,7 +368,7 @@ public class TestBlockManager {
       DatanodeStorageInfo[] pipeline) throws IOException {
     for (int i = 1; i < pipeline.length; i++) {
       DatanodeStorageInfo storage = pipeline[i];
-      bm.addBlock(storage.getDatanodeDescriptor(), storage.getStorageID(), blockInfo, null);
+      bm.addBlock(storage, blockInfo, null);
       blockInfo.addStorage(storage);
     }
   }
@@ -549,12 +549,12 @@ public class TestBlockManager {
     // send block report, should be processed
     reset(node);
     
-    bm.processReport(node, new DatanodeStorage(ds.getStorageID()), "pool", 
+    bm.processReport(node, new DatanodeStorage(ds.getStorageID()),
         new BlockListAsLongs(null, null));
     assertEquals(1, ds.getBlockReportCount());
     // send block report again, should NOT be processed
     reset(node);
-    bm.processReport(node, new DatanodeStorage(ds.getStorageID()), "pool",
+    bm.processReport(node, new DatanodeStorage(ds.getStorageID()),
         new BlockListAsLongs(null, null));
     assertEquals(1, ds.getBlockReportCount());
 
@@ -566,7 +566,7 @@ public class TestBlockManager {
     assertEquals(0, ds.getBlockReportCount()); // ready for report again
     // send block report, should be processed after restart
     reset(node);
-    bm.processReport(node, new DatanodeStorage(ds.getStorageID()), "pool",
+    bm.processReport(node, new DatanodeStorage(ds.getStorageID()),
         new BlockListAsLongs(null, null));
     assertEquals(1, ds.getBlockReportCount());
   }
@@ -595,7 +595,7 @@ public class TestBlockManager {
     // send block report while pretending to already have blocks
     reset(node);
     doReturn(1).when(node).numBlocks();
-    bm.processReport(node, new DatanodeStorage(ds.getStorageID()), "pool",
+    bm.processReport(node, new DatanodeStorage(ds.getStorageID()),
         new BlockListAsLongs(null, null));
     assertEquals(1, ds.getBlockReportCount());
   }

+ 10 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestCorruptReplicaInfo.java

@@ -33,6 +33,7 @@ import org.apache.commons.logging.Log;
 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.blockmanagement.CorruptReplicasMap.Reason;
 import org.junit.Test;
 
 
@@ -89,14 +90,14 @@ public class TestCorruptReplicaInfo {
       DatanodeDescriptor dn1 = DFSTestUtil.getLocalDatanodeDescriptor();
       DatanodeDescriptor dn2 = DFSTestUtil.getLocalDatanodeDescriptor();
       
-      crm.addToCorruptReplicasMap(getBlock(0), dn1, "TEST");
+      addToCorruptReplicasMap(crm, getBlock(0), dn1);
       assertEquals("Number of corrupt blocks not returning correctly",
                    1, crm.size());
-      crm.addToCorruptReplicasMap(getBlock(1), dn1, "TEST");
+      addToCorruptReplicasMap(crm, getBlock(1), dn1);
       assertEquals("Number of corrupt blocks not returning correctly",
                    2, crm.size());
       
-      crm.addToCorruptReplicasMap(getBlock(1), dn2, "TEST");
+      addToCorruptReplicasMap(crm, getBlock(1), dn2);
       assertEquals("Number of corrupt blocks not returning correctly",
                    2, crm.size());
       
@@ -109,7 +110,7 @@ public class TestCorruptReplicaInfo {
                    0, crm.size());
       
       for (Long block_id: block_ids) {
-        crm.addToCorruptReplicasMap(getBlock(block_id), dn1, "TEST");
+        addToCorruptReplicasMap(crm, getBlock(block_id), dn1);
       }
             
       assertEquals("Number of corrupt blocks not returning correctly",
@@ -127,4 +128,9 @@ public class TestCorruptReplicaInfo {
                               crm.getCorruptReplicaBlockIds(10, 7L)));
       
   }
+  
+  private static void addToCorruptReplicasMap(CorruptReplicasMap crm,
+      Block blk, DatanodeDescriptor dn) {
+    crm.addToCorruptReplicasMap(blk, dn, "TEST", Reason.NONE);
+  }
 }

Kaikkia tiedostoja ei voida näyttää, sillä liian monta tiedostoa muuttui tässä diffissä