Browse Source

Merge from trunk to branch.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/fs-encryption@1616589 13f79535-47bb-0310-9956-ffa450edef68
Andrew Wang 10 năm trước cách đây
mục cha
commit
311d2f0773
81 tập tin đã thay đổi với 2983 bổ sung738 xóa
  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. 18 2
      hadoop-common-project/hadoop-common/CHANGES.txt
  15. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java
  16. 7 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/jmx/JMXJsonServlet.java
  17. 22 4
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java
  18. 19 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/FileBasedKeyStoresFactory.java
  19. 16 179
      hadoop-common-project/hadoop-common/src/site/apt/CommandsManual.apt.vm
  20. 7 7
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderFactory.java
  21. 3 3
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java
  22. 58 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMapping.java
  23. 41 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/KeyStoreTestUtil.java
  24. 57 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/ssl/TestSSLFactory.java
  25. 33 0
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSJMXServlet.java
  26. 1 1
      hadoop-common-project/hadoop-kms/src/main/webapp/WEB-INF/web.xml
  27. 2 1
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java
  28. 22 3
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  29. 0 44
      hadoop-hdfs-project/hadoop-hdfs/src/main/conf/hadoop-metrics2.properties
  30. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  31. 28 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
  32. 41 147
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
  33. 173 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/KeyManager.java
  34. 51 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Matcher.java
  35. 124 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/MovedBlocks.java
  36. 66 143
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java
  37. 4 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
  38. 9 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  39. 9 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
  40. 3 16
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CorruptReplicasMap.java
  41. 3 36
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
  42. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingDataNodeMessages.java
  43. 408 0
      hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HDFSCommands.apt.vm
  44. 14 15
      hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HdfsUserGuide.apt.vm
  45. 73 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
  46. 10 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
  47. 10 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestCorruptReplicaInfo.java
  48. 87 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java
  49. 6 0
      hadoop-mapreduce-project/CHANGES.txt
  50. 2 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java
  51. 3 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksBlock.java
  52. 3 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestBlocks.java
  53. 2 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTaskPage.java
  54. 3 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestBlocks.java
  55. 9 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
  56. 73 31
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestYARNRunner.java
  57. 1 0
      hadoop-project/src/site/site.xml
  58. 5 0
      hadoop-tools/hadoop-openstack/pom.xml
  59. 17 0
      hadoop-yarn-project/CHANGES.txt
  60. 71 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
  61. 0 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/pom.xml
  62. 8 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml
  63. 0 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java
  64. 0 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineAuthenticator.java
  65. 0 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
  66. 21 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/package-info.java
  67. 21 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/package-info.java
  68. 7 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DefaultResourceCalculator.java
  69. 8 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
  70. 9 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java
  71. 5 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
  72. 0 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineAuthenticator.java
  73. 0 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClient.java
  74. 72 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/LeveldbTimelineStore.java
  75. 49 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestLeveldbTimelineStore.java
  76. 3 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/webapp/TestTimelineWebServices.java
  77. 7 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
  78. 53 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueueUtils.java
  79. 7 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
  80. 27 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
  81. 262 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCSQueueUtils.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]);
+  }
+}

+ 18 - 2
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -418,6 +418,9 @@ Trunk (Unreleased)
     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)
@@ -484,6 +487,9 @@ Release 2.6.0 - UNRELEASED
     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
@@ -527,6 +533,18 @@ Release 2.6.0 - UNRELEASED
     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)
+
+    HADOOP-10931 compile error on tools/hadoop-openstack (xukun via stevel)
+
 Release 2.5.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -667,8 +685,6 @@ Release 2.5.0 - UNRELEASED
 
   BUG FIXES 
 
-    HADOOP-10759. Remove hardcoded JAVA_HEAP_MAX. (Sam Liu via Eric Yang)
-
     HADOOP-10378. Typo in help printed by hdfs dfs -help.
     (Mit Desai via suresh)
 

+ 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;
     }

+ 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

+ 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.
    */

+ 16 - 179
hadoop-common-project/hadoop-common/src/site/apt/CommandsManual.apt.vm

@@ -114,57 +114,18 @@ User Commands
 
 * <<<fs>>>
 
