You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by sz...@apache.org on 2012/10/19 04:28:07 UTC

svn commit: r1399950 [3/27] - in /hadoop/common/branches/HDFS-2802/hadoop-hdfs-project: ./ hadoop-hdfs-httpfs/ hadoop-hdfs-httpfs/dev-support/ hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/ hadoop-hdfs-httpfs/src/main/java/org/apach...

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/servlet/ServerWebApp.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/servlet/ServerWebApp.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/servlet/ServerWebApp.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/servlet/ServerWebApp.java Fri Oct 19 02:25:55 2012
@@ -18,27 +18,37 @@
 
 package org.apache.hadoop.lib.servlet;
 
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.lib.server.Server;
 import org.apache.hadoop.lib.server.ServerException;
 
 import javax.servlet.ServletContextEvent;
 import javax.servlet.ServletContextListener;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.UnknownHostException;
 import java.text.MessageFormat;
 
 /**
  * {@link Server} subclass that implements <code>ServletContextListener</code>
  * and uses its lifecycle to start and stop the server.
  */
+@InterfaceAudience.Private
 public abstract class ServerWebApp extends Server implements ServletContextListener {
 
   private static final String HOME_DIR = ".home.dir";
   private static final String CONFIG_DIR = ".config.dir";
   private static final String LOG_DIR = ".log.dir";
   private static final String TEMP_DIR = ".temp.dir";
+  private static final String HTTP_HOSTNAME = ".http.hostname";
+  private static final String HTTP_PORT = ".http.port";
 
   private static ThreadLocal<String> HOME_DIR_TL = new ThreadLocal<String>();
 
+  private InetSocketAddress authority;
+
   /**
    * Method for testing purposes.
    */
@@ -137,6 +147,7 @@ public abstract class ServerWebApp exten
    *
    * @param event servelt context event.
    */
+  @Override
   public void contextInitialized(ServletContextEvent event) {
     try {
       init();
@@ -147,13 +158,71 @@ public abstract class ServerWebApp exten
   }
 
   /**
+   * Resolves the host & port InetSocketAddress the web server is listening to.
+   * <p/>
+   * This implementation looks for the following 2 properties:
+   * <ul>
+   *   <li>#SERVER_NAME#.http.hostname</li>
+   *   <li>#SERVER_NAME#.http.port</li>
+   * </ul>
+   *
+   * @return the host & port InetSocketAddress the web server is listening to.
+   * @throws ServerException thrown if any of the above 2 properties is not defined.
+   */
+  protected InetSocketAddress resolveAuthority() throws ServerException {
+    String hostnameKey = getName() + HTTP_HOSTNAME;
+    String portKey = getName() + HTTP_PORT;
+    String host = System.getProperty(hostnameKey);
+    String port = System.getProperty(portKey);
+    if (host == null) {
+      throw new ServerException(ServerException.ERROR.S13, hostnameKey);
+    }
+    if (port == null) {
+      throw new ServerException(ServerException.ERROR.S13, portKey);
+    }
+    try {
+      InetAddress add = InetAddress.getByName(host);
+      int portNum = Integer.parseInt(port);
+      return new InetSocketAddress(add, portNum);
+    } catch (UnknownHostException ex) {
+      throw new ServerException(ServerException.ERROR.S14, ex.toString(), ex);
+    }
+  }
+
+  /**
    * Destroys the <code>ServletContextListener</code> which destroys
    * the Server.
    *
    * @param event servelt context event.
    */
+  @Override
   public void contextDestroyed(ServletContextEvent event) {
     destroy();
   }
 
+  /**
+   * Returns the hostname:port InetSocketAddress the webserver is listening to.
+   *
+   * @return the hostname:port InetSocketAddress the webserver is listening to.
+   */
+  public InetSocketAddress getAuthority() throws ServerException {
+    synchronized (this) {
+      if (authority == null) {
+          authority = resolveAuthority();
+      }
+    }
+    return authority;
+  }
+
+  /**
+   * Sets an alternate hostname:port InetSocketAddress to use.
+   * <p/>
+   * For testing purposes.
+   * 
+   * @param authority alterante authority.
+   */
+  @VisibleForTesting
+  public void setAuthority(InetSocketAddress authority) {
+    this.authority = authority;
+  }
 }

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/util/Check.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/util/Check.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/util/Check.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/util/Check.java Fri Oct 19 02:25:55 2012
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.lib.util;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+
 import java.text.MessageFormat;
 import java.util.List;
 import java.util.regex.Pattern;
@@ -27,6 +29,7 @@ import java.util.regex.Pattern;
  * <p/>
  * Commonly used for method arguments preconditions.
  */
+@InterfaceAudience.Private
 public class Check {
 
   /**

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/util/ConfigurationUtils.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/util/ConfigurationUtils.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/util/ConfigurationUtils.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/util/ConfigurationUtils.java Fri Oct 19 02:25:55 2012
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.lib.util;
 
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.w3c.dom.DOMException;
 import org.w3c.dom.Document;
@@ -37,6 +38,7 @@ import java.util.Map;
 /**
  * Configuration utilities.
  */
+@InterfaceAudience.Private
 public abstract class ConfigurationUtils {
 
   /**

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/BooleanParam.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/BooleanParam.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/BooleanParam.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/BooleanParam.java Fri Oct 19 02:25:55 2012
@@ -18,14 +18,18 @@
 
 package org.apache.hadoop.lib.wsrs;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+
 import java.text.MessageFormat;
 
+@InterfaceAudience.Private
 public abstract class BooleanParam extends Param<Boolean> {
 
   public BooleanParam(String name, Boolean defaultValue) {
     super(name, defaultValue);
   }
 
+  @Override
   protected Boolean parse(String str) throws Exception {
     if (str.equalsIgnoreCase("true")) {
       return true;

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/ByteParam.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/ByteParam.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/ByteParam.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/ByteParam.java Fri Oct 19 02:25:55 2012
@@ -18,12 +18,16 @@
 
 package org.apache.hadoop.lib.wsrs;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+
+@InterfaceAudience.Private
 public abstract class ByteParam extends Param<Byte> {
 
   public ByteParam(String name, Byte defaultValue) {
     super(name, defaultValue);
   }
 
+  @Override
   protected Byte parse(String str) throws Exception {
     return Byte.parseByte(str);
   }

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/EnumParam.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/EnumParam.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/EnumParam.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/EnumParam.java Fri Oct 19 02:25:55 2012
@@ -18,10 +18,12 @@
 
 package org.apache.hadoop.lib.wsrs;
 
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.util.StringUtils;
 
 import java.util.Arrays;
 
+@InterfaceAudience.Private
 public abstract class EnumParam<E extends Enum<E>> extends Param<E> {
   Class<E> klass;
 
@@ -30,6 +32,7 @@ public abstract class EnumParam<E extend
     klass = e;
   }
 
+  @Override
   protected E parse(String str) throws Exception {
     return Enum.valueOf(klass, str.toUpperCase());
   }

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/ExceptionProvider.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/ExceptionProvider.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/ExceptionProvider.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/ExceptionProvider.java Fri Oct 19 02:25:55 2012
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.lib.wsrs;
 
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.http.client.HttpFSFileSystem;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -28,6 +29,7 @@ import javax.ws.rs.ext.ExceptionMapper;
 import java.util.LinkedHashMap;
 import java.util.Map;
 
+@InterfaceAudience.Private
 public class ExceptionProvider implements ExceptionMapper<Throwable> {
   private static Logger LOG = LoggerFactory.getLogger(ExceptionProvider.class);
 

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/InputStreamEntity.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/InputStreamEntity.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/InputStreamEntity.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/InputStreamEntity.java Fri Oct 19 02:25:55 2012
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.lib.wsrs;
 
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.io.IOUtils;
 
 import javax.ws.rs.core.StreamingOutput;
@@ -25,6 +26,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 
+@InterfaceAudience.Private
 public class InputStreamEntity implements StreamingOutput {
   private InputStream is;
   private long offset;
@@ -42,10 +44,7 @@ public class InputStreamEntity implement
 
   @Override
   public void write(OutputStream os) throws IOException {
-    long skipped = is.skip(offset);
-    if (skipped < offset) {
-      throw new IOException("Requested offset beyond stream size");
-    }
+    IOUtils.skipFully(is, offset);
     if (len == -1) {
       IOUtils.copyBytes(is, os, 4096, true);
     } else {

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/IntegerParam.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/IntegerParam.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/IntegerParam.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/IntegerParam.java Fri Oct 19 02:25:55 2012
@@ -18,12 +18,16 @@
 
 package org.apache.hadoop.lib.wsrs;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+
+@InterfaceAudience.Private
 public abstract class IntegerParam extends Param<Integer> {
 
   public IntegerParam(String name, Integer defaultValue) {
     super(name, defaultValue);
   }
 
+  @Override
   protected Integer parse(String str) throws Exception {
     return Integer.parseInt(str);
   }

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/JSONMapProvider.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/JSONMapProvider.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/JSONMapProvider.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/JSONMapProvider.java Fri Oct 19 02:25:55 2012
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.lib.wsrs;
 
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.json.simple.JSONObject;
 
 import javax.ws.rs.Produces;
@@ -36,6 +37,7 @@ import java.util.Map;
 
 @Provider
 @Produces(MediaType.APPLICATION_JSON)
+@InterfaceAudience.Private
 public class JSONMapProvider implements MessageBodyWriter<Map> {
   private static final String ENTER = System.getProperty("line.separator");
 

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/JSONProvider.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/JSONProvider.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/JSONProvider.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/JSONProvider.java Fri Oct 19 02:25:55 2012
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.lib.wsrs;
 
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.json.simple.JSONStreamAware;
 
 import javax.ws.rs.Produces;
@@ -35,6 +36,7 @@ import java.lang.reflect.Type;
 
 @Provider
 @Produces(MediaType.APPLICATION_JSON)
+@InterfaceAudience.Private
 public class JSONProvider implements MessageBodyWriter<JSONStreamAware> {
   private static final String ENTER = System.getProperty("line.separator");
 

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/LongParam.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/LongParam.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/LongParam.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/LongParam.java Fri Oct 19 02:25:55 2012
@@ -18,12 +18,16 @@
 
 package org.apache.hadoop.lib.wsrs;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+
+@InterfaceAudience.Private
 public abstract class LongParam extends Param<Long> {
 
   public LongParam(String name, Long defaultValue) {
     super(name, defaultValue);
   }
 
+  @Override
   protected Long parse(String str) throws Exception {
     return Long.parseLong(str);
   }

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/Param.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/Param.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/Param.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/Param.java Fri Oct 19 02:25:55 2012
@@ -18,10 +18,11 @@
 
 package org.apache.hadoop.lib.wsrs;
 
-import org.apache.hadoop.lib.util.Check;
+import org.apache.hadoop.classification.InterfaceAudience;
 
 import java.text.MessageFormat;
 
+@InterfaceAudience.Private
 public abstract class Param<T> {
   private String name;
   protected T value;
@@ -54,6 +55,7 @@ public abstract class Param<T> {
 
   protected abstract T parse(String str) throws Exception;
 
+  @Override
   public String toString() {
     return (value != null) ? value.toString() : "NULL";
   }

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/Parameters.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/Parameters.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/Parameters.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/Parameters.java Fri Oct 19 02:25:55 2012
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.lib.wsrs;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+
 import java.util.Map;
 
 /**
@@ -24,6 +26,7 @@ import java.util.Map;
  * <p/>
  * Instances are created by the {@link ParametersProvider} class.
  */
+@InterfaceAudience.Private
 public class Parameters {
   private Map<String, Param<?>> params;
 

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/ParametersProvider.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/ParametersProvider.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/ParametersProvider.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/ParametersProvider.java Fri Oct 19 02:25:55 2012
@@ -24,6 +24,7 @@ import com.sun.jersey.core.spi.component
 import com.sun.jersey.server.impl.inject.AbstractHttpContextInjectable;
 import com.sun.jersey.spi.inject.Injectable;
 import com.sun.jersey.spi.inject.InjectableProvider;
+import org.apache.hadoop.classification.InterfaceAudience;
 
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MultivaluedMap;
@@ -36,6 +37,7 @@ import java.util.Map;
  * Jersey provider that parses the request parameters based on the
  * given parameter definition. 
  */
+@InterfaceAudience.Private
 public class ParametersProvider
   extends AbstractHttpContextInjectable<Parameters>
   implements InjectableProvider<Context, Type> {

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/ShortParam.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/ShortParam.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/ShortParam.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/ShortParam.java Fri Oct 19 02:25:55 2012
@@ -18,6 +18,9 @@
 
 package org.apache.hadoop.lib.wsrs;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+
+@InterfaceAudience.Private
 public abstract class ShortParam extends Param<Short> {
 
   private int radix;
@@ -31,6 +34,7 @@ public abstract class ShortParam extends
     this(name, defaultValue, 10);
   }
 
+  @Override
   protected Short parse(String str) throws Exception {
     return Short.parseShort(str, radix);
   }

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/StringParam.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/StringParam.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/StringParam.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/StringParam.java Fri Oct 19 02:25:55 2012
@@ -17,9 +17,12 @@
  */
 package org.apache.hadoop.lib.wsrs;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+
 import java.text.MessageFormat;
 import java.util.regex.Pattern;
 
+@InterfaceAudience.Private
 public abstract class StringParam extends Param<String> {
   private Pattern pattern;
 
@@ -33,6 +36,7 @@ public abstract class StringParam extend
     parseParam(defaultValue);
   }
 
+  @Override
   public String parseParam(String str) {
     try {
       if (str != null) {
@@ -49,6 +53,7 @@ public abstract class StringParam extend
     return value;
   }
 
+  @Override
   protected String parse(String str) throws Exception {
     if (pattern != null) {
       if (!pattern.matcher(str).matches()) {

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/UserProvider.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/UserProvider.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/UserProvider.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/UserProvider.java Fri Oct 19 02:25:55 2012
@@ -24,6 +24,7 @@ import com.sun.jersey.core.spi.component
 import com.sun.jersey.server.impl.inject.AbstractHttpContextInjectable;
 import com.sun.jersey.spi.inject.Injectable;
 import com.sun.jersey.spi.inject.InjectableProvider;
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.slf4j.MDC;
 
 import javax.ws.rs.core.Context;
@@ -33,6 +34,7 @@ import java.security.Principal;
 import java.util.regex.Pattern;
 
 @Provider
+@InterfaceAudience.Private
 public class UserProvider extends AbstractHttpContextInjectable<Principal> implements
   InjectableProvider<Context, Type> {
 

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/resources/httpfs-default.xml
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/resources/httpfs-default.xml?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/resources/httpfs-default.xml (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/resources/httpfs-default.xml Fri Oct 19 02:25:55 2012
@@ -35,6 +35,7 @@
       org.apache.hadoop.lib.service.scheduler.SchedulerService,
       org.apache.hadoop.lib.service.security.GroupsService,
       org.apache.hadoop.lib.service.security.ProxyUserService,
+      org.apache.hadoop.lib.service.security.DelegationTokenManagerService,
       org.apache.hadoop.lib.service.hadoop.FileSystemAccessService
     </value>
     <description>
@@ -88,12 +89,12 @@
     <description>
       Defines the authentication mechanism used by httpfs for its HTTP clients.
 
-      Valid values are 'simple' and 'kerberos'.
+      Valid values are 'simple' or 'kerberos'.
 
       If using 'simple' HTTP clients must specify the username with the
       'user.name' query string parameter.
 
-      If using 'kerberos' HTTP clients must use HTTP SPNEGO.
+      If using 'kerberos' HTTP clients must use HTTP SPNEGO or delegation tokens.
     </description>
   </property>
 
@@ -153,6 +154,32 @@
     </description>
   </property>
 
+  <!-- HttpFS Delegation Token configuration -->
+
+  <property>
+    <name>httpfs.delegation.token.manager.update.interval</name>
+    <value>86400</value>
+    <description>
+      HttpFS delegation token update interval, default 1 day, in seconds.
+    </description>
+  </property>
+
+  <property>
+    <name>httpfs.delegation.token.manager.max.lifetime</name>
+    <value>604800</value>
+    <description>
+      HttpFS delegation token maximum lifetime, default 7 days, in seconds
+    </description>
+  </property>
+
+  <property>
+    <name>httpfs.delegation.token.manager.renewal.interval</name>
+    <value>86400</value>
+    <description>
+      HttpFS delegation token update interval, default 1 day, in seconds.
+    </description>
+  </property>
+
   <!-- FileSystemAccess Namenode Security Configuration -->
 
   <property>
@@ -183,4 +210,20 @@
     </description>
   </property>
 
+  <property>
+    <name>httpfs.hadoop.filesystem.cache.purge.frequency</name>
+    <value>60</value>
+    <description>
+      Frequency, in seconds, for the idle filesystem purging daemon runs.
+    </description>
+  </property>
+
+  <property>
+    <name>httpfs.hadoop.filesystem.cache.purge.timeout</name>
+    <value>60</value>
+    <description>
+      Timeout, in seconds, for an idle filesystem to be purged.
+    </description>
+  </property>
+
 </configuration>

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/webapp/WEB-INF/web.xml
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/webapp/WEB-INF/web.xml?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/webapp/WEB-INF/web.xml (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/webapp/WEB-INF/web.xml Fri Oct 19 02:25:55 2012
@@ -47,7 +47,7 @@
 
   <filter>
     <filter-name>authFilter</filter-name>
-    <filter-class>org.apache.hadoop.fs.http.server.AuthFilter</filter-class>
+    <filter-class>org.apache.hadoop.fs.http.server.HttpFSAuthenticationFilter</filter-class>
   </filter>
 
   <filter>

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/apt/index.apt.vm
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/apt/index.apt.vm?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/apt/index.apt.vm (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/apt/index.apt.vm Fri Oct 19 02:25:55 2012
@@ -81,8 +81,3 @@ Hadoop HDFS over HTTP - Documentation Se
 
   * {{{./UsingHttpTools.html}Using HTTP Tools}}
 
-* Current Limitations
-
-  <<<GETDELEGATIONTOKEN, RENEWDELEGATIONTOKEN and CANCELDELEGATIONTOKEN>>>
-  operations are not supported.
-

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestCheckUploadContentTypeFilter.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestCheckUploadContentTypeFilter.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestCheckUploadContentTypeFilter.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestCheckUploadContentTypeFilter.java Fri Oct 19 02:25:55 2012
@@ -18,15 +18,15 @@
 
 package org.apache.hadoop.fs.http.server;
 
-import org.apache.hadoop.fs.http.client.HttpFSFileSystem;
-import org.junit.Test;
-import org.mockito.Mockito;
-
 import javax.servlet.Filter;
 import javax.servlet.FilterChain;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
+import org.apache.hadoop.fs.http.client.HttpFSFileSystem;
+import org.junit.Test;
+import org.mockito.Mockito;
+
 public class TestCheckUploadContentTypeFilter {
 
   @Test

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java Fri Oct 19 02:25:55 2012
@@ -15,17 +15,36 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.fs.http.server;
 
-import junit.framework.Assert;
+import org.junit.Assert;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.Writer;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.text.MessageFormat;
+import java.util.Arrays;
+import java.util.List;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.http.client.HttpFSKerberosAuthenticator;
 import org.apache.hadoop.lib.server.Service;
 import org.apache.hadoop.lib.server.ServiceException;
 import org.apache.hadoop.lib.service.Groups;
+import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
+import org.apache.hadoop.security.authentication.server.AuthenticationToken;
+import org.apache.hadoop.security.authentication.util.Signer;
 import org.apache.hadoop.test.HFSTestCase;
 import org.apache.hadoop.test.HadoopUsersConfTestHelper;
 import org.apache.hadoop.test.TestDir;
@@ -34,24 +53,12 @@ import org.apache.hadoop.test.TestHdfs;
 import org.apache.hadoop.test.TestHdfsHelper;
 import org.apache.hadoop.test.TestJetty;
 import org.apache.hadoop.test.TestJettyHelper;
+import org.json.simple.JSONObject;
+import org.json.simple.parser.JSONParser;
 import org.junit.Test;
 import org.mortbay.jetty.Server;
 import org.mortbay.jetty.webapp.WebAppContext;
 
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.io.OutputStream;
-import java.io.Writer;
-import java.net.HttpURLConnection;
-import java.net.URL;
-import java.text.MessageFormat;
-import java.util.Arrays;
-import java.util.List;
-
 public class TestHttpFSServer extends HFSTestCase {
 
   @Test
@@ -101,7 +108,9 @@ public class TestHttpFSServer extends HF
     }
 
   }
-  private void createHttpFSServer() throws Exception {
+
+  private void createHttpFSServer(boolean addDelegationTokenAuthHandler)
+    throws Exception {
     File homeDir = TestDirHelper.getTestDir();
     Assert.assertTrue(new File(homeDir, "conf").mkdir());
     Assert.assertTrue(new File(homeDir, "log").mkdir());
@@ -126,6 +135,10 @@ public class TestHttpFSServer extends HF
 
     //HTTPFS configuration
     conf = new Configuration(false);
+    if (addDelegationTokenAuthHandler) {
+     conf.set("httpfs.authentication.type",
+              HttpFSKerberosAuthenticationHandlerForTesting.class.getName());
+    }
     conf.set("httpfs.services.ext", MockGroups.class.getName());
     conf.set("httpfs.admin.group", HadoopUsersConfTestHelper.
       getHadoopUserGroups(HadoopUsersConfTestHelper.getHadoopUsers()[0])[0]);
@@ -134,6 +147,7 @@ public class TestHttpFSServer extends HF
     conf.set("httpfs.proxyuser." + HadoopUsersConfTestHelper.getHadoopProxyUser() + ".hosts",
              HadoopUsersConfTestHelper.getHadoopProxyUserHosts());
     conf.set("httpfs.authentication.signature.secret.file", secretFile.getAbsolutePath());
+    conf.set("httpfs.hadoop.config.dir", hadoopConfDir.toString());
     File httpfsSite = new File(new File(homeDir, "conf"), "httpfs-site.xml");
     os = new FileOutputStream(httpfsSite);
     conf.writeXml(os);
@@ -145,6 +159,9 @@ public class TestHttpFSServer extends HF
     Server server = TestJettyHelper.getJettyServer();
     server.addHandler(context);
     server.start();
+    if (addDelegationTokenAuthHandler) {
+      HttpFSServerWebApp.get().setAuthority(TestJettyHelper.getAuthority());
+    }
   }
 
   @Test
@@ -152,7 +169,7 @@ public class TestHttpFSServer extends HF
   @TestJetty
   @TestHdfs
   public void instrumentation() throws Exception {
-    createHttpFSServer();
+    createHttpFSServer(false);
 
     URL url = new URL(TestJettyHelper.getJettyURL(),
                       MessageFormat.format("/webhdfs/v1?user.name={0}&op=instrumentation", "nobody"));
@@ -181,7 +198,7 @@ public class TestHttpFSServer extends HF
   @TestJetty
   @TestHdfs
   public void testHdfsAccess() throws Exception {
-    createHttpFSServer();
+    createHttpFSServer(false);
 
     String user = HadoopUsersConfTestHelper.getHadoopUsers()[0];
     URL url = new URL(TestJettyHelper.getJettyURL(),
@@ -198,7 +215,7 @@ public class TestHttpFSServer extends HF
   @TestJetty
   @TestHdfs
   public void testGlobFilter() throws Exception {
-    createHttpFSServer();
+    createHttpFSServer(false);
 
     FileSystem fs = FileSystem.get(TestHdfsHelper.getHdfsConf());
     fs.mkdirs(new Path("/tmp"));
@@ -218,8 +235,33 @@ public class TestHttpFSServer extends HF
   @TestDir
   @TestJetty
   @TestHdfs
+  public void testOpenOffsetLength() throws Exception {
+    createHttpFSServer(false);
+
+    byte[] array = new byte[]{0, 1, 2, 3};
+    FileSystem fs = FileSystem.get(TestHdfsHelper.getHdfsConf());
+    fs.mkdirs(new Path("/tmp"));
+    OutputStream os = fs.create(new Path("/tmp/foo"));
+    os.write(array);
+    os.close();
+
+    String user = HadoopUsersConfTestHelper.getHadoopUsers()[0];
+    URL url = new URL(TestJettyHelper.getJettyURL(),
+                      MessageFormat.format("/webhdfs/v1/tmp/foo?user.name={0}&op=open&offset=1&length=2", user));
+    HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+    Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
+    InputStream is = conn.getInputStream();
+    Assert.assertEquals(1, is.read());
+    Assert.assertEquals(2, is.read());
+    Assert.assertEquals(-1, is.read());
+  }
+
+  @Test
+  @TestDir
+  @TestJetty
+  @TestHdfs
   public void testPutNoOperation() throws Exception {
-    createHttpFSServer();
+    createHttpFSServer(false);
 
     String user = HadoopUsersConfTestHelper.getHadoopUsers()[0];
     URL url = new URL(TestJettyHelper.getJettyURL(),
@@ -231,4 +273,84 @@ public class TestHttpFSServer extends HF
     Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_BAD_REQUEST);
   }
 
+  @Test
+  @TestDir
+  @TestJetty
+  @TestHdfs
+  public void testDelegationTokenOperations() throws Exception {
+    createHttpFSServer(true);
+
+    URL url = new URL(TestJettyHelper.getJettyURL(),
+                      "/webhdfs/v1/?op=GETHOMEDIRECTORY");
+    HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+    Assert.assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED,
+                        conn.getResponseCode());
+
+
+    AuthenticationToken token =
+      new AuthenticationToken("u", "p",
+        HttpFSKerberosAuthenticationHandlerForTesting.TYPE);
+    token.setExpires(System.currentTimeMillis() + 100000000);
+    Signer signer = new Signer("secret".getBytes());
+    String tokenSigned = signer.sign(token.toString());
+
+    url = new URL(TestJettyHelper.getJettyURL(),
+                  "/webhdfs/v1/?op=GETHOMEDIRECTORY");
+    conn = (HttpURLConnection) url.openConnection();
+    conn.setRequestProperty("Cookie",
+                            AuthenticatedURL.AUTH_COOKIE  + "=" + tokenSigned);
+    Assert.assertEquals(HttpURLConnection.HTTP_OK,
+                        conn.getResponseCode());
+
+    url = new URL(TestJettyHelper.getJettyURL(),
+                  "/webhdfs/v1/?op=GETDELEGATIONTOKEN");
+    conn = (HttpURLConnection) url.openConnection();
+    conn.setRequestProperty("Cookie",
+                            AuthenticatedURL.AUTH_COOKIE  + "=" + tokenSigned);
+    Assert.assertEquals(HttpURLConnection.HTTP_OK,
+                        conn.getResponseCode());
+
+    JSONObject json = (JSONObject)
+      new JSONParser().parse(new InputStreamReader(conn.getInputStream()));
+    json = (JSONObject)
+      json.get(HttpFSKerberosAuthenticator.DELEGATION_TOKEN_JSON);
+    String tokenStr = (String)
+        json.get(HttpFSKerberosAuthenticator.DELEGATION_TOKEN_URL_STRING_JSON);
+
+    url = new URL(TestJettyHelper.getJettyURL(),
+                  "/webhdfs/v1/?op=GETHOMEDIRECTORY&delegation=" + tokenStr);
+    conn = (HttpURLConnection) url.openConnection();
+    Assert.assertEquals(HttpURLConnection.HTTP_OK,
+                        conn.getResponseCode());
+
+    url = new URL(TestJettyHelper.getJettyURL(),
+                  "/webhdfs/v1/?op=RENEWDELEGATIONTOKEN&token=" + tokenStr);
+    conn = (HttpURLConnection) url.openConnection();
+    conn.setRequestMethod("PUT");
+    Assert.assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED,
+                        conn.getResponseCode());
+
+    url = new URL(TestJettyHelper.getJettyURL(),
+                  "/webhdfs/v1/?op=RENEWDELEGATIONTOKEN&token=" + tokenStr);
+    conn = (HttpURLConnection) url.openConnection();
+    conn.setRequestMethod("PUT");
+    conn.setRequestProperty("Cookie",
+                            AuthenticatedURL.AUTH_COOKIE  + "=" + tokenSigned);
+    Assert.assertEquals(HttpURLConnection.HTTP_OK,
+                        conn.getResponseCode());
+
+    url = new URL(TestJettyHelper.getJettyURL(),
+                  "/webhdfs/v1/?op=CANCELDELEGATIONTOKEN&token=" + tokenStr);
+    conn = (HttpURLConnection) url.openConnection();
+    conn.setRequestMethod("PUT");
+    Assert.assertEquals(HttpURLConnection.HTTP_OK,
+                        conn.getResponseCode());
+
+    url = new URL(TestJettyHelper.getJettyURL(),
+                  "/webhdfs/v1/?op=GETHOMEDIRECTORY&delegation=" + tokenStr);
+    conn = (HttpURLConnection) url.openConnection();
+    Assert.assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED,
+                        conn.getResponseCode());
+  }
+
 }

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/lang/TestRunnableCallable.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/lang/TestRunnableCallable.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/lang/TestRunnableCallable.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/lang/TestRunnableCallable.java Fri Oct 19 02:25:55 2012
@@ -19,12 +19,14 @@
 package org.apache.hadoop.lib.lang;
 
 
-import junit.framework.Assert;
-import org.apache.hadoop.test.HTestCase;
-import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import java.util.concurrent.Callable;
 
+import org.apache.hadoop.test.HTestCase;
+import org.junit.Test;
+
 public class TestRunnableCallable extends HTestCase {
 
   public static class R implements Runnable {
@@ -59,14 +61,14 @@ public class TestRunnableCallable extend
     R r = new R();
     RunnableCallable rc = new RunnableCallable(r);
     rc.run();
-    Assert.assertTrue(r.RUN);
+    assertTrue(r.RUN);
 
     r = new R();
     rc = new RunnableCallable(r);
     rc.call();
-    Assert.assertTrue(r.RUN);
+    assertTrue(r.RUN);
 
-    Assert.assertEquals(rc.toString(), "R");
+    assertEquals(rc.toString(), "R");
   }
 
   @Test
@@ -74,14 +76,14 @@ public class TestRunnableCallable extend
     C c = new C();
     RunnableCallable rc = new RunnableCallable(c);
     rc.run();
-    Assert.assertTrue(c.RUN);
+    assertTrue(c.RUN);
 
     c = new C();
     rc = new RunnableCallable(c);
     rc.call();
-    Assert.assertTrue(c.RUN);
+    assertTrue(c.RUN);
 
-    Assert.assertEquals(rc.toString(), "C");
+    assertEquals(rc.toString(), "C");
   }
 
   @Test(expected = RuntimeException.class)

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/lang/TestXException.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/lang/TestXException.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/lang/TestXException.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/lang/TestXException.java Fri Oct 19 02:25:55 2012
@@ -19,7 +19,9 @@
 package org.apache.hadoop.lib.lang;
 
 
-import junit.framework.Assert;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
 import org.apache.hadoop.test.HTestCase;
 import org.junit.Test;
 
@@ -37,26 +39,26 @@ public class TestXException extends HTes
   @Test
   public void testXException() throws Exception {
     XException ex = new XException(TestERROR.TC);
-    Assert.assertEquals(ex.getError(), TestERROR.TC);
-    Assert.assertEquals(ex.getMessage(), "TC: {0}");
-    Assert.assertNull(ex.getCause());
+    assertEquals(ex.getError(), TestERROR.TC);
+    assertEquals(ex.getMessage(), "TC: {0}");
+    assertNull(ex.getCause());
 
     ex = new XException(TestERROR.TC, "msg");
-    Assert.assertEquals(ex.getError(), TestERROR.TC);
-    Assert.assertEquals(ex.getMessage(), "TC: msg");
-    Assert.assertNull(ex.getCause());
+    assertEquals(ex.getError(), TestERROR.TC);
+    assertEquals(ex.getMessage(), "TC: msg");
+    assertNull(ex.getCause());
 
     Exception cause = new Exception();
     ex = new XException(TestERROR.TC, cause);
-    Assert.assertEquals(ex.getError(), TestERROR.TC);
-    Assert.assertEquals(ex.getMessage(), "TC: " + cause.toString());
-    Assert.assertEquals(ex.getCause(), cause);
+    assertEquals(ex.getError(), TestERROR.TC);
+    assertEquals(ex.getMessage(), "TC: " + cause.toString());
+    assertEquals(ex.getCause(), cause);
 
     XException xcause = ex;
     ex = new XException(xcause);
-    Assert.assertEquals(ex.getError(), TestERROR.TC);
-    Assert.assertEquals(ex.getMessage(), xcause.getMessage());
-    Assert.assertEquals(ex.getCause(), xcause);
+    assertEquals(ex.getError(), TestERROR.TC);
+    assertEquals(ex.getMessage(), xcause.getMessage());
+    assertEquals(ex.getCause(), xcause);
   }
 
 }

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/server/TestBaseService.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/server/TestBaseService.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/server/TestBaseService.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/server/TestBaseService.java Fri Oct 19 02:25:55 2012
@@ -18,7 +18,10 @@
 
 package org.apache.hadoop.lib.server;
 
-import junit.framework.Assert;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.test.HTestCase;
 import org.junit.Test;
@@ -47,9 +50,9 @@ public class TestBaseService extends HTe
   @Test
   public void baseService() throws Exception {
     BaseService service = new MyService();
-    Assert.assertNull(service.getInterface());
-    Assert.assertEquals(service.getPrefix(), "myservice");
-    Assert.assertEquals(service.getServiceDependencies().length, 0);
+    assertNull(service.getInterface());
+    assertEquals(service.getPrefix(), "myservice");
+    assertEquals(service.getServiceDependencies().length, 0);
 
     Server server = Mockito.mock(Server.class);
     Configuration conf = new Configuration(false);
@@ -60,9 +63,9 @@ public class TestBaseService extends HTe
     Mockito.when(server.getPrefixedName("myservice.")).thenReturn("server.myservice.");
 
     service.init(server);
-    Assert.assertEquals(service.getPrefixedName("foo"), "server.myservice.foo");
-    Assert.assertEquals(service.getServiceConfig().size(), 1);
-    Assert.assertEquals(service.getServiceConfig().get("foo"), "FOO");
-    Assert.assertTrue(MyService.INIT);
+    assertEquals(service.getPrefixedName("foo"), "server.myservice.foo");
+    assertEquals(service.getServiceConfig().size(), 1);
+    assertEquals(service.getServiceConfig().get("foo"), "FOO");
+    assertTrue(MyService.INIT);
   }
 }

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/server/TestServer.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/server/TestServer.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/server/TestServer.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/server/TestServer.java Fri Oct 19 02:25:55 2012
@@ -18,16 +18,12 @@
 
 package org.apache.hadoop.lib.server;
 
-import junit.framework.Assert;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.lib.lang.XException;
-import org.apache.hadoop.test.HTestCase;
-import org.apache.hadoop.test.TestDir;
-import org.apache.hadoop.test.TestDirHelper;
-import org.apache.hadoop.test.TestException;
-import org.apache.hadoop.util.StringUtils;
-import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.File;
 import java.io.FileOutputStream;
@@ -39,50 +35,60 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.lib.lang.XException;
+import org.apache.hadoop.test.HTestCase;
+import org.apache.hadoop.test.TestDir;
+import org.apache.hadoop.test.TestDirHelper;
+import org.apache.hadoop.test.TestException;
+import org.apache.hadoop.util.StringUtils;
+import org.junit.Test;
+
 public class TestServer extends HTestCase {
 
   @Test
   @TestDir
   public void constructorsGetters() throws Exception {
     Server server = new Server("server", "/a", "/b", "/c", "/d", new Configuration(false));
-    Assert.assertEquals(server.getHomeDir(), "/a");
-    Assert.assertEquals(server.getConfigDir(), "/b");
-    Assert.assertEquals(server.getLogDir(), "/c");
-    Assert.assertEquals(server.getTempDir(), "/d");
-    Assert.assertEquals(server.getName(), "server");
-    Assert.assertEquals(server.getPrefix(), "server");
-    Assert.assertEquals(server.getPrefixedName("name"), "server.name");
-    Assert.assertNotNull(server.getConfig());
+    assertEquals(server.getHomeDir(), "/a");
+    assertEquals(server.getConfigDir(), "/b");
+    assertEquals(server.getLogDir(), "/c");
+    assertEquals(server.getTempDir(), "/d");
+    assertEquals(server.getName(), "server");
+    assertEquals(server.getPrefix(), "server");
+    assertEquals(server.getPrefixedName("name"), "server.name");
+    assertNotNull(server.getConfig());
 
     server = new Server("server", "/a", "/b", "/c", "/d");
-    Assert.assertEquals(server.getHomeDir(), "/a");
-    Assert.assertEquals(server.getConfigDir(), "/b");
-    Assert.assertEquals(server.getLogDir(), "/c");
-    Assert.assertEquals(server.getTempDir(), "/d");
-    Assert.assertEquals(server.getName(), "server");
-    Assert.assertEquals(server.getPrefix(), "server");
-    Assert.assertEquals(server.getPrefixedName("name"), "server.name");
-    Assert.assertNull(server.getConfig());
+    assertEquals(server.getHomeDir(), "/a");
+    assertEquals(server.getConfigDir(), "/b");
+    assertEquals(server.getLogDir(), "/c");
+    assertEquals(server.getTempDir(), "/d");
+    assertEquals(server.getName(), "server");
+    assertEquals(server.getPrefix(), "server");
+    assertEquals(server.getPrefixedName("name"), "server.name");
+    assertNull(server.getConfig());
 
     server = new Server("server", TestDirHelper.getTestDir().getAbsolutePath(), new Configuration(false));
-    Assert.assertEquals(server.getHomeDir(), TestDirHelper.getTestDir().getAbsolutePath());
-    Assert.assertEquals(server.getConfigDir(), TestDirHelper.getTestDir() + "/conf");
-    Assert.assertEquals(server.getLogDir(), TestDirHelper.getTestDir() + "/log");
-    Assert.assertEquals(server.getTempDir(), TestDirHelper.getTestDir() + "/temp");
-    Assert.assertEquals(server.getName(), "server");
-    Assert.assertEquals(server.getPrefix(), "server");
-    Assert.assertEquals(server.getPrefixedName("name"), "server.name");
-    Assert.assertNotNull(server.getConfig());
+    assertEquals(server.getHomeDir(), TestDirHelper.getTestDir().getAbsolutePath());
+    assertEquals(server.getConfigDir(), TestDirHelper.getTestDir() + "/conf");
+    assertEquals(server.getLogDir(), TestDirHelper.getTestDir() + "/log");
+    assertEquals(server.getTempDir(), TestDirHelper.getTestDir() + "/temp");
+    assertEquals(server.getName(), "server");
+    assertEquals(server.getPrefix(), "server");
+    assertEquals(server.getPrefixedName("name"), "server.name");
+    assertNotNull(server.getConfig());
 
     server = new Server("server", TestDirHelper.getTestDir().getAbsolutePath());
-    Assert.assertEquals(server.getHomeDir(), TestDirHelper.getTestDir().getAbsolutePath());
-    Assert.assertEquals(server.getConfigDir(), TestDirHelper.getTestDir() + "/conf");
-    Assert.assertEquals(server.getLogDir(), TestDirHelper.getTestDir() + "/log");
-    Assert.assertEquals(server.getTempDir(), TestDirHelper.getTestDir() + "/temp");
-    Assert.assertEquals(server.getName(), "server");
-    Assert.assertEquals(server.getPrefix(), "server");
-    Assert.assertEquals(server.getPrefixedName("name"), "server.name");
-    Assert.assertNull(server.getConfig());
+    assertEquals(server.getHomeDir(), TestDirHelper.getTestDir().getAbsolutePath());
+    assertEquals(server.getConfigDir(), TestDirHelper.getTestDir() + "/conf");
+    assertEquals(server.getLogDir(), TestDirHelper.getTestDir() + "/log");
+    assertEquals(server.getTempDir(), TestDirHelper.getTestDir() + "/temp");
+    assertEquals(server.getName(), "server");
+    assertEquals(server.getPrefix(), "server");
+    assertEquals(server.getPrefixedName("name"), "server.name");
+    assertNull(server.getConfig());
   }
 
   @Test
@@ -113,9 +119,9 @@ public class TestServer extends HTestCas
   @TestDir
   public void initNoConfigDir() throws Exception {
     File homeDir = new File(TestDirHelper.getTestDir(), "home");
-    Assert.assertTrue(homeDir.mkdir());
-    Assert.assertTrue(new File(homeDir, "log").mkdir());
-    Assert.assertTrue(new File(homeDir, "temp").mkdir());
+    assertTrue(homeDir.mkdir());
+    assertTrue(new File(homeDir, "log").mkdir());
+    assertTrue(new File(homeDir, "temp").mkdir());
     Configuration conf = new Configuration(false);
     conf.set("server.services", TestService.class.getName());
     Server server = new Server("server", homeDir.getAbsolutePath(), conf);
@@ -127,9 +133,9 @@ public class TestServer extends HTestCas
   @TestDir
   public void initConfigDirNotDir() throws Exception {
     File homeDir = new File(TestDirHelper.getTestDir(), "home");
-    Assert.assertTrue(homeDir.mkdir());
-    Assert.assertTrue(new File(homeDir, "log").mkdir());
-    Assert.assertTrue(new File(homeDir, "temp").mkdir());
+    assertTrue(homeDir.mkdir());
+    assertTrue(new File(homeDir, "log").mkdir());
+    assertTrue(new File(homeDir, "temp").mkdir());
     File configDir = new File(homeDir, "conf");
     new FileOutputStream(configDir).close();
     Configuration conf = new Configuration(false);
@@ -143,9 +149,9 @@ public class TestServer extends HTestCas
   @TestDir
   public void initNoLogDir() throws Exception {
     File homeDir = new File(TestDirHelper.getTestDir(), "home");
-    Assert.assertTrue(homeDir.mkdir());
-    Assert.assertTrue(new File(homeDir, "conf").mkdir());
-    Assert.assertTrue(new File(homeDir, "temp").mkdir());
+    assertTrue(homeDir.mkdir());
+    assertTrue(new File(homeDir, "conf").mkdir());
+    assertTrue(new File(homeDir, "temp").mkdir());
     Configuration conf = new Configuration(false);
     conf.set("server.services", TestService.class.getName());
     Server server = new Server("server", homeDir.getAbsolutePath(), conf);
@@ -157,9 +163,9 @@ public class TestServer extends HTestCas
   @TestDir
   public void initLogDirNotDir() throws Exception {
     File homeDir = new File(TestDirHelper.getTestDir(), "home");
-    Assert.assertTrue(homeDir.mkdir());
-    Assert.assertTrue(new File(homeDir, "conf").mkdir());
-    Assert.assertTrue(new File(homeDir, "temp").mkdir());
+    assertTrue(homeDir.mkdir());
+    assertTrue(new File(homeDir, "conf").mkdir());
+    assertTrue(new File(homeDir, "temp").mkdir());
     File logDir = new File(homeDir, "log");
     new FileOutputStream(logDir).close();
     Configuration conf = new Configuration(false);
@@ -173,9 +179,9 @@ public class TestServer extends HTestCas
   @TestDir
   public void initNoTempDir() throws Exception {
     File homeDir = new File(TestDirHelper.getTestDir(), "home");
-    Assert.assertTrue(homeDir.mkdir());
-    Assert.assertTrue(new File(homeDir, "conf").mkdir());
-    Assert.assertTrue(new File(homeDir, "log").mkdir());
+    assertTrue(homeDir.mkdir());
+    assertTrue(new File(homeDir, "conf").mkdir());
+    assertTrue(new File(homeDir, "log").mkdir());
     Configuration conf = new Configuration(false);
     conf.set("server.services", TestService.class.getName());
     Server server = new Server("server", homeDir.getAbsolutePath(), conf);
@@ -187,9 +193,9 @@ public class TestServer extends HTestCas
   @TestDir
   public void initTempDirNotDir() throws Exception {
     File homeDir = new File(TestDirHelper.getTestDir(), "home");
-    Assert.assertTrue(homeDir.mkdir());
-    Assert.assertTrue(new File(homeDir, "conf").mkdir());
-    Assert.assertTrue(new File(homeDir, "log").mkdir());
+    assertTrue(homeDir.mkdir());
+    assertTrue(new File(homeDir, "conf").mkdir());
+    assertTrue(new File(homeDir, "log").mkdir());
     File tempDir = new File(homeDir, "temp");
     new FileOutputStream(tempDir).close();
     Configuration conf = new Configuration(false);
@@ -204,7 +210,7 @@ public class TestServer extends HTestCas
   public void siteFileNotAFile() throws Exception {
     String homeDir = TestDirHelper.getTestDir().getAbsolutePath();
     File siteFile = new File(homeDir, "server-site.xml");
-    Assert.assertTrue(siteFile.mkdir());
+    assertTrue(siteFile.mkdir());
     Server server = new Server("server", homeDir, homeDir, homeDir, homeDir);
     server.init();
   }
@@ -234,12 +240,12 @@ public class TestServer extends HTestCas
 
     @Override
     protected void init() throws ServiceException {
-      Assert.assertEquals(getServer().getStatus(), Server.Status.BOOTING);
+      assertEquals(getServer().getStatus(), Server.Status.BOOTING);
     }
 
     @Override
     public void destroy() {
-      Assert.assertEquals(getServer().getStatus(), Server.Status.SHUTTING_DOWN);
+      assertEquals(getServer().getStatus(), Server.Status.SHUTTING_DOWN);
       super.destroy();
     }
 
@@ -255,12 +261,12 @@ public class TestServer extends HTestCas
     Configuration conf = new Configuration(false);
     conf.set("server.services", LifeCycleService.class.getName());
     Server server = createServer(conf);
-    Assert.assertEquals(server.getStatus(), Server.Status.UNDEF);
+    assertEquals(server.getStatus(), Server.Status.UNDEF);
     server.init();
-    Assert.assertNotNull(server.get(LifeCycleService.class));
-    Assert.assertEquals(server.getStatus(), Server.Status.NORMAL);
+    assertNotNull(server.get(LifeCycleService.class));
+    assertEquals(server.getStatus(), Server.Status.NORMAL);
     server.destroy();
-    Assert.assertEquals(server.getStatus(), Server.Status.SHUTDOWN);
+    assertEquals(server.getStatus(), Server.Status.SHUTDOWN);
   }
 
   @Test
@@ -270,7 +276,7 @@ public class TestServer extends HTestCas
     conf.set("server.startup.status", "ADMIN");
     Server server = createServer(conf);
     server.init();
-    Assert.assertEquals(server.getStatus(), Server.Status.ADMIN);
+    assertEquals(server.getStatus(), Server.Status.ADMIN);
     server.destroy();
   }
 
@@ -334,7 +340,7 @@ public class TestServer extends HTestCas
     Server server = createServer(conf);
     server.init();
     server.setStatus(Server.Status.ADMIN);
-    Assert.assertTrue(TestService.LIFECYCLE.contains("serverStatusChange"));
+    assertTrue(TestService.LIFECYCLE.contains("serverStatusChange"));
   }
 
   @Test
@@ -357,7 +363,7 @@ public class TestServer extends HTestCas
     server.init();
     TestService.LIFECYCLE.clear();
     server.setStatus(server.getStatus());
-    Assert.assertFalse(TestService.LIFECYCLE.contains("serverStatusChange"));
+    assertFalse(TestService.LIFECYCLE.contains("serverStatusChange"));
   }
 
   @Test
@@ -368,9 +374,9 @@ public class TestServer extends HTestCas
     conf.set("server.services", TestService.class.getName());
     Server server = createServer(conf);
     server.init();
-    Assert.assertNotNull(server.get(TestService.class));
+    assertNotNull(server.get(TestService.class));
     server.destroy();
-    Assert.assertEquals(TestService.LIFECYCLE, Arrays.asList("init", "postInit", "serverStatusChange", "destroy"));
+    assertEquals(TestService.LIFECYCLE, Arrays.asList("init", "postInit", "serverStatusChange", "destroy"));
   }
 
   @Test
@@ -379,7 +385,7 @@ public class TestServer extends HTestCas
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
     Server server = new Server("testserver", dir, dir, dir, dir);
     server.init();
-    Assert.assertEquals(server.getConfig().get("testserver.a"), "default");
+    assertEquals(server.getConfig().get("testserver.a"), "default");
   }
 
   @Test
@@ -392,7 +398,7 @@ public class TestServer extends HTestCas
     w.close();
     Server server = new Server("testserver", dir, dir, dir, dir);
     server.init();
-    Assert.assertEquals(server.getConfig().get("testserver.a"), "site");
+    assertEquals(server.getConfig().get("testserver.a"), "site");
   }
 
   @Test
@@ -407,7 +413,7 @@ public class TestServer extends HTestCas
       w.close();
       Server server = new Server("testserver", dir, dir, dir, dir);
       server.init();
-      Assert.assertEquals(server.getConfig().get("testserver.a"), "sysprop");
+      assertEquals(server.getConfig().get("testserver.a"), "sysprop");
     } finally {
       System.getProperties().remove("testserver.a");
     }
@@ -633,7 +639,7 @@ public class TestServer extends HTestCas
     conf = new Configuration(false);
     server = new Server("server", dir, dir, dir, dir, conf);
     server.init();
-    Assert.assertEquals(ORDER.size(), 0);
+    assertEquals(ORDER.size(), 0);
 
     // 2 services init/destroy
     ORDER.clear();
@@ -643,17 +649,17 @@ public class TestServer extends HTestCas
     conf.set("server.services", services);
     server = new Server("server", dir, dir, dir, dir, conf);
     server.init();
-    Assert.assertEquals(server.get(MyService1.class).getInterface(), MyService1.class);
-    Assert.assertEquals(server.get(MyService3.class).getInterface(), MyService3.class);
-    Assert.assertEquals(ORDER.size(), 4);
-    Assert.assertEquals(ORDER.get(0), "s1.init");
-    Assert.assertEquals(ORDER.get(1), "s3.init");
-    Assert.assertEquals(ORDER.get(2), "s1.postInit");
-    Assert.assertEquals(ORDER.get(3), "s3.postInit");
+    assertEquals(server.get(MyService1.class).getInterface(), MyService1.class);
+    assertEquals(server.get(MyService3.class).getInterface(), MyService3.class);
+    assertEquals(ORDER.size(), 4);
+    assertEquals(ORDER.get(0), "s1.init");
+    assertEquals(ORDER.get(1), "s3.init");
+    assertEquals(ORDER.get(2), "s1.postInit");
+    assertEquals(ORDER.get(3), "s3.postInit");
     server.destroy();
-    Assert.assertEquals(ORDER.size(), 6);
-    Assert.assertEquals(ORDER.get(4), "s3.destroy");
-    Assert.assertEquals(ORDER.get(5), "s1.destroy");
+    assertEquals(ORDER.size(), 6);
+    assertEquals(ORDER.get(4), "s3.destroy");
+    assertEquals(ORDER.get(5), "s1.destroy");
 
     // 3 services, 2nd one fails on init
     ORDER.clear();
@@ -665,16 +671,16 @@ public class TestServer extends HTestCas
     server = new Server("server", dir, dir, dir, dir, conf);
     try {
       server.init();
-      Assert.fail();
+      fail();
     } catch (ServerException ex) {
-      Assert.assertEquals(MyService2.class, ex.getError().getClass());
+      assertEquals(MyService2.class, ex.getError().getClass());
     } catch (Exception ex) {
-      Assert.fail();
+      fail();
     }
-    Assert.assertEquals(ORDER.size(), 3);
-    Assert.assertEquals(ORDER.get(0), "s1.init");
-    Assert.assertEquals(ORDER.get(1), "s2.init");
-    Assert.assertEquals(ORDER.get(2), "s1.destroy");
+    assertEquals(ORDER.size(), 3);
+    assertEquals(ORDER.get(0), "s1.init");
+    assertEquals(ORDER.get(1), "s2.init");
+    assertEquals(ORDER.get(2), "s1.destroy");
 
     // 2 services one fails on destroy
     ORDER.clear();
@@ -683,15 +689,15 @@ public class TestServer extends HTestCas
     conf.set("server.services", services);
     server = new Server("server", dir, dir, dir, dir, conf);
     server.init();
-    Assert.assertEquals(ORDER.size(), 4);
-    Assert.assertEquals(ORDER.get(0), "s1.init");
-    Assert.assertEquals(ORDER.get(1), "s5.init");
-    Assert.assertEquals(ORDER.get(2), "s1.postInit");
-    Assert.assertEquals(ORDER.get(3), "s5.postInit");
+    assertEquals(ORDER.size(), 4);
+    assertEquals(ORDER.get(0), "s1.init");
+    assertEquals(ORDER.get(1), "s5.init");
+    assertEquals(ORDER.get(2), "s1.postInit");
+    assertEquals(ORDER.get(3), "s5.postInit");
     server.destroy();
-    Assert.assertEquals(ORDER.size(), 6);
-    Assert.assertEquals(ORDER.get(4), "s5.destroy");
-    Assert.assertEquals(ORDER.get(5), "s1.destroy");
+    assertEquals(ORDER.size(), 6);
+    assertEquals(ORDER.get(4), "s5.destroy");
+    assertEquals(ORDER.get(5), "s1.destroy");
 
 
     // service override via ext
@@ -705,16 +711,16 @@ public class TestServer extends HTestCas
     server = new Server("server", dir, dir, dir, dir, conf);
     server.init();
 
-    Assert.assertEquals(server.get(MyService1.class).getClass(), MyService1a.class);
-    Assert.assertEquals(ORDER.size(), 4);
-    Assert.assertEquals(ORDER.get(0), "s1a.init");
-    Assert.assertEquals(ORDER.get(1), "s3.init");
-    Assert.assertEquals(ORDER.get(2), "s1a.postInit");
-    Assert.assertEquals(ORDER.get(3), "s3.postInit");
+    assertEquals(server.get(MyService1.class).getClass(), MyService1a.class);
+    assertEquals(ORDER.size(), 4);
+    assertEquals(ORDER.get(0), "s1a.init");
+    assertEquals(ORDER.get(1), "s3.init");
+    assertEquals(ORDER.get(2), "s1a.postInit");
+    assertEquals(ORDER.get(3), "s3.postInit");
     server.destroy();
-    Assert.assertEquals(ORDER.size(), 6);
-    Assert.assertEquals(ORDER.get(4), "s3.destroy");
-    Assert.assertEquals(ORDER.get(5), "s1a.destroy");
+    assertEquals(ORDER.size(), 6);
+    assertEquals(ORDER.get(4), "s3.destroy");
+    assertEquals(ORDER.get(5), "s1a.destroy");
 
     // service override via setService
     ORDER.clear();
@@ -725,16 +731,16 @@ public class TestServer extends HTestCas
     server.init();
 
     server.setService(MyService1a.class);
-    Assert.assertEquals(ORDER.size(), 6);
-    Assert.assertEquals(ORDER.get(4), "s1.destroy");
-    Assert.assertEquals(ORDER.get(5), "s1a.init");
+    assertEquals(ORDER.size(), 6);
+    assertEquals(ORDER.get(4), "s1.destroy");
+    assertEquals(ORDER.get(5), "s1a.init");
 
-    Assert.assertEquals(server.get(MyService1.class).getClass(), MyService1a.class);
+    assertEquals(server.get(MyService1.class).getClass(), MyService1a.class);
 
     server.destroy();
-    Assert.assertEquals(ORDER.size(), 8);
-    Assert.assertEquals(ORDER.get(6), "s3.destroy");
-    Assert.assertEquals(ORDER.get(7), "s1a.destroy");
+    assertEquals(ORDER.size(), 8);
+    assertEquals(ORDER.get(6), "s3.destroy");
+    assertEquals(ORDER.get(7), "s1a.destroy");
 
     // service add via setService
     ORDER.clear();
@@ -745,16 +751,16 @@ public class TestServer extends HTestCas
     server.init();
 
     server.setService(MyService5.class);
-    Assert.assertEquals(ORDER.size(), 5);
-    Assert.assertEquals(ORDER.get(4), "s5.init");
+    assertEquals(ORDER.size(), 5);
+    assertEquals(ORDER.get(4), "s5.init");
 
-    Assert.assertEquals(server.get(MyService5.class).getClass(), MyService5.class);
+    assertEquals(server.get(MyService5.class).getClass(), MyService5.class);
 
     server.destroy();
-    Assert.assertEquals(ORDER.size(), 8);
-    Assert.assertEquals(ORDER.get(5), "s5.destroy");
-    Assert.assertEquals(ORDER.get(6), "s3.destroy");
-    Assert.assertEquals(ORDER.get(7), "s1.destroy");
+    assertEquals(ORDER.size(), 8);
+    assertEquals(ORDER.get(5), "s5.destroy");
+    assertEquals(ORDER.get(6), "s3.destroy");
+    assertEquals(ORDER.get(7), "s1.destroy");
 
     // service add via setService exception
     ORDER.clear();
@@ -765,15 +771,15 @@ public class TestServer extends HTestCas
     server.init();
     try {
       server.setService(MyService7.class);
-      Assert.fail();
+      fail();
     } catch (ServerException ex) {
-      Assert.assertEquals(ServerException.ERROR.S09, ex.getError());
+      assertEquals(ServerException.ERROR.S09, ex.getError());
     } catch (Exception ex) {
-      Assert.fail();
+      fail();
     }
-    Assert.assertEquals(ORDER.size(), 6);
-    Assert.assertEquals(ORDER.get(4), "s3.destroy");
-    Assert.assertEquals(ORDER.get(5), "s1.destroy");
+    assertEquals(ORDER.size(), 6);
+    assertEquals(ORDER.get(4), "s3.destroy");
+    assertEquals(ORDER.get(5), "s1.destroy");
 
     // service with dependency
     ORDER.clear();
@@ -782,8 +788,8 @@ public class TestServer extends HTestCas
     conf.set("server.services", services);
     server = new Server("server", dir, dir, dir, dir, conf);
     server.init();
-    Assert.assertEquals(server.get(MyService1.class).getInterface(), MyService1.class);
-    Assert.assertEquals(server.get(MyService6.class).getInterface(), MyService6.class);
+    assertEquals(server.get(MyService1.class).getInterface(), MyService1.class);
+    assertEquals(server.get(MyService6.class).getInterface(), MyService6.class);
     server.destroy();
   }
 

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/server/TestServerConstructor.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/server/TestServerConstructor.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/server/TestServerConstructor.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/server/TestServerConstructor.java Fri Oct 19 02:25:55 2012
@@ -18,15 +18,15 @@
 
 package org.apache.hadoop.lib.server;
 
+import java.util.Arrays;
+import java.util.Collection;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.test.HTestCase;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
-import java.util.Arrays;
-import java.util.Collection;
-
 @RunWith(value = Parameterized.class)
 public class TestServerConstructor extends HTestCase {
 

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/hadoop/TestFileSystemAccessService.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/hadoop/TestFileSystemAccessService.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/hadoop/TestFileSystemAccessService.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/hadoop/TestFileSystemAccessService.java Fri Oct 19 02:25:55 2012
@@ -18,7 +18,12 @@
 
 package org.apache.hadoop.lib.service.hadoop;
 
-import junit.framework.Assert;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Arrays;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
@@ -28,6 +33,7 @@ import org.apache.hadoop.lib.server.Serv
 import org.apache.hadoop.lib.service.FileSystemAccess;
 import org.apache.hadoop.lib.service.FileSystemAccessException;
 import org.apache.hadoop.lib.service.instrumentation.InstrumentationService;
+import org.apache.hadoop.lib.service.scheduler.SchedulerService;
 import org.apache.hadoop.test.HFSTestCase;
 import org.apache.hadoop.test.TestDir;
 import org.apache.hadoop.test.TestDirHelper;
@@ -35,15 +41,10 @@ import org.apache.hadoop.test.TestExcept
 import org.apache.hadoop.test.TestHdfs;
 import org.apache.hadoop.test.TestHdfsHelper;
 import org.apache.hadoop.util.StringUtils;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.Arrays;
-
 public class TestFileSystemAccessService extends HFSTestCase {
 
   private void createHadoopConf(Configuration hadoopConf) throws Exception {
@@ -65,8 +66,10 @@ public class TestFileSystemAccessService
   @TestDir
   public void simpleSecurity() throws Exception {
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
-    String services = StringUtils.join(",", Arrays.asList(InstrumentationService.class.getName(),
-                                                          FileSystemAccessService.class.getName()));
+    String services = StringUtils.join(",",
+      Arrays.asList(InstrumentationService.class.getName(),
+                    SchedulerService.class.getName(),
+                    FileSystemAccessService.class.getName()));
     Configuration conf = new Configuration(false);
     conf.set("server.services", services);
     Server server = new Server("server", dir, dir, dir, dir, conf);
@@ -80,8 +83,10 @@ public class TestFileSystemAccessService
   @TestDir
   public void noKerberosKeytabProperty() throws Exception {
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
-    String services = StringUtils.join(",", Arrays.asList(InstrumentationService.class.getName(),
-                                                          FileSystemAccessService.class.getName()));
+    String services = StringUtils.join(",",
+    Arrays.asList(InstrumentationService.class.getName(),
+                  SchedulerService.class.getName(),
+                  FileSystemAccessService.class.getName()));
     Configuration conf = new Configuration(false);
     conf.set("server.services", services);
     conf.set("server.hadoop.authentication.type", "kerberos");
@@ -95,8 +100,10 @@ public class TestFileSystemAccessService
   @TestDir
   public void noKerberosPrincipalProperty() throws Exception {
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
-    String services = StringUtils.join(",", Arrays.asList(InstrumentationService.class.getName(),
-                                                          FileSystemAccessService.class.getName()));
+    String services = StringUtils.join(",",
+      Arrays.asList(InstrumentationService.class.getName(),
+                    SchedulerService.class.getName(),
+                    FileSystemAccessService.class.getName()));
     Configuration conf = new Configuration(false);
     conf.set("server.services", services);
     conf.set("server.hadoop.authentication.type", "kerberos");
@@ -111,8 +118,10 @@ public class TestFileSystemAccessService
   @TestDir
   public void kerberosInitializationFailure() throws Exception {
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
-    String services = StringUtils.join(",", Arrays.asList(InstrumentationService.class.getName(),
-                                                          FileSystemAccessService.class.getName()));
+    String services = StringUtils.join(",",
+      Arrays.asList(InstrumentationService.class.getName(),
+                    SchedulerService.class.getName(),
+                    FileSystemAccessService.class.getName()));
     Configuration conf = new Configuration(false);
     conf.set("server.services", services);
     conf.set("server.hadoop.authentication.type", "kerberos");
@@ -127,8 +136,10 @@ public class TestFileSystemAccessService
   @TestDir
   public void invalidSecurity() throws Exception {
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
-    String services = StringUtils.join(",", Arrays.asList(InstrumentationService.class.getName(),
-                                                          FileSystemAccessService.class.getName()));
+    String services = StringUtils.join(",",
+      Arrays.asList(InstrumentationService.class.getName(),
+                    SchedulerService.class.getName(),
+                    FileSystemAccessService.class.getName()));
     Configuration conf = new Configuration(false);
     conf.set("server.services", services);
     conf.set("server.hadoop.authentication.type", "foo");
@@ -140,8 +151,10 @@ public class TestFileSystemAccessService
   @TestDir
   public void serviceHadoopConf() throws Exception {
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
-    String services = StringUtils.join(",", Arrays.asList(InstrumentationService.class.getName(),
-                                                          FileSystemAccessService.class.getName()));
+    String services = StringUtils.join(",",
+      Arrays.asList(InstrumentationService.class.getName(),
+                    SchedulerService.class.getName(),
+                    FileSystemAccessService.class.getName()));
     Configuration conf = new Configuration(false);
     conf.set("server.services", services);
 
@@ -158,8 +171,10 @@ public class TestFileSystemAccessService
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
     String hadoopConfDir = new File(dir, "confx").getAbsolutePath();
     new File(hadoopConfDir).mkdirs();
-    String services = StringUtils.join(",", Arrays.asList(InstrumentationService.class.getName(),
-                                                          FileSystemAccessService.class.getName()));
+    String services = StringUtils.join(",",
+      Arrays.asList(InstrumentationService.class.getName(),
+                    SchedulerService.class.getName(),
+                    FileSystemAccessService.class.getName()));
     Configuration conf = new Configuration(false);
     conf.set("server.services", services);
     conf.set("server.hadoop.config.dir", hadoopConfDir);
@@ -182,8 +197,10 @@ public class TestFileSystemAccessService
   @TestDir
   public void inWhitelists() throws Exception {
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
-    String services = StringUtils.join(",", Arrays.asList(InstrumentationService.class.getName(),
-                                                          FileSystemAccessService.class.getName()));
+    String services = StringUtils.join(",",
+      Arrays.asList(InstrumentationService.class.getName(),
+                    SchedulerService.class.getName(),
+                    FileSystemAccessService.class.getName()));
     Configuration conf = new Configuration(false);
     conf.set("server.services", services);
     Server server = new Server("server", dir, dir, dir, dir, conf);
@@ -216,8 +233,10 @@ public class TestFileSystemAccessService
   @TestDir
   public void NameNodeNotinWhitelists() throws Exception {
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
-    String services = StringUtils.join(",", Arrays.asList(InstrumentationService.class.getName(),
-                                                          FileSystemAccessService.class.getName()));
+    String services = StringUtils.join(",",
+      Arrays.asList(InstrumentationService.class.getName(),
+                    SchedulerService.class.getName(),
+                    FileSystemAccessService.class.getName()));
     Configuration conf = new Configuration(false);
     conf.set("server.services", services);
     conf.set("server.hadoop.name.node.whitelist", "NN");
@@ -232,8 +251,10 @@ public class TestFileSystemAccessService
   @TestHdfs
   public void createFileSystem() throws Exception {
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
-    String services = StringUtils.join(",", Arrays.asList(InstrumentationService.class.getName(),
-                                                          FileSystemAccessService.class.getName()));
+    String services = StringUtils.join(",",
+      Arrays.asList(InstrumentationService.class.getName(),
+                    SchedulerService.class.getName(),
+                    FileSystemAccessService.class.getName()));
 
     Configuration hadoopConf = new Configuration(false);
     hadoopConf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, TestHdfsHelper.getHdfsConf().get(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY));
@@ -241,6 +262,7 @@ public class TestFileSystemAccessService
 
     Configuration conf = new Configuration(false);
     conf.set("server.services", services);
+    conf.set("server.hadoop.filesystem.cache.purge.timeout", "0");
     Server server = new Server("server", dir, dir, dir, dir, conf);
     server.init();
     FileSystemAccess hadoop = server.get(FileSystemAccess.class);
@@ -263,8 +285,10 @@ public class TestFileSystemAccessService
   @TestHdfs
   public void fileSystemExecutor() throws Exception {
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
-    String services = StringUtils.join(",", Arrays.asList(InstrumentationService.class.getName(),
-                                                          FileSystemAccessService.class.getName()));
+    String services = StringUtils.join(",",
+      Arrays.asList(InstrumentationService.class.getName(),
+                    SchedulerService.class.getName(),
+                    FileSystemAccessService.class.getName()));
 
     Configuration hadoopConf = new Configuration(false);
     hadoopConf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, TestHdfsHelper.getHdfsConf().get(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY));
@@ -272,6 +296,7 @@ public class TestFileSystemAccessService
 
     Configuration conf = new Configuration(false);
     conf.set("server.services", services);
+    conf.set("server.hadoop.filesystem.cache.purge.timeout", "0");
     Server server = new Server("server", dir, dir, dir, dir, conf);
     server.init();
     FileSystemAccess hadoop = server.get(FileSystemAccess.class);
@@ -302,8 +327,10 @@ public class TestFileSystemAccessService
   @TestHdfs
   public void fileSystemExecutorNoNameNode() throws Exception {
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
-    String services = StringUtils.join(",", Arrays.asList(InstrumentationService.class.getName(),
-                                                          FileSystemAccessService.class.getName()));
+    String services = StringUtils.join(",",
+      Arrays.asList(InstrumentationService.class.getName(),
+                    SchedulerService.class.getName(),
+                    FileSystemAccessService.class.getName()));
     Configuration hadoopConf = new Configuration(false);
     hadoopConf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, TestHdfsHelper.getHdfsConf().get(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY));
     createHadoopConf(hadoopConf);
@@ -329,8 +356,10 @@ public class TestFileSystemAccessService
   @TestHdfs
   public void fileSystemExecutorException() throws Exception {
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
-    String services = StringUtils.join(",", Arrays.asList(InstrumentationService.class.getName(),
-                                                          FileSystemAccessService.class.getName()));
+    String services = StringUtils.join(",",
+      Arrays.asList(InstrumentationService.class.getName(),
+                    SchedulerService.class.getName(),
+                    FileSystemAccessService.class.getName()));
 
     Configuration hadoopConf = new Configuration(false);
     hadoopConf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, TestHdfsHelper.getHdfsConf().get(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY));
@@ -338,6 +367,7 @@ public class TestFileSystemAccessService
 
     Configuration conf = new Configuration(false);
     conf.set("server.services", services);
+    conf.set("server.hadoop.filesystem.cache.purge.timeout", "0");
     Server server = new Server("server", dir, dir, dir, dir, conf);
     server.init();
     FileSystemAccess hadoop = server.get(FileSystemAccess.class);
@@ -368,4 +398,69 @@ public class TestFileSystemAccessService
     server.destroy();
   }
 
+  @Test
+  @TestDir
+  @TestHdfs
+  public void fileSystemCache() throws Exception {
+    String dir = TestDirHelper.getTestDir().getAbsolutePath();
+    String services = StringUtils.join(",",
+      Arrays.asList(InstrumentationService.class.getName(),
+                    SchedulerService.class.getName(),
+                    FileSystemAccessService.class.getName()));
+
+    Configuration hadoopConf = new Configuration(false);
+    hadoopConf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY,
+      TestHdfsHelper.getHdfsConf().get(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY));
+    createHadoopConf(hadoopConf);
+
+    Configuration conf = new Configuration(false);
+    conf.set("server.services", services);
+    conf.set("server.hadoop.filesystem.cache.purge.frequency", "1");
+    conf.set("server.hadoop.filesystem.cache.purge.timeout", "1");
+    Server server = new Server("server", dir, dir, dir, dir, conf);
+    try {
+      server.init();
+      FileSystemAccess hadoop = server.get(FileSystemAccess.class);
+
+      FileSystem fs1 =
+        hadoop.createFileSystem("u", hadoop.getFileSystemConfiguration());
+      Assert.assertNotNull(fs1);
+      fs1.mkdirs(new Path("/tmp/foo1"));
+      hadoop.releaseFileSystem(fs1);
+
+      //still around because of caching
+      fs1.mkdirs(new Path("/tmp/foo2"));
+
+      FileSystem fs2 =
+        hadoop.createFileSystem("u", hadoop.getFileSystemConfiguration());
+
+      //should be same instance because of caching
+      Assert.assertEquals(fs1, fs2);
+
+      Thread.sleep(4 * 1000);
+
+      //still around because of lease count is 1 (fs2 is out)
+      fs1.mkdirs(new Path("/tmp/foo2"));
+
+      Thread.sleep(4 * 1000);
+
+      //still around because of lease count is 1 (fs2 is out)
+      fs2.mkdirs(new Path("/tmp/foo"));
+
+      hadoop.releaseFileSystem(fs2);
+      Thread.sleep(4 * 1000);
+
+      //should not be around as lease count is 0
+      try {
+        fs2.mkdirs(new Path("/tmp/foo"));
+        Assert.fail();
+      } catch (IOException ex) {
+      } catch (Exception ex) {
+        Assert.fail();
+      }
+    } finally {
+      server.destroy();
+    }
+  }
+
 }