浏览代码

HADOOP-10791. AuthenticationFilter should support externalizing the secret for signing and provide rotation support. (rkanter via tucu)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1616006 13f79535-47bb-0310-9956-ffa450edef68
Alejandro Abdelnur 10 年之前
父节点
当前提交
891d93bbcf
共有 15 个文件被更改,包括 807 次插入47 次删除
  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. 3 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  15. 2 1
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.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

@@ -155,6 +155,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]);
+  }
+}

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

@@ -60,6 +60,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

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