-   Usage: <<<hadoop fs [GENERIC_OPTIONS] [COMMAND_OPTIONS]>>>
-
-   Deprecated, use <<<hdfs dfs>>> instead.
-
-   Runs a generic filesystem user client.
-
-   The various COMMAND_OPTIONS can be found at File System Shell Guide.
+   Deprecated, use {{{../hadoop-hdfs/HDFSCommands.html#dfs}<<<hdfs dfs>>>}}
+   instead.
 
 * <<<fsck>>>
 
-   Runs a HDFS filesystem checking utility.
-   See {{{../hadoop-hdfs/HdfsUserGuide.html#fsck}fsck}} for more info.
-
-   Usage: <<<hadoop fsck [GENERIC_OPTIONS] <path> [-move | -delete | -openforwrite] [-files [-blocks [-locations | -racks]]] [-showprogress]>>>
-
-*------------------+---------------------------------------------+
-||  COMMAND_OPTION || Description
-*------------------+---------------------------------------------+
-|   <path>         | Start checking from this path.
-*------------------+---------------------------------------------+
-|   -move          | Move corrupted files to /lost+found
-*------------------+---------------------------------------------+
-|   -delete        | Delete corrupted files.
-*------------------+---------------------------------------------+
-|   -openforwrite  | Print out files opened for write.
-*------------------+---------------------------------------------+
-|   -files         | Print out files being checked.
-*------------------+---------------------------------------------+
-|   -blocks        | Print out block report.
-*------------------+---------------------------------------------+
-|   -locations     | Print out locations for every block.
-*------------------+---------------------------------------------+
-|   -racks         | Print out network topology for data-node locations.
-*------------------+---------------------------------------------+
-|   -showprogress  | Print out show progress in output. Default is OFF (no progress).
-*------------------+---------------------------------------------+
+   Deprecated, use {{{../hadoop-hdfs/HDFSCommands.html#fsck}<<<hdfs fsck>>>}}
+   instead.
 
 * <<<fetchdt>>>
 
-   Gets Delegation Token from a NameNode.
-   See {{{../hadoop-hdfs/HdfsUserGuide.html#fetchdt}fetchdt}} for more info.
-
-   Usage: <<<hadoop fetchdt [GENERIC_OPTIONS] [--webservice <namenode_http_addr>] <path> >>>
-
-*------------------------------+---------------------------------------------+
-|| COMMAND_OPTION              || Description
-*------------------------------+---------------------------------------------+
-| <fileName>                   | File name to store the token into.
-*------------------------------+---------------------------------------------+
-| --webservice <https_address> | use http protocol instead of RPC
-*------------------------------+---------------------------------------------+
+   Deprecated, use {{{../hadoop-hdfs/HDFSCommands.html#fetchdt}
+   <<<hdfs fetchdt>>>}} instead.
 
 * <<<jar>>>
 
@@ -321,23 +282,8 @@ Administration Commands
 
 * <<<balancer>>>
 
-   Runs a cluster balancing utility. An administrator can simply press Ctrl-C
-   to stop the rebalancing process. See
-   {{{../hadoop-hdfs/HdfsUserGuide.html#Balancer}Balancer}} for more details.
-
-   Usage: <<<hadoop balancer [-threshold <threshold>] [-policy <policy>]>>>
-
-*------------------------+-----------------------------------------------------------+
-|| COMMAND_OPTION        | Description
-*------------------------+-----------------------------------------------------------+
-| -threshold <threshold> | Percentage of disk capacity. This overwrites the
-                         | default threshold.
-*------------------------+-----------------------------------------------------------+
-| -policy <policy>       | <<<datanode>>> (default): Cluster is balanced if each datanode is balanced. \
-                         | <<<blockpool>>>: Cluster is balanced if each block pool in each datanode is balanced.
-*------------------------+-----------------------------------------------------------+
-
-   Note that the <<<blockpool>>> policy is more strict than the <<<datanode>>> policy.
+   Deprecated, use {{{../hadoop-hdfs/HDFSCommands.html#balancer}
+   <<<hdfs balancer>>>}} instead.
 
 * <<<daemonlog>>>
 
@@ -360,84 +306,13 @@ Administration Commands
 
 * <<<datanode>>>
 
-   Runs a HDFS datanode.
-
-   Usage: <<<hadoop datanode [-rollback]>>>
-
-*-----------------+-----------------------------------------------------------+
-|| COMMAND_OPTION || Description
-*-----------------+-----------------------------------------------------------+
-| -rollback       | Rollsback the datanode to the previous version. This should
-                  | be used after stopping the datanode and distributing the old
-                  | hadoop version.
-*-----------------+-----------------------------------------------------------+
+   Deprecated, use {{{../hadoop-hdfs/HDFSCommands.html#datanode}
+   <<<hdfs datanode>>>}} instead.
 
 * <<<dfsadmin>>>
 
-   Runs a HDFS dfsadmin client.
-
-   Usage: <<<hadoop dfsadmin [GENERIC_OPTIONS] [-report] [-safemode enter | leave | get | wait] [-refreshNodes] [-finalizeUpgrade] [-upgradeProgress status | details | force] [-metasave filename] [-setQuota <quota> <dirname>...<dirname>] [-clrQuota <dirname>...<dirname>] [-restoreFailedStorage true|false|check] [-help [cmd]]>>>
-
-*-----------------+-----------------------------------------------------------+
-|| COMMAND_OPTION || Description
-*-----------------+-----------------------------------------------------------+
-| -report         | Reports basic filesystem information and statistics.
-*-----------------+-----------------------------------------------------------+
-| -safemode enter / leave / get / wait | Safe mode maintenance command. Safe
-                  | mode is a Namenode state in which it \
-                  | 1. does not accept changes to the name space (read-only) \
-                  | 2. does not replicate or delete blocks. \
-                  | Safe mode is entered automatically at Namenode startup, and
-                  | leaves safe mode automatically when the configured minimum
-                  | percentage of blocks satisfies the minimum replication
-                  | condition. Safe mode can also be entered manually, but then
-                  | it can only be turned off manually as well.
-*-----------------+-----------------------------------------------------------+
-| -refreshNodes   | Re-read the hosts and exclude files to update the set of
-                  | Datanodes that are allowed to connect to the Namenode and
-                  | those that should be decommissioned or recommissioned.
-*-----------------+-----------------------------------------------------------+
-| -finalizeUpgrade| Finalize upgrade of HDFS. Datanodes delete their previous
-                  | version working directories, followed by Namenode doing the
-                  | same. This completes the upgrade process.
-*-----------------+-----------------------------------------------------------+
-| -upgradeProgress status / details / force | Request current distributed
-                  | upgrade status, a detailed status or force the upgrade to
-                  | proceed.
-*-----------------+-----------------------------------------------------------+
-| -metasave filename | Save Namenode's primary data structures to <filename> in
-                  | the directory specified by hadoop.log.dir property.
-                  | <filename> is overwritten if it exists.
-                  | <filename> will contain one line for each of the following\
-                  | 1. Datanodes heart beating with Namenode\
-                  | 2. Blocks waiting to be replicated\
-                  | 3. Blocks currrently being replicated\
-                  | 4. Blocks waiting to be deleted\
-*-----------------+-----------------------------------------------------------+
-| -setQuota <quota> <dirname>...<dirname> | Set the quota <quota> for each
-                  | directory <dirname>. The directory quota is a long integer
-                  | that puts a hard limit on the number of names in the
-                  | directory tree.  Best effort for the directory, with faults
-                  | reported if \
-                  | 1. N is not a positive integer, or \
-                  | 2. user is not an administrator, or \
-                  | 3. the directory does not exist or is a file, or \
-                  | 4. the directory would immediately exceed the new quota. \
-*-----------------+-----------------------------------------------------------+
-| -clrQuota <dirname>...<dirname> | Clear the quota for each directory
-                  | <dirname>.  Best effort for the directory. with fault
-                  | reported if \
-                  | 1. the directory does not exist or is a file, or \
-                  | 2. user is not an administrator.  It does not fault if the
-                  | directory has no quota.
-*-----------------+-----------------------------------------------------------+
-| -restoreFailedStorage true / false / check | This option will turn on/off automatic attempt to restore failed storage replicas.
-                  | If a failed storage becomes available again the system will attempt to restore
-                  | edits and/or fsimage during checkpoint. 'check' option will return current setting.
-*-----------------+-----------------------------------------------------------+
-| -help [cmd]     | Displays help for the given command or all commands if none
-                  | is specified.
-*-----------------+-----------------------------------------------------------+
+   Deprecated, use {{{../hadoop-hdfs/HDFSCommands.html#dfsadmin}
+   <<<hdfs dfsadmin>>>}} instead.
 
 * <<<mradmin>>>
 
@@ -470,51 +345,13 @@ Administration Commands
 
 * <<<namenode>>>
 
-   Runs the namenode. More info about the upgrade, rollback and finalize is
-   at {{{../hadoop-hdfs/HdfsUserGuide.html#Upgrade_and_Rollback}Upgrade Rollback}}.
-
-   Usage: <<<hadoop namenode [-format] | [-upgrade] | [-rollback] | [-finalize] | [-importCheckpoint]>>>
-
-*--------------------+-----------------------------------------------------------+
-|| COMMAND_OPTION    || Description
-*--------------------+-----------------------------------------------------------+
-| -format            | Formats the namenode. It starts the namenode, formats
-                     | it and then shut it down.
-*--------------------+-----------------------------------------------------------+
-| -upgrade           | Namenode should be started with upgrade option after
-                     | the distribution of new hadoop version.
-*--------------------+-----------------------------------------------------------+
-| -rollback          | Rollsback the namenode to the previous version. This
-                     | should be used after stopping the cluster and
-                     | distributing the old hadoop version.
-*--------------------+-----------------------------------------------------------+
-| -finalize          | Finalize will remove the previous state of the files
-                     | system. Recent upgrade will become permanent.  Rollback
-                     | option will not be available anymore. After finalization
-                     | it shuts the namenode down.
-*--------------------+-----------------------------------------------------------+
-| -importCheckpoint  | Loads image from a checkpoint directory and save it
-                     | into the current one. Checkpoint dir is read from
-                     | property fs.checkpoint.dir
-*--------------------+-----------------------------------------------------------+
+   Deprecated, use {{{../hadoop-hdfs/HDFSCommands.html#namenode}
+   <<<hdfs namenode>>>}} instead.
 
 * <<<secondarynamenode>>>
 
-   Runs the HDFS secondary namenode.
-   See {{{../hadoop-hdfs/HdfsUserGuide.html#Secondary_NameNode}Secondary Namenode}}
-   for more info.
-
-   Usage: <<<hadoop secondarynamenode [-checkpoint [force]] | [-geteditsize]>>>
-
-*----------------------+-----------------------------------------------------------+
-|| COMMAND_OPTION      || Description
-*----------------------+-----------------------------------------------------------+
-| -checkpoint [-force] | Checkpoints the Secondary namenode if EditLog size
-                       | >= fs.checkpoint.size. If <<<-force>>> is used,
-                       | checkpoint irrespective of EditLog size.
-*----------------------+-----------------------------------------------------------+
-| -geteditsize         | Prints the EditLog size.
-*----------------------+-----------------------------------------------------------+
+   Deprecated, use {{{../hadoop-hdfs/HDFSCommands.html#secondarynamenode}
+   <<<hdfs secondarynamenode>>>}} instead.
 
 * <<<tasktracker>>>
 

+ 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");

+ 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", ""));
+  }
 }

+ 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,

+ 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;
+  }
+}

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

+ 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(),

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

@@ -136,6 +136,9 @@ Trunk (Unreleased)
   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 +360,16 @@ Release 2.6.0 - UNRELEASED
 
     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)
+
+    HDFS-6781. Separate HDFS commands from CommandsManual.apt.vm. (Akira
+    Ajisaka via Arpit Agarwal)
+
   OPTIMIZATIONS
 
     HDFS-6690. Deduplicate xattr names in memory. (wang)
@@ -440,6 +453,15 @@ Release 2.6.0 - UNRELEASED
     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
@@ -992,9 +1014,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
-

+ 3 - 0
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";

+ 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;
@@ -1534,15 +1537,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.
    */

+ 41 - 147
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);

+ 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;
   }
   
   /**

+ 9 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -3177,6 +3177,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();
   }
 

+ 3 - 36
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) {
@@ -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);

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

@@ -23,9 +23,9 @@ 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;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 
 /**
  * In the Standby Node, we can receive messages about blocks
@@ -123,7 +123,7 @@ class PendingDataNodeMessages {
     return queue;
   }
   
-  public int count() {
+  int count() {
     return count ;
   }
 
@@ -140,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()) {

+ 408 - 0
hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HDFSCommands.apt.vm

@@ -0,0 +1,408 @@
+~~ 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.
+
+  ---
+  HDFS Commands Guide
+  ---
+  ---
+  ${maven.build.timestamp}
+
+HDFS Commands Guide
+
+%{toc|section=1|fromDepth=2|toDepth=4}
+
+* Overview
+
+   All HDFS commands are invoked by the <<<bin/hdfs>>> script. Running the
+   hdfs script without any arguments prints the description for all
+   commands.
+
+   Usage: <<<hdfs [--config confdir] [COMMAND] [GENERIC_OPTIONS]
+          [COMMAND_OPTIONS]>>>
+
+   Hadoop has an option parsing framework that employs parsing generic options
+   as well as running classes.
+
+*-----------------------+---------------+
+|| COMMAND_OPTION       || Description
+*-----------------------+---------------+
+| <<<--config confdir>>>| Overwrites the default Configuration directory.
+|                       | Default is <<<${HADOOP_HOME}/conf>>>.
+*-----------------------+---------------+
+| GENERIC_OPTIONS       | The common set of options supported by multiple
+|                       | commands. Full list is
+|                       | {{{../hadoop-common/CommandsManual.html#Generic_Options}here}}.
+*-----------------------+---------------+
+| COMMAND_OPTIONS       | Various commands with their options are described in
+|                       | the following sections. The commands have been
+|                       | grouped into {{{User Commands}}} and
+|                       | {{{Administration Commands}}}.
+*-----------------------+---------------+
+
+* User Commands
+
+   Commands useful for users of a hadoop cluster.
+
+** <<<dfs>>>
+
+   Usage: <<<hdfs dfs [GENERIC_OPTIONS] [COMMAND_OPTIONS]>>>
+
+   Run a filesystem command on the file system supported in Hadoop.
+   The various COMMAND_OPTIONS can be found at
+   {{{../hadoop-common/FileSystemShell.html}File System Shell Guide}}.
+
+** <<<fetchdt>>>
+
+   Gets Delegation Token from a NameNode.
+   See {{{./HdfsUserGuide.html#fetchdt}fetchdt}} for more info.
+
+   Usage: <<<hdfs fetchdt [GENERIC_OPTIONS]
+          [--webservice <namenode_http_addr>] <path> >>>
+
+*------------------------------+---------------------------------------------+
+|| COMMAND_OPTION              || Description
+*------------------------------+---------------------------------------------+
+| <fileName>                   | File name to store the token into.
+*------------------------------+---------------------------------------------+
+| --webservice <https_address> | use http protocol instead of RPC
+*------------------------------+---------------------------------------------+
+
+** <<<fsck>>>
+
+   Runs a HDFS filesystem checking utility.
+   See {{{./HdfsUserGuide.html#fsck}fsck}} for more info.
+
+   Usage: <<<hdfs fsck [GENERIC_OPTIONS] <path>
+          [-move | -delete | -openforwrite]
+          [-files [-blocks [-locations | -racks]]]
+          [-showprogress]>>>
+
+*------------------+---------------------------------------------+
+||  COMMAND_OPTION || Description
+*------------------+---------------------------------------------+
+|   <path>         | Start checking from this path.
+*------------------+---------------------------------------------+
+|   -move          | Move corrupted files to /lost+found
+*------------------+---------------------------------------------+
+|   -delete        | Delete corrupted files.
+*------------------+---------------------------------------------+
+|   -openforwrite  | Print out files opened for write.
+*------------------+---------------------------------------------+
+|   -files         | Print out files being checked.
+*------------------+---------------------------------------------+
+|   -blocks        | Print out block report.
+*------------------+---------------------------------------------+
+|   -locations     | Print out locations for every block.
+*------------------+---------------------------------------------+
+|   -racks         | Print out network topology for data-node locations.
+*------------------+---------------------------------------------+
+|   -showprogress  | Print out dots for progress in output. Default is OFF
+|                  | (no progress).
+*------------------+---------------------------------------------+
+
+* Administration Commands
+
+   Commands useful for administrators of a hadoop cluster.
+
+** <<<balancer>>>
+
+   Runs a cluster balancing utility. An administrator can simply press Ctrl-C
+   to stop the rebalancing process. See
+   {{{./HdfsUserGuide.html#Balancer}Balancer}} for more details.
+
+   Usage: <<<hdfs balancer [-threshold <threshold>] [-policy <policy>]>>>
+
+*------------------------+----------------------------------------------------+
+|| COMMAND_OPTION        | Description
+*------------------------+----------------------------------------------------+
+| -threshold <threshold> | Percentage of disk capacity. This overwrites the
+|                        | default threshold.
+*------------------------+----------------------------------------------------+
+| -policy <policy>       | <<<datanode>>> (default): Cluster is balanced if
+|                        | each datanode is balanced. \
+|                        | <<<blockpool>>>: Cluster is balanced if each block
+|                        | pool in each datanode is balanced.
+*------------------------+----------------------------------------------------+
+
+   Note that the <<<blockpool>>> policy is more strict than the <<<datanode>>>
+   policy.
+
+** <<<datanode>>>
+
+   Runs a HDFS datanode.
+
+   Usage: <<<hdfs datanode [-regular | -rollback | -rollingupgrace rollback]>>>
+
+*-----------------+-----------------------------------------------------------+
+|| COMMAND_OPTION || Description
+*-----------------+-----------------------------------------------------------+
+| -regular        | Normal datanode startup (default).
+*-----------------+-----------------------------------------------------------+
+| -rollback       | Rollsback the datanode to the previous version. This should
+|                 | be used after stopping the datanode and distributing the
+|                 | old hadoop version.
+*-----------------+-----------------------------------------------------------+
+| -rollingupgrade rollback | Rollsback a rolling upgrade operation.
+*-----------------+-----------------------------------------------------------+
+
+** <<<dfsadmin>>>
+
+   Runs a HDFS dfsadmin client.
+
+   Usage: <<<hdfs dfsadmin [GENERIC_OPTIONS]
+          [-report [-live] [-dead] [-decommissioning]]
+          [-safemode enter | leave | get | wait]
+          [-saveNamespace]
+          [-rollEdits]
+          [-restoreFailedStorage true|false|check]
+          [-refreshNodes]
+          [-setQuota <quota> <dirname>...<dirname>]
+          [-clrQuota <dirname>...<dirname>]
+          [-setSpaceQuota <quota> <dirname>...<dirname>]
+          [-clrSpaceQuota <dirname>...<dirname>]
+          [-finalizeUpgrade]
+          [-rollingUpgrade [<query>|<prepare>|<finalize>]]
+          [-metasave filename]
+          [-refreshServiceAcl]
+          [-refreshUserToGroupsMappings]
+          [-refreshSuperUserGroupsConfiguration]
+          [-refreshCallQueue]
+          [-refresh <host:ipc_port> <key> [arg1..argn]]
+          [-printTopology]
+          [-refreshNamenodes datanodehost:port]
+          [-deleteBlockPool datanode-host:port blockpoolId [force]]
+          [-setBalancerBandwidth <bandwidth in bytes per second>]
+          [-allowSnapshot <snapshotDir>]
+          [-disallowSnapshot <snapshotDir>]
+          [-fetchImage <local directory>]
+          [-shutdownDatanode <datanode_host:ipc_port> [upgrade]]
+          [-getDatanodeInfo <datanode_host:ipc_port>]
+          [-help [cmd]]>>>
+
+*-----------------+-----------------------------------------------------------+
+|| COMMAND_OPTION || Description
+*-----------------+-----------------------------------------------------------+
+| -report [-live] [-dead] [-decommissioning] | Reports basic filesystem
+                  | information and statistics. Optional flags may be used to
+                  | filter the list of displayed DataNodes.
+*-----------------+-----------------------------------------------------------+
+| -safemode enter\|leave\|get\|wait | Safe mode maintenance command. Safe
+                  | mode is a Namenode state in which it \
+                  | 1. does not accept changes to the name space (read-only) \
+                  | 2. does not replicate or delete blocks. \
+                  | Safe mode is entered automatically at Namenode startup, and
+                  | leaves safe mode automatically when the configured minimum
+                  | percentage of blocks satisfies the minimum replication
+                  | condition. Safe mode can also be entered manually, but then
+                  | it can only be turned off manually as well.
+*-----------------+-----------------------------------------------------------+
+| -saveNamespace  | Save current namespace into storage directories and reset
+                  | edits log. Requires safe mode.
+*-----------------+-----------------------------------------------------------+
+| -rollEdits      | Rolls the edit log on the active NameNode.
+*-----------------+-----------------------------------------------------------+
+| -restoreFailedStorage true\|false\|check | This option will turn on/off
+                  | automatic attempt to restore failed storage replicas.
+                  | If a failed storage becomes available again the system will
+                  | attempt to restore edits and/or fsimage during checkpoint.
+                  | 'check' option will return current setting.
+*-----------------+-----------------------------------------------------------+
+| -refreshNodes   | Re-read the hosts and exclude files to update the set of
+                  | Datanodes that are allowed to connect to the Namenode and
+                  | those that should be decommissioned or recommissioned.
+*-----------------+-----------------------------------------------------------+
+| -setQuota \<quota\> \<dirname\>...\<dirname\> | See
+                  | {{{../hadoop-hdfs/HdfsQuotaAdminGuide.html#Administrative_Commands}HDFS Quotas Guide}}
+                  | for the detail.
+*-----------------+-----------------------------------------------------------+
+| -clrQuota \<dirname\>...\<dirname\> | See
+                  | {{{../hadoop-hdfs/HdfsQuotaAdminGuide.html#Administrative_Commands}HDFS Quotas Guide}}
+                  | for the detail.
+*-----------------+-----------------------------------------------------------+
+| -setSpaceQuota \<quota\> \<dirname\>...\<dirname\> | See
+                  | {{{../hadoop-hdfs/HdfsQuotaAdminGuide.html#Administrative_Commands}HDFS Quotas Guide}}
+                  | for the detail.
+*-----------------+-----------------------------------------------------------+
+| -clrSpaceQuota \<dirname\>...\<dirname\> | See
+                  | {{{../hadoop-hdfs/HdfsQuotaAdminGuide.html#Administrative_Commands}HDFS Quotas Guide}}
+                  | for the detail.
+*-----------------+-----------------------------------------------------------+
+| -finalizeUpgrade| Finalize upgrade of HDFS. Datanodes delete their previous
+                  | version working directories, followed by Namenode doing the
+                  | same. This completes the upgrade process.
+*-----------------+-----------------------------------------------------------+
+| -rollingUpgrade [\<query\>\|\<prepare\>\|\<finalize\>] | See
+                  | {{{../hadoop-hdfs/HdfsRollingUpgrade.html#dfsadmin_-rollingUpgrade}Rolling Upgrade document}}
+                  | for the detail.
+*-----------------+-----------------------------------------------------------+
+| -metasave filename | Save Namenode's primary data structures to <filename> in
+                  | the directory specified by hadoop.log.dir property.
+                  | <filename> is overwritten if it exists.
+                  | <filename> will contain one line for each of the following\
+                  | 1. Datanodes heart beating with Namenode\
+                  | 2. Blocks waiting to be replicated\
+                  | 3. Blocks currrently being replicated\
+                  | 4. Blocks waiting to be deleted
+*-----------------+-----------------------------------------------------------+
+| -refreshServiceAcl | Reload the service-level authorization policy file.
+*-----------------+-----------------------------------------------------------+
+| -refreshUserToGroupsMappings | Refresh user-to-groups mappings.
+*-----------------+-----------------------------------------------------------+
+| -refreshSuperUserGroupsConfiguration |Refresh superuser proxy groups mappings
+*-----------------+-----------------------------------------------------------+
+| -refreshCallQueue | Reload the call queue from config.
+*-----------------+-----------------------------------------------------------+
+| -refresh \<host:ipc_port\> \<key\> [arg1..argn] | Triggers a runtime-refresh
+                  | of the resource specified by \<key\> on \<host:ipc_port\>.
+                  | All other args after are sent to the host.
+*-----------------+-----------------------------------------------------------+
+| -printTopology  | Print a tree of the racks and their nodes as reported by
+                  | the Namenode
+*-----------------+-----------------------------------------------------------+
+| -refreshNamenodes datanodehost:port | For the given datanode, reloads the
+                  | configuration files, stops serving the removed block-pools
+                  | and starts serving new block-pools.
+*-----------------+-----------------------------------------------------------+
+| -deleteBlockPool datanode-host:port blockpoolId [force] | If force is passed,
+                  | block pool directory for the given blockpool id on the
+                  | given datanode is deleted along with its contents,
+                  | otherwise the directory is deleted only if it is empty.
+                  | The command will fail if datanode is still serving the
+                  | block pool. Refer to refreshNamenodes to shutdown a block
+                  | pool service on a datanode.
+*-----------------+-----------------------------------------------------------+
+| -setBalancerBandwidth \<bandwidth in bytes per second\> | Changes the network
+                  | bandwidth used by each datanode during HDFS block
+                  | balancing. \<bandwidth\> is the maximum number of bytes per
+                  | second that will be used by each datanode. This value
+                  | overrides the dfs.balance.bandwidthPerSec parameter.\
+                  | NOTE: The new value is not persistent on the DataNode.
+*-----------------+-----------------------------------------------------------+
+| -allowSnapshot \<snapshotDir\> | Allowing snapshots of a directory to be
+                  | created. If the operation completes successfully, the
+                  | directory becomes snapshottable.
+*-----------------+-----------------------------------------------------------+
+| -disallowSnapshot \<snapshotDir\> | Disallowing snapshots of a directory to
+                  | be created. All snapshots of the directory must be deleted
+                  | before disallowing snapshots.
+*-----------------+-----------------------------------------------------------+
+| -fetchImage \<local directory\> | Downloads the most recent fsimage from the
+                  | NameNode and saves it in the specified local directory.
+*-----------------+-----------------------------------------------------------+
+| -shutdownDatanode \<datanode_host:ipc_port\> [upgrade] | Submit a shutdown
+                  | request for the given datanode. See
+                  | {{{./HdfsRollingUpgrade.html#dfsadmin_-shutdownDatanode}Rolling Upgrade document}}
+                  | for the detail.
+*-----------------+-----------------------------------------------------------+
+| -getDatanodeInfo \<datanode_host:ipc_port\> | Get the information about the
+                  | given datanode. See
+                  | {{{./HdfsRollingUpgrade.html#dfsadmin_-getDatanodeInfo}Rolling Upgrade document}}
+                  | for the detail.
+*-----------------+-----------------------------------------------------------+
+| -help [cmd]     | Displays help for the given command or all commands if none
+                  | is specified.
+*-----------------+-----------------------------------------------------------+
+
+** <<<namenode>>>
+
+   Runs the namenode. More info about the upgrade, rollback and finalize is at
+   {{{./HdfsUserGuide.html#Upgrade_and_Rollback}Upgrade Rollback}}.
+
+   Usage: <<<hdfs namenode [-backup] |
+          [-checkpoint] |
+          [-format [-clusterid cid ] [-force] [-nonInteractive] ] |
+          [-upgrade [-clusterid cid] [-renameReserved<k-v pairs>] ] |
+          [-upgradeOnly [-clusterid cid] [-renameReserved<k-v pairs>] ] |
+          [-rollback] |
+          [-rollingUpgrade <downgrade|rollback> ] |
+          [-finalize] |
+          [-importCheckpoint] |
+          [-initializeSharedEdits] |
+          [-bootstrapStandby] |
+          [-recover [-force] ] |
+          [-metadataVersion ]>>>
+
+*--------------------+--------------------------------------------------------+
+|| COMMAND_OPTION    || Description
+*--------------------+--------------------------------------------------------+
+| -backup            | Start backup node.
+*--------------------+--------------------------------------------------------+
+| -checkpoint        | Start checkpoint node.
+*--------------------+--------------------------------------------------------+
+| -format [-clusterid cid] [-force] [-nonInteractive] | Formats the specified
+                     | NameNode. It starts the NameNode, formats it and then
+                     | shut it down. -force option formats if the name
+                     | directory exists. -nonInteractive option aborts if the
+                     | name directory exists, unless -force option is specified.
+*--------------------+--------------------------------------------------------+
+| -upgrade [-clusterid cid] [-renameReserved\<k-v pairs\>] | Namenode should be
+                     | started with upgrade option after
+                     | the distribution of new Hadoop version.
+*--------------------+--------------------------------------------------------+
+| -upgradeOnly [-clusterid cid] [-renameReserved\<k-v pairs\>] | Upgrade the
+                     | specified NameNode and then shutdown it.
+*--------------------+--------------------------------------------------------+
+| -rollback          | Rollsback the NameNode to the previous version. This
+                     | should be used after stopping the cluster and
+                     | distributing the old Hadoop version.
+*--------------------+--------------------------------------------------------+
+| -rollingUpgrade \<downgrade\|rollback\|started\> | See
+                     | {{{./HdfsRollingUpgrade.html#NameNode_Startup_Options}Rolling Upgrade document}}
+                     | for the detail.
+*--------------------+--------------------------------------------------------+
+| -finalize          | Finalize will remove the previous state of the files
+                     | system. Recent upgrade will become permanent.  Rollback
+                     | option will not be available anymore. After finalization
+                     | it shuts the NameNode down.
+*--------------------+--------------------------------------------------------+
+| -importCheckpoint  | Loads image from a checkpoint directory and save it
+                     | into the current one. Checkpoint dir is read from
+                     | property fs.checkpoint.dir
+*--------------------+--------------------------------------------------------+
+| -initializeSharedEdits | Format a new shared edits dir and copy in enough
+                     | edit log segments so that the standby NameNode can start
+                     | up.
+*--------------------+--------------------------------------------------------+
+| -bootstrapStandby  | Allows the standby NameNode's storage directories to be
+                     | bootstrapped by copying the latest namespace snapshot
+                     | from the active NameNode. This is used when first
+                     | configuring an HA cluster.
+*--------------------+--------------------------------------------------------+
+| -recover [-force]  | Recover lost metadata on a corrupt filesystem. See
+                     | {{{./HdfsUserGuide.html#Recovery_Mode}HDFS User Guide}}
+                     | for the detail.
+*--------------------+--------------------------------------------------------+
+| -metadataVersion   | Verify that configured directories exist, then print the
+                     | metadata versions of the software and the image.
+*--------------------+--------------------------------------------------------+
+
+** <<<secondarynamenode>>>
+
+   Runs the HDFS secondary namenode.
+   See {{{./HdfsUserGuide.html#Secondary_NameNode}Secondary Namenode}}
+   for more info.
+
+   Usage: <<<hdfs secondarynamenode [-checkpoint [force]] | [-format] |
+          [-geteditsize]>>>
+
+*----------------------+------------------------------------------------------+
+|| COMMAND_OPTION      || Description
+*----------------------+------------------------------------------------------+
+| -checkpoint [force]  | Checkpoints the SecondaryNameNode if EditLog size
+                       | >= fs.checkpoint.size. If <<<force>>> is used,
+                       | checkpoint irrespective of EditLog size.
+*----------------------+------------------------------------------------------+
+| -format              | Format the local storage during startup.
+*----------------------+------------------------------------------------------+
+| -geteditsize         | Prints the number of uncheckpointed transactions on
+                       | the NameNode.
+*----------------------+------------------------------------------------------+

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

@@ -143,8 +143,8 @@ HDFS Users Guide
 
 **  DFSAdmin Command
 
-   The <<<bin/hadoop dfsadmin>>> command supports a few HDFS administration
-   related operations. The <<<bin/hadoop dfsadmin -help>>> command lists all the
+   The <<<bin/hdfs dfsadmin>>> command supports a few HDFS administration
+   related operations. The <<<bin/hdfs dfsadmin -help>>> command lists all the
    commands currently supported. For e.g.:
 
      * <<<-report>>>: reports basic statistics of HDFS. Some of this
@@ -172,7 +172,7 @@ HDFS Users Guide
        of racks and datanodes attached to the tracks as viewed by the
        NameNode.
 
-   For command usage, see {{{../hadoop-common/CommandsManual.html#dfsadmin}dfsadmin}}.
+   For command usage, see {{{./HDFSCommands.html#dfsadmin}dfsadmin}}.
 
 * Secondary NameNode
 
@@ -207,7 +207,7 @@ HDFS Users Guide
    primary NameNode if necessary.
 
    For command usage,
-   see {{{../hadoop-common/CommandsManual.html#secondarynamenode}secondarynamenode}}.
+   see {{{./HDFSCommands.html#secondarynamenode}secondarynamenode}}.
 
 * Checkpoint Node
 
@@ -249,7 +249,7 @@ HDFS Users Guide
    Multiple checkpoint nodes may be specified in the cluster configuration
    file.
 
-   For command usage, see {{{../hadoop-common/CommandsManual.html#namenode}namenode}}.
+   For command usage, see {{{./HDFSCommands.html#namenode}namenode}}.
 
 * Backup Node
 
@@ -291,7 +291,7 @@ HDFS Users Guide
 
    For a complete discussion of the motivation behind the creation of the
    Backup node and Checkpoint node, see {{{https://issues.apache.org/jira/browse/HADOOP-4539}HADOOP-4539}}.
-   For command usage, see {{{../hadoop-common/CommandsManual.html#namenode}namenode}}.
+   For command usage, see {{{./HDFSCommands.html#namenode}namenode}}.
 
 * Import Checkpoint
 
@@ -314,7 +314,7 @@ HDFS Users Guide
    verifies that the image in <<<dfs.namenode.checkpoint.dir>>> is consistent,
    but does not modify it in any way.
 
-   For command usage, see {{{../hadoop-common/CommandsManual.html#namenode}namenode}}.
+   For command usage, see {{{./HDFSCommands.html#namenode}namenode}}.
 
 * Balancer
 
@@ -341,7 +341,7 @@ HDFS Users Guide
    A brief administrator's guide for balancer is available at 
    {{{https://issues.apache.org/jira/browse/HADOOP-1652}HADOOP-1652}}.
 
-   For command usage, see {{{../hadoop-common/CommandsManual.html#balancer}balancer}}.
+   For command usage, see {{{./HDFSCommands.html#balancer}balancer}}.
 
 * Rack Awareness
 
@@ -368,7 +368,7 @@ HDFS Users Guide
    allow any modifications to file system or blocks. Normally the NameNode
    leaves Safemode automatically after the DataNodes have reported that
    most file system blocks are available. If required, HDFS could be
-   placed in Safemode explicitly using <<<bin/hadoop dfsadmin -safemode>>>
+   placed in Safemode explicitly using <<<bin/hdfs dfsadmin -safemode>>>
    command. NameNode front page shows whether Safemode is on or off. A
    more detailed description and configuration is maintained as JavaDoc
    for <<<setSafeMode()>>>.
@@ -383,8 +383,8 @@ HDFS Users Guide
    most of the recoverable failures. By default fsck ignores open files
    but provides an option to select all files during reporting. The HDFS
    fsck command is not a Hadoop shell command. It can be run as
-   <<<bin/hadoop fsck>>>. For command usage, see 
-   {{{../hadoop-common/CommandsManual.html#fsck}fsck}}. fsck can be run on
+   <<<bin/hdfs fsck>>>. For command usage, see
+   {{{./HDFSCommands.html#fsck}fsck}}. fsck can be run on
    the whole file system or on a subset of files.
 
 * fetchdt
@@ -395,11 +395,11 @@ HDFS Users Guide
    Utility uses either RPC or HTTPS (over Kerberos) to get the token, and
    thus requires kerberos tickets to be present before the run (run kinit
    to get the tickets). The HDFS fetchdt command is not a Hadoop shell
-   command. It can be run as <<<bin/hadoop fetchdt DTfile>>>. After you got
+   command. It can be run as <<<bin/hdfs fetchdt DTfile>>>. After you got
    the token you can run an HDFS command without having Kerberos tickets,
    by pointing <<<HADOOP_TOKEN_FILE_LOCATION>>> environmental variable to the
    delegation token file. For command usage, see
-   {{{../hadoop-common/CommandsManual.html#fetchdt}fetchdt}} command.
+   {{{./HDFSCommands.html#fetchdt}fetchdt}} command.
 
 * Recovery Mode
 
@@ -533,5 +533,4 @@ HDFS Users Guide
      * Explore {{{./hdfs-default.xml}hdfs-default.xml}}. It includes
        brief description of most of the configuration variables available.
 
-     * {{{../hadoop-common/CommandsManual.html}Hadoop Commands Guide}}:
-       Hadoop commands usage.
+     * {{{./HDFSCommands.html}HDFS Commands Guide}}: HDFS commands usage.

+ 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"));
+  }
 }

+ 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);
+  }
+
 }

+ 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);
+  }
 }

+ 87 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java

@@ -31,19 +31,24 @@ import java.util.Random;
 
 import org.apache.commons.io.output.ByteArrayOutputStream;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
@@ -89,6 +94,8 @@ public class TestDecommissioningStatus {
         4);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1000);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY, 1);
+    conf.setLong(DFSConfigKeys.DFS_DATANODE_BALANCE_BANDWIDTHPERSEC_KEY, 1);
+
     writeConfigFile(localFileSys, excludeFile, null);
     writeConfigFile(localFileSys, includeFile, null);
 
@@ -99,6 +106,7 @@ public class TestDecommissioningStatus {
 
   @AfterClass
   public static void tearDown() throws Exception {
+    if (localFileSys != null ) cleanupFile(localFileSys, dir);
     if(fileSys != null) fileSys.close();
     if(cluster != null) cluster.shutdown();
   }
@@ -138,7 +146,8 @@ public class TestDecommissioningStatus {
     return stm;
   }
   
-  private void cleanupFile(FileSystem fileSys, Path name) throws IOException {
+  static private void cleanupFile(FileSystem fileSys, Path name)
+      throws IOException {
     assertTrue(fileSys.exists(name));
     fileSys.delete(name, true);
     assertTrue(!fileSys.exists(name));
@@ -147,19 +156,26 @@ public class TestDecommissioningStatus {
   /*
    * Decommissions the node at the given index
    */
-  private String decommissionNode(FSNamesystem namesystem,
-      DFSClient client, FileSystem localFileSys, int nodeIndex)
-      throws IOException {
+  private String decommissionNode(FSNamesystem namesystem, DFSClient client,
+      FileSystem localFileSys, int nodeIndex) throws IOException {
     DatanodeInfo[] info = client.datanodeReport(DatanodeReportType.LIVE);
 
     String nodename = info[nodeIndex].getXferAddr();
-    System.out.println("Decommissioning node: " + nodename);
+    decommissionNode(namesystem, localFileSys, nodename);
+    return nodename;
+  }
+
+  /*
+   * Decommissions the node by name
+   */
+  private void decommissionNode(FSNamesystem namesystem,
+      FileSystem localFileSys, String dnName) throws IOException {
+    System.out.println("Decommissioning node: " + dnName);
 
     // write nodename into the exclude file.
     ArrayList<String> nodes = new ArrayList<String>(decommissionedNodes);
-    nodes.add(nodename);
+    nodes.add(dnName);
     writeConfigFile(localFileSys, excludeFile, nodes);
-    return nodename;
   }
 
   private void checkDecommissionStatus(DatanodeDescriptor decommNode,
@@ -276,6 +292,69 @@ public class TestDecommissioningStatus {
     st1.close();
     cleanupFile(fileSys, file1);
     cleanupFile(fileSys, file2);
-    cleanupFile(localFileSys, dir);
+  }
+
+  /**
+   * Verify a DN remains in DECOMMISSION_INPROGRESS state if it is marked
+   * as dead before decommission has completed. That will allow DN to resume
+   * the replication process after it rejoins the cluster.
+   */
+  @Test(timeout=120000)
+  public void testDecommissionStatusAfterDNRestart()
+      throws IOException, InterruptedException {
+    DistributedFileSystem fileSys =
+        (DistributedFileSystem)cluster.getFileSystem();
+
+    // Create a file with one block. That block has one replica.
+    Path f = new Path("decommission.dat");
+    DFSTestUtil.createFile(fileSys, f, fileSize, fileSize, fileSize,
+        (short)1, seed);
+
+    // Find the DN that owns the only replica.
+    RemoteIterator<LocatedFileStatus> fileList = fileSys.listLocatedStatus(f);
+    BlockLocation[] blockLocations = fileList.next().getBlockLocations();
+    String dnName = blockLocations[0].getNames()[0];
+
+    // Decommission the DN.
+    FSNamesystem fsn = cluster.getNamesystem();
+    final DatanodeManager dm = fsn.getBlockManager().getDatanodeManager();
+    decommissionNode(fsn, localFileSys, dnName);
+    dm.refreshNodes(conf);
+
+    // Stop the DN when decommission is in progress.
+    // Given DFS_DATANODE_BALANCE_BANDWIDTHPERSEC_KEY is to 1 and the size of
+    // the block, it will take much longer time that test timeout value for
+    // the decommission to complete. So when stopDataNode is called,
+    // decommission should be in progress.
+    DataNodeProperties dataNodeProperties = cluster.stopDataNode(dnName);
+    final List<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
+    while (true) {
+      dm.fetchDatanodes(null, dead, false);
+      if (dead.size() == 1) {
+        break;
+      }
+      Thread.sleep(1000);
+    }
+
+    // Force removal of the dead node's blocks.
+    BlockManagerTestUtil.checkHeartbeat(fsn.getBlockManager());
+
+    // Force DatanodeManager to check decommission state.
+    BlockManagerTestUtil.checkDecommissionState(dm, dead.get(0));
+
+    // Verify that the DN remains in DECOMMISSION_INPROGRESS state.
+    assertTrue("the node is in decommissioned state ",
+        !dead.get(0).isDecommissioned());
+
+    // Add the node back
+    cluster.restartDataNode(dataNodeProperties, true);
+    cluster.waitActive();
+
+    // Call refreshNodes on FSNamesystem with empty exclude file.
+    // This will remove the datanodes from decommissioning list and
+    // make them available again.
+    writeConfigFile(localFileSys, excludeFile, null);
+    dm.refreshNodes(conf);
+    cleanupFile(fileSys, f);
   }
 }

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

@@ -181,6 +181,12 @@ Release 2.6.0 - UNRELEASED
     MAPREDUCE-5756. CombineFileInputFormat.getSplits() including directories
     in its results (Jason Dere via jlowe)
 
+    MAPREDUCE-6014. New task status field in task attempts table can lead to
+    an empty web page (Mit Desai via jlowe)
+
+    MAPREDUCE-6021. MR AM should have working directory in LD_LIBRARY_PATH
+    (jlowe)
+
 Release 2.5.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 2 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java

@@ -85,7 +85,8 @@ public class TaskPage extends AppView {
         .append(ta.getId()).append("\",\"")
         .append(progress).append("\",\"")
         .append(ta.getState().toString()).append("\",\"")
-        .append(ta.getStatus()).append("\",\"")
+        .append(StringEscapeUtils.escapeJavaScript(
+              StringEscapeUtils.escapeHtml(ta.getStatus()))).append("\",\"")
 
         .append(nodeHttpAddr == null ? "N/A" :
           "<a class='nodelink' href='" + MRWebAppUtil.getYARNWebappScheme() + nodeHttpAddr + "'>"

+ 3 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksBlock.java

@@ -25,6 +25,7 @@ import static org.apache.hadoop.yarn.util.StringHelper.percent;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.C_PROGRESSBAR;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.C_PROGRESSBAR_VALUE;
 
+import org.apache.commons.lang.StringEscapeUtils;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
 import org.apache.hadoop.mapreduce.v2.app.job.Task;
 import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskInfo;
@@ -102,7 +103,8 @@ public class TasksBlock extends HtmlBlock {
       .append(join(pct, '%')).append("'> ").append("<div class='")
       .append(C_PROGRESSBAR_VALUE).append("' style='")
       .append(join("width:", pct, '%')).append("'> </div> </div>\",\"")
-      .append(info.getStatus()).append("\",\"")
+      .append(StringEscapeUtils.escapeJavaScript(
+              StringEscapeUtils.escapeHtml(info.getStatus()))).append("\",\"")
 
       .append(info.getState()).append("\",\"")
       .append(info.getStartTime()).append("\",\"")

+ 3 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestBlocks.java

@@ -106,6 +106,7 @@ public class TestBlocks {
     when(report.getTaskState()).thenReturn(TaskState.SUCCEEDED);
     when(report.getStartTime()).thenReturn(100001L);
     when(report.getFinishTime()).thenReturn(100011L);
+    when(report.getStatus()).thenReturn("Dummy Status \n*");
 
 
     when(task.getReport()).thenReturn(report);
@@ -134,6 +135,8 @@ public class TestBlocks {
     assertTrue(data.toString().contains("SUCCEEDED"));
     assertTrue(data.toString().contains("100001"));
     assertTrue(data.toString().contains("100011"));
+    assertFalse(data.toString().contains("Dummy Status \n*"));
+    assertTrue(data.toString().contains("Dummy Status \\n*"));
 
 
   }

+ 2 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTaskPage.java

@@ -149,7 +149,8 @@ public class HsTaskPage extends HsView {
         attemptsTableData.append("[\"")
         .append(sortId + " ").append(taid).append("\",\"")
         .append(ta.getState()).append("\",\"")
-        .append(ta.getStatus()).append("\",\"")
+        .append(StringEscapeUtils.escapeJavaScript(
+              StringEscapeUtils.escapeHtml(ta.getStatus()))).append("\",\"")
 
         .append("<a class='nodelink' href='" + MRWebAppUtil.getYARNWebappScheme() + nodeHttpAddr + "'>")
         .append(nodeRackName + "/" + nodeHttpAddr + "</a>\",\"")

+ 3 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestBlocks.java

@@ -159,7 +159,7 @@ public class TestBlocks {
     when(taReport.getSortFinishTime()).thenReturn(taSortFinishTime);
     when(taReport.getContainerId()).thenReturn(containerId);
     when(taReport.getProgress()).thenReturn(1.0f);
-    when(taReport.getStateString()).thenReturn("Processed 128/128 records");
+    when(taReport.getStateString()).thenReturn("Processed 128/128 records <p> \n");
     when(taReport.getTaskAttemptState()).thenReturn(taState);
     when(taReport.getDiagnosticInfo()).thenReturn("");
 
@@ -184,6 +184,8 @@ public class TestBlocks {
     // should be printed information about attempts
     assertTrue(data.toString().contains("0 attempt_0_0001_r_000000_0"));
     assertTrue(data.toString().contains("SUCCEEDED"));
+    assertFalse(data.toString().contains("Processed 128/128 records <p> \n"));
+    assertTrue(data.toString().contains("Processed 128\\/128 records &lt;p&gt; \\n"));
     assertTrue(data.toString().contains(
             "_0005_01_000001:attempt_0_0001_r_000000_0:User:"));
     assertTrue(data.toString().contains("100002"));

+ 9 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java

@@ -447,6 +447,15 @@ public class YARNRunner implements ClientProtocol {
     Map<String, String> environment = new HashMap<String, String>();
     MRApps.setClasspath(environment, conf);
 
+    // Shell
+    environment.put(Environment.SHELL.name(),
+        conf.get(MRJobConfig.MAPRED_ADMIN_USER_SHELL,
+            MRJobConfig.DEFAULT_SHELL));
+
+    // Add the container working directory at the front of LD_LIBRARY_PATH
+    MRApps.addToEnvironment(environment, Environment.LD_LIBRARY_PATH.name(),
+        MRApps.crossPlatformifyMREnv(conf, Environment.PWD), conf);
+
     // Setup the environment variables for Admin first
     MRApps.setEnvFromInputString(environment, 
         conf.get(MRJobConfig.MR_AM_ADMIN_USER_ENV), conf);

+ 73 - 31
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestYARNRunner.java

@@ -29,6 +29,7 @@ import static org.mockito.Mockito.when;
 
 import java.io.ByteArrayOutputStream;
 import java.io.File;
+import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
@@ -36,6 +37,7 @@ import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
 import java.security.PrivilegedExceptionAction;
 import java.util.List;
+import java.util.Map;
 
 import junit.framework.TestCase;
 
@@ -44,22 +46,27 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.JobPriority;
 import org.apache.hadoop.mapreduce.JobStatus.State;
+import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.TypeConverter;
 import org.apache.hadoop.mapreduce.v2.api.MRClientProtocol;
 import org.apache.hadoop.mapreduce.v2.api.MRDelegationTokenIdentifier;
 import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetDelegationTokenRequest;
 import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetDelegationTokenResponse;
+import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
@@ -94,6 +101,7 @@ import org.apache.log4j.Layout;
 import org.apache.log4j.Logger;
 import org.apache.log4j.SimpleLayout;
 import org.apache.log4j.WriterAppender;
+import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.invocation.InvocationOnMock;
@@ -146,8 +154,12 @@ public class TestYARNRunner extends TestCase {
       FileContext.getLocalFSFileContext().delete(new Path(testWorkDir.toString()), true);
     }
     testWorkDir.mkdirs();
-   }
+  }
 
+  @After
+  public void cleanup() {
+    FileUtil.fullyDelete(testWorkDir);
+  }
 
   @Test(timeout=20000)
   public void testJobKill() throws Exception {
@@ -397,21 +409,8 @@ public class TestYARNRunner extends TestCase {
     
     YARNRunner yarnRunner = new YARNRunner(jobConf);
     
-    File jobxml = new File(testWorkDir, MRJobConfig.JOB_CONF_FILE);
-    OutputStream out = new FileOutputStream(jobxml);
-    conf.writeXml(out);
-    out.close();
-    
-    File jobsplit = new File(testWorkDir, MRJobConfig.JOB_SPLIT);
-    out = new FileOutputStream(jobsplit);
-    out.close();
-    
-    File jobsplitmetainfo = new File(testWorkDir, MRJobConfig.JOB_SPLIT_METAINFO);
-    out = new FileOutputStream(jobsplitmetainfo);
-    out.close();
-    
-    ApplicationSubmissionContext submissionContext = 
-        yarnRunner.createApplicationSubmissionContext(jobConf, testWorkDir.toString(), new Credentials());
+    ApplicationSubmissionContext submissionContext =
+        buildSubmitContext(yarnRunner, jobConf);
     
     ContainerLaunchContext containerSpec = submissionContext.getAMContainerSpec();
     List<String> commands = containerSpec.getCommands();
@@ -463,22 +462,9 @@ public class TestYARNRunner extends TestCase {
     
     YARNRunner yarnRunner = new YARNRunner(jobConf);
     
-    File jobxml = new File(testWorkDir, MRJobConfig.JOB_CONF_FILE);
-    OutputStream out = new FileOutputStream(jobxml);
-    conf.writeXml(out);
-    out.close();
-    
-    File jobsplit = new File(testWorkDir, MRJobConfig.JOB_SPLIT);
-    out = new FileOutputStream(jobsplit);
-    out.close();
-    
-    File jobsplitmetainfo = new File(testWorkDir, MRJobConfig.JOB_SPLIT_METAINFO);
-    out = new FileOutputStream(jobsplitmetainfo);
-    out.close();
-    
     @SuppressWarnings("unused")
-    ApplicationSubmissionContext submissionContext = 
-        yarnRunner.createApplicationSubmissionContext(jobConf, testWorkDir.toString(), new Credentials());
+    ApplicationSubmissionContext submissionContext =
+        buildSubmitContext(yarnRunner, jobConf);
    
     String logMsg = bout.toString();
     assertTrue(logMsg.contains("WARN - Usage of -Djava.library.path in " + 
@@ -492,4 +478,60 @@ public class TestYARNRunner extends TestCase {
         "be set as part of the LD_LIBRARY_PATH in the app master JVM env " +
         "using yarn.app.mapreduce.am.env config settings."));
   }
+
+  @Test
+  public void testAMStandardEnv() throws Exception {
+    final String ADMIN_LIB_PATH = "foo";
+    final String USER_LIB_PATH = "bar";
+    final String USER_SHELL = "shell";
+    JobConf jobConf = new JobConf();
+
+    jobConf.set(MRJobConfig.MR_AM_ADMIN_USER_ENV, "LD_LIBRARY_PATH=" +
+        ADMIN_LIB_PATH);
+    jobConf.set(MRJobConfig.MR_AM_ENV, "LD_LIBRARY_PATH="
+        + USER_LIB_PATH);
+    jobConf.set(MRJobConfig.MAPRED_ADMIN_USER_SHELL, USER_SHELL);
+
+    YARNRunner yarnRunner = new YARNRunner(jobConf);
+    ApplicationSubmissionContext appSubCtx =
+        buildSubmitContext(yarnRunner, jobConf);
+
+    // make sure PWD is first in the lib path
+    ContainerLaunchContext clc = appSubCtx.getAMContainerSpec();
+    Map<String, String> env = clc.getEnvironment();
+    String libPath = env.get(Environment.LD_LIBRARY_PATH.name());
+    assertNotNull("LD_LIBRARY_PATH not set", libPath);
+    String cps = jobConf.getBoolean(
+        MRConfig.MAPREDUCE_APP_SUBMISSION_CROSS_PLATFORM,
+        MRConfig.DEFAULT_MAPREDUCE_APP_SUBMISSION_CROSS_PLATFORM)
+        ? ApplicationConstants.CLASS_PATH_SEPARATOR : File.pathSeparator;
+    assertEquals("Bad AM LD_LIBRARY_PATH setting",
+        MRApps.crossPlatformifyMREnv(conf, Environment.PWD)
+        + cps + ADMIN_LIB_PATH + cps + USER_LIB_PATH, libPath);
+
+    // make sure SHELL is set
+    String shell = env.get(Environment.SHELL.name());
+    assertNotNull("SHELL not set", shell);
+    assertEquals("Bad SHELL setting", USER_SHELL, shell);
+  }
+
+  private ApplicationSubmissionContext buildSubmitContext(
+      YARNRunner yarnRunner, JobConf jobConf) throws IOException {
+    File jobxml = new File(testWorkDir, MRJobConfig.JOB_CONF_FILE);
+    OutputStream out = new FileOutputStream(jobxml);
+    conf.writeXml(out);
+    out.close();
+
+    File jobsplit = new File(testWorkDir, MRJobConfig.JOB_SPLIT);
+    out = new FileOutputStream(jobsplit);
+    out.close();
+
+    File jobsplitmetainfo = new File(testWorkDir,
+        MRJobConfig.JOB_SPLIT_METAINFO);
+    out = new FileOutputStream(jobsplitmetainfo);
+    out.close();
+
+    return yarnRunner.createApplicationSubmissionContext(jobConf,
+        testWorkDir.toString(), new Credentials());
+  }
 }

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

@@ -69,6 +69,7 @@
     
     <menu name="HDFS" inherit="top">
       <item name="HDFS User Guide" href="hadoop-project-dist/hadoop-hdfs/HdfsUserGuide.html"/>
+      <item name="HDFS Commands Reference" href="hadoop-project-dist/hadoop-hdfs/HDFSCommands.html"/>
       <item name="High Availability With QJM" href="hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithQJM.html"/>
       <item name="High Availability With NFS" href="hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html"/>
       <item name="Federation" href="hadoop-project-dist/hadoop-hdfs/Federation.html"/>

+ 5 - 0
hadoop-tools/hadoop-openstack/pom.xml

@@ -145,6 +145,11 @@
       <artifactId>junit</artifactId>
       <scope>compile</scope>
     </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>compile</scope>
+    </dependency>
     <dependency>
       <groupId>com.google.guava</groupId>
       <artifactId>guava</artifactId>

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

@@ -88,6 +88,12 @@ Release 2.6.0 - UNRELEASED
     YARN-2370. Fix comment in o.a.h.y.server.resourcemanager.schedulerAppSchedulingInfo 
     (Wenwu Peng via junping_du)
 
+    YARN-2298. Move TimelineClient to yarn-common project (Zhijie Shen via 
+    junping_du)
+
+    YARN-2288. Made persisted data in LevelDB timeline store be versioned. (Junping Du
+    via zjshen)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -131,6 +137,17 @@ Release 2.6.0 - UNRELEASED
     YARN-2051. Fix bug in PBimpls and add more unit tests with reflection. 
     (Binglin Chang via junping_du)
 
+    YARN-2374. Fixed TestDistributedShell#testDSShell failure due to hostname
+    dismatch. (Varun Vasudev via jianhe)
+
+    YARN-2359. Application hangs when it fails to launch AM container. 
+    (Zhihai Xu via kasha)
+
+    YARN-2388. Fixed TestTimelineWebServices failure due to HADOOP-10791. (zjshen)
+
+    YARN-2008. Fixed CapacityScheduler to calculate headroom based on max available
+    capacity instead of configured max capacity. (Craig Welch via jianhe)
+
 Release 2.5.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 71 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java

@@ -26,13 +26,13 @@ import java.io.FileReader;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.io.PrintWriter;
+import java.net.InetAddress;
 import java.net.URL;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.junit.Assert;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -169,7 +169,9 @@ public class TestDistributedShell {
     yarnClient.init(new Configuration(yarnCluster.getConfig()));
     yarnClient.start();
     String hostName = NetUtils.getHostname();
+
     boolean verified = false;
+    String errorMessage = "";
     while(!verified) {
       List<ApplicationReport> apps = yarnClient.getApplications();
       if (apps.size() == 0 ) {
@@ -177,15 +179,22 @@ public class TestDistributedShell {
         continue;
       }
       ApplicationReport appReport = apps.get(0);
-      if (appReport.getHost().startsWith(hostName)
-          && appReport.getRpcPort() == -1) {
+      if(appReport.getHost().equals("N/A")) {
+        Thread.sleep(10);
+        continue;
+      }
+      errorMessage =
+          "Expected host name to start with '" + hostName + "', was '"
+              + appReport.getHost() + "'. Expected rpc port to be '-1', was '"
+              + appReport.getRpcPort() + "'.";
+      if (checkHostname(appReport.getHost()) && appReport.getRpcPort() == -1) {
         verified = true;
       }
       if (appReport.getYarnApplicationState() == YarnApplicationState.FINISHED) {
         break;
       }
     }
-    Assert.assertTrue(verified);
+    Assert.assertTrue(errorMessage, verified);
     t.join();
     LOG.info("Client run completed. Result=" + result);
     Assert.assertTrue(result.get());
@@ -212,6 +221,64 @@ public class TestDistributedShell {
         .toString(), ApplicationMaster.DSEntity.DS_CONTAINER.toString());
   }
 
+  /*
+   * NetUtils.getHostname() returns a string in the form "hostname/ip".
+   * Sometimes the hostname we get is the FQDN and sometimes the short name. In
+   * addition, on machines with multiple network interfaces, it runs any one of
+   * the ips. The function below compares the returns values for
+   * NetUtils.getHostname() accounting for the conditions mentioned.
+   */
+  private boolean checkHostname(String appHostname) throws Exception {
+
+    String hostname = NetUtils.getHostname();
+    if (hostname.equals(appHostname)) {
+      return true;
+    }
+
+    Assert.assertTrue("Unknown format for hostname " + appHostname,
+      appHostname.contains("/"));
+    Assert.assertTrue("Unknown format for hostname " + hostname,
+      hostname.contains("/"));
+
+    String[] appHostnameParts = appHostname.split("/");
+    String[] hostnameParts = hostname.split("/");
+
+    return (compareFQDNs(appHostnameParts[0], hostnameParts[0]) && checkIPs(
+      hostnameParts[0], hostnameParts[1], appHostnameParts[1]));
+  }
+
+  private boolean compareFQDNs(String appHostname, String hostname)
+      throws Exception {
+    if (appHostname.equals(hostname)) {
+      return true;
+    }
+    String appFQDN = InetAddress.getByName(appHostname).getCanonicalHostName();
+    String localFQDN = InetAddress.getByName(hostname).getCanonicalHostName();
+    return appFQDN.equals(localFQDN);
+  }
+
+  private boolean checkIPs(String hostname, String localIP, String appIP)
+      throws Exception {
+
+    if (localIP.equals(appIP)) {
+      return true;
+    }
+    boolean appIPCheck = false;
+    boolean localIPCheck = false;
+    InetAddress[] addresses = InetAddress.getAllByName(hostname);
+    for (InetAddress ia : addresses) {
+      if (ia.getHostAddress().equals(appIP)) {
+        appIPCheck = true;
+        continue;
+      }
+      if (ia.getHostAddress().equals(localIP)) {
+        localIPCheck = true;
+      }
+    }
+    return (appIPCheck && localIPCheck);
+
+  }
+
   @Test(timeout=90000)
   public void testDSRestartWithPreviousRunningContainers() throws Exception {
     String[] args = {

+ 0 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/pom.xml

@@ -57,14 +57,6 @@
       <groupId>log4j</groupId>
       <artifactId>log4j</artifactId>
     </dependency>
-    <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>jetty-util</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>com.sun.jersey</groupId>
-      <artifactId>jersey-client</artifactId>
-    </dependency>
 
     <!-- 'mvn dependency:analyze' fails to detect use of this dependency -->
     <dependency>

+ 8 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml

@@ -66,10 +66,18 @@
       <groupId>commons-codec</groupId>
       <artifactId>commons-codec</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.mortbay.jetty</groupId>
+      <artifactId>jetty-util</artifactId>
+    </dependency>
     <dependency>
       <groupId>com.sun.jersey</groupId>
       <artifactId>jersey-core</artifactId>
     </dependency>
+    <dependency>
+      <groupId>com.sun.jersey</groupId>
+      <artifactId>jersey-client</artifactId>
+    </dependency>
     <dependency>
       <groupId>org.codehaus.jackson</groupId>
       <artifactId>jackson-core-asl</artifactId>

+ 0 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java


+ 0 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineAuthenticator.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineAuthenticator.java


+ 0 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java


+ 21 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/package-info.java

@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+@InterfaceAudience.Public
+package org.apache.hadoop.yarn.client.api.impl;
+import org.apache.hadoop.classification.InterfaceAudience;
+

+ 21 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/package-info.java

@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+@InterfaceAudience.Public
+package org.apache.hadoop.yarn.client.api;
+import org.apache.hadoop.classification.InterfaceAudience;
+

+ 7 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DefaultResourceCalculator.java

@@ -42,6 +42,13 @@ public class DefaultResourceCalculator extends ResourceCalculator {
       Resource numerator, Resource denominator) {
     return ratio(numerator, denominator);
   }
+  
+  public boolean isInvalidDivisor(Resource r) {
+    if (r.getMemory() == 0.0f) {
+      return true;
+    }
+    return false;
+  }
 
   @Override
   public float ratio(Resource a, Resource b) {

+ 8 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java

@@ -109,6 +109,14 @@ public class DominantResourceCalculator extends ResourceCalculator {
         getResourceAsValue(clusterResource, numerator, true) / 
         getResourceAsValue(clusterResource, denominator, true);
   }
+  
+  @Override
+  public boolean isInvalidDivisor(Resource r) {
+    if (r.getMemory() == 0.0f || r.getVirtualCores() == 0.0f) {
+      return true;
+    }
+    return false;
+  }
 
   @Override
   public float ratio(Resource a, Resource b) {

+ 9 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java

@@ -149,6 +149,15 @@ public abstract class ResourceCalculator {
   public abstract float divide(
       Resource clusterResource, Resource numerator, Resource denominator);
   
+  /**
+   * Determine if a resource is not suitable for use as a divisor
+   * (will result in divide by 0, etc)
+   *
+   * @param r resource
+   * @return true if divisor is invalid (should not be used), false else
+   */
+  public abstract boolean isInvalidDivisor(Resource r);
+
   /**
    * Ratio of resource <code>a</code> to resource <code>b</code>.
    * 

+ 5 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java

@@ -184,6 +184,11 @@ public class Resources {
     return calculator.roundDown(lhs, factor);
   }
   
+  public static boolean isInvalidDivisor(
+      ResourceCalculator resourceCalculator, Resource divisor) {
+    return resourceCalculator.isInvalidDivisor(divisor);
+  }
+
   public static float ratio(
       ResourceCalculator resourceCalculator, Resource lhs, Resource rhs) {
     return resourceCalculator.ratio(lhs, rhs);

+ 0 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineAuthenticator.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineAuthenticator.java


+ 0 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClient.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClient.java


+ 72 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/LeveldbTimelineStore.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.timeline;
 
 import static org.apache.hadoop.yarn.server.timeline.GenericObjectMapper.readReverseOrderedLong;
 import static org.apache.hadoop.yarn.server.timeline.GenericObjectMapper.writeReverseOrderedLong;
+import static org.fusesource.leveldbjni.JniDBFactory.bytes;
 
 import java.io.ByteArrayOutputStream;
 import java.io.File;
@@ -60,8 +61,12 @@ import org.apache.hadoop.yarn.api.records.timeline.TimelineEvents.EventsOfOneEnt
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse.TimelinePutError;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.VersionProto;
+import org.apache.hadoop.yarn.server.records.Version;
+import org.apache.hadoop.yarn.server.records.impl.pb.VersionPBImpl;
 import org.fusesource.leveldbjni.JniDBFactory;
 import org.iq80.leveldb.DB;
+import org.iq80.leveldb.DBException;
 import org.iq80.leveldb.DBIterator;
 import org.iq80.leveldb.Options;
 import org.iq80.leveldb.ReadOptions;
@@ -141,6 +146,11 @@ public class LeveldbTimelineStore extends AbstractService
       "z".getBytes();
 
   private static final byte[] EMPTY_BYTES = new byte[0];
+  
+  private static final String TIMELINE_STORE_VERSION_KEY = "timeline-store-version";
+  
+  private static final Version CURRENT_VERSION_INFO = Version
+      .newInstance(1, 0);
 
   @Private
   @VisibleForTesting
@@ -193,6 +203,7 @@ public class LeveldbTimelineStore extends AbstractService
     }
     LOG.info("Using leveldb path " + dbPath);
     db = factory.open(new File(dbPath.toString()), options);
+    checkVersion();
     startTimeWriteCache =
         Collections.synchronizedMap(new LRUMap(getStartTimeWriteCacheSize(
             conf)));
@@ -1270,8 +1281,6 @@ public class LeveldbTimelineStore extends AbstractService
             DEFAULT_TIMELINE_SERVICE_LEVELDB_START_TIME_WRITE_CACHE_SIZE);
   }
 
-  // warning is suppressed to prevent eclipse from noting unclosed resource
-  @SuppressWarnings("resource")
   @VisibleForTesting
   List<String> getEntityTypes() throws IOException {
     DBIterator iterator = null;
@@ -1489,4 +1498,65 @@ public class LeveldbTimelineStore extends AbstractService
     readOptions.fillCache(fillCache);
     return db.iterator(readOptions);
   }
+  
+  Version loadVersion() throws IOException {
+    byte[] data = db.get(bytes(TIMELINE_STORE_VERSION_KEY));
+    // if version is not stored previously, treat it as 1.0.
+    if (data == null || data.length == 0) {
+      return Version.newInstance(1, 0);
+    }
+    Version version =
+        new VersionPBImpl(VersionProto.parseFrom(data));
+    return version;
+  }
+  
+  // Only used for test
+  @VisibleForTesting
+  void storeVersion(Version state) throws IOException {
+    dbStoreVersion(state);
+  }
+  
+  private void dbStoreVersion(Version state) throws IOException {
+    String key = TIMELINE_STORE_VERSION_KEY;
+    byte[] data = 
+        ((VersionPBImpl) state).getProto().toByteArray();
+    try {
+      db.put(bytes(key), data);
+    } catch (DBException e) {
+      throw new IOException(e);
+    }
+  }
+
+  Version getCurrentVersion() {
+    return CURRENT_VERSION_INFO;
+  }
+  
+  /**
+   * 1) Versioning timeline store: major.minor. For e.g. 1.0, 1.1, 1.2...1.25, 2.0 etc.
+   * 2) Any incompatible change of TS-store is a major upgrade, and any
+   *    compatible change of TS-store is a minor upgrade.
+   * 3) Within a minor upgrade, say 1.1 to 1.2:
+   *    overwrite the version info and proceed as normal.
+   * 4) Within a major upgrade, say 1.2 to 2.0:
+   *    throw exception and indicate user to use a separate upgrade tool to
+   *    upgrade timeline store or remove incompatible old state.
+   */
+  private void checkVersion() throws IOException {
+    Version loadedVersion = loadVersion();
+    LOG.info("Loaded timeline store version info " + loadedVersion);
+    if (loadedVersion.equals(getCurrentVersion())) {
+      return;
+    }
+    if (loadedVersion.isCompatibleTo(getCurrentVersion())) {
+      LOG.info("Storing timeline store version info " + getCurrentVersion());
+      dbStoreVersion(CURRENT_VERSION_INFO);
+    } else {
+      String incompatibleMessage = 
+          "Incompatible version for timeline store: expecting version " 
+              + getCurrentVersion() + ", but loading version " + loadedVersion;
+      LOG.fatal(incompatibleMessage);
+      throw new IOException(incompatibleMessage);
+    }
+  }
+  
 }

+ 49 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestLeveldbTimelineStore.java

@@ -36,14 +36,17 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.service.ServiceStateException;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.records.Version;
 import org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore;
 import org.apache.hadoop.yarn.server.timeline.NameValuePair;
 import org.iq80.leveldb.DBIterator;
 import org.junit.After;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -52,19 +55,19 @@ import org.junit.Test;
 public class TestLeveldbTimelineStore extends TimelineStoreTestUtils {
   private FileContext fsContext;
   private File fsPath;
+  private Configuration config = new YarnConfiguration();
 
   @Before
   public void setup() throws Exception {
     fsContext = FileContext.getLocalFSFileContext();
-    Configuration conf = new YarnConfiguration();
     fsPath = new File("target", this.getClass().getSimpleName() +
         "-tmpDir").getAbsoluteFile();
     fsContext.delete(new Path(fsPath.getAbsolutePath()), true);
-    conf.set(YarnConfiguration.TIMELINE_SERVICE_LEVELDB_PATH,
+    config.set(YarnConfiguration.TIMELINE_SERVICE_LEVELDB_PATH,
         fsPath.getAbsolutePath());
-    conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_TTL_ENABLE, false);
+    config.setBoolean(YarnConfiguration.TIMELINE_SERVICE_TTL_ENABLE, false);
     store = new LeveldbTimelineStore();
-    store.init(conf);
+    store.init(config);
     store.start();
     loadTestData();
     loadVerificationData();
@@ -263,5 +266,47 @@ public class TestLeveldbTimelineStore extends TimelineStoreTestUtils {
     assertEquals(1, getEntities("type_2").size());
     assertEquals(2, getEntitiesWithPrimaryFilter("type_1", userFilter).size());
   }
+  
+  @Test
+  public void testCheckVersion() throws IOException {
+    LeveldbTimelineStore dbStore = (LeveldbTimelineStore) store;
+    // default version
+    Version defaultVersion = dbStore.getCurrentVersion();
+    Assert.assertEquals(defaultVersion, dbStore.loadVersion());
+
+    // compatible version
+    Version compatibleVersion =
+        Version.newInstance(defaultVersion.getMajorVersion(),
+          defaultVersion.getMinorVersion() + 2);
+    dbStore.storeVersion(compatibleVersion);
+    Assert.assertEquals(compatibleVersion, dbStore.loadVersion());
+    restartTimelineStore();
+    dbStore = (LeveldbTimelineStore) store;
+    // overwrite the compatible version
+    Assert.assertEquals(defaultVersion, dbStore.loadVersion());
+
+    // incompatible version
+    Version incompatibleVersion =
+      Version.newInstance(defaultVersion.getMajorVersion() + 1,
+          defaultVersion.getMinorVersion());
+    dbStore.storeVersion(incompatibleVersion);
+    try {
+      restartTimelineStore();
+      Assert.fail("Incompatible version, should expect fail here.");
+    } catch (ServiceStateException e) {
+      Assert.assertTrue("Exception message mismatch", 
+        e.getMessage().contains("Incompatible version for timeline store"));
+    }
+  }
+  
+  private void restartTimelineStore() throws IOException {
+    // need to close so leveldb releases database lock
+    if (store != null) {
+      store.close();
+    }
+    store = new LeveldbTimelineStore();
+    store.init(config);
+    store.start();
+  }
 
 }

+ 3 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/webapp/TestTimelineWebServices.java

@@ -33,6 +33,7 @@ import java.util.Map;
 import java.util.Set;
 
 import javax.servlet.FilterConfig;
+import javax.servlet.ServletContext;
 import javax.servlet.ServletException;
 import javax.ws.rs.core.MediaType;
 
@@ -105,6 +106,8 @@ public class TestTimelineWebServices extends JerseyTest {
           .thenReturn("simple");
       when(filterConfig.getInitParameter(
           PseudoAuthenticationHandler.ANONYMOUS_ALLOWED)).thenReturn("true");
+      ServletContext context = mock(ServletContext.class);
+      when(filterConfig.getServletContext()).thenReturn(context);
       Enumeration<Object> names = mock(Enumeration.class);
       when(names.hasMoreElements()).thenReturn(true, true, false);
       when(names.nextElement()).thenReturn(

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

@@ -217,7 +217,13 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
           RMAppAttemptEventType.KILL,
           new FinalSavingTransition(new BaseFinalTransition(
             RMAppAttemptState.KILLED), RMAppAttemptState.KILLED))
-          
+      .addTransition(RMAppAttemptState.SCHEDULED,
+          RMAppAttemptState.FINAL_SAVING,
+          RMAppAttemptEventType.CONTAINER_FINISHED,
+          new FinalSavingTransition(
+            new AMContainerCrashedBeforeRunningTransition(),
+            RMAppAttemptState.FAILED))
+
        // Transitions from ALLOCATED_SAVING State
       .addTransition(RMAppAttemptState.ALLOCATED_SAVING, 
           RMAppAttemptState.ALLOCATED,

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

@@ -17,6 +17,9 @@
 */
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.utils.Lock;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
@@ -24,6 +27,8 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 
 class CSQueueUtils {
   
+  private static final Log LOG = LogFactory.getLog(CSQueueUtils.class);
+
   final static float EPSILON = 0.0001f;
   
   public static void checkMaxCapacity(String queueName, 
@@ -113,4 +118,52 @@ class CSQueueUtils {
             )
         );
    }
+
+   public static float getAbsoluteMaxAvailCapacity(
+      ResourceCalculator resourceCalculator, Resource clusterResource, CSQueue queue) {
+      CSQueue parent = queue.getParent();
+      if (parent == null) {
+        return queue.getAbsoluteMaximumCapacity();
+      }
+
+      //Get my parent's max avail, needed to determine my own
+      float parentMaxAvail = getAbsoluteMaxAvailCapacity(
+        resourceCalculator, clusterResource, parent);
+      //...and as a resource
+      Resource parentResource = Resources.multiply(clusterResource, parentMaxAvail);
+
+      //check for no resources parent before dividing, if so, max avail is none
+      if (Resources.isInvalidDivisor(resourceCalculator, parentResource)) {
+        return 0.0f;
+      }
+      //sibling used is parent used - my used...
+      float siblingUsedCapacity = Resources.ratio(
+                 resourceCalculator,
+                 Resources.subtract(parent.getUsedResources(), queue.getUsedResources()),
+                 parentResource);
+      //my max avail is the lesser of my max capacity and what is unused from my parent
+      //by my siblings (if they are beyond their base capacity)
+      float maxAvail = Math.min(
+        queue.getMaximumCapacity(),
+        1.0f - siblingUsedCapacity);
+      //and, mutiply by parent to get absolute (cluster relative) value
+      float absoluteMaxAvail = maxAvail * parentMaxAvail;
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("qpath " + queue.getQueuePath());
+        LOG.debug("parentMaxAvail " + parentMaxAvail);
+        LOG.debug("siblingUsedCapacity " + siblingUsedCapacity);
+        LOG.debug("getAbsoluteMaximumCapacity " + queue.getAbsoluteMaximumCapacity());
+        LOG.debug("maxAvail " + maxAvail);
+        LOG.debug("absoluteMaxAvail " + absoluteMaxAvail);
+      }
+
+      if (absoluteMaxAvail < 0.0f) {
+        absoluteMaxAvail = 0.0f;
+      } else if (absoluteMaxAvail > 1.0f) {
+        absoluteMaxAvail = 1.0f;
+      }
+
+      return absoluteMaxAvail;
+   }
 }

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

@@ -976,13 +976,18 @@ public class LeafQueue implements CSQueue {
 
     Resource userLimit =                          // User limit
         computeUserLimit(application, clusterResource, required);
-    
+
+    //Max avail capacity needs to take into account usage by ancestor-siblings
+    //which are greater than their base capacity, so we are interested in "max avail"
+    //capacity
+    float absoluteMaxAvailCapacity = CSQueueUtils.getAbsoluteMaxAvailCapacity(
+      resourceCalculator, clusterResource, this);
 
     Resource queueMaxCap =                        // Queue Max-Capacity
         Resources.multiplyAndNormalizeDown(
             resourceCalculator, 
             clusterResource, 
-            absoluteMaxCapacity, 
+            absoluteMaxAvailCapacity,
             minimumAllocation);
     
     Resource userConsumed = getUser(user).getConsumedResources(); 

+ 27 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java

@@ -89,6 +89,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAlloca
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
@@ -782,6 +783,32 @@ public class TestRMAppAttemptTransitions {
     testAppAttemptKilledState(null, EMPTY_DIAGNOSTICS);
   }
 
+  @Test
+  public void testAMCrashAtScheduled() {
+    // This is to test sending CONTAINER_FINISHED event at SCHEDULED state.
+    // Verify the state transition is correct.
+    scheduleApplicationAttempt();
+    ContainerStatus cs =
+        SchedulerUtils.createAbnormalContainerStatus(
+            BuilderUtils.newContainerId(
+                applicationAttempt.getAppAttemptId(), 1),
+            SchedulerUtils.LOST_CONTAINER);
+    // send CONTAINER_FINISHED event at SCHEDULED state,
+    // The state should be FINAL_SAVING with previous state SCHEDULED
+    applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
+        applicationAttempt.getAppAttemptId(), cs));
+    // createApplicationAttemptState will return previous state (SCHEDULED),
+    // if the current state is FINAL_SAVING.
+    assertEquals(YarnApplicationAttemptState.SCHEDULED,
+        applicationAttempt.createApplicationAttemptState());
+    // send ATTEMPT_UPDATE_SAVED event,
+    // verify the state is changed to state FAILED.
+    sendAttemptUpdateSavedEvent(applicationAttempt);
+    assertEquals(RMAppAttemptState.FAILED,
+        applicationAttempt.getAppAttemptState());
+    verifyApplicationAttemptFinished(RMAppAttemptState.FAILED);
+  }
+
   @Test
   public void testAllocatedToKilled() {
     Container amContainer = allocateApplicationAttempt();

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

@@ -0,0 +1,262 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.inOrder;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.when;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.junit.Assert;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.Resources;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.InOrder;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+public class TestCSQueueUtils {
+
+  private static final Log LOG = LogFactory.getLog(TestCSQueueUtils.class);
+
+  final static int GB = 1024;
+
+  @Test
+  public void testAbsoluteMaxAvailCapacityInvalidDivisor() throws Exception {
+    runInvalidDivisorTest(false);
+    runInvalidDivisorTest(true);
+  }
+    
+  public void runInvalidDivisorTest(boolean useDominant) throws Exception {
+  
+    ResourceCalculator resourceCalculator;
+    Resource clusterResource;
+    if (useDominant) {
+      resourceCalculator = new DominantResourceCalculator();
+      clusterResource = Resources.createResource(10, 0);
+    } else {
+      resourceCalculator = new DefaultResourceCalculator();
+      clusterResource = Resources.createResource(0, 99);
+    }
+    
+    YarnConfiguration conf = new YarnConfiguration();
+    CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration();
+  
+    CapacitySchedulerContext csContext = mock(CapacitySchedulerContext.class);
+    when(csContext.getConf()).thenReturn(conf);
+    when(csContext.getConfiguration()).thenReturn(csConf);
+    when(csContext.getClusterResource()).thenReturn(clusterResource);
+    when(csContext.getResourceCalculator()).thenReturn(resourceCalculator);
+    when(csContext.getMinimumResourceCapability()).
+        thenReturn(Resources.createResource(GB, 1));
+    when(csContext.getMaximumResourceCapability()).
+        thenReturn(Resources.createResource(0, 0));
+  
+    final String L1Q1 = "L1Q1";
+    csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {L1Q1});
+    
+    final String L1Q1P = CapacitySchedulerConfiguration.ROOT + "." + L1Q1;
+    csConf.setCapacity(L1Q1P, 90);
+    csConf.setMaximumCapacity(L1Q1P, 90);
+    
+    ParentQueue root = new ParentQueue(csContext, 
+        CapacitySchedulerConfiguration.ROOT, null, null);
+    LeafQueue l1q1 = new LeafQueue(csContext, L1Q1, root, null);
+    
+    LOG.info("t1 root " + CSQueueUtils.getAbsoluteMaxAvailCapacity(
+      resourceCalculator, clusterResource, root));
+    
+    LOG.info("t1 l1q1 " + CSQueueUtils.getAbsoluteMaxAvailCapacity(
+      resourceCalculator, clusterResource, l1q1));
+    
+    assertEquals(0.0f, CSQueueUtils.getAbsoluteMaxAvailCapacity(
+      resourceCalculator, clusterResource, l1q1), 0.000001f);
+    
+  }
+  
+  @Test
+  public void testAbsoluteMaxAvailCapacityNoUse() throws Exception {
+    
+    ResourceCalculator resourceCalculator = new DefaultResourceCalculator();
+    Resource clusterResource = Resources.createResource(100 * 16 * GB, 100 * 32);
+    
+    YarnConfiguration conf = new YarnConfiguration();
+    CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration();
+    
+    CapacitySchedulerContext csContext = mock(CapacitySchedulerContext.class);
+    when(csContext.getConf()).thenReturn(conf);
+    when(csContext.getConfiguration()).thenReturn(csConf);
+    when(csContext.getClusterResource()).thenReturn(clusterResource);
+    when(csContext.getResourceCalculator()).thenReturn(resourceCalculator);
+    when(csContext.getMinimumResourceCapability()).
+        thenReturn(Resources.createResource(GB, 1));
+    when(csContext.getMaximumResourceCapability()).
+        thenReturn(Resources.createResource(16*GB, 32));
+    
+    final String L1Q1 = "L1Q1";
+    csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {L1Q1});
+    
+    final String L1Q1P = CapacitySchedulerConfiguration.ROOT + "." + L1Q1;
+    csConf.setCapacity(L1Q1P, 90);
+    csConf.setMaximumCapacity(L1Q1P, 90);
+    
+    ParentQueue root = new ParentQueue(csContext, 
+        CapacitySchedulerConfiguration.ROOT, null, null);
+    LeafQueue l1q1 = new LeafQueue(csContext, L1Q1, root, null);
+    
+    LOG.info("t1 root " + CSQueueUtils.getAbsoluteMaxAvailCapacity(
+      resourceCalculator, clusterResource, root));
+    
+    LOG.info("t1 l1q1 " + CSQueueUtils.getAbsoluteMaxAvailCapacity(
+      resourceCalculator, clusterResource, l1q1));
+    
+    assertEquals(1.0f, CSQueueUtils.getAbsoluteMaxAvailCapacity(
+      resourceCalculator, clusterResource, root), 0.000001f);
+    
+    assertEquals(0.9f, CSQueueUtils.getAbsoluteMaxAvailCapacity(
+      resourceCalculator, clusterResource, l1q1), 0.000001f);
+    
+  }
+  
+  @Test
+  public void testAbsoluteMaxAvailCapacityWithUse() throws Exception {
+    
+    ResourceCalculator resourceCalculator = new DefaultResourceCalculator();
+    Resource clusterResource = Resources.createResource(100 * 16 * GB, 100 * 32);
+    
+    YarnConfiguration conf = new YarnConfiguration();
+    CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration();
+    
+    CapacitySchedulerContext csContext = mock(CapacitySchedulerContext.class);
+    when(csContext.getConf()).thenReturn(conf);
+    when(csContext.getConfiguration()).thenReturn(csConf);
+    when(csContext.getClusterResource()).thenReturn(clusterResource);
+    when(csContext.getResourceCalculator()).thenReturn(resourceCalculator);
+    when(csContext.getMinimumResourceCapability()).
+        thenReturn(Resources.createResource(GB, 1));
+    when(csContext.getMaximumResourceCapability()).
+        thenReturn(Resources.createResource(16*GB, 32));
+    
+    final String L1Q1 = "L1Q1";
+    final String L1Q2 = "L1Q2";
+    final String L2Q1 = "L2Q1";
+    final String L2Q2 = "L2Q2";
+    csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {L1Q1, L1Q2,
+                     L2Q1, L2Q2});
+    
+    final String L1Q1P = CapacitySchedulerConfiguration.ROOT + "." + L1Q1;
+    csConf.setCapacity(L1Q1P, 80);
+    csConf.setMaximumCapacity(L1Q1P, 80);
+    
+    final String L1Q2P = CapacitySchedulerConfiguration.ROOT + "." + L1Q2;
+    csConf.setCapacity(L1Q2P, 20);
+    csConf.setMaximumCapacity(L1Q2P, 100);
+    
+    final String L2Q1P = L1Q1P + "." + L2Q1;
+    csConf.setCapacity(L2Q1P, 50);
+    csConf.setMaximumCapacity(L2Q1P, 50);
+    
+    final String L2Q2P = L1Q1P + "." + L2Q2;
+    csConf.setCapacity(L2Q2P, 50);
+    csConf.setMaximumCapacity(L2Q2P, 50);
+    
+    float result;
+    
+    ParentQueue root = new ParentQueue(csContext, 
+        CapacitySchedulerConfiguration.ROOT, null, null);
+    
+    LeafQueue l1q1 = new LeafQueue(csContext, L1Q1, root, null);
+    LeafQueue l1q2 = new LeafQueue(csContext, L1Q2, root, null);
+    LeafQueue l2q2 = new LeafQueue(csContext, L2Q2, l1q1, null);
+    LeafQueue l2q1 = new LeafQueue(csContext, L2Q1, l1q1, null);
+    
+    //no usage, all based on maxCapacity (prior behavior)
+    result = CSQueueUtils.getAbsoluteMaxAvailCapacity(
+      resourceCalculator, clusterResource, l2q2);
+    assertEquals( 0.4f, result, 0.000001f);
+    LOG.info("t2 l2q2 " + result);
+    
+    //some usage, but below the base capacity
+    Resources.addTo(root.getUsedResources(), Resources.multiply(clusterResource, 0.1f));
+    Resources.addTo(l1q2.getUsedResources(), Resources.multiply(clusterResource, 0.1f));
+    result = CSQueueUtils.getAbsoluteMaxAvailCapacity(
+      resourceCalculator, clusterResource, l2q2);
+    assertEquals( 0.4f, result, 0.000001f);
+    LOG.info("t2 l2q2 " + result);
+    
+    //usage gt base on parent sibling
+    Resources.addTo(root.getUsedResources(), Resources.multiply(clusterResource, 0.3f));
+    Resources.addTo(l1q2.getUsedResources(), Resources.multiply(clusterResource, 0.3f));
+    result = CSQueueUtils.getAbsoluteMaxAvailCapacity(
+      resourceCalculator, clusterResource, l2q2);
+    assertEquals( 0.3f, result, 0.000001f);
+    LOG.info("t2 l2q2 " + result);
+    
+    //same as last, but with usage also on direct parent
+    Resources.addTo(root.getUsedResources(), Resources.multiply(clusterResource, 0.1f));
+    Resources.addTo(l1q1.getUsedResources(), Resources.multiply(clusterResource, 0.1f));
+    result = CSQueueUtils.getAbsoluteMaxAvailCapacity(
+      resourceCalculator, clusterResource, l2q2);
+    assertEquals( 0.3f, result, 0.000001f);
+    LOG.info("t2 l2q2 " + result);
+    
+    //add to direct sibling, below the threshold of effect at present
+    Resources.addTo(root.getUsedResources(), Resources.multiply(clusterResource, 0.2f));
+    Resources.addTo(l1q1.getUsedResources(), Resources.multiply(clusterResource, 0.2f));
+    Resources.addTo(l2q1.getUsedResources(), Resources.multiply(clusterResource, 0.2f));
+    result = CSQueueUtils.getAbsoluteMaxAvailCapacity(
+      resourceCalculator, clusterResource, l2q2);
+    assertEquals( 0.3f, result, 0.000001f);
+    LOG.info("t2 l2q2 " + result);
+    
+    //add to direct sibling, now above the threshold of effect
+    //(it's cumulative with prior tests)
+    Resources.addTo(root.getUsedResources(), Resources.multiply(clusterResource, 0.2f));
+    Resources.addTo(l1q1.getUsedResources(), Resources.multiply(clusterResource, 0.2f));
+    Resources.addTo(l2q1.getUsedResources(), Resources.multiply(clusterResource, 0.2f));
+    result = CSQueueUtils.getAbsoluteMaxAvailCapacity(
+      resourceCalculator, clusterResource, l2q2);
+    assertEquals( 0.1f, result, 0.000001f);
+    LOG.info("t2 l2q2 " + result);
+    
+    
+  }
+
+